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 cn...@apache.org on 2013/12/13 18:28:18 UTC

svn commit: r1550774 [6/10] - in /hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs: ./ src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/ src/main/java/ src/main/java/org/apache/hadoop/hdfs/ src/main/java/org/apach...

Modified: hadoop/common/branches/HDFS-4685/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/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java Fri Dec 13 17:28:14 2013
@@ -149,6 +149,7 @@ import org.apache.hadoop.hdfs.DFSConfigK
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
@@ -176,14 +177,7 @@ import org.apache.hadoop.hdfs.security.t
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
-import org.apache.hadoop.hdfs.server.blockmanagement.OutOfV1GenerationStampsException;
+import org.apache.hadoop.hdfs.server.blockmanagement.*;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
@@ -220,7 +214,8 @@ import org.apache.hadoop.hdfs.server.pro
 import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
-import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
+import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.util.ChunkedArrayList;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
@@ -2584,7 +2579,7 @@ public class FSNamesystem implements Nam
     }
 
     // choose targets for the new block to be allocated.
-    final DatanodeDescriptor targets[] = getBlockManager().chooseTarget( 
+    final DatanodeStorageInfo targets[] = getBlockManager().chooseTarget( 
         src, replication, clientNode, excludedNodes, blockSize, favoredNodes);
 
     // Part II.
@@ -2711,7 +2706,7 @@ public class FSNamesystem implements Nam
             src + ". Returning previously allocated block " + lastBlockInFile);
         long offset = pendingFile.computeFileSize();
         onRetryBlock[0] = makeLocatedBlock(lastBlockInFile,
-            ((BlockInfoUnderConstruction)lastBlockInFile).getExpectedLocations(),
+            ((BlockInfoUnderConstruction)lastBlockInFile).getExpectedStorageLocations(),
             offset);
         return iip;
       } else {
@@ -2729,11 +2724,10 @@ public class FSNamesystem implements Nam
     return iip;
   }
 
