Kaynağa Gözat

svn merge -c 1238969 from trunk for HDFS-2864.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1238972 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 13 yıl önce
ebeveyn
işleme
5d9745fde4
14 değiştirilmiş dosya ile 65 ekleme ve 61 silme
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
  3. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
  4. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java
  5. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
  6. 18 20
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
  7. 6 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java
  8. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
  9. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
  10. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
  11. 5 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
  12. 16 22
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
  13. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
  14. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -118,6 +118,9 @@ Release 0.23.1 - UNRELEASED
     HDFS-2826. Add test case for HDFS-1476 (safemode can initialize
     replication queues before exiting) (todd)
 
+    HDFS-2864. Remove some redundant methods and the constant METADATA_VERSION
+    from FSDataset.  (szetszwo)
+
   BUG FIXES
 
     HDFS-2541. For a sufficiently large value of blocks, the DN Scanner 

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java

@@ -31,13 +31,12 @@ import java.util.Map;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
-import org.apache.hadoop.hdfs.server.datanode.FSDataset;
 import org.apache.hadoop.hdfs.util.DirectBufferPool;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.security.token.Token;
@@ -183,7 +182,7 @@ class BlockReaderLocal implements BlockReader {
         BlockMetadataHeader header = BlockMetadataHeader
             .readHeader(new DataInputStream(checksumIn));
         short version = header.getVersion();
-        if (version != FSDataset.METADATA_VERSION) {
+        if (version != BlockMetadataHeader.VERSION) {
           LOG.warn("Wrong version (" + version + ") for metadata file for "
               + blk + " ignoring ...");
         }

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

@@ -42,7 +42,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceStability.Evolving
 public class BlockMetadataHeader {
 
-  static final short METADATA_VERSION = FSDataset.METADATA_VERSION;
+  public static final short VERSION = 1;
   
   /**
    * Header includes everything except the checksum(s) themselves.
@@ -138,7 +138,7 @@ public class BlockMetadataHeader {
    */
   static void writeHeader(DataOutputStream out, DataChecksum checksum)
                          throws IOException {
-    writeHeader(out, new BlockMetadataHeader(METADATA_VERSION, checksum));
+    writeHeader(out, new BlockMetadataHeader(VERSION, checksum));
   }
 
   /**

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

@@ -425,9 +425,8 @@ class BlockPoolSliceScanner {
         updateScanStatus(block.getLocalBlock(), ScanType.VERIFICATION_SCAN, false);
 
         // If the block does not exists anymore, then its not an error
-        if ( dataset.getFile(block.getBlockPoolId(), block.getLocalBlock()) == null ) {
-          LOG.info("Verification failed for " + block + ". Its ok since " +
-          "it not in datanode dataset anymore.");
+        if (!dataset.contains(block)) {
+          LOG.info(block + " is no longer in the dataset.");
           deleteBlock(block.getLocalBlock());
           return;
         }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java

@@ -226,7 +226,7 @@ class BlockSender implements java.io.Closeable {
         // read and handle the common header here. For now just a version
         BlockMetadataHeader header = BlockMetadataHeader.readHeader(checksumIn);
         short version = header.getVersion();
-        if (version != FSDataset.METADATA_VERSION) {
+        if (version != BlockMetadataHeader.VERSION) {
           LOG.warn("Wrong version (" + version + ") for metadata file for "
               + block + " ignoring ...");
         }

+ 18 - 20
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java

@@ -471,7 +471,7 @@ public class FSDataset implements FSDatasetInterface {
         // read and handle the common header here. For now just a version
         BlockMetadataHeader header = BlockMetadataHeader.readHeader(checksumIn);
         short version = header.getVersion();
-        if (version != FSDataset.METADATA_VERSION) {
+        if (version != BlockMetadataHeader.VERSION) {
           DataNode.LOG.warn("Wrong version (" + version + ") for metadata file "
               + metaFile + " ignoring ...");
         }
@@ -946,8 +946,7 @@ public class FSDataset implements FSDatasetInterface {
   //////////////////////////////////////////////////////
 
   //Find better place?
-  public static final String METADATA_EXTENSION = ".meta";
-  public static final short METADATA_VERSION = 1;
+  static final String METADATA_EXTENSION = ".meta";
   static final String UNLINK_BLOCK_SUFFIX = ".unlinked";
 
   private static boolean isUnlinkTmpFile(File f) {
@@ -1032,15 +1031,10 @@ public class FSDataset implements FSDatasetInterface {
     }
   }
 
-  /** Return the block file for the given ID */ 
-  public File findBlockFile(String bpid, long blockId) {
-    return getFile(bpid, blockId);
-  }
-
   @Override // FSDatasetInterface
   public synchronized Block getStoredBlock(String bpid, long blkid)
       throws IOException {
-    File blockfile = findBlockFile(bpid, blkid);
+    File blockfile = getFile(bpid, blkid);
     if (blockfile == null) {
       return null;
     }
@@ -1260,8 +1254,7 @@ public class FSDataset implements FSDatasetInterface {
   /**
    * Get File name for a given block.
    */
-  public File getBlockFile(String bpid, Block b)
-      throws IOException {
+  File getBlockFile(String bpid, Block b) throws IOException {
     File f = validateBlockFile(bpid, b);
     if(f == null) {
       if (InterDatanodeProtocol.LOG.isDebugEnabled()) {
@@ -1292,7 +1285,10 @@ public class FSDataset implements FSDatasetInterface {
    */
   private File getBlockFileNoExistsCheck(ExtendedBlock b)
       throws IOException {
-    File f = getFile(b.getBlockPoolId(), b.getLocalBlock());
+    final File f;
+    synchronized(this) {
+      f = getFile(b.getBlockPoolId(), b.getLocalBlock().getBlockId());
+    }
     if (f == null) {
       throw new IOException("Block " + b + " is not valid");
     }
@@ -2022,7 +2018,10 @@ public class FSDataset implements FSDatasetInterface {
    */
   File validateBlockFile(String bpid, Block b) throws IOException {
     //Should we check for metadata file too?
-    File f = getFile(bpid, b);
+    final File f;
+    synchronized(this) {
+      f = getFile(bpid, b.getBlockId());
+    }
     
     if(f != null ) {
       if(f.exists())
@@ -2072,7 +2071,7 @@ public class FSDataset implements FSDatasetInterface {
       File f = null;
       FSVolume v;
       synchronized (this) {
-        f = getFile(bpid, invalidBlks[i]);
+        f = getFile(bpid, invalidBlks[i].getBlockId());
         ReplicaInfo dinfo = volumeMap.get(bpid, invalidBlks[i]);
         if (dinfo == null || 
             dinfo.getGenerationStamp() != invalidBlks[i].getGenerationStamp()) {
@@ -2127,11 +2126,10 @@ public class FSDataset implements FSDatasetInterface {
     }
   }
 
-  /**
-   * Turn the block identifier into a filename; ignore generation stamp!!!
-   */
-  public synchronized File getFile(String bpid, Block b) {
-    return getFile(bpid, b.getBlockId());
+  @Override // {@link FSDatasetInterface}
+  public synchronized boolean contains(final ExtendedBlock block) {
+    final long blockId = block.getLocalBlock().getBlockId();
+    return getFile(block.getBlockPoolId(), blockId) != null;
   }
 
   /**
@@ -2140,7 +2138,7 @@ public class FSDataset implements FSDatasetInterface {
    * @param blockId a block's id
    * @return on disk data file path; null if the replica does not exist
    */
-  private File getFile(String bpid, long blockId) {
+  File getFile(final String bpid, final long blockId) {
     ReplicaInfo info = volumeMap.get(bpid, blockId);
     if (info != null) {
       return info.getBlockFile();

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDatasetInterface.java

@@ -19,7 +19,6 @@ package org.apache.hadoop.hdfs.server.datanode;
 
 
 import java.io.Closeable;
-import java.io.File;
 import java.io.FilterInputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -27,13 +26,13 @@ import java.io.OutputStream;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
-import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
-import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
+import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
+import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
@@ -303,6 +302,9 @@ public interface FSDatasetInterface extends FSDatasetMBean {
    */
   public BlockListAsLongs getBlockReport(String bpid);
 
+  /** Does the dataset contain the block? */
+  public boolean contains(ExtendedBlock block);
+
   /**
    * Is the block valid?
    * @param b

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java

@@ -45,6 +45,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.FSDataset;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.io.IOUtils;
@@ -1165,7 +1166,7 @@ public class TestDFSShell extends TestCase {
     for(int i = 0; i < blocks.length; i++) {
       FSDataset ds = (FSDataset)datanodes.get(i).getFSDataset();
       for(Block b : blocks[i]) {
-        files.add(ds.getBlockFile(poolId, b));
+        files.add(DataNodeTestUtils.getBlockFile(ds, poolId, b.getBlockId()));
       }        
     }
     return files;

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java

@@ -37,6 +37,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.FSDataset;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 
@@ -139,7 +140,8 @@ public class TestFileAppend{
       //
       for (int i = 0; i < blocks.size(); i = i + 2) {
         ExtendedBlock b = blocks.get(i).getBlock();
-        File f = dataset.getFile(b.getBlockPoolId(), b.getLocalBlock());
+        final File f = DataNodeTestUtils.getBlockFile(dataset,
+            b.getBlockPoolId(), b.getLocalBlock().getBlockId());
         File link = new File(f.toString() + ".link");
         System.out.println("Creating hardlink for File " + f + " to " + link);
         HardLink.createHardLink(f, link);

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java

@@ -44,6 +44,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.FSDataset;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -831,7 +832,8 @@ public class TestFileCreation extends junit.framework.TestCase {
         FSDataset dataset = (FSDataset)datanode.data;
         ExtendedBlock blk = locatedblock.getBlock();
         Block b = dataset.getStoredBlock(blk.getBlockPoolId(), blk.getBlockId());
-        File blockfile = dataset.findBlockFile(blk.getBlockPoolId(), b.getBlockId());
+        final File blockfile = DataNodeTestUtils.getBlockFile(dataset,
+            blk.getBlockPoolId(), b.getBlockId());
         System.out.println("blockfile=" + blockfile);
         if (blockfile != null) {
           BufferedReader in = new BufferedReader(new FileReader(blockfile));

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java

@@ -19,6 +19,7 @@
 
 package org.apache.hadoop.hdfs.server.datanode;
 
+import java.io.File;
 import java.io.IOException;
 
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -37,5 +38,8 @@ public class DataNodeTestUtils {
   getDNRegistrationForBP(DataNode dn, String bpid) throws IOException {
     return dn.getDNRegistrationForBP(bpid);
   }
-  
+
+  public static File getBlockFile(FSDataset fsdataset, String bpid, long bid) {
+    return fsdataset.getFile(bpid, bid);
+  }
 }

+ 16 - 22
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java

@@ -81,8 +81,8 @@ public class SimulatedFSDataset  implements FSDatasetInterface, Configurable{
                               CHECKSUM_NULL, 16*1024 );
     byte[] nullCrcHeader = checksum.getHeader();
     nullCrcFileData =  new byte[2 + nullCrcHeader.length];
-    nullCrcFileData[0] = (byte) ((FSDataset.METADATA_VERSION >>> 8) & 0xff);
-    nullCrcFileData[1] = (byte) (FSDataset.METADATA_VERSION & 0xff);
+    nullCrcFileData[0] = (byte) ((BlockMetadataHeader.VERSION >>> 8) & 0xff);
+    nullCrcFileData[1] = (byte) (BlockMetadataHeader.VERSION & 0xff);
     for (int i = 0; i < nullCrcHeader.length; i++) {
       nullCrcFileData[i+2] = nullCrcHeader[i];
     }
@@ -390,9 +390,7 @@ public class SimulatedFSDataset  implements FSDatasetInterface, Configurable{
       Iterable<Block> injectBlocks) throws IOException {
     ExtendedBlock blk = new ExtendedBlock();
     if (injectBlocks != null) {
-      int numInjectedBlocks = 0;
       for (Block b: injectBlocks) { // if any blocks in list is bad, reject list
-        numInjectedBlocks++;
         if (b == null) {
           throw new NullPointerException("Null blocks in block list");
         }
@@ -555,31 +553,27 @@ public class SimulatedFSDataset  implements FSDatasetInterface, Configurable{
     }
   }
 
+  private BInfo getBInfo(final ExtendedBlock b) {
+    final Map<Block, BInfo> map = blockMap.get(b.getBlockPoolId());
+    return map == null? null: map.get(b.getLocalBlock());
+  }
+
+  @Override // {@link FSDatasetInterface}
+  public boolean contains(ExtendedBlock block) {
+    return getBInfo(block) != null;
+  }
+
   @Override // FSDatasetInterface
   public synchronized boolean isValidBlock(ExtendedBlock b) {
-    final Map<Block, BInfo> map = blockMap.get(b.getBlockPoolId());
-    if (map == null) {
-      return false;
-    }
-    BInfo binfo = map.get(b.getLocalBlock());
-    if (binfo == null) {
-      return false;
-    }
-    return binfo.isFinalized();
+    final BInfo binfo = getBInfo(b);
+    return binfo != null && binfo.isFinalized();
   }
 
   /* check if a block is created but not finalized */
   @Override
   public synchronized boolean isValidRbw(ExtendedBlock b) {
-    final Map<Block, BInfo> map = blockMap.get(b.getBlockPoolId());
-    if (map == null) {
-      return false;
-    }
-    BInfo binfo = map.get(b.getLocalBlock());
-    if (binfo == null) {
-      return false;
-    }
-    return !binfo.isFinalized();  
+    final BInfo binfo = getBInfo(b);
+    return binfo != null && !binfo.isFinalized();  
   }
 
   @Override

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

@@ -352,7 +352,7 @@ public class TestDirectoryScanner extends TestCase {
 
     // Added block has the same file as the one created by the test
     File file = new File(getBlockFile(blockId));
-    assertEquals(file.getName(), fds.findBlockFile(bpid, blockId).getName());
+    assertEquals(file.getName(), fds.getFile(bpid, blockId).getName());
 
     // Generation stamp is same as that of created file
     assertEquals(genStamp, replicainfo.getGenerationStamp());

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestSimulatedFSDataset.java

@@ -101,7 +101,7 @@ public class TestSimulatedFSDataset extends TestCase {
     InputStream metaInput = fsdataset.getMetaDataInputStream(b);
     DataInputStream metaDataInput = new DataInputStream(metaInput);
     short version = metaDataInput.readShort();
-    assertEquals(FSDataset.METADATA_VERSION, version);
+    assertEquals(BlockMetadataHeader.VERSION, version);
     DataChecksum checksum = DataChecksum.newDataChecksum(metaDataInput);
     assertEquals(DataChecksum.CHECKSUM_NULL, checksum.getChecksumType());
     assertEquals(0, checksum.getChecksumSize());