Browse Source

HDFS-510. Reverting incorrect commit.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hdfs/trunk@799139 13f79535-47bb-0310-9956-ffa450edef68
Konstantin Shvachko 16 years ago
parent
commit
843ac403d3

+ 0 - 3
CHANGES.txt

@@ -54,9 +54,6 @@ Trunk (unreleased changes)
 
 
     HDFS-508. Factor out BlockInfo from BlocksMap. (shv)
     HDFS-508. Factor out BlockInfo from BlocksMap. (shv)
 
 
-    HDFS-510. Rename DatanodeBlockInfo to be ReplicaInfo.
-    (Jakob Homan & Hairong Kuang via shv)
-
   BUG FIXES
   BUG FIXES
     HDFS-76. Better error message to users when commands fail because of 
     HDFS-76. Better error message to users when commands fail because of 
     lack of quota. Allow quota to be set even if the limit is lower than
     lack of quota. Allow quota to be set even if the limit is lower than

+ 3 - 3
src/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java → src/java/org/apache/hadoop/hdfs/server/datanode/DatanodeBlockInfo.java

@@ -32,19 +32,19 @@ import org.apache.hadoop.io.IOUtils;
  * This class is used by the datanode to maintain the map from a block 
  * This class is used by the datanode to maintain the map from a block 
  * to its metadata.
  * to its metadata.
  */
  */
-class ReplicaInfo {
+class DatanodeBlockInfo {
 
 
   private FSVolume volume;       // volume where the block belongs
   private FSVolume volume;       // volume where the block belongs
   private File     file;         // block file
   private File     file;         // block file
   private boolean detached;      // copy-on-write done for block
   private boolean detached;      // copy-on-write done for block
 
 
-  ReplicaInfo(FSVolume vol, File file) {
+  DatanodeBlockInfo(FSVolume vol, File file) {
     this.volume = vol;
     this.volume = vol;
     this.file = file;
     this.file = file;
     detached = false;
     detached = false;
   }
   }
 
 
-  ReplicaInfo(FSVolume vol) {
+  DatanodeBlockInfo(FSVolume vol) {
     this.volume = vol;
     this.volume = vol;
     this.file = null;
     this.file = null;
     detached = false;
     detached = false;

+ 22 - 22
src/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java

@@ -171,7 +171,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
       return Block.GRANDFATHER_GENERATION_STAMP;
       return Block.GRANDFATHER_GENERATION_STAMP;
     }
     }
 
 
-    void getVolumeMap(HashMap<Block, ReplicaInfo> volumeMap, FSVolume volume) {
+    void getVolumeMap(HashMap<Block, DatanodeBlockInfo> volumeMap, FSVolume volume) {
       if (children != null) {
       if (children != null) {
         for (int i = 0; i < children.length; i++) {
         for (int i = 0; i < children.length; i++) {
           children[i].getVolumeMap(volumeMap, volume);
           children[i].getVolumeMap(volumeMap, volume);
@@ -183,7 +183,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
         if (Block.isBlockFilename(blockFiles[i])) {
         if (Block.isBlockFilename(blockFiles[i])) {
           long genStamp = getGenerationStampFromFile(blockFiles, blockFiles[i]);
           long genStamp = getGenerationStampFromFile(blockFiles, blockFiles[i]);
           volumeMap.put(new Block(blockFiles[i], blockFiles[i].length(), genStamp), 
           volumeMap.put(new Block(blockFiles[i], blockFiles[i].length(), genStamp), 
-                        new ReplicaInfo(volume, blockFiles[i]));
+                        new DatanodeBlockInfo(volume, blockFiles[i]));
         }
         }
       }
       }
     }
     }
@@ -403,7 +403,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
       DiskChecker.checkDir(tmpDir);
       DiskChecker.checkDir(tmpDir);
     }
     }
       
       
-    void getVolumeMap(HashMap<Block, ReplicaInfo> volumeMap) {
+    void getVolumeMap(HashMap<Block, DatanodeBlockInfo> volumeMap) {
       dataDir.getVolumeMap(volumeMap, this);
       dataDir.getVolumeMap(volumeMap, this);
     }
     }
       
       
@@ -496,7 +496,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
       return remaining;
       return remaining;
     }
     }
       
       