-  LocatedBlock makeLocatedBlock(Block blk,
-                                        DatanodeInfo[] locs,
+  LocatedBlock makeLocatedBlock(Block blk, DatanodeStorageInfo[] locs,
                                         long offset) throws IOException {
     LocatedBlock lBlk = new LocatedBlock(
-        getExtendedBlock(blk), locs, offset);
+        getExtendedBlock(blk), locs, offset, false);
     getBlockManager().setBlockToken(
         lBlk, BlockTokenSecretManager.AccessMode.WRITE);
     return lBlk;
@@ -2741,7 +2735,8 @@ public class FSNamesystem implements Nam
 
   /** @see NameNode#getAdditionalDatanode(String, ExtendedBlock, DatanodeInfo[], DatanodeInfo[], int, String) */
   LocatedBlock getAdditionalDatanode(String src, final ExtendedBlock blk,
-      final DatanodeInfo[] existings,  final Set<Node> excludes,
+      final DatanodeInfo[] existings, final String[] storageIDs,
+      final Set<Node> excludes,
       final int numAdditionalNodes, final String clientName
       ) throws IOException {
     //check if the feature is enabled
@@ -2749,7 +2744,7 @@ public class FSNamesystem implements Nam
 
     final DatanodeDescriptor clientnode;
     final long preferredblocksize;
-    final List<DatanodeDescriptor> chosen;
+    final List<DatanodeStorageInfo> chosen;
     checkOperation(OperationCategory.READ);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     readLock();
@@ -2764,23 +2759,18 @@ public class FSNamesystem implements Nam
       clientnode = file.getFileUnderConstructionFeature().getClientNode();
       preferredblocksize = file.getPreferredBlockSize();
 
-      //find datanode descriptors
-      chosen = new ArrayList<DatanodeDescriptor>();
-      for(DatanodeInfo d : existings) {
-        final DatanodeDescriptor descriptor = blockManager.getDatanodeManager(
-            ).getDatanode(d);
-        if (descriptor != null) {
-          chosen.add(descriptor);
-        }
-      }
+      //find datanode storages
+      final DatanodeManager dm = blockManager.getDatanodeManager();
+      chosen = Arrays.asList(dm.getDatanodeStorageInfos(existings, storageIDs));
     } finally {
       readUnlock();
     }
 
     // choose new datanodes.
-    final DatanodeInfo[] targets = blockManager.getBlockPlacementPolicy(
+    final DatanodeStorageInfo[] targets = blockManager.getBlockPlacementPolicy(
         ).chooseTarget(src, numAdditionalNodes, clientnode, chosen, true,
-        excludes, preferredblocksize);
+            // TODO: get storage type from the file
+        excludes, preferredblocksize, StorageType.DEFAULT);
     final LocatedBlock lb = new LocatedBlock(blk, targets);
     blockManager.setBlockToken(lb, AccessMode.COPY);
     return lb;
@@ -2951,14 +2941,13 @@ public class FSNamesystem implements Nam
    * @throws QuotaExceededException If addition of block exceeds space quota
    */
   BlockInfo saveAllocatedBlock(String src, INodesInPath inodesInPath,
-      Block newBlock, DatanodeDescriptor targets[]) throws IOException {
+      Block newBlock, DatanodeStorageInfo[] targets)
+          throws IOException {
     assert hasWriteLock();
     BlockInfo b = dir.addBlock(src, inodesInPath, newBlock, targets);
     NameNode.stateChangeLog.info("BLOCK* allocateBlock: " + src + ". "
         + getBlockPoolId() + " " + b);
-    for (DatanodeDescriptor dn : targets) {
-      dn.incBlocksScheduled();
-    }
+    DatanodeStorageInfo.incrementBlocksScheduled(targets);
     return b;
   }
 
@@ -3421,7 +3410,7 @@ public class FSNamesystem implements Nam
   boolean isFileClosed(String src) 
       throws AccessControlException, UnresolvedLinkException,
       StandbyException, IOException {
-    FSPermissionChecker pc = getPermissionChecker();	
+    FSPermissionChecker pc = getPermissionChecker();  
     checkOperation(OperationCategory.READ);
     readLock();
     try {
@@ -3718,7 +3707,7 @@ public class FSNamesystem implements Nam
       final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)lastBlock;
       // setup the last block locations from the blockManager if not known
       if (uc.getNumExpectedLocations() == 0) {
-        uc.setExpectedLocations(blockManager.getNodes(lastBlock));
+        uc.setExpectedLocations(blockManager.getStorages(lastBlock));
       }
 
       if (uc.getNumExpectedLocations() == 0 && uc.getNumBytes() == 0) {
@@ -3917,32 +3906,39 @@ public class FSNamesystem implements Nam
         // find the DatanodeDescriptor objects
         // There should be no locations in the blockManager till now because the
         // file is underConstruction
-        List<DatanodeDescriptor> targetList =
+        ArrayList<DatanodeDescriptor> trimmedTargets =
             new ArrayList<DatanodeDescriptor>(newtargets.length);
+        ArrayList<String> trimmedStorages =
+            new ArrayList<String>(newtargets.length);
         if (newtargets.length > 0) {
-          for (DatanodeID newtarget : newtargets) {
+          for (int i = 0; i < newtargets.length; ++i) {
             // try to get targetNode
             DatanodeDescriptor targetNode =
-                blockManager.getDatanodeManager().getDatanode(newtarget);
-            if (targetNode != null)
-              targetList.add(targetNode);
-            else if (LOG.isDebugEnabled()) {
-              LOG.debug("DatanodeDescriptor (=" + newtarget + ") not found");
+                blockManager.getDatanodeManager().getDatanode(newtargets[i]);
+            if (targetNode != null) {
+              trimmedTargets.add(targetNode);
+              trimmedStorages.add(newtargetstorages[i]);
+            } else if (LOG.isDebugEnabled()) {
+              LOG.debug("DatanodeDescriptor (=" + newtargets[i] + ") not found");
             }
           }
         }
-        if ((closeFile) && !targetList.isEmpty()) {
+        if ((closeFile) && !trimmedTargets.isEmpty()) {
           // the file is getting closed. Insert block locations into blockManager.
           // Otherwise fsck will report these blocks as MISSING, especially if the
           // blocksReceived from Datanodes take a long time to arrive.
-          for (DatanodeDescriptor targetNode : targetList) {
-            targetNode.addBlock(storedBlock);
+          for (int i = 0; i < trimmedTargets.size(); i++) {
+            trimmedTargets.get(i).addBlock(
+              trimmedStorages.get(i), storedBlock);
           }
         }
+
         // add pipeline locations into the INodeUnderConstruction
-        DatanodeDescriptor[] targetArray =
-            new DatanodeDescriptor[targetList.size()];
-        iFile.setLastBlock(storedBlock, targetList.toArray(targetArray));
+        DatanodeStorageInfo[] trimmedStorageInfos =
+            blockManager.getDatanodeManager().getDatanodeStorageInfos(
+                trimmedTargets.toArray(new DatanodeID[trimmedTargets.size()]),
+                trimmedStorages.toArray(new String[trimmedStorages.size()]));
+        iFile.setLastBlock(storedBlock, trimmedStorageInfos);
       }
 
       if (closeFile) {
@@ -4144,16 +4140,16 @@ public class FSNamesystem implements Nam
    * @throws IOException
    */
   HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg,
-      long capacity, long dfsUsed, long remaining, long blockPoolUsed,
-      long cacheCapacity, long cacheUsed, int xceiverCount, int xmitsInProgress,
-      int failedVolumes) throws IOException {
+      StorageReport[] reports, long cacheCapacity, long cacheUsed,
+      int xceiverCount, int xmitsInProgress, int failedVolumes)
+        throws IOException {
     readLock();
     try {
       final int maxTransfer = blockManager.getMaxReplicationStreams()
           - xmitsInProgress;
       DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat(
-          nodeReg, blockPoolId, capacity, dfsUsed, remaining, blockPoolUsed,
-          cacheCapacity, cacheUsed, xceiverCount, maxTransfer, failedVolumes);
+          nodeReg, reports, blockPoolId, cacheCapacity, cacheUsed,
+          xceiverCount, maxTransfer, failedVolumes);
       return new HeartbeatResponse(cmds, createHaStatusHeartbeat());
     } finally {
       readUnlock();
@@ -5314,11 +5310,11 @@ public class FSNamesystem implements Nam
   }
 
   public void processIncrementalBlockReport(final DatanodeID nodeID,
-      final String poolId, final ReceivedDeletedBlockInfo blockInfos[])
+      final String poolId, final StorageReceivedDeletedBlocks srdb)
       throws IOException {
     writeLock();
     try {
-      blockManager.processIncrementalBlockReport(nodeID, poolId, blockInfos);
+      blockManager.processIncrementalBlockReport(nodeID, poolId, srdb);
     } finally {
       writeUnlock();
     }
@@ -5807,9 +5803,10 @@ public class FSNamesystem implements Nam
       for (int i = 0; i < blocks.length; i++) {
         ExtendedBlock blk = blocks[i].getBlock();
         DatanodeInfo[] nodes = blocks[i].getLocations();
+        String[] storageIDs = blocks[i].getStorageIDs();
         for (int j = 0; j < nodes.length; j++) {
-          DatanodeInfo dn = nodes[j];
-          blockManager.findAndMarkBlockAsCorrupt(blk, dn,
+          blockManager.findAndMarkBlockAsCorrupt(blk, nodes[j],
+              storageIDs == null ? null: storageIDs[j], 
               "client machine reported it");
         }
       }
@@ -5864,7 +5861,7 @@ public class FSNamesystem implements Nam
    * @throws IOException if any error occurs
    */
   void updatePipeline(String clientName, ExtendedBlock oldBlock, 
-      ExtendedBlock newBlock, DatanodeID[] newNodes)
+      ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
       throws IOException {
     checkOperation(OperationCategory.WRITE);
     CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
@@ -5885,7 +5882,7 @@ public class FSNamesystem implements Nam
       assert newBlock.getBlockId()==oldBlock.getBlockId() : newBlock + " and "
         + oldBlock + " has different block identifier";
       updatePipelineInternal(clientName, oldBlock, newBlock, newNodes,
-          cacheEntry != null);
+          newStorageIDs, cacheEntry != null);
       success = true;
     } finally {
       writeUnlock();
@@ -5897,7 +5894,8 @@ public class FSNamesystem implements Nam
 
   /** @see #updatePipeline(String, ExtendedBlock, ExtendedBlock, DatanodeID[]) */
   private void updatePipelineInternal(String clientName, ExtendedBlock oldBlock, 
-      ExtendedBlock newBlock, DatanodeID[] newNodes, boolean logRetryCache)
+      ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs,
+      boolean logRetryCache)
       throws IOException {
     assert hasWriteLock();
     // check the vadility of the block and lease holder name
@@ -5920,15 +5918,9 @@ public class FSNamesystem implements Nam
     blockinfo.setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp());
 
     // find the DatanodeDescriptor objects
-    final DatanodeManager dm = getBlockManager().getDatanodeManager();
-    DatanodeDescriptor[] descriptors = null;
-    if (newNodes.length > 0) {
-      descriptors = new DatanodeDescriptor[newNodes.length];
-      for(int i = 0; i < newNodes.length; i++) {
-        descriptors[i] = dm.getDatanode(newNodes[i]);
-      }
-    }
-    blockinfo.setExpectedLocations(descriptors);
+    final DatanodeStorageInfo[] storages = blockManager.getDatanodeManager()
+        .getDatanodeStorageInfos(newNodes, newStorageIDs);
+    blockinfo.setExpectedLocations(storages);
 
     String src = leaseManager.findPath(pendingFile);
     dir.persistBlocks(src, pendingFile, logRetryCache);
@@ -6055,7 +6047,7 @@ public class FSNamesystem implements Nam
    * @throws IOException
    */
   Collection<CorruptFileBlockInfo> listCorruptFileBlocks(String path,
-	String[] cookieTab) throws IOException {
+  String[] cookieTab) throws IOException {
     checkSuperuserPrivilege();
     checkOperation(OperationCategory.READ);
     readLock();
@@ -7400,3 +7392,4 @@ public class FSNamesystem implements Nam
     }
   }
 }
+

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java Fri Dec 13 17:28:14 2013
@@ -167,19 +167,13 @@ public class FileJournalManager implemen
   /**
    * Find all editlog segments starting at or above the given txid.
    * @param fromTxId the txnid which to start looking
-   * @param forReading whether or not the caller intends to read from the edit
-   *        logs
    * @param inProgressOk whether or not to include the in-progress edit log 
    *        segment       
    * @return a list of remote edit logs
    * @throws IOException if edit logs cannot be listed.
    */
   public List<RemoteEditLog> getRemoteEditLogs(long firstTxId,
-      boolean forReading, boolean inProgressOk) throws IOException {
-    // make sure not reading in-progress edit log, i.e., if forReading is true,
-    // we should ignore the in-progress edit log.
-    Preconditions.checkArgument(!(forReading && inProgressOk));
-    
+      boolean inProgressOk) throws IOException {
     File currentDir = sd.getCurrentDir();
     List<EditLogFile> allLogFiles = matchEditLogs(currentDir);
     List<RemoteEditLog> ret = Lists.newArrayListWithCapacity(
@@ -192,14 +186,9 @@ public class FileJournalManager implemen
       if (elf.getFirstTxId() >= firstTxId) {
         ret.add(new RemoteEditLog(elf.firstTxId, elf.lastTxId));
       } else if (elf.getFirstTxId() < firstTxId && firstTxId <= elf.getLastTxId()) {
-        // If the firstTxId is in the middle of an edit log segment
-        if (forReading) {
-          // Note that this behavior is different from getLogFiles below.
-          throw new IllegalStateException("Asked for firstTxId " + firstTxId
-              + " which is in the middle of file " + elf.file);
-        } else {
-          ret.add(new RemoteEditLog(elf.firstTxId, elf.lastTxId));
-        }
+        // If the firstTxId is in the middle of an edit log segment. Return this
+        // anyway and let the caller figure out whether it wants to use it.
+        ret.add(new RemoteEditLog(elf.firstTxId, elf.lastTxId));
       }
     }
     
@@ -260,7 +249,7 @@ public class FileJournalManager implemen
   @Override
   synchronized public void selectInputStreams(
       Collection<EditLogInputStream> streams, long fromTxId,
-      boolean inProgressOk, boolean forReading) throws IOException {
+      boolean inProgressOk) throws IOException {
     List<EditLogFile> elfs = matchEditLogs(sd.getCurrentDir());
     LOG.debug(this + ": selecting input streams starting at " + fromTxId + 
         (inProgressOk ? " (inProgress ok) " : " (excluding inProgress) ") +

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java Fri Dec 13 17:28:14 2013
@@ -29,7 +29,7 @@ import org.apache.hadoop.hdfs.server.nam
  * Feature for under-construction file.
  */
 @InterfaceAudience.Private
-public class FileUnderConstructionFeature extends INodeFile.Feature {
+public class FileUnderConstructionFeature implements INode.Feature {
   private String clientName; // lease holder
   private final String clientMachine;
   // if client is a cluster node too.

Modified: hadoop/common/branches/HDFS-4685/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/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java Fri Dec 13 17:28:14 2013
@@ -756,47 +756,6 @@ public abstract class INode implements I
   /** INode feature such as {@link FileUnderConstructionFeature}
    *  and {@link DirectoryWithQuotaFeature}.
    */
-  interface Feature<F extends Feature<F>> {
-    /** @return the next feature. */
-    public F getNextFeature();
-
-    /** Set the next feature. */
-    public void setNextFeature(F next);
-
-    /** Utility methods such as addFeature and removeFeature. */
-    static class Util {
-      /**
-       * Add a feature to the linked list.
-       * @return the new head.
-       */
-      static <F extends Feature<F>> F addFeature(F feature, F head) {
-        feature.setNextFeature(head);
-        return feature;
-      }
-
-      /**
-       * Remove a feature from the linked list.
-       * @return the new head.
-       */
-      static <F extends Feature<F>> F removeFeature(F feature, F head) {
-        if (feature == head) {
-          final F newHead = head.getNextFeature();
-          head.setNextFeature(null);
-          return newHead;
-        } else if (head != null) {
-          F prev = head;
-          F curr = head.getNextFeature();
-          for (; curr != null && curr != feature;
-              prev = curr, curr = curr.getNextFeature())
-            ;
-          if (curr != null) {
-            prev.setNextFeature(curr.getNextFeature());
-            curr.setNextFeature(null);
-            return head;
-          }
-        }
-        throw new IllegalStateException("Feature " + feature + " not found.");
-      }
-    }
+  public interface Feature {
   }
 }

Modified: hadoop/common/branches/HDFS-4685/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/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectory.java Fri Dec 13 17:28:14 2013
@@ -45,20 +45,6 @@ import com.google.common.base.Preconditi
  */
 public class INodeDirectory extends INodeWithAdditionalFields
     implements INodeDirectoryAttributes {
-  /** Directory related features such as quota and snapshots. */
-  public static abstract class Feature implements INode.Feature<Feature> {
-    private Feature nextFeature;
-
-    @Override
-    public Feature getNextFeature() {
-      return nextFeature;
-    }
-
-    @Override
-    public void setNextFeature(Feature next) {
-      this.nextFeature = next;
-    }
-  }
 
   /** Cast INode to INodeDirectory. */
   public static INodeDirectory valueOf(INode inode, Object path
@@ -78,9 +64,6 @@ public class INodeDirectory extends INod
 
   private List<INode> children = null;
   
-  /** A linked list of {@link Feature}s. */
-  private Feature headFeature = null;
-
   /** constructor */
   public INodeDirectory(long id, byte[] name, PermissionStatus permissions,
       long mtime) {
@@ -102,7 +85,7 @@ public class INodeDirectory extends INod
       }
     }
     if (copyFeatures) {
-      this.headFeature = other.headFeature;
+      this.features = other.features;
     }
   }
 
@@ -160,7 +143,7 @@ public class INodeDirectory extends INod
    * otherwise, return null.
    */
   public final DirectoryWithQuotaFeature getDirectoryWithQuotaFeature() {
-    for(Feature f = headFeature; f != null; f = f.nextFeature) {
+    for (Feature f : features) {
       if (f instanceof DirectoryWithQuotaFeature) {
         return (DirectoryWithQuotaFeature)f;
       }
@@ -182,14 +165,6 @@ public class INodeDirectory extends INod
     return quota;
   }
 
-  private void addFeature(Feature f) {
-    headFeature = INode.Feature.Util.addFeature(f, headFeature);
-  }
-
-  private void removeFeature(Feature f) {
-    headFeature = INode.Feature.Util.removeFeature(f, headFeature);
-  }
-
   private int searchChildren(byte[] name) {
     return children == null? -1: Collections.binarySearch(children, name);
   }

Modified: hadoop/common/branches/HDFS-4685/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/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java Fri Dec 13 17:28:14 2013
@@ -27,10 +27,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.*;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
@@ -44,23 +41,6 @@ import com.google.common.base.Preconditi
 @InterfaceAudience.Private
 public class INodeFile extends INodeWithAdditionalFields
     implements INodeFileAttributes, BlockCollection {
-  /**
-   * A feature contains specific information for a type of INodeFile. E.g.,
-   * we can have separate features for Under-Construction and Snapshot.
-   */
-  public static abstract class Feature implements INode.Feature<Feature> {
-    private Feature nextFeature;
-
-    @Override
-    public Feature getNextFeature() {
-      return nextFeature;
-    }
-
-    @Override
-    public void setNextFeature(Feature next) {
-      this.nextFeature = next;
-    }
-  }
 
   /** The same as valueOf(inode, path, false). */
   public static INodeFile valueOf(INode inode, String path
@@ -123,8 +103,6 @@ public class INodeFile extends INodeWith
 
   private BlockInfo[] blocks;
 
-  private Feature headFeature;
-
   INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
       long atime, BlockInfo[] blklist, short replication,
       long preferredBlockSize) {
@@ -138,7 +116,7 @@ public class INodeFile extends INodeWith
     super(that);
     this.header = that.header;
     this.blocks = that.blocks;
-    this.headFeature = that.headFeature;
+    this.features = that.features;
   }
   
   public INodeFile(INodeFile that, FileDiffList diffs) {
@@ -147,14 +125,6 @@ public class INodeFile extends INodeWith
     this.addSnapshotFeature(diffs);
   }
 
-  private void addFeature(Feature f) {
-    headFeature = INode.Feature.Util.addFeature(f, headFeature);
-  }
-
-  private void removeFeature(Feature f) {
-    headFeature = INode.Feature.Util.removeFeature(f, headFeature);
-  }
-
   /** @return true unconditionally. */
   @Override
   public final boolean isFile() {
@@ -174,7 +144,7 @@ public class INodeFile extends INodeWith
    * otherwise, return null.
    */
   public final FileUnderConstructionFeature getFileUnderConstructionFeature() {
-    for (Feature f = this.headFeature; f != null; f = f.nextFeature) {
+    for (Feature f : features) {
       if (f instanceof FileUnderConstructionFeature) {
         return (FileUnderConstructionFeature) f;
       }
@@ -234,7 +204,7 @@ public class INodeFile extends INodeWith
 
   @Override // BlockCollection, the file should be under construction
   public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
-      DatanodeDescriptor[] locations) throws IOException {
+      DatanodeStorageInfo[] locations) throws IOException {
     Preconditions.checkState(isUnderConstruction(),
         "file is no longer under construction");
 
@@ -286,7 +256,7 @@ public class INodeFile extends INodeWith
    * otherwise, return null.
    */
   public final FileWithSnapshotFeature getFileWithSnapshotFeature() {
-    for (Feature f = headFeature; f != null; f = f.nextFeature) {
+    for (Feature f: features) {
       if (f instanceof FileWithSnapshotFeature) {
         return (FileWithSnapshotFeature) f;
       }

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeWithAdditionalFields.java Fri Dec 13 17:28:14 2013
@@ -95,6 +95,9 @@ public abstract class INodeWithAdditiona
 
   /** For implementing {@link LinkedElement}. */
   private LinkedElement next = null;
+  /** An array {@link Feature}s. */
+  private static final Feature[] EMPTY_FEATURE = new Feature[0];
+  protected Feature[] features = EMPTY_FEATURE;
 
   private INodeWithAdditionalFields(INode parent, long id, byte[] name,
       long permission, long modificationTime, long accessTime) {
@@ -262,4 +265,45 @@ public abstract class INodeWithAdditiona
   public final void setAccessTime(long accessTime) {
     this.accessTime = accessTime;
   }
+
+  protected void addFeature(Feature f) {
+    int size = features.length;
+    Feature[] arr = new Feature[size + 1];
+    if (size != 0) {
+      System.arraycopy(features, 0, arr, 0, size);
+    }
+    arr[size] = f;
+    features = arr;
+  }
+
+  protected void removeFeature(Feature f) {
+    int size = features.length;
+    Preconditions.checkState(size > 0, "Feature "
+        + f.getClass().getSimpleName() + " not found.");
+
+    if (size == 1) {
+      Preconditions.checkState(features[0] == f, "Feature "
+          + f.getClass().getSimpleName() + " not found.");
+      features = EMPTY_FEATURE;
+      return;
+    }
+
+    Feature[] arr = new Feature[size - 1];
+    int j = 0;
+    boolean overflow = false;
+    for (Feature f1 : features) {
+      if (f1 != f) {
+        if (j == size - 1) {
+          overflow = true;
+          break;
+        } else {
+          arr[j++] = f1;
+        }
+      }
+    }
+
+    Preconditions.checkState(!overflow && j == size - 1, "Feature "
+        + f.getClass().getSimpleName() + " not found.");
+    features = arr;
+  }
 }

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java Fri Dec 13 17:28:14 2013
@@ -233,12 +233,10 @@ public class JournalSet implements Journ
    *                         may not be sorted-- this is up to the caller.
    * @param fromTxId         The transaction ID to start looking for streams at
    * @param inProgressOk     Should we consider unfinalized streams?
-   * @param forReading       Whether or not the caller intends to read from
-   *                         the returned streams.
    */
   @Override
   public void selectInputStreams(Collection<EditLogInputStream> streams,
-      long fromTxId, boolean inProgressOk, boolean forReading) {
+      long fromTxId, boolean inProgressOk) {
     final PriorityQueue<EditLogInputStream> allStreams = 
         new PriorityQueue<EditLogInputStream>(64,
             EDIT_LOG_INPUT_STREAM_COMPARATOR);
@@ -248,8 +246,7 @@ public class JournalSet implements Journ
         continue;
       }
       try {
-        jas.getManager().selectInputStreams(allStreams, fromTxId, inProgressOk,
-            forReading);
+        jas.getManager().selectInputStreams(allStreams, fromTxId, inProgressOk);
       } catch (IOException ioe) {
         LOG.warn("Unable to determine input streams from " + jas.getManager() +
             ". Skipping.", ioe);
@@ -582,20 +579,20 @@ public class JournalSet implements Journ
   
   /**
    * Return a manifest of what finalized edit logs are available. All available
-   * edit logs are returned starting from the transaction id passed.
+   * edit logs are returned starting from the transaction id passed. If
+   * 'fromTxId' falls in the middle of a log, that log is returned as well.
    * 
    * @param fromTxId Starting transaction id to read the logs.
    * @return RemoteEditLogManifest object.
    */
-  public synchronized RemoteEditLogManifest getEditLogManifest(long fromTxId,
-      boolean forReading) {
+  public synchronized RemoteEditLogManifest getEditLogManifest(long fromTxId) {
     // Collect RemoteEditLogs available from each FileJournalManager
     List<RemoteEditLog> allLogs = Lists.newArrayList();
     for (JournalAndStream j : journals) {
       if (j.getManager() instanceof FileJournalManager) {
         FileJournalManager fjm = (FileJournalManager)j.getManager();
         try {
-          allLogs.addAll(fjm.getRemoteEditLogs(fromTxId, forReading, false));
+          allLogs.addAll(fjm.getRemoteEditLogs(fromTxId, false));
         } catch (Throwable t) {
           LOG.warn("Cannot list edit logs in " + fjm, t);
         }

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LogsPurgeable.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LogsPurgeable.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LogsPurgeable.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LogsPurgeable.java Fri Dec 13 17:28:14 2013
@@ -42,13 +42,11 @@ interface LogsPurgeable {
    * 
    * @param fromTxId the first transaction id we want to read
    * @param inProgressOk whether or not in-progress streams should be returned
-   * @param forReading whether or not the caller intends to read from the edit logs
-   *
    * @return a list of streams
    * @throws IOException if the underlying storage has an error or is otherwise
    * inaccessible
    */
   void selectInputStreams(Collection<EditLogInputStream> streams,
-      long fromTxId, boolean inProgressOk, boolean forReading) throws IOException;
+      long fromTxId, boolean inProgressOk) throws IOException;
   
 }

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorageRetentionManager.java Fri Dec 13 17:28:14 2013
@@ -108,7 +108,7 @@ public class NNStorageRetentionManager {
     long purgeLogsFrom = Math.max(0, minimumRequiredTxId - numExtraEditsToRetain);
     
     ArrayList<EditLogInputStream> editLogs = new ArrayList<EditLogInputStream>();
-    purgeableLogs.selectInputStreams(editLogs, purgeLogsFrom, false, false);
+    purgeableLogs.selectInputStreams(editLogs, purgeLogsFrom, false);
     Collections.sort(editLogs, new Comparator<EditLogInputStream>() {
       @Override
       public int compare(EditLogInputStream a, EditLogInputStream b) {

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java Fri Dec 13 17:28:14 2013
@@ -43,8 +43,8 @@ import org.apache.hadoop.fs.FileAlreadyE
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.Options;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
 import org.apache.hadoop.fs.permission.AclEntry;
@@ -67,21 +67,21 @@ import org.apache.hadoop.hdfs.protocol.C
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
+import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
@@ -97,9 +97,9 @@ import org.apache.hadoop.hdfs.protocolPB
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
@@ -200,9 +200,9 @@ class NameNodeRpcServer implements Namen
 
     NamenodeProtocolServerSideTranslatorPB namenodeProtocolXlator = 
         new NamenodeProtocolServerSideTranslatorPB(this);
-	  BlockingService NNPbService = NamenodeProtocolService
+    BlockingService NNPbService = NamenodeProtocolService
           .newReflectiveBlockingService(namenodeProtocolXlator);
-	  
+    
     RefreshAuthorizationPolicyProtocolServerSideTranslatorPB refreshAuthPolicyXlator = 
         new RefreshAuthorizationPolicyProtocolServerSideTranslatorPB(this);
     BlockingService refreshAuthService = RefreshAuthorizationPolicyProtocolService
@@ -222,7 +222,7 @@ class NameNodeRpcServer implements Namen
         new HAServiceProtocolServerSideTranslatorPB(this);
     BlockingService haPbService = HAServiceProtocolService
         .newReflectiveBlockingService(haServiceProtocolXlator);
-	  
+    
     WritableRpcEngine.ensureInitialized();
 
     InetSocketAddress serviceRpcAddr = nn.getServiceRpcServerAddress(conf);
@@ -573,7 +573,8 @@ class NameNodeRpcServer implements Namen
 
   @Override // ClientProtocol
   public LocatedBlock getAdditionalDatanode(final String src, final ExtendedBlock blk,
-      final DatanodeInfo[] existings, final DatanodeInfo[] excludes,
+      final DatanodeInfo[] existings, final String[] existingStorageIDs,
+      final DatanodeInfo[] excludes,
       final int numAdditionalNodes, final String clientName
       ) throws IOException {
     if (LOG.isDebugEnabled()) {
@@ -594,8 +595,8 @@ class NameNodeRpcServer implements Namen
         excludeSet.add(node);
       }
     }
-    return namesystem.getAdditionalDatanode(src, blk,
-        existings, excludeSet, numAdditionalNodes, clientName);
+    return namesystem.getAdditionalDatanode(src, blk, existings,
+        existingStorageIDs, excludeSet, numAdditionalNodes, clientName);
   }
   /**
    * The client needs to give up on the block.
@@ -643,9 +644,9 @@ class NameNodeRpcServer implements Namen
 
   @Override // ClientProtocol
   public void updatePipeline(String clientName, ExtendedBlock oldBlock,
-      ExtendedBlock newBlock, DatanodeID[] newNodes)
+      ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
       throws IOException {
-    namesystem.updatePipeline(clientName, oldBlock, newBlock, newNodes);
+    namesystem.updatePipeline(clientName, oldBlock, newBlock, newNodes, newStorageIDs);
   }
   
   @Override // DatanodeProtocol
@@ -967,24 +968,25 @@ class NameNodeRpcServer implements Namen
       int xmitsInProgress, int xceiverCount,
       int failedVolumes) throws IOException {
     verifyRequest(nodeReg);
-    return namesystem.handleHeartbeat(nodeReg, report[0].getCapacity(),
-        report[0].getDfsUsed(), report[0].getRemaining(),
-        report[0].getBlockPoolUsed(), dnCacheCapacity, dnCacheUsed,
-        xceiverCount, xmitsInProgress, failedVolumes);
+    return namesystem.handleHeartbeat(nodeReg, report,
+        dnCacheCapacity, dnCacheUsed, xceiverCount, xmitsInProgress,
+        failedVolumes);
   }
 
   @Override // DatanodeProtocol
   public DatanodeCommand blockReport(DatanodeRegistration nodeReg,
       String poolId, StorageBlockReport[] reports) throws IOException {
     verifyRequest(nodeReg);
-    BlockListAsLongs blist = new BlockListAsLongs(reports[0].getBlocks());
     if(blockStateChangeLog.isDebugEnabled()) {
       blockStateChangeLog.debug("*BLOCK* NameNode.blockReport: "
-           + "from " + nodeReg + " " + blist.getNumberOfBlocks()
-           + " blocks");
+           + "from " + nodeReg + ", reports.length=" + reports.length);
+    }
+    final BlockManager bm = namesystem.getBlockManager(); 
+    for(StorageBlockReport r : reports) {
+      final BlockListAsLongs blocks = new BlockListAsLongs(r.getBlocks());
+      bm.processReport(nodeReg, r.getStorage(), poolId, blocks);
     }
 
-    namesystem.getBlockManager().processReport(nodeReg, poolId, blist);
     if (nn.getFSImage().isUpgradeFinalized() && !nn.isStandbyState())
       return new FinalizeCommand(poolId);
     return null;
@@ -1011,8 +1013,9 @@ class NameNodeRpcServer implements Namen
           +"from "+nodeReg+" "+receivedAndDeletedBlocks.length
           +" blocks.");
     }
-    namesystem.processIncrementalBlockReport(
-        nodeReg, poolId, receivedAndDeletedBlocks[0].getBlocks());
+    for(StorageReceivedDeletedBlocks r : receivedAndDeletedBlocks) {
+      namesystem.processIncrementalBlockReport(nodeReg, poolId, r);
+    }
   }
 
   @Override // DatanodeProtocol
@@ -1314,3 +1317,4 @@ class NameNodeRpcServer implements Namen
     return namesystem.getAclStatus(src);
   }
 }
+

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeJspHelper.java Fri Dec 13 17:28:14 2013
@@ -27,13 +27,7 @@ import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URLEncoder;
 import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Date;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 
 import javax.servlet.ServletContext;
 import javax.servlet.http.HttpServletRequest;
@@ -50,6 +44,7 @@ import org.apache.hadoop.hdfs.security.t
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@@ -1113,13 +1108,12 @@ class NamenodeJspHelper {
         } 
 
         doc.startTag("replicas");
-        for (final Iterator<DatanodeDescriptor> it = blockManager != null ?
-            blockManager.datanodeIterator(block) :
-            Collections.<DatanodeDescriptor>emptyList().iterator();
-            it.hasNext();) {
+        for(DatanodeStorageInfo storage : (blockManager != null ?
+                blockManager.getStorages(block) :
+                Collections.<DatanodeStorageInfo>emptyList())) {
           doc.startTag("replica");
 
-          DatanodeDescriptor dd = it.next();
+          DatanodeDescriptor dd = storage.getDatanodeDescriptor();
 
           doc.startTag("host_name");
           doc.pcdata(dd.getHostName());

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java Fri Dec 13 17:28:14 2013
@@ -827,7 +827,7 @@ public class SecondaryNameNode implement
 
       @Override
       public void selectInputStreams(Collection<EditLogInputStream> streams,
-          long fromTxId, boolean inProgressOk, boolean forReading) {
+          long fromTxId, boolean inProgressOk) {
         Iterator<StorageDirectory> iter = storage.dirIterator();
         while (iter.hasNext()) {
           StorageDirectory dir = iter.next();

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java Fri Dec 13 17:28:14 2013
@@ -228,7 +228,7 @@ public class BootstrapStandby implements
     try {
       Collection<EditLogInputStream> streams =
         image.getEditLog().selectInputStreams(
-          firstTxIdInLogs, curTxIdOnOtherNode, null, true, false);
+          firstTxIdInLogs, curTxIdOnOtherNode, null, true);
       for (EditLogInputStream stream : streams) {
         IOUtils.closeStream(stream);
       }

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java Fri Dec 13 17:28:14 2013
@@ -165,7 +165,7 @@ public class EditLogTailer {
   }
   
   @VisibleForTesting
-  void setEditLog(FSEditLog editLog) {
+  public void setEditLog(FSEditLog editLog) {
     this.editLog = editLog;
   }
   

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java Fri Dec 13 17:28:14 2013
@@ -32,7 +32,7 @@ import org.apache.hadoop.hdfs.server.nam
  * Feature for file with snapshot-related information.
  */
 @InterfaceAudience.Private
-public class FileWithSnapshotFeature extends INodeFile.Feature {
+public class FileWithSnapshotFeature implements INode.Feature {
   private final FileDiffList diffs;
   private boolean isCurrentFileDeleted = false;
   

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java Fri Dec 13 17:28:14 2013
@@ -53,6 +53,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.hdfs.StorageType;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -61,6 +62,7 @@ import org.apache.hadoop.hdfs.security.t
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
@@ -160,7 +162,7 @@ public class NamenodeWebHdfsMethods {
 
   static DatanodeInfo chooseDatanode(final NameNode namenode,
       final String path, final HttpOpParam.Op op, final long openOffset,
-      final long blocksize, Configuration conf) throws IOException {
+      final long blocksize, final Configuration conf) throws IOException {
     final BlockManager bm = namenode.getNamesystem().getBlockManager();
 
     if (op == PutOpParam.Op.CREATE) {
@@ -168,11 +170,13 @@ public class NamenodeWebHdfsMethods {
       final DatanodeDescriptor clientNode = bm.getDatanodeManager(
           ).getDatanodeByHost(getRemoteAddress());
       if (clientNode != null) {
-        final DatanodeDescriptor[] datanodes = bm.getBlockPlacementPolicy()
+        final DatanodeStorageInfo[] storages = bm.getBlockPlacementPolicy()
             .chooseTarget(path, 1, clientNode,
-                new ArrayList<DatanodeDescriptor>(), false, null, blocksize);
-        if (datanodes.length > 0) {
-          return datanodes[0];
+                new ArrayList<DatanodeStorageInfo>(), false, null, blocksize,
+                // TODO: get storage type from the file
+                StorageType.DEFAULT);
+        if (storages.length > 0) {
+          return storages[0].getDatanodeDescriptor();
         }
       }
     } else if (op == GetOpParam.Op.OPEN

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockCommand.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockCommand.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockCommand.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockCommand.java Fri Dec 13 17:28:14 2013
@@ -24,6 +24,7 @@ import org.apache.hadoop.classification.
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 
 /****************************************************
  * A BlockCommand is an instruction to a datanode 
@@ -46,9 +47,10 @@ public class BlockCommand extends Datano
    */
   public static final long NO_ACK = Long.MAX_VALUE;
   
-  String poolId;
-  Block blocks[];
-  DatanodeInfo targets[][];
+  final String poolId;
+  final Block[] blocks;
+  final DatanodeInfo[][] targets;
+  final String[][] targetStorageIDs;
 
   /**
    * Create BlockCommand for transferring blocks to another datanode
@@ -60,21 +62,26 @@ public class BlockCommand extends Datano
     this.poolId = poolId;
     blocks = new Block[blocktargetlist.size()]; 
     targets = new DatanodeInfo[blocks.length][];
+    targetStorageIDs = new String[blocks.length][];
+
     for(int i = 0; i < blocks.length; i++) {
       BlockTargetPair p = blocktargetlist.get(i);
       blocks[i] = p.block;
-      targets[i] = p.targets;
+      targets[i] = DatanodeStorageInfo.toDatanodeInfos(p.targets);
+      targetStorageIDs[i] = DatanodeStorageInfo.toStorageIDs(p.targets);
     }
   }
 
-  private static final DatanodeInfo[][] EMPTY_TARGET = {};
+  private static final DatanodeInfo[][] EMPTY_TARGET_DATANODES = {};
+  private static final String[][] EMPTY_TARGET_STORAGEIDS = {};
 
   /**
    * Create BlockCommand for the given action
    * @param blocks blocks related to the action
    */
   public BlockCommand(int action, String poolId, Block blocks[]) {
-    this(action, poolId, blocks, EMPTY_TARGET);
+    this(action, poolId, blocks, EMPTY_TARGET_DATANODES,
+        EMPTY_TARGET_STORAGEIDS);
   }
 
   /**
@@ -82,11 +89,12 @@ public class BlockCommand extends Datano
    * @param blocks blocks related to the action
    */
   public BlockCommand(int action, String poolId, Block[] blocks,
-      DatanodeInfo[][] targets) {
+      DatanodeInfo[][] targets, String[][] targetStorageIDs) {
     super(action);
     this.poolId = poolId;
     this.blocks = blocks;
     this.targets = targets;
+    this.targetStorageIDs = targetStorageIDs;
   }
   
   public String getBlockPoolId() {
@@ -100,4 +108,8 @@ public class BlockCommand extends Datano
   public DatanodeInfo[][] getTargets() {
     return targets;
   }
+
+  public String[][] getTargetStorageIDs() {
+    return targetStorageIDs;
+  }
 }

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksWithLocations.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksWithLocations.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksWithLocations.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksWithLocations.java Fri Dec 13 17:28:14 2013
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.protocol;
 
+import java.util.Arrays;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -34,12 +36,14 @@ public class BlocksWithLocations {
   @InterfaceAudience.Private
   @InterfaceStability.Evolving
   public static class BlockWithLocations {
-    Block block;
-    String storageIDs[];
+    final Block block;
+    final String[] datanodeUuids;
+    final String[] storageIDs;
     
     /** constructor */
-    public BlockWithLocations(Block block, String[] storageIDs) {
+    public BlockWithLocations(Block block, String[] datanodeUuids, String[] storageIDs) {
       this.block = block;
+      this.datanodeUuids = datanodeUuids;
       this.storageIDs = storageIDs;
     }
     
@@ -48,10 +52,30 @@ public class BlocksWithLocations {
       return block;
     }
     
-    /** get the block's locations */
+    /** get the block's datanode locations */
+    public String[] getDatanodeUuids() {
+      return datanodeUuids;
+    }
+
+    /** get the block's storage locations */
     public String[] getStorageIDs() {
       return storageIDs;
     }
+    
+    @Override
+    public String toString() {
+      final StringBuilder b = new StringBuilder();
+      b.append(block);
+      if (datanodeUuids.length == 0) {
+        return b.append("[]").toString();
+      }
+      
+      b.append(storageIDs[0]).append('@').append(datanodeUuids[0]);
+      for(int i = 1; i < datanodeUuids.length; i++) {
+        b.append(", ").append(storageIDs[i]).append("@").append(datanodeUuids[i]);
+      }
+      return b.append("]").toString();
+    }
   }
 
   private BlockWithLocations[] blocks;

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java Fri Dec 13 17:28:14 2013
@@ -82,7 +82,7 @@ public class DatanodeRegistration extend
   public String toString() {
     return getClass().getSimpleName()
       + "(" + getIpAddr()
-      + ", storageID=" + getStorageID()
+      + ", datanodeUuid=" + getDatanodeUuid()
       + ", infoPort=" + getInfoPort()
       + ", ipcPort=" + getIpcPort()
       + ", storageInfo=" + storageInfo

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java Fri Dec 13 17:28:14 2013
@@ -17,6 +17,10 @@
  */
 package org.apache.hadoop.hdfs.server.protocol;
 
+import org.apache.hadoop.hdfs.StorageType;
+
+import java.util.UUID;
+
 /**
  * Class captures information of a storage in Datanode.
  */
@@ -29,18 +33,21 @@ public class DatanodeStorage {
   
   private final String storageID;
   private final State state;
+  private final StorageType storageType;
 
   /**
-   * Create a storage with {@link State#NORMAL}.
+   * Create a storage with {@link State#NORMAL} and {@link StorageType#DEFAULT}.
+   *
    * @param storageID
    */
   public DatanodeStorage(String storageID) {
-    this(storageID, State.NORMAL);
+    this(storageID, State.NORMAL, StorageType.DEFAULT);
   }
 
-  public DatanodeStorage(String sid, State s) {
-    storageID = sid;
-    state = s;
+  public DatanodeStorage(String sid, State s, StorageType sm) {
+    this.storageID = sid;
+    this.state = s;
+    this.storageType = sm;
   }
 
   public String getStorageID() {
@@ -50,4 +57,37 @@ public class DatanodeStorage {
   public State getState() {
     return state;
   }
+
+  public StorageType getStorageType() {
+    return storageType;
+  }
+
+  /**
+   * Generate new storage ID. The format of this string can be changed
+   * in the future without requiring that old storage IDs be updated.
+   *
+   * @return unique storage ID
+   */
+  public static String generateUuid() {
+    return "DS-" + UUID.randomUUID();
+  }
+
+  @Override
+  public boolean equals(Object other){
+    if (other == this) {
+      return true;
+    }
+
+    if ((other == null) ||
+        !(other instanceof DatanodeStorage)) {
+      return false;
+    }
+    DatanodeStorage otherStorage = (DatanodeStorage) other;
+    return otherStorage.getStorageID().compareTo(getStorageID()) == 0;
+  }
+
+  @Override
+  public int hashCode() {
+    return getStorageID().hashCode();
+  }
 }

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java Fri Dec 13 17:28:14 2013
@@ -27,6 +27,8 @@ public class StorageReport {
   private final long dfsUsed;
   private final long remaining;
   private final long blockPoolUsed;
+
+  public static final StorageReport[] EMPTY_ARRAY = {};
   
   public StorageReport(String sid, boolean failed, long capacity, long dfsUsed,
       long remaining, long bpUsed) {

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java Fri Dec 13 17:28:14 2013
@@ -126,7 +126,7 @@ class ImageLoaderCurrent implements Imag
                                       new SimpleDateFormat("yyyy-MM-dd HH:mm");
   private static int[] versions = { -16, -17, -18, -19, -20, -21, -22, -23,
       -24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39,
-      -40, -41, -42, -43, -44, -45, -46, -47, -48 };
+      -40, -41, -42, -43, -44, -45, -46, -47, -48, -49 };
   private int imageVersion = 0;
   
   private final Map<Long, Boolean> subtreeMap = new HashMap<Long, Boolean>();

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java Fri Dec 13 17:28:14 2013
@@ -276,10 +276,11 @@ public class JsonUtil {
       return null;
     }
 
+    // TODO: Fix storageID
     final Map<String, Object> m = new TreeMap<String, Object>();
     m.put("ipAddr", datanodeinfo.getIpAddr());
     m.put("hostName", datanodeinfo.getHostName());
-    m.put("storageID", datanodeinfo.getStorageID());
+    m.put("storageID", datanodeinfo.getDatanodeUuid());
     m.put("xferPort", datanodeinfo.getXferPort());
     m.put("infoPort", datanodeinfo.getInfoPort());
     m.put("infoSecurePort", datanodeinfo.getInfoSecurePort());
@@ -329,6 +330,7 @@ public class JsonUtil {
       return null;
     }
 
+    // TODO: Fix storageID
     return new DatanodeInfo(
         (String)m.get("ipAddr"),
         (String)m.get("hostName"),
@@ -412,7 +414,7 @@ public class JsonUtil {
         (Object[])m.get("cachedLocations"));
 
     final LocatedBlock locatedblock = new LocatedBlock(b, locations,
-        startOffset, isCorrupt, cachedLocations);
+        null, null, startOffset, isCorrupt, cachedLocations);
     locatedblock.setBlockToken(toBlockToken((Map<?, ?>)m.get("blockToken")));
     return locatedblock;
   }

Propchange: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/native/
------------------------------------------------------------------------------
  Merged /hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/native:r1513717-1550362
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/native:r1549906-1550771

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto Fri Dec 13 17:28:14 2013
@@ -142,6 +142,7 @@ message GetAdditionalDatanodeRequestProt
   repeated DatanodeInfoProto excludes = 4;
   required uint32 numAdditionalNodes = 5;
   required string clientName = 6;
+  repeated string existingStorageUuids = 7;
 }
 
 message GetAdditionalDatanodeResponseProto {
@@ -546,6 +547,7 @@ message UpdatePipelineRequestProto {
   required ExtendedBlockProto oldBlock = 2;
   required ExtendedBlockProto newBlock = 3;
   repeated DatanodeIDProto newNodes = 4;
+  repeated string storageIDs = 5;
 }
 
 message UpdatePipelineResponseProto { // void response

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto Fri Dec 13 17:28:14 2013
@@ -53,8 +53,9 @@ message DatanodeStorageProto {
     READ_ONLY = 1;
   }
 
-  required string storageID = 1;    // Unique identifier for the storage
+  required string storageUuid = 1;
   optional StorageState state = 2 [default = NORMAL];
+  optional StorageTypeProto storageType = 3 [default = DISK];
 }
 
 /**
@@ -106,10 +107,12 @@ message BlockCommandProto {
     INVALIDATE = 2; // Invalidate blocks
     SHUTDOWN = 3;   // Shutdown the datanode
   }
+
   required Action action = 1;
   required string blockPoolId = 2;
   repeated BlockProto blocks = 3;
   repeated DatanodeInfosProto targets = 4;
+  repeated StorageUuidsProto targetStorageUuids = 5;
 }
 
 /**
@@ -193,7 +196,7 @@ message HeartbeatRequestProto {
 }
 
 message StorageReportProto {
-  required string storageID = 1;
+  required string storageUuid = 1;
   optional bool failed = 2 [ default = false ];
   optional uint64 capacity = 3 [ default = 0 ];
   optional uint64 dfsUsed = 4 [ default = 0 ];
@@ -284,7 +287,7 @@ message ReceivedDeletedBlockInfoProto {
  * List of blocks received and deleted for a storage.
  */
 message StorageReceivedDeletedBlocksProto {
-  required string storageID = 1;
+  required string storageUuid = 1;
   repeated ReceivedDeletedBlockInfoProto blocks = 2;
 }
 

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/InterDatanodeProtocol.proto Fri Dec 13 17:28:14 2013
@@ -65,7 +65,7 @@ message UpdateReplicaUnderRecoveryReques
  * Response returns updated block information
  */
 message UpdateReplicaUnderRecoveryResponseProto {
-  required string storageID = 1; // ID of the storage that stores replica
+  optional string storageUuid = 1; // ID of the storage that stores replica
 }
 
 /**

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/QJournalProtocol.proto Fri Dec 13 17:28:14 2013
@@ -178,7 +178,7 @@ message GetEditLogManifestRequestProto {
   required JournalIdProto jid = 1;
   required uint64 sinceTxId = 2;  // Transaction ID
   // Whether or not the client will be reading from the returned streams.
-  optional bool forReading = 3 [default = true];
+  // optional bool forReading = 3 [default = true]; <obsolete, do not reuse>
   optional bool inProgressOk = 4 [default = false];
 }
 

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto Fri Dec 13 17:28:14 2013
@@ -50,7 +50,10 @@ message ExtendedBlockProto {
 message DatanodeIDProto {
   required string ipAddr = 1;    // IP address
   required string hostName = 2;  // hostname
-  required string storageID = 3; // unique storage id
+  required string datanodeUuid = 3;     // UUID assigned to the Datanode. For
+                                        // upgraded clusters this is the same
+                                        // as the original StorageID of the
+                                        // Datanode.
   required uint32 xferPort = 4;  // data streaming port
   required uint32 infoPort = 5;  // datanode http port
   required uint32 ipcPort = 6;   // ipc server port
@@ -116,6 +119,20 @@ message FsPermissionProto {
   required uint32 perm = 1;       // Actually a short - only 16bits used
 }
 
+/**
+ * Types of recognized storage media.
+ */
+enum StorageTypeProto {
+  DISK = 1;
+  SSD = 2;
+}
+
+/**
+ * A list of storage IDs. 
+ */
+message StorageUuidsProto {
+  repeated string storageUuids = 1;
+}
 
 /**
  * A LocatedBlock gives information about a block and its location.
@@ -130,6 +147,8 @@ message LocatedBlockProto {
 
   required hadoop.common.TokenProto blockToken = 5;
   repeated bool isCached = 6 [packed=true]; // if a location in locs is cached
+  repeated StorageTypeProto storageTypes = 7;
+  repeated string storageIDs = 8;
 }
 
 message DataEncryptionKeyProto {
@@ -336,7 +355,8 @@ message BlockProto {
  */
 message BlockWithLocationsProto {
   required BlockProto block = 1;   // Block
-  repeated string storageIDs = 2;  // Datanodes with replicas of the block
+  repeated string datanodeUuids = 2; // Datanodes with replicas of the block
+  repeated string storageUuids = 3;  // Storages with replicas of the block
 }
 
 /**
@@ -439,3 +459,4 @@ message SnapshotInfoProto {
   // TODO: do we need access time?
 }
 
+

Propchange: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode:r1549906-1550771
  Merged /hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode:r1513717-1550362

Propchange: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs:r1549906-1550771
  Merged /hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs:r1513717-1550362

Propchange: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/
------------------------------------------------------------------------------
  Merged /hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary:r1513717-1550362
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary:r1549906-1550771

Propchange: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/test/hdfs/
------------------------------------------------------------------------------
  Merged /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/hdfs:r1549906-1550771
  Merged /hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/hdfs:r1513717-1550362

Modified: hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java?rev=1550774&r1=1550773&r2=1550774&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java (original)
+++ hadoop/common/branches/HDFS-4685/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java Fri Dec 13 17:28:14 2013
@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.fs.Options.Rename;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.MiniDFSCluster.NameNodeInfo;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
@@ -39,6 +40,7 @@ import org.apache.hadoop.hdfs.security.t
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -46,6 +48,7 @@ import org.apache.hadoop.hdfs.server.dat
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
@@ -770,7 +773,8 @@ public class DFSTestUtil {
   }
   
   private static DatanodeID getDatanodeID(String ipAddr) {
-    return new DatanodeID(ipAddr, "localhost", "",
+    return new DatanodeID(ipAddr, "localhost",
+        UUID.randomUUID().toString(),
         DFSConfigKeys.DFS_DATANODE_DEFAULT_PORT,
         DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
         DFSConfigKeys.DFS_DATANODE_HTTPS_DEFAULT_PORT,
@@ -782,7 +786,8 @@ public class DFSTestUtil {
   }
 
   public static DatanodeID getLocalDatanodeID(int port) {
-    return new DatanodeID("127.0.0.1", "localhost", "",
+    return new DatanodeID("127.0.0.1", "localhost",
+        UUID.randomUUID().toString(),
         port, port, port, port);
   }
 
@@ -804,8 +809,9 @@ public class DFSTestUtil {
 
   public static DatanodeInfo getDatanodeInfo(String ipAddr, 
       String host, int port) {
-    return new DatanodeInfo(new DatanodeID(ipAddr, host, "",
-        port, DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
+    return new DatanodeInfo(new DatanodeID(ipAddr, host,
+        UUID.randomUUID().toString(), port,
+        DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
         DFSConfigKeys.DFS_DATANODE_HTTPS_DEFAULT_PORT,
         DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT));
   }
@@ -826,9 +832,43 @@ public class DFSTestUtil {
         rackLocation);
   }
 
+  public static DatanodeStorageInfo createDatanodeStorageInfo(
+      String storageID, String ip) {
+    return createDatanodeStorageInfo(storageID, ip, "defaultRack");
+  }
+  public static DatanodeStorageInfo[] createDatanodeStorageInfos(String[] racks) {
+    return createDatanodeStorageInfos(racks.length, racks);
+  }
+  public static DatanodeStorageInfo[] createDatanodeStorageInfos(int n, String... racks) {
+    DatanodeStorageInfo[] storages = new DatanodeStorageInfo[n];
+    for(int i = storages.length; i > 0; ) {
+      final String storageID = "s" + i;
+      final String ip = i + "." + i + "." + i + "." + i;
+      i--;
+      final String rack = i < racks.length? racks[i]: "defaultRack";
+      storages[i] = createDatanodeStorageInfo(storageID, ip, rack);
+    }
+    return storages;
+  }
+  public static DatanodeStorageInfo createDatanodeStorageInfo(
+      String storageID, String ip, String rack) {
+    final DatanodeStorage storage = new DatanodeStorage(storageID);
+    final DatanodeDescriptor dn = BlockManagerTestUtil.getDatanodeDescriptor(ip, rack, storage);
+    return BlockManagerTestUtil.newDatanodeStorageInfo(dn, storage);
+  }
+  public static DatanodeDescriptor[] toDatanodeDescriptor(
+      DatanodeStorageInfo[] storages) {
+    DatanodeDescriptor[] datanodes = new DatanodeDescriptor[storages.length];
+    for(int i = 0; i < datanodes.length; i++) {
+      datanodes[i] = storages[i].getDatanodeDescriptor();
+    }
+    return datanodes;
+  }
+
   public static DatanodeDescriptor getDatanodeDescriptor(String ipAddr,
       int port, String rackLocation) {
-    DatanodeID dnId = new DatanodeID(ipAddr, "host", "", port,
+    DatanodeID dnId = new DatanodeID(ipAddr, "host",
+        UUID.randomUUID().toString(), port,
         DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
         DFSConfigKeys.DFS_DATANODE_HTTPS_DEFAULT_PORT,
         DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT);