Browse Source

HDFS-1298 - Add support in HDFS for new statistics added in FileSystem to track the file system operations. Contributed by Suresh Srinivas.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hdfs/trunk@964947 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 15 years ago
parent
commit
cbe388729c

+ 4 - 0
CHANGES.txt

@@ -76,6 +76,10 @@ Trunk (unreleased changes)
     HDFS-1272. Fixes to take care of the changes in HADOOP-6845.
     HDFS-1272. Fixes to take care of the changes in HADOOP-6845.
     (Jitendra Pandey via ddas)
     (Jitendra Pandey via ddas)
 
 
+    HDFS-1298 - Add support in HDFS for new statistics added in FileSystem
+    to track the file system operations. (suresh)
+    
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HDFS-1140. Speedup INode.getPathComponents. (Dmytro Molkov via shv)
     HDFS-1140. Speedup INode.getPathComponents. (Dmytro Molkov via shv)

+ 26 - 4
src/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -186,12 +186,13 @@ public class DistributedFileSystem extends FileSystem {
     if (file == null) {
     if (file == null) {
       return null;
       return null;
     }
     }
-    return dfs.getBlockLocations(getPathName(file.getPath()), start, len);
+    return getFileBlockLocations(file.getPath(), start, len);
   }
   }
   
   
   @Override
   @Override
   public BlockLocation[] getFileBlockLocations(Path p, 
   public BlockLocation[] getFileBlockLocations(Path p, 
       long start, long len) throws IOException {
       long start, long len) throws IOException {
+    statistics.incrementReadOps(1);
     return dfs.getBlockLocations(getPathName(p), start, len);
     return dfs.getBlockLocations(getPathName(p), start, len);
 
 
   }
   }
