Parcourir la source

HADOOP-1703. DFS client code cleanups, removing several uses of the obsolete UTF8. Contributed by Christophe Taton.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@567877 13f79535-47bb-0310-9956-ffa450edef68
Doug Cutting il y a 18 ans
Parent
commit
361512fddd

+ 3 - 0
CHANGES.txt

@@ -65,6 +65,9 @@ Trunk (unreleased changes)
     not subclass ToolBase, and generic parameter parser is public.
     (Enis Soztutar via cutting)
 
+    HADOOP-1703.  DFS-internal code cleanups, removing several uses of
+    the obsolete UTF8.  (Christophe Taton via cutting)
+
 
 Release 0.14.0 - 2007-08-17
 

+ 15 - 15
src/java/org/apache/hadoop/dfs/DFSClient.java

@@ -209,9 +209,9 @@ class DFSClient implements FSConstants {
     return defaultBlockSize;
   }
     
-  public long getBlockSize(UTF8 f) throws IOException {
+  public long getBlockSize(String f) throws IOException {
     try {
-      return namenode.getBlockSize(f.toString());
+      return namenode.getBlockSize(f);
     } catch (IOException ie) {
       LOG.warn("Problem getting block size: " + 
           StringUtils.stringifyException(ie));
@@ -286,7 +286,7 @@ class DFSClient implements FSConstants {
    * @return output stream
    * @throws IOException
    */
-  public OutputStream create(UTF8 src, 
+  public OutputStream create(String src, 
                              boolean overwrite
                              ) throws IOException {
     return create(src, overwrite, defaultReplication, defaultBlockSize, null);
@@ -301,7 +301,7 @@ class DFSClient implements FSConstants {
    * @return output stream
    * @throws IOException
    */
-  public OutputStream create(UTF8 src, 
+  public OutputStream create(String src, 
                              boolean overwrite,
                              Progressable progress
                              ) throws IOException {
@@ -318,7 +318,7 @@ class DFSClient implements FSConstants {
    * @return output stream
    * @throws IOException
    */
-  public OutputStream create(UTF8 src, 
+  public OutputStream create(String src, 
                              boolean overwrite, 
                              short replication,
                              long blockSize
@@ -338,7 +338,7 @@ class DFSClient implements FSConstants {
    * @return output stream
    * @throws IOException
    */
-  public OutputStream create(UTF8 src, 
+  public OutputStream create(String src, 
                              boolean overwrite, 
                              short replication,
                              long blockSize,
@@ -358,7 +358,7 @@ class DFSClient implements FSConstants {
    * @return output stream
    * @throws IOException
    */
-  public OutputStream create(UTF8 src, 
+  public OutputStream create(String src, 
                              boolean overwrite, 
                              short replication,
                              long blockSize,
@@ -381,10 +381,10 @@ class DFSClient implements FSConstants {
    * @throws IOException
    * @return true is successful or false if file does not exist 
    */
-  public boolean setReplication(UTF8 src, 
+  public boolean setReplication(String src, 
                                 short replication
                                 ) throws IOException {
-    return namenode.setReplication(src.toString(), replication);
+    return namenode.setReplication(src, replication);
   }
 
   /**
@@ -421,14 +421,14 @@ class DFSClient implements FSConstants {
 
   /**
    */
-  public DFSFileInfo[] listPaths(UTF8 src) throws IOException {
+  public DFSFileInfo[] listPaths(String src) throws IOException {
     checkOpen();
-    return namenode.getListing(src.toString());
+    return namenode.getListing(src);
   }
 
-  public DFSFileInfo getFileInfo(UTF8 src) throws IOException {
+  public DFSFileInfo getFileInfo(String src) throws IOException {
     checkOpen();
-    return namenode.getFileInfo(src.toString());
+    return namenode.getFileInfo(src);
   }
 
   public DiskStatus getDiskStatus() throws IOException {
@@ -1358,7 +1358,7 @@ class DFSClient implements FSConstants {
     private Socket s;
     boolean closed = false;
 
-    private UTF8 src;
+    private String src;
     private short replication;
     private DataOutputStream blockStream;
     private DataInputStream blockReplyStream;
@@ -1375,7 +1375,7 @@ class DFSClient implements FSConstants {
     /**
      * Create a new output stream to the given DataNode.
      */
-    public DFSOutputStream(UTF8 src, boolean overwrite, 
+    public DFSOutputStream(String src, boolean overwrite, 
                            short replication, long blockSize,
                            Progressable progress,
                            int buffersize

+ 16 - 15
src/java/org/apache/hadoop/dfs/DistributedFileSystem.java

@@ -37,14 +37,12 @@ public class DistributedFileSystem extends FileSystem {
   private Path workingDir =
     new Path("/user", System.getProperty("user.name")); 
   private URI uri;
-  private FileSystem localFs;
 
   DFSClient dfs;
 
   public DistributedFileSystem() {
   }
 
-
   /** @deprecated */
   public DistributedFileSystem(InetSocketAddress namenode,
     Configuration conf) throws IOException {
@@ -65,7 +63,6 @@ public class DistributedFileSystem extends FileSystem {
     int port = uri.getPort();
     this.dfs = new DFSClient(new InetSocketAddress(host, port), conf);
     this.uri = URI.create("hdfs://"+host+":"+port);
-    this.localFs = getLocal(conf);
   }
 
   public Path getWorkingDirectory() {
@@ -114,7 +111,8 @@ public class DistributedFileSystem extends FileSystem {
     return result;
   }
 
-  public String[][] getFileCacheHints(Path f, long start, long len) throws IOException {
+  public String[][] getFileCacheHints(Path f, long start, long len)
+      throws IOException {
     return dfs.getHints(getPathName(f), start, len);
   }
 
@@ -133,7 +131,7 @@ public class DistributedFileSystem extends FileSystem {
       throw new IOException("Mkdirs failed to create " + parent);
     }
 
-    return new FSDataOutputStream( dfs.create(getPath(f), overwrite, 
+    return new FSDataOutputStream( dfs.create(getPathName(f), overwrite, 
                                               replication, blockSize, 
                                               progress, bufferSize) );
   }
@@ -141,7 +139,7 @@ public class DistributedFileSystem extends FileSystem {
   public boolean setReplication(Path src, 
                                 short replication
                                ) throws IOException {
-    return dfs.setReplication(getPath(src), replication);
+    return dfs.setReplication(getPathName(src), replication);
   }
 
   /**
@@ -167,16 +165,16 @@ public class DistributedFileSystem extends FileSystem {
       return ((DfsPath)f).getContentsLength();
     }
 
-    DFSFileInfo info[] = dfs.listPaths(getPath(f));
+    DFSFileInfo info[] = dfs.listPaths(getPathName(f));
     return (info == null) ? 0 : info[0].getLen();
   }
 
   public FileStatus[] listStatus(Path f) throws IOException {
-    return dfs.listPaths(getPath(f));
+    return dfs.listPaths(getPathName(f));
   }
 
   public Path[] listPaths(Path f) throws IOException {
-    DFSFileInfo info[] = dfs.listPaths(getPath(f));
+    DFSFileInfo info[] = dfs.listPaths(getPathName(f));
     if (info == null) {
       return new Path[0];
     } else {
@@ -253,7 +251,8 @@ public class DistributedFileSystem extends FileSystem {
   /**
    * Enter, leave or get safe mode.
    *  
-   * @see org.apache.hadoop.dfs.ClientProtocol#setSafeMode(FSConstants.SafeModeAction)
+   * @see org.apache.hadoop.dfs.ClientProtocol#setSafeMode(
+   *    FSConstants.SafeModeAction)
    */
   public boolean setSafeMode(FSConstants.SafeModeAction action) 
   throws IOException {
@@ -309,8 +308,9 @@ public class DistributedFileSystem extends FileSystem {
     }
     DatanodeInfo[] dataNode = {dfsIn.getCurrentDatanode()}; 
     lblocks[0] = new LocatedBlock(dataBlock, dataNode);
-    LOG.info("Found checksum error in data stream at block=" + dataBlock.getBlockName() + 
-             " on datanode=" + dataNode[0].getName());
+    LOG.info("Found checksum error in data stream at block="
+        + dataBlock.getBlockName() + " on datanode="
+        + dataNode[0].getName());
 
     // Find block in checksum stream
     DFSClient.DFSDataInputStream dfsSums = (DFSClient.DFSDataInputStream) sums;
@@ -321,8 +321,9 @@ public class DistributedFileSystem extends FileSystem {
     }
     DatanodeInfo[] sumsNode = {dfsSums.getCurrentDatanode()}; 
     lblocks[1] = new LocatedBlock(sumsBlock, sumsNode);
-    LOG.info("Found checksum error in checksum stream at block=" + sumsBlock.getBlockName() + 
-             " on datanode=" + sumsNode[0].getName());
+    LOG.info("Found checksum error in checksum stream at block="
+        + sumsBlock.getBlockName() + " on datanode="
+        + sumsNode[0].getName());
 
     // Ask client to delete blocks.
     dfs.reportChecksumFailure(f.toString(), lblocks);
@@ -339,7 +340,7 @@ public class DistributedFileSystem extends FileSystem {
       return p.info;
     }
     else {
-      DFSFileInfo p = dfs.getFileInfo(getPath(f));
+      DFSFileInfo p = dfs.getFileInfo(getPathName(f));
       return p;
     }
   }

+ 1 - 1
src/java/org/apache/hadoop/dfs/NamenodeFsck.java

@@ -270,7 +270,7 @@ public class NamenodeFsck {
           continue;
         }
         if (fos == null) {
-          fos = dfs.create(new UTF8(target + "/" + chain), true);
+          fos = dfs.create(target + "/" + chain, true);
           if (fos != null) chain++;
         }
         if (fos == null) {

+ 1 - 1
src/test/org/apache/hadoop/dfs/ClusterTestDFS.java

@@ -258,7 +258,7 @@ public class ClusterTestDFS extends TestCase implements FSConstants {
         for (int iFileNumber = 0; iFileNumber < numFiles; iFileNumber++) {
           testFileName = new UTF8("/f" + iFileNumber);
           testfilesList.add(testFileName);
-          OutputStream nos = dfsClient.create(testFileName, false);
+          OutputStream nos = dfsClient.create(testFileName.toString(), false);
           try {
             for (long nBytesWritten = 0L;
                  nBytesWritten < nBytes;

+ 1 - 1
src/test/org/apache/hadoop/dfs/ClusterTestDFSNamespaceLogging.java

@@ -223,7 +223,7 @@ public class ClusterTestDFSNamespaceLogging extends TestCase implements FSConsta
     byte[] buffer = new byte[BUFFER_SIZE];
     UTF8 testFileName = new UTF8(filename); // hardcode filename
     OutputStream nos;
-    nos = dfsClient.create(testFileName, false);
+    nos = dfsClient.create(testFileName.toString(), false);
     try {
       for (long nBytesWritten = 0L;
            nBytesWritten < fileSize;

+ 1 - 1
src/test/org/apache/hadoop/dfs/TestDFSUpgradeFromImage.java

@@ -130,7 +130,7 @@ public class TestDFSUpgradeFromImage extends TestCase {
   private void verifyDir(DFSClient client, String dir) 
                                            throws IOException {
     
-    DFSFileInfo[] fileArr = client.listPaths(new UTF8(dir));
+    DFSFileInfo[] fileArr = client.listPaths(dir);
     TreeMap<String, Boolean> fileMap = new TreeMap<String, Boolean>();
     
     for(DFSFileInfo file : fileArr) {

+ 1 - 1
src/webapps/datanode/browseDirectory.jsp

@@ -67,7 +67,7 @@
         return;
       }
       // directory
-      DFSFileInfo[] files = dfs.listPaths(target);
+      DFSFileInfo[] files = dfs.listPaths(target.toString());
       //generate a table and dump the info
       String [] headings = new String[6];
       headings[0] = "Name"; headings[1] = "Type"; headings[2] = "Size";