-    synchronized void getVolumeMap(HashMap<Block, ReplicaInfo> volumeMap) {
+    synchronized void getVolumeMap(HashMap<Block, DatanodeBlockInfo> volumeMap) {
       for (int idx = 0; idx < volumes.length; idx++) {
       for (int idx = 0; idx < volumes.length; idx++) {
         volumes[idx].getVolumeMap(volumeMap);
         volumes[idx].getVolumeMap(volumeMap);
       }
       }
@@ -653,7 +653,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
   FSVolumeSet volumes;
   FSVolumeSet volumes;
   private HashMap<Block,ActiveFile> ongoingCreates = new HashMap<Block,ActiveFile>();
   private HashMap<Block,ActiveFile> ongoingCreates = new HashMap<Block,ActiveFile>();
   private int maxBlocksPerDir = 0;
   private int maxBlocksPerDir = 0;
-  HashMap<Block,ReplicaInfo> volumeMap = null;
+  HashMap<Block,DatanodeBlockInfo> volumeMap = null;
   static  Random random = new Random();
   static  Random random = new Random();
 
 
   // Used for synchronizing access to usage stats
   // Used for synchronizing access to usage stats
@@ -669,7 +669,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
       volArray[idx] = new FSVolume(storage.getStorageDir(idx).getCurrentDir(), conf);
       volArray[idx] = new FSVolume(storage.getStorageDir(idx).getCurrentDir(), conf);
     }
     }
     volumes = new FSVolumeSet(volArray);
     volumes = new FSVolumeSet(volArray);
-    volumeMap = new HashMap<Block, ReplicaInfo>();
+    volumeMap = new HashMap<Block, DatanodeBlockInfo>();
     volumes.getVolumeMap(volumeMap);
     volumes.getVolumeMap(volumeMap);
     registerMBean(storage.getStorageID());
     registerMBean(storage.getStorageID());
   }
   }
@@ -742,7 +742,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
   public synchronized BlockInputStreams getTmpInputStreams(Block b, 
   public synchronized BlockInputStreams getTmpInputStreams(Block b, 
                           long blkOffset, long ckoff) throws IOException {
                           long blkOffset, long ckoff) throws IOException {
 
 
-    ReplicaInfo info = volumeMap.get(b);
+    DatanodeBlockInfo info = volumeMap.get(b);
     if (info == null) {
     if (info == null) {
       throw new IOException("Block " + b + " does not exist in volumeMap.");
       throw new IOException("Block " + b + " does not exist in volumeMap.");
     }
     }
@@ -777,7 +777,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
    * @return - true if the specified block was detached
    * @return - true if the specified block was detached
    */
    */
   public boolean detachBlock(Block block, int numLinks) throws IOException {
   public boolean detachBlock(Block block, int numLinks) throws IOException {
-    ReplicaInfo info = null;
+    DatanodeBlockInfo info = null;
 
 
     synchronized (this) {
     synchronized (this) {
       info = volumeMap.get(block);
       info = volumeMap.get(block);
@@ -1006,12 +1006,12 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
         v = volumes.getNextVolume(blockSize);
         v = volumes.getNextVolume(blockSize);
         // create temporary file to hold block in the designated volume
         // create temporary file to hold block in the designated volume
         f = createTmpFile(v, b);
         f = createTmpFile(v, b);
-        volumeMap.put(b, new ReplicaInfo(v));
+        volumeMap.put(b, new DatanodeBlockInfo(v));
       } else if (f != null) {
       } else if (f != null) {
         DataNode.LOG.info("Reopen already-open Block for append " + b);
         DataNode.LOG.info("Reopen already-open Block for append " + b);
         // create or reuse temporary file to hold block in the designated volume
         // create or reuse temporary file to hold block in the designated volume
         v = volumeMap.get(b).getVolume();
         v = volumeMap.get(b).getVolume();
-        volumeMap.put(b, new ReplicaInfo(v));
+        volumeMap.put(b, new DatanodeBlockInfo(v));
       } else {
       } else {
         // reopening block for appending to it.
         // reopening block for appending to it.
         DataNode.LOG.info("Reopen Block for append " + b);
         DataNode.LOG.info("Reopen Block for append " + b);
@@ -1042,7 +1042,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
                                   " to tmp dir " + f);
                                   " to tmp dir " + f);
           }
           }
         }
         }
