فهرست منبع

HADOOP-4824. Should not use File.setWritable() in 0.18. Contributed by Hairong Kuang.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/branches/branch-0.18@725801 13f79535-47bb-0310-9956-ffa450edef68
Hairong Kuang 16 سال پیش
والد
کامیت
0e8828fd96
2فایلهای تغییر یافته به همراه18 افزوده شده و 8 حذف شده
  1. 2 0
      CHANGES.txt
  2. 16 8
      src/test/org/apache/hadoop/dfs/TestDiskError.java

+ 2 - 0
CHANGES.txt

@@ -97,6 +97,8 @@ Release 0.18.3 - Unreleased
     HADOOP-4823. Use SortedMap instead of NavigableMap in 0.18 due to the
     Java 5 requirement.  (szetszwo)
 
+    HADOOP-4824. Should not use File.setWritable() in 0.18. (hairong)
+
 Release 0.18.2 - 2008-11-03
 
   BUG FIXES

+ 16 - 8
src/test/org/apache/hadoop/dfs/TestDiskError.java

@@ -25,6 +25,8 @@ import java.net.Socket;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.dfs.DFSTestUtil;
 import org.apache.hadoop.dfs.FSConstants;
 import org.apache.hadoop.dfs.MiniDFSCluster;
@@ -43,15 +45,21 @@ public class TestDiskError extends TestCase {
     cluster.waitActive();
     FileSystem fs = cluster.getFileSystem();
     final int dnIndex = 0;
-    File dataDir = new File(
+
+    FileSystem localFs = FileSystem.getLocal(conf);
+    Path dataDir = new Path(
       System.getProperty("test.build.data", "build/test/data"), "dfs");
-    dataDir = new File(dataDir, "data");
-    File dir1 = new File(new File(dataDir, "data"+(2*dnIndex+1)), "tmp");
-    File dir2 = new File(new File(dataDir, "data"+(2*dnIndex+2)), "tmp");
+    dataDir = new Path(dataDir, "data");
+    Path dir1 = new Path(new Path(dataDir, "data"+(2*dnIndex+1)), "tmp");
+    Path dir2 = new Path(new Path(dataDir, "data"+(2*dnIndex+2)), "tmp");
+    FsPermission oldPerm1 = localFs.getFileStatus(dir1).getPermission();
+    FsPermission oldPerm2 = localFs.getFileStatus(dir2).getPermission();
     try {
       // make the data directory of the first datanode to be readonly
-      assertTrue(dir1.setReadOnly());
-      assertTrue(dir2.setReadOnly());
+      final FsPermission readPermission =
+        new FsPermission(FsAction.READ, FsAction.READ, FsAction.READ);
+      localFs.setPermission(dir1, readPermission);
+      localFs.setPermission(dir2, readPermission);
 
       // create files and make sure that first datanode will be down
       DataNode dn = cluster.getDataNodes().get(dnIndex);
@@ -63,8 +71,8 @@ public class TestDiskError extends TestCase {
       }
     } finally {
       // restore its old permission
-      dir1.setWritable(true);
-      dir2.setWritable(true);
+      localFs.setPermission(dir1, oldPerm1);
+      localFs.setPermission(dir2, oldPerm2);
       cluster.shutdown();
     }
   }