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 sz...@apache.org on 2012/02/03 08:26:03 UTC

svn commit: r1240020 - in /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/server/datanode/ src/test/java/org/apache/hadoop/hdfs/ src/test/java/org/apache/hadoop/hdfs/server/datanode/

Author: szetszwo
Date: Fri Feb  3 07:26:02 2012
New Revision: 1240020

URL: http://svn.apache.org/viewvc?rev=1240020&view=rev
Log:
HDFS-2879. Change FSDataset to package private.

Added:
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeUtil.java
Modified:
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1240020&r1=1240019&r2=1240020&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Fri Feb  3 07:26:02 2012
@@ -315,6 +315,8 @@ Release 0.23.1 - UNRELEASED
     HDFS-2864. Remove some redundant methods and the constant METADATA_VERSION
     from FSDataset.  (szetszwo)
 
+    HDFS-2879. Change FSDataset to package private.  (szetszwo)
+
   BUG FIXES
 
     HDFS-2541. For a sufficiently large value of blocks, the DN Scanner 

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java?rev=1240020&r1=1240019&r2=1240020&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java Fri Feb  3 07:26:02 2012
@@ -222,7 +222,7 @@ class BlockReceiver implements Closeable
       cleanupBlock();
       
       // check if there is a disk error
-      IOException cause = FSDataset.getCauseIfDiskError(ioe);
+      IOException cause = DatanodeUtil.getCauseIfDiskError(ioe);
       DataNode.LOG.warn("IOException in BlockReceiver constructor. Cause is ",
           cause);
       

Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeUtil.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeUtil.java?rev=1240020&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeUtil.java (added)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DatanodeUtil.java Fri Feb  3 07:26:02 2012
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.Block;
+
+/** Provide utility methods for Datanode. */
+@InterfaceAudience.Private
+class DatanodeUtil {
+  private final static String DISK_ERROR = "Possible disk error on file creation: ";
+
+  /** Get the cause of an I/O exception if caused by a possible disk error
+   * @param ioe an I/O exception
+   * @return cause if the I/O exception is caused by a possible disk error;
+   *         null otherwise.
+   */ 
+  static IOException getCauseIfDiskError(IOException ioe) {
+    if (ioe.getMessage()!=null && ioe.getMessage().startsWith(DISK_ERROR)) {
+      return (IOException)ioe.getCause();
+    } else {
+      return null;
+    }
+  }
+
+  /**
+   * Create a new file.
+   * @throws IOException 
+   * if the file already exists or if the file cannot be created.
+   */
+  static File createTmpFile(Block b, File f) throws IOException {
+    if (f.exists()) {
+      throw new IOException("Unexpected problem in creating temporary file for "
+          + b + ".  File " + f + " should not be present, but is.");
+    }
+    // Create the zero-length temp file
+    final boolean fileCreated;
+    try {
+      fileCreated = f.createNewFile();
+    } catch (IOException ioe) {
+      throw (IOException)new IOException(DISK_ERROR + f).initCause(ioe);
+    }
+    if (!fileCreated) {
+      throw new IOException("Unexpected problem in creating temporary file for "
+          + b + ".  File " + f + " should be creatable, but is already present.");
+    }
+    return f;
+  }
+}

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java?rev=1240020&r1=1240019&r2=1240020&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FSDataset.java Fri Feb  3 07:26:02 2012
@@ -75,7 +75,7 @@ import org.apache.hadoop.util.Reflection
  *
  ***************************************************/
 @InterfaceAudience.Private