-        volumeMap.put(b, new ReplicaInfo(v));
+        volumeMap.put(b, new DatanodeBlockInfo(v));
       }
       }
       if (f == null) {
       if (f == null) {
         DataNode.LOG.warn("Block " + b + " reopen failed " +
         DataNode.LOG.warn("Block " + b + " reopen failed " +
@@ -1147,7 +1147,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
         
         
     File dest = null;
     File dest = null;
     dest = v.addBlock(b, f);
     dest = v.addBlock(b, f);
-    volumeMap.put(b, new ReplicaInfo(v, dest));
+    volumeMap.put(b, new DatanodeBlockInfo(v, dest));
     ongoingCreates.remove(b);
     ongoingCreates.remove(b);
   }
   }
 
 
@@ -1248,7 +1248,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
 
 
   /** {@inheritDoc} */
   /** {@inheritDoc} */
   public void validateBlockMetadata(Block b) throws IOException {
   public void validateBlockMetadata(Block b) throws IOException {
-    ReplicaInfo info = volumeMap.get(b);
+    DatanodeBlockInfo info = volumeMap.get(b);
     if (info == null) {
     if (info == null) {
       throw new IOException("Block " + b + " does not exist in volumeMap.");
       throw new IOException("Block " + b + " does not exist in volumeMap.");
     }
     }
@@ -1306,7 +1306,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
       FSVolume v;
       FSVolume v;
       synchronized (this) {
       synchronized (this) {
         f = getFile(invalidBlks[i]);
         f = getFile(invalidBlks[i]);
-        ReplicaInfo dinfo = volumeMap.get(invalidBlks[i]);
+        DatanodeBlockInfo dinfo = volumeMap.get(invalidBlks[i]);
         if (dinfo == null) {
         if (dinfo == null) {
           DataNode.LOG.warn("Unexpected error trying to delete block "
           DataNode.LOG.warn("Unexpected error trying to delete block "
                            + invalidBlks[i] + 
                            + invalidBlks[i] + 
@@ -1369,7 +1369,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
    * Turn the block identifier into a filename.
    * Turn the block identifier into a filename.
    */
    */
   public synchronized File getFile(Block b) {
   public synchronized File getFile(Block b) {
-    ReplicaInfo info = volumeMap.get(b);
+    DatanodeBlockInfo info = volumeMap.get(b);
     if (info != null) {
     if (info != null) {
       return info.getFile();
       return info.getFile();
     }
     }
@@ -1448,8 +1448,8 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
    * generation stamp</li>
    * generation stamp</li>
    * <li>If the block length in memory does not match the actual block file length
    * <li>If the block length in memory does not match the actual block file length
    * then mark the block as corrupt and update the block length in memory</li>
    * then mark the block as corrupt and update the block length in memory</li>
-   * <li>If the file in {@link ReplicaInfo} does not match the file on
-   * the disk, update {@link ReplicaInfo} with the correct file</li>
+   * <li>If the file in {@link DatanodeBlockInfo} does not match the file on
+   * the disk, update {@link DatanodeBlockInfo} with the correct file</li>
    * </ul>
    * </ul>
    *
    *
    * @param blockId Block that differs
    * @param blockId Block that differs
@@ -1472,7 +1472,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
           Block.getGenerationStamp(diskMetaFile.getName()) :
           Block.getGenerationStamp(diskMetaFile.getName()) :
             Block.GRANDFATHER_GENERATION_STAMP;
             Block.GRANDFATHER_GENERATION_STAMP;
 
 
-      ReplicaInfo memBlockInfo = volumeMap.get(block);
+      DatanodeBlockInfo memBlockInfo = volumeMap.get(block);
       if (diskFile == null || !diskFile.exists()) {
       if (diskFile == null || !diskFile.exists()) {
         if (memBlockInfo == null) {
         if (memBlockInfo == null) {
           // Block file does not exist and block does not exist in memory
           // Block file does not exist and block does not exist in memory
@@ -1507,7 +1507,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
        */
        */
       if (memBlockInfo == null) {
       if (memBlockInfo == null) {
         // Block is missing in memory - add the block to volumeMap
         // Block is missing in memory - add the block to volumeMap
-        ReplicaInfo diskBlockInfo = new ReplicaInfo(vol, diskFile);
+        DatanodeBlockInfo diskBlockInfo = new DatanodeBlockInfo(vol, diskFile);
         Block diskBlock = new Block(diskFile, diskFile.length(), diskGS);
         Block diskBlock = new Block(diskFile, diskFile.length(), diskGS);
         volumeMap.put(diskBlock, diskBlockInfo);
         volumeMap.put(diskBlock, diskBlockInfo);
         if (datanode.blockScanner != null) {
         if (datanode.blockScanner != null) {
@@ -1540,7 +1540,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
             + memFile.getAbsolutePath()
             + memFile.getAbsolutePath()
             + " does not exist. Updating it to the file found during scan "
             + " does not exist. Updating it to the file found during scan "
             + diskFile.getAbsolutePath());
             + diskFile.getAbsolutePath());
-        ReplicaInfo info = volumeMap.remove(memBlock);
+        DatanodeBlockInfo info = volumeMap.remove(memBlock);
         info.setFile(diskFile);
         info.setFile(diskFile);
         memFile = diskFile;
         memFile = diskFile;
 
 
@@ -1571,7 +1571,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
           DataNode.LOG.warn("Updating generation stamp for block " + blockId
           DataNode.LOG.warn("Updating generation stamp for block " + blockId
               + " from " + memBlock.getGenerationStamp() + " to " + gs);
               + " from " + memBlock.getGenerationStamp() + " to " + gs);
 
 
-          ReplicaInfo info = volumeMap.remove(memBlock);
+          DatanodeBlockInfo info = volumeMap.remove(memBlock);
           memBlock.setGenerationStamp(gs);
           memBlock.setGenerationStamp(gs);
           volumeMap.put(memBlock, info);
           volumeMap.put(memBlock, info);
         }
         }
@@ -1583,7 +1583,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
         corruptBlock = new Block(memBlock);
         corruptBlock = new Block(memBlock);
         DataNode.LOG.warn("Updating size of block " + blockId + " from "
         DataNode.LOG.warn("Updating size of block " + blockId + " from "
             + memBlock.getNumBytes() + " to " + memFile.length());
             + memBlock.getNumBytes() + " to " + memFile.length());
-        ReplicaInfo info = volumeMap.remove(memBlock);
+        DatanodeBlockInfo info = volumeMap.remove(memBlock);
         memBlock.setNumBytes(memFile.length());
         memBlock.setNumBytes(memFile.length());
         volumeMap.put(memBlock, info);
         volumeMap.put(memBlock, info);
       }
       }

+ 5 - 5
src/test/hdfs/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java

@@ -67,7 +67,7 @@ public class TestDirectoryScanner extends TestCase {
   /** Truncate a block file */
   /** Truncate a block file */
   private long truncateBlockFile() throws IOException {
   private long truncateBlockFile() throws IOException {
     synchronized (fds) {
     synchronized (fds) {
-      for (Entry<Block, ReplicaInfo> entry : fds.volumeMap.entrySet()) {
+      for (Entry<Block, DatanodeBlockInfo> entry : fds.volumeMap.entrySet()) {
         Block b = entry.getKey();
         Block b = entry.getKey();
         File f = entry.getValue().getFile();
         File f = entry.getValue().getFile();
         File mf = FSDataset.getMetaFile(f, b);
         File mf = FSDataset.getMetaFile(f, b);
@@ -87,7 +87,7 @@ public class TestDirectoryScanner extends TestCase {
   /** Delete a block file */
   /** Delete a block file */
   private long deleteBlockFile() {
   private long deleteBlockFile() {
     synchronized(fds) {
     synchronized(fds) {
-      for (Entry<Block, ReplicaInfo> entry : fds.volumeMap.entrySet()) {
+      for (Entry<Block, DatanodeBlockInfo> entry : fds.volumeMap.entrySet()) {
         Block b = entry.getKey();
         Block b = entry.getKey();
         File f = entry.getValue().getFile();
         File f = entry.getValue().getFile();
         File mf = FSDataset.getMetaFile(f, b);
         File mf = FSDataset.getMetaFile(f, b);
@@ -104,7 +104,7 @@ public class TestDirectoryScanner extends TestCase {
   /** Delete block meta file */
   /** Delete block meta file */
   private long deleteMetaFile() {
   private long deleteMetaFile() {
     synchronized(fds) {
     synchronized(fds) {
-      for (Entry<Block, ReplicaInfo> entry : fds.volumeMap.entrySet()) {
+      for (Entry<Block, DatanodeBlockInfo> entry : fds.volumeMap.entrySet()) {
         Block b = entry.getKey();
         Block b = entry.getKey();
         String blkfile = entry.getValue().getFile().getAbsolutePath();
         String blkfile = entry.getValue().getFile().getAbsolutePath();
         long genStamp = b.getGenerationStamp();
         long genStamp = b.getGenerationStamp();
@@ -126,7 +126,7 @@ public class TestDirectoryScanner extends TestCase {
     while (true) {
     while (true) {
       id = rand.nextLong();
       id = rand.nextLong();
       Block b = new Block(id);
       Block b = new Block(id);
-      ReplicaInfo info = null;
+      DatanodeBlockInfo info = null;
       synchronized(fds) {
       synchronized(fds) {
         info = fds.volumeMap.get(b);
         info = fds.volumeMap.get(b);
       }
       }
@@ -326,7 +326,7 @@ public class TestDirectoryScanner extends TestCase {
   private void verifyAddition(long blockId, long genStamp, long size) {
   private void verifyAddition(long blockId, long genStamp, long size) {
     Block memBlock = fds.getBlockKey(blockId);
     Block memBlock = fds.getBlockKey(blockId);
     assertNotNull(memBlock);
     assertNotNull(memBlock);
-    ReplicaInfo blockInfo;
+    DatanodeBlockInfo blockInfo;
     synchronized(fds) {
     synchronized(fds) {
       blockInfo = fds.volumeMap.get(memBlock);
       blockInfo = fds.volumeMap.get(memBlock);
     }
     }