@@ -204,6 +205,7 @@ public class DistributedFileSystem extends FileSystem {
   @SuppressWarnings("deprecation")
   @SuppressWarnings("deprecation")
   @Override
   @Override
   public FSDataInputStream open(Path f, int bufferSize) throws IOException {
   public FSDataInputStream open(Path f, int bufferSize) throws IOException {
+    statistics.incrementReadOps(1);
     return new DFSClient.DFSDataInputStream(
     return new DFSClient.DFSDataInputStream(
           dfs.open(getPathName(f), bufferSize, verifyChecksum, statistics));
           dfs.open(getPathName(f), bufferSize, verifyChecksum, statistics));
   }
   }
@@ -213,6 +215,7 @@ public class DistributedFileSystem extends FileSystem {
   public FSDataOutputStream append(Path f, int bufferSize,
   public FSDataOutputStream append(Path f, int bufferSize,
       Progressable progress) throws IOException {
       Progressable progress) throws IOException {
 
 
+    statistics.incrementWriteOps(1);
     DFSOutputStream op = (DFSOutputStream)dfs.append(getPathName(f), bufferSize, progress);
     DFSOutputStream op = (DFSOutputStream)dfs.append(getPathName(f), bufferSize, progress);
     return new FSDataOutputStream(op, statistics, op.getInitialLen());
     return new FSDataOutputStream(op, statistics, op.getInitialLen());
   }
   }
@@ -221,7 +224,7 @@ public class DistributedFileSystem extends FileSystem {
   public FSDataOutputStream create(Path f, FsPermission permission,
   public FSDataOutputStream create(Path f, FsPermission permission,
     boolean overwrite, int bufferSize, short replication, long blockSize,
     boolean overwrite, int bufferSize, short replication, long blockSize,
     Progressable progress) throws IOException {
     Progressable progress) throws IOException {
-
+    statistics.incrementWriteOps(1);
     return new FSDataOutputStream(dfs.create(getPathName(f), permission,
     return new FSDataOutputStream(dfs.create(getPathName(f), permission,
         overwrite ? EnumSet.of(CreateFlag.OVERWRITE) : EnumSet.of(CreateFlag.CREATE),
         overwrite ? EnumSet.of(CreateFlag.OVERWRITE) : EnumSet.of(CreateFlag.CREATE),
         replication, blockSize, progress, bufferSize),
         replication, blockSize, progress, bufferSize),
@@ -234,6 +237,7 @@ public class DistributedFileSystem extends FileSystem {
     FsPermission absolutePermission, EnumSet<CreateFlag> flag, int bufferSize,
     FsPermission absolutePermission, EnumSet<CreateFlag> flag, int bufferSize,
     short replication, long blockSize, Progressable progress,
     short replication, long blockSize, Progressable progress,
     int bytesPerChecksum) throws IOException {
     int bytesPerChecksum) throws IOException {
+    statistics.incrementReadOps(1);
     return new FSDataOutputStream(dfs.primitiveCreate(getPathName(f),
     return new FSDataOutputStream(dfs.primitiveCreate(getPathName(f),
         absolutePermission, flag, true, replication, blockSize,
         absolutePermission, flag, true, replication, blockSize,
         progress, bufferSize, bytesPerChecksum),statistics);
         progress, bufferSize, bytesPerChecksum),statistics);
@@ -245,7 +249,7 @@ public class DistributedFileSystem extends FileSystem {
   public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
   public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
       EnumSet<CreateFlag> flag, int bufferSize, short replication,
       EnumSet<CreateFlag> flag, int bufferSize, short replication,
       long blockSize, Progressable progress) throws IOException {
       long blockSize, Progressable progress) throws IOException {
-
+    statistics.incrementWriteOps(1);
     return new FSDataOutputStream(dfs.create(getPathName(f), permission, flag,
     return new FSDataOutputStream(dfs.create(getPathName(f), permission, flag,
         false, replication, blockSize, progress, bufferSize), statistics);
         false, replication, blockSize, progress, bufferSize), statistics);
   }
   }
@@ -254,6 +258,7 @@ public class DistributedFileSystem extends FileSystem {
   public boolean setReplication(Path src, 
   public boolean setReplication(Path src, 
                                 short replication
                                 short replication
                                ) throws IOException {
                                ) throws IOException {
+    statistics.incrementWriteOps(1);
     return dfs.setReplication(getPathName(src), replication);
     return dfs.setReplication(getPathName(src), replication);
   }
   }
   
   
@@ -271,6 +276,7 @@ public class DistributedFileSystem extends FileSystem {
     for(int i=0; i<psrcs.length; i++) {
     for(int i=0; i<psrcs.length; i++) {
       srcs[i] = getPathName(psrcs[i]);
       srcs[i] = getPathName(psrcs[i]);
     }
     }
+    statistics.incrementWriteOps(1);
     dfs.concat(getPathName(trg), srcs);
     dfs.concat(getPathName(trg), srcs);
   }
   }
 
 
@@ -278,6 +284,7 @@ public class DistributedFileSystem extends FileSystem {
   @SuppressWarnings("deprecation")
   @SuppressWarnings("deprecation")
   @Override
   @Override
   public boolean rename(Path src, Path dst) throws IOException {
   public boolean rename(Path src, Path dst) throws IOException {
+    statistics.incrementWriteOps(1);
     return dfs.rename(getPathName(src), getPathName(dst));
     return dfs.rename(getPathName(src), getPathName(dst));
   }
   }
 
 
@@ -288,17 +295,20 @@ public class DistributedFileSystem extends FileSystem {
   @SuppressWarnings("deprecation")
   @SuppressWarnings("deprecation")
   @Override
   @Override
   public void rename(Path src, Path dst, Options.Rename... options) throws IOException {
   public void rename(Path src, Path dst, Options.Rename... options) throws IOException {
+    statistics.incrementWriteOps(1);
     dfs.rename(getPathName(src), getPathName(dst), options);
     dfs.rename(getPathName(src), getPathName(dst), options);
   }
   }
   
   
   @Override
   @Override
   public boolean delete(Path f, boolean recursive) throws IOException {
   public boolean delete(Path f, boolean recursive) throws IOException {
-   return dfs.delete(getPathName(f), recursive);
+    statistics.incrementWriteOps(1);
+    return dfs.delete(getPathName(f), recursive);
   }
   }
   
   
   /** {@inheritDoc} */
   /** {@inheritDoc} */
   @Override
   @Override
   public ContentSummary getContentSummary(Path f) throws IOException {
   public ContentSummary getContentSummary(Path f) throws IOException {
+    statistics.incrementReadOps(1);
     return dfs.getContentSummary(getPathName(f));
     return dfs.getContentSummary(getPathName(f));
   }
   }
 
 
@@ -345,6 +355,7 @@ public class DistributedFileSystem extends FileSystem {
       for (int i = 0; i < partialListing.length; i++) {
       for (int i = 0; i < partialListing.length; i++) {
         stats[i] = makeQualified(partialListing[i], p);
         stats[i] = makeQualified(partialListing[i], p);
       }
       }
+      statistics.incrementReadOps(1);
       return stats;
       return stats;
     }
     }
 
 
@@ -358,6 +369,7 @@ public class DistributedFileSystem extends FileSystem {
     for (HdfsFileStatus fileStatus : partialListing) {
     for (HdfsFileStatus fileStatus : partialListing) {
       listing.add(makeQualified(fileStatus, p));
       listing.add(makeQualified(fileStatus, p));
     }
     }
+    statistics.incrementLargeReadOps(1);
  
  
     // now fetch more entries
     // now fetch more entries
     do {
     do {
@@ -371,6 +383,7 @@ public class DistributedFileSystem extends FileSystem {
       for (HdfsFileStatus fileStatus : partialListing) {
       for (HdfsFileStatus fileStatus : partialListing) {
         listing.add(makeQualified(fileStatus, p));
         listing.add(makeQualified(fileStatus, p));
       }
       }
+      statistics.incrementLargeReadOps(1);
     } while (thisListing.hasMore());
     } while (thisListing.hasMore());
  
  
     return listing.toArray(new FileStatus[listing.size()]);
     return listing.toArray(new FileStatus[listing.size()]);
@@ -381,11 +394,13 @@ public class DistributedFileSystem extends FileSystem {
    * parent directory exists.
    * parent directory exists.
    */
    */
   public boolean mkdir(Path f, FsPermission permission) throws IOException {
   public boolean mkdir(Path f, FsPermission permission) throws IOException {
+    statistics.incrementWriteOps(1);
     return dfs.mkdirs(getPathName(f), permission, false);
     return dfs.mkdirs(getPathName(f), permission, false);
   }
   }
 
 
   @Override
   @Override
   public boolean mkdirs(Path f, FsPermission permission) throws IOException {
   public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+    statistics.incrementWriteOps(1);
     return dfs.mkdirs(getPathName(f), permission, true);
     return dfs.mkdirs(getPathName(f), permission, true);
   }
   }
 
 
@@ -393,6 +408,7 @@ public class DistributedFileSystem extends FileSystem {
   @Override
   @Override
   protected boolean primitiveMkdir(Path f, FsPermission absolutePermission)
   protected boolean primitiveMkdir(Path f, FsPermission absolutePermission)
     throws IOException {
     throws IOException {
+    statistics.incrementWriteOps(1);
     return dfs.primitiveMkdir(getPathName(f), absolutePermission);
     return dfs.primitiveMkdir(getPathName(f), absolutePermission);
   }
   }
 
 
@@ -436,6 +452,7 @@ public class DistributedFileSystem extends FileSystem {
   /** {@inheritDoc} */
   /** {@inheritDoc} */
   @Override
   @Override
   public FsStatus getStatus(Path p) throws IOException {
   public FsStatus getStatus(Path p) throws IOException {
+    statistics.incrementReadOps(1);
     return dfs.getDiskStatus();
     return dfs.getDiskStatus();
   }
   }
 
 
@@ -617,6 +634,7 @@ public class DistributedFileSystem extends FileSystem {
    */
    */
   @Override
   @Override
   public FileStatus getFileStatus(Path f) throws IOException {
   public FileStatus getFileStatus(Path f) throws IOException {
+    statistics.incrementReadOps(1);
     HdfsFileStatus fi = dfs.getFileInfo(getPathName(f));
     HdfsFileStatus fi = dfs.getFileInfo(getPathName(f));
     if (fi != null) {
     if (fi != null) {
       return makeQualified(fi, f);
       return makeQualified(fi, f);
@@ -628,6 +646,7 @@ public class DistributedFileSystem extends FileSystem {
   /** {@inheritDoc} */
   /** {@inheritDoc} */
   @Override
   @Override
   public MD5MD5CRC32FileChecksum getFileChecksum(Path f) throws IOException {
   public MD5MD5CRC32FileChecksum getFileChecksum(Path f) throws IOException {
+    statistics.incrementReadOps(1);
     return dfs.getFileChecksum(getPathName(f));
     return dfs.getFileChecksum(getPathName(f));
   }
   }
 
 
@@ -635,6 +654,7 @@ public class DistributedFileSystem extends FileSystem {
   @Override
   @Override
   public void setPermission(Path p, FsPermission permission
   public void setPermission(Path p, FsPermission permission
       ) throws IOException {
       ) throws IOException {
+    statistics.incrementWriteOps(1);
     dfs.setPermission(getPathName(p), permission);
     dfs.setPermission(getPathName(p), permission);
   }
   }
 
 
@@ -645,6 +665,7 @@ public class DistributedFileSystem extends FileSystem {
     if (username == null && groupname == null) {
     if (username == null && groupname == null) {
       throw new IOException("username == null && groupname == null");
       throw new IOException("username == null && groupname == null");
     }
     }
+    statistics.incrementWriteOps(1);
     dfs.setOwner(getPathName(p), username, groupname);
     dfs.setOwner(getPathName(p), username, groupname);
   }
   }
 
 
@@ -652,6 +673,7 @@ public class DistributedFileSystem extends FileSystem {
   @Override
   @Override
   public void setTimes(Path p, long mtime, long atime
   public void setTimes(Path p, long mtime, long atime
       ) throws IOException {
       ) throws IOException {
+    statistics.incrementWriteOps(1);
     dfs.setTimes(getPathName(p), mtime, atime);
     dfs.setTimes(getPathName(p), mtime, atime);
   }
   }
   
   

+ 5 - 0
src/test/hdfs/org/apache/hadoop/hdfs/DFSTestUtil.java

@@ -39,6 +39,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream;
 import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -371,4 +372,8 @@ public class DFSTestUtil {
 
 
     return result;
     return result;
   }
   }
+  
+  public static Statistics getStatistics(FileSystem fs) {
+    return FileSystem.getStatistics(fs.getUri().getScheme(), fs.getClass());
+  }
 }
 }

+ 110 - 1
src/test/hdfs/org/apache/hadoop/hdfs/TestDistributedFileSystem.java

@@ -18,6 +18,8 @@
 
 
 package org.apache.hadoop.hdfs;
 package org.apache.hadoop.hdfs;
 
 
+import static org.junit.Assert.*;
+
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
@@ -28,11 +30,15 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileChecksum;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.log4j.Level;
 import org.apache.log4j.Level;
+import org.junit.Test;
 
 
-public class TestDistributedFileSystem extends junit.framework.TestCase {
+public class TestDistributedFileSystem {
   private static final Random RAN = new Random();
   private static final Random RAN = new Random();
 
 
   private boolean dualPortTesting = false;
   private boolean dualPortTesting = false;
@@ -46,6 +52,7 @@ public class TestDistributedFileSystem extends junit.framework.TestCase {
     return conf;
     return conf;
   }
   }
 
 
+  @Test
   public void testFileSystemCloseAll() throws Exception {
   public void testFileSystemCloseAll() throws Exception {
     Configuration conf = getTestConfiguration();
     Configuration conf = getTestConfiguration();
     MiniDFSCluster cluster = new MiniDFSCluster(conf, 0, true, null);
     MiniDFSCluster cluster = new MiniDFSCluster(conf, 0, true, null);
@@ -69,6 +76,7 @@ public class TestDistributedFileSystem extends junit.framework.TestCase {
    * Tests DFSClient.close throws no ConcurrentModificationException if 
    * Tests DFSClient.close throws no ConcurrentModificationException if 
    * multiple files are open.
    * multiple files are open.
    */
    */
+  @Test
   public void testDFSClose() throws Exception {
   public void testDFSClose() throws Exception {
     Configuration conf = getTestConfiguration();
     Configuration conf = getTestConfiguration();
     MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
     MiniDFSCluster cluster = new MiniDFSCluster(conf, 2, true, null);
@@ -86,6 +94,7 @@ public class TestDistributedFileSystem extends junit.framework.TestCase {
     }
     }
   }
   }
 
 
+  @Test
   public void testDFSClient() throws Exception {
   public void testDFSClient() throws Exception {
     Configuration conf = getTestConfiguration();
     Configuration conf = getTestConfiguration();
     MiniDFSCluster cluster = null;
     MiniDFSCluster cluster = null;
@@ -169,6 +178,7 @@ public class TestDistributedFileSystem extends junit.framework.TestCase {
     }
     }
   }
   }
   
   
+  @Test
   public void testFileChecksum() throws IOException {
   public void testFileChecksum() throws IOException {
     ((Log4JLogger)HftpFileSystem.LOG).getLogger().setLevel(Level.ALL);
     ((Log4JLogger)HftpFileSystem.LOG).getLogger().setLevel(Level.ALL);
 
 
@@ -242,6 +252,7 @@ public class TestDistributedFileSystem extends junit.framework.TestCase {
     cluster.shutdown();
     cluster.shutdown();
   }
   }
   
   
+  @Test
   public void testAllWithDualPort() throws Exception {
   public void testAllWithDualPort() throws Exception {
     dualPortTesting = true;
     dualPortTesting = true;
 
 
@@ -250,4 +261,102 @@ public class TestDistributedFileSystem extends junit.framework.TestCase {
     testDFSClient();
     testDFSClient();
     testFileChecksum();
     testFileChecksum();
   }
   }
+  
+  @Test
+  public void testStatistics() throws Exception {
+    int lsLimit = 2;
+    final Configuration conf = getTestConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_LIST_LIMIT, lsLimit);
+    final MiniDFSCluster cluster = new MiniDFSCluster(conf, 1, true, null);
+    try {
+      final FileSystem fs = cluster.getFileSystem();
+      Path dir = new Path("/test");
+      Path file = new Path(dir, "file");
+      
+      int readOps = DFSTestUtil.getStatistics(fs).getReadOps();
+      int writeOps = DFSTestUtil.getStatistics(fs).getWriteOps();
+      int largeReadOps = DFSTestUtil.getStatistics(fs).getLargeReadOps();
+      fs.mkdirs(dir);
+      checkStatistics(fs, readOps, ++writeOps, largeReadOps);
+      
+      FSDataOutputStream out = fs.create(file, (short)1);
+      out.close();
+      checkStatistics(fs, readOps, ++writeOps, largeReadOps);
+      
+      FileStatus status = fs.getFileStatus(file);
+      checkStatistics(fs, ++readOps, writeOps, largeReadOps);
+      
+      fs.getFileBlockLocations(file, 0, 0);
+      checkStatistics(fs, ++readOps, writeOps, largeReadOps);
+      
+      fs.getFileBlockLocations(status, 0, 0);
+      checkStatistics(fs, ++readOps, writeOps, largeReadOps);
+      
+      FSDataInputStream in = fs.open(file);
+      in.close();
+      checkStatistics(fs, ++readOps, writeOps, largeReadOps);
+      
+      fs.setReplication(file, (short)2);
+      checkStatistics(fs, readOps, ++writeOps, largeReadOps);
+      
+      Path file1 = new Path(dir, "file1");
+      fs.rename(file, file1);
+      checkStatistics(fs, readOps, ++writeOps, largeReadOps);
+      
+      fs.getContentSummary(file1);
+      checkStatistics(fs, ++readOps, writeOps, largeReadOps);
+      
+      
+      // Iterative ls test
+      for (int i = 0; i < 10; i++) {
+        Path p = new Path(dir, Integer.toString(i));
+        fs.mkdirs(p);
+        FileStatus[] list = fs.listStatus(dir);
+        if (list.length > lsLimit) {
+          // if large directory, then count readOps and largeReadOps by 
+          // number times listStatus iterates
+          int iterations = (int)Math.ceil((double)list.length/lsLimit);
+          largeReadOps += iterations;
+          readOps += iterations;
+        } else {
+          // Single iteration in listStatus - no large read operation done
+          readOps++;
+        }
+        
+        // writeOps incremented by 1 for mkdirs
+        // readOps and largeReadOps incremented by 1 or more
+        checkStatistics(fs, readOps, ++writeOps, largeReadOps);
+      }
+      
+      fs.getStatus(file1);
+      checkStatistics(fs, ++readOps, writeOps, largeReadOps);
+      
+      fs.getFileChecksum(file1);
+      checkStatistics(fs, ++readOps, writeOps, largeReadOps);
+      
+      fs.setPermission(file1, new FsPermission((short)0777));
+      checkStatistics(fs, readOps, ++writeOps, largeReadOps);
+      
+      fs.setTimes(file1, 0L, 0L);
+      checkStatistics(fs, readOps, ++writeOps, largeReadOps);
+      
+      UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+      fs.setOwner(file1, ugi.getUserName(), ugi.getGroupNames()[0]);
+      checkStatistics(fs, readOps, ++writeOps, largeReadOps);
+      
+      fs.delete(dir, true);
+      checkStatistics(fs, readOps, ++writeOps, largeReadOps);
+      
+    } finally {
+      if (cluster != null) cluster.shutdown();
+    }
+    
+  }
+  
+  /** Checks statistics. -1 indicates do not check for the operations */
+  private void checkStatistics(FileSystem fs, int readOps, int writeOps, int largeReadOps) {
+    assertEquals(readOps, DFSTestUtil.getStatistics(fs).getReadOps());
+    assertEquals(writeOps, DFSTestUtil.getStatistics(fs).getWriteOps());
+    assertEquals(largeReadOps, DFSTestUtil.getStatistics(fs).getLargeReadOps());
+  }
 }
 }

+ 0 - 30
src/test/hdfs/org/apache/hadoop/hdfs/TestFileCreation.java

@@ -96,36 +96,6 @@ public class TestFileCreation extends junit.framework.TestCase {
     stm.write(buffer, 0, size);
     stm.write(buffer, 0, size);
   }
   }
 
 
-  static private void checkData(byte[] actual, int from, byte[] expected, String message) {
-    for (int idx = 0; idx < actual.length; idx++) {
-      assertEquals(message+" byte "+(from+idx)+" differs. expected "+
-                   expected[from+idx]+" actual "+actual[idx],
-                   expected[from+idx], actual[idx]);
-      actual[idx] = 0;
-    }
-  }
-
-  static void checkFullFile(FileSystem fs, Path name) throws IOException {
-    FileStatus stat = fs.getFileStatus(name);
-    BlockLocation[] locations = fs.getFileBlockLocations(stat, 0, 
-                                                         fileSize);
-    for (int idx = 0; idx < locations.length; idx++) {
-      String[] hosts = locations[idx].getNames();
-      for (int i = 0; i < hosts.length; i++) {
-        System.out.print( hosts[i] + " ");
-      }
-      System.out.println(" off " + locations[idx].getOffset() +
-                         " len " + locations[idx].getLength());
-    }
-
-    byte[] expected = AppendTestUtil.randomBytes(seed, fileSize);
-    FSDataInputStream stm = fs.open(name);
-    byte[] actual = new byte[fileSize];
-    stm.readFully(0, actual);
-    checkData(actual, 0, expected, "Read 2");
-    stm.close();
-  }
-
   /**
   /**
    * Test that server default values can be retrieved on the client side
    * Test that server default values can be retrieved on the client side
    */
    */