-public class FSDataset implements FSDatasetInterface {
+class FSDataset implements FSDatasetInterface {
 
   /**
    * A node type that can be built into a tree reflecting the
@@ -373,7 +373,7 @@ public class FSDataset implements FSData
      */
     File createTmpFile(Block b) throws IOException {
       File f = new File(tmpDir, b.getBlockName());
-      return FSDataset.createTmpFile(b, f);
+      return DatanodeUtil.createTmpFile(b, f);
     }
 
     /**
@@ -382,7 +382,7 @@ public class FSDataset implements FSData
      */
     File createRbwFile(Block b) throws IOException {
       File f = new File(rbwDir, b.getBlockName());
-      return FSDataset.createTmpFile(b, f);
+      return DatanodeUtil.createTmpFile(b, f);
     }
 
     File addBlock(Block b, File f) throws IOException {
@@ -536,15 +536,15 @@ public class FSDataset implements FSData
     }
 
     /** Return storage directory corresponding to the volume */
-    public File getDir() {
+    File getDir() {
       return currentDir.getParentFile();
     }
     
-    public File getCurrentDir() {
+    File getCurrentDir() {
       return currentDir;
     }
     
-    public File getRbwDir(String bpid) throws IOException {
+    File getRbwDir(String bpid) throws IOException {
       BlockPoolSlice bp = getBlockPoolSlice(bpid);
       return bp.getRbwDir();
     }
@@ -1085,26 +1085,6 @@ public class FSDataset implements FSData
     return new MetaDataInputStream(new FileInputStream(checksumFile),
                                                     checksumFile.length());
   }
-
-  static File createTmpFile(Block b, File f) throws IOException {
-    if (f.exists()) {
-      throw new IOException("Unexpected problem in creating temporary file for "+
-                            b + ".  File " + f + " should not be present, but is.");
-    }
-    // Create the zero-length temp file
-    //
-    boolean fileCreated = false;
-    try {
-      fileCreated = f.createNewFile();
-    } catch (IOException ioe) {
-      throw (IOException)new IOException(DISK_ERROR +f).initCause(ioe);
-    }
-    if (!fileCreated) {
-      throw new IOException("Unexpected problem in creating temporary file for "+
-                            b + ".  File " + f + " should be creatable, but is already present.");
-    }
-    return f;
-  }
     
   private final DataNode datanode;
   final FSVolumeSet volumes;
@@ -1246,7 +1226,7 @@ public class FSDataset implements FSData
   /**
    * Get File name for a given block.
    */
-  public File getBlockFile(ExtendedBlock b) throws IOException {
+  private File getBlockFile(ExtendedBlock b) throws IOException {
     return getBlockFile(b.getBlockPoolId(), b.getLocalBlock());
   }
   
@@ -1320,7 +1300,7 @@ public class FSDataset implements FSData
    * @throws ReplicaNotFoundException if no entry is in the map or 
    *                        there is a generation stamp mismatch
    */
-  private ReplicaInfo getReplicaInfo(ExtendedBlock b)
+  ReplicaInfo getReplicaInfo(ExtendedBlock b)
       throws ReplicaNotFoundException {
     ReplicaInfo info = volumeMap.get(b.getBlockPoolId(), b.getLocalBlock());
     if (info == null) {
@@ -1456,19 +1436,6 @@ public class FSDataset implements FSData
     }
   }
 
-  private final static String DISK_ERROR = "Possible disk error on file creation: ";
-  /** Get the cause of an I/O exception if caused by a possible disk error
-   * @param ioe an I/O exception
-   * @return cause if the I/O exception is caused by a possible disk error;
-   *         null otherwise.
-   */ 
-  static IOException getCauseIfDiskError(IOException ioe) {
-    if (ioe.getMessage()!=null && ioe.getMessage().startsWith(DISK_ERROR)) {
-      return (IOException)ioe.getCause();
-    } else {
-      return null;
-    }
-  }
 
   @Override  // FSDatasetInterface
   public synchronized ReplicaInPipelineInterface append(ExtendedBlock b,

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java?rev=1240020&r1=1240019&r2=1240020&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java Fri Feb  3 07:26:02 2012
@@ -162,7 +162,7 @@ abstract public class ReplicaInfo extend
    * be recovered (especially on Windows) on datanode restart.
    */
   private void unlinkFile(File file, Block b) throws IOException {
-    File tmpFile = FSDataset.createTmpFile(b, FSDataset.getUnlinkTmpFile(file));
+    File tmpFile = DatanodeUtil.createTmpFile(b, FSDataset.getUnlinkTmpFile(file));
     try {
       FileInputStream in = new FileInputStream(file);
       try {

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java?rev=1240020&r1=1240019&r2=1240020&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientReportBadBlock.java Fri Feb  3 07:26:02 2012
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs;
 
 import java.io.ByteArrayOutputStream;
+import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.PrintStream;
@@ -36,16 +37,15 @@ import org.apache.hadoop.hdfs.protocol.E
 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.FSDataset;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck;
 import org.apache.hadoop.hdfs.tools.DFSck;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.util.ToolRunner;
-
-import org.junit.Before;
 import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Test;
-import junit.framework.Assert;
 
 /**
  * Class is used to test client reporting corrupted block replica to name node.
@@ -306,9 +306,9 @@ public class TestClientReportBadBlock {
    */
   private static void corruptBlock(final ExtendedBlock block, final DataNode dn)
       throws FileNotFoundException, IOException {
-    final FSDataset data = (FSDataset) dn.getFSDataset();
-    final RandomAccessFile raFile = new RandomAccessFile(
-        data.getBlockFile(block), "rw");
+    final File f = DataNodeTestUtils.getBlockFile(
+        dn, block.getBlockPoolId(), block.getLocalBlock());
+    final RandomAccessFile raFile = new RandomAccessFile(f, "rw");
     final byte[] bytes = new byte[(int) BLOCK_SIZE];
     for (int i = 0; i < BLOCK_SIZE; i++) {
       bytes[i] = (byte) (i);

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java?rev=1240020&r1=1240019&r2=1240020&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java Fri Feb  3 07:26:02 2012
@@ -46,7 +46,6 @@ import org.apache.hadoop.fs.permission.F
 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;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -1164,9 +1163,9 @@ public class TestDFSShell extends TestCa
     String poolId = cluster.getNamesystem().getBlockPoolId();
     Iterable<Block>[] blocks = cluster.getAllBlockReports(poolId);
     for(int i = 0; i < blocks.length; i++) {
-      FSDataset ds = (FSDataset)datanodes.get(i).getFSDataset();
+      DataNode dn = datanodes.get(i);
       for(Block b : blocks[i]) {
-        files.add(DataNodeTestUtils.getBlockFile(ds, poolId, b.getBlockId()));
+        files.add(DataNodeTestUtils.getFile(dn, poolId, b.getBlockId()));
       }        
     }
     return files;

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java?rev=1240020&r1=1240019&r2=1240020&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend.java Fri Feb  3 07:26:02 2012
@@ -17,29 +17,27 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertTrue;
 
 import java.io.File;
-import java.io.IOException;
 import java.io.FileNotFoundException;
+import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.List;
 
-import org.junit.Test;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.HardLink;
+import org.apache.hadoop.fs.Path;
 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;
+import org.junit.Test;
 
 /**
  * This class tests the building blocks that are needed to
@@ -133,14 +131,13 @@ public class TestFileAppend{
       LocatedBlocks locations = client.getNamenode().getBlockLocations(
                                   file1.toString(), 0, Long.MAX_VALUE);
       List<LocatedBlock> blocks = locations.getLocatedBlocks();
-      FSDataset dataset = (FSDataset) dn[0].data;
 
       //
       // Create hard links for a few of the blocks
       //
       for (int i = 0; i < blocks.size(); i = i + 2) {
         ExtendedBlock b = blocks.get(i).getBlock();
-        final File f = DataNodeTestUtils.getBlockFile(dataset,
+        final File f = DataNodeTestUtils.getFile(dn[0],
             b.getBlockPoolId(), b.getLocalBlock().getBlockId());
         File link = new File(f.toString() + ".link");
         System.out.println("Creating hardlink for File " + f + " to " + link);
@@ -154,7 +151,7 @@ public class TestFileAppend{
         ExtendedBlock b = blocks.get(i).getBlock();
         System.out.println("testCopyOnWrite detaching block " + b);
         assertTrue("Detaching block " + b + " should have returned true",
-            dataset.unlinkBlock(b, 1));
+            DataNodeTestUtils.unlinkBlock(dn[0], b, 1));
       }
 
       // Since the blocks were already detached earlier, these calls should
@@ -164,7 +161,7 @@ public class TestFileAppend{
         ExtendedBlock b = blocks.get(i).getBlock();
         System.out.println("testCopyOnWrite detaching block " + b);
         assertTrue("Detaching block " + b + " should have returned false",
-            !dataset.unlinkBlock(b, 1));
+            !DataNodeTestUtils.unlinkBlock(dn[0], b, 1));
       }
 
     } finally {

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java?rev=1240020&r1=1240019&r2=1240020&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java Fri Feb  3 07:26:02 2012
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs;
 
+import java.io.File;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 
@@ -36,7 +37,7 @@ import org.apache.hadoop.hdfs.protocol.E
 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.FSDataset;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -200,8 +201,9 @@ public class TestFileAppend3 extends jun
     DatanodeInfo[] datanodeinfos = lb.getLocations();
     assertEquals(repl, datanodeinfos.length);
     final DataNode dn = cluster.getDataNode(datanodeinfos[0].getIpcPort());
-    final FSDataset data = (FSDataset)dn.getFSDataset();
-    final RandomAccessFile raf = new RandomAccessFile(data.getBlockFile(blk), "rw");
+    final File f = DataNodeTestUtils.getBlockFile(
+        dn, blk.getBlockPoolId(), blk.getLocalBlock());
+    final RandomAccessFile raf = new RandomAccessFile(f, "rw");
     AppendTestUtil.LOG.info("dn=" + dn + ", blk=" + blk + " (length=" + blk.getNumBytes() + ")");
     assertEquals(len1, raf.length());
     raf.setLength(0);

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java?rev=1240020&r1=1240019&r2=1240020&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java Fri Feb  3 07:26:02 2012
@@ -17,6 +17,21 @@
  */
 package org.apache.hadoop.hdfs;
 
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SYNCONCLOSE_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
+
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileNotFoundException;
@@ -37,7 +52,6 @@ import org.apache.hadoop.fs.FsServerDefa
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -46,7 +60,6 @@ import org.apache.hadoop.hdfs.protocol.L
 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;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
@@ -829,10 +842,9 @@ public class TestFileCreation extends ju
       int successcount = 0;
       for(DatanodeInfo datanodeinfo: locatedblock.getLocations()) {
         DataNode datanode = cluster.getDataNode(datanodeinfo.ipcPort);
-        FSDataset dataset = (FSDataset)datanode.data;
         ExtendedBlock blk = locatedblock.getBlock();
-        Block b = dataset.getStoredBlock(blk.getBlockPoolId(), blk.getBlockId());
-        final File blockfile = DataNodeTestUtils.getBlockFile(dataset,
+        Block b = datanode.data.getStoredBlock(blk.getBlockPoolId(), blk.getBlockId());
+        final File blockfile = DataNodeTestUtils.getFile(datanode,
             blk.getBlockPoolId(), b.getBlockId());
         System.out.println("blockfile=" + blockfile);
         if (blockfile != null) {

Modified: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java?rev=1240020&r1=1240019&r2=1240020&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java (original)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java Fri Feb  3 07:26:02 2012
@@ -22,6 +22,8 @@ package org.apache.hadoop.hdfs.server.da
 import java.io.File;
 import java.io.IOException;
 
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 
 /**
@@ -39,7 +41,18 @@ public class DataNodeTestUtils {
     return dn.getDNRegistrationForBP(bpid);
   }
 
-  public static File getBlockFile(FSDataset fsdataset, String bpid, long bid) {
-    return fsdataset.getFile(bpid, bid);
+  public static File getFile(DataNode dn, String bpid, long bid) {
+    return ((FSDataset)dn.getFSDataset()).getFile(bpid, bid);
+  }
+
+  public static File getBlockFile(DataNode dn, String bpid, Block b
+      ) throws IOException {
+    return ((FSDataset)dn.getFSDataset()).getBlockFile(bpid, b);
+  }
+
+  public static boolean unlinkBlock(DataNode dn, ExtendedBlock block, int numLinks
+      ) throws IOException {
+    ReplicaInfo info = ((FSDataset)dn.getFSDataset()).getReplicaInfo(block);
+    return info.unlinkBlock(numLinks);
   }
 }