فهرست منبع

HADOOP-4326. ChecksumFileSystem does not override create(...) correctly. (szetszwo)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/branches/branch-0.19@701439 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 16 سال پیش
والد
کامیت
ee5ec6b5ee
3فایلهای تغییر یافته به همراه19 افزوده شده و 20 حذف شده
  1. 3 0
      CHANGES.txt
  2. 12 17
      src/core/org/apache/hadoop/fs/ChecksumFileSystem.java
  3. 4 3
      src/test/org/apache/hadoop/hdfs/TestFSInputChecker.java

+ 3 - 0
CHANGES.txt

@@ -1635,6 +1635,9 @@ Release 0.17.3 - Unreleased
 
     HADOOP-4318. DistCp should use absolute paths for cleanup.  (szetszwo)
 
+    HADOOP-4326. ChecksumFileSystem does not override create(...) correctly.
+    (szetszwo)
+
 Release 0.17.2 - 2008-08-11
 
   BUG FIXES

+ 12 - 17
src/core/org/apache/hadoop/fs/ChecksumFileSystem.java

@@ -25,6 +25,7 @@ import java.util.zip.CRC32;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.StringUtils;
 
@@ -341,28 +342,22 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
     }
   }
 
-  /**
-   * Opens an FSDataOutputStream at the indicated Path with write-progress
-   * reporting.
-   * @param f the file name to open
-   * @param overwrite if a file with this name already exists, then if true,
-   *   the file will be overwritten, and if false an error will be thrown.
-   * @param bufferSize the size of the buffer to be used.
-   * @param replication required block replication for the file. 
-   */
+  /** {@inheritDoc} */
   @Override
-  public FSDataOutputStream create(Path f, boolean overwrite, int bufferSize,
-                                   short replication, long blockSize, Progressable progress)
-    throws IOException {
+  public FSDataOutputStream create(Path f, FsPermission permission,
+      boolean overwrite, int bufferSize, short replication, long blockSize,
+      Progressable progress) throws IOException {
     Path parent = f.getParent();
     if (parent != null && !mkdirs(parent)) {
       throw new IOException("Mkdirs failed to create " + parent);
     }
-    return new FSDataOutputStream
-            (new ChecksumFSOutputSummer
-                (this, f, overwrite, bufferSize, replication, 
-                 blockSize, progress),
-             null);
+    final FSDataOutputStream out = new FSDataOutputStream(
+        new ChecksumFSOutputSummer(this, f, overwrite, bufferSize, replication,
+            blockSize, progress), null);
+    if (permission != null) {
+      setPermission(f, permission);
+    }
+    return out;
   }
 
   /**

+ 4 - 3
src/test/org/apache/hadoop/hdfs/TestFSInputChecker.java

@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.IOUtils;
 
 /**
@@ -48,9 +49,9 @@ public class TestFSInputChecker extends TestCase {
   /* create a file */
   private void writeFile(FileSystem fileSys, Path name) throws IOException {
     // create and write a file that contains three blocks of data
-    FSDataOutputStream stm = fileSys.create(name, true, 
-                     fileSys.getConf().getInt("io.file.buffer.size", 4096),
-                     NUM_OF_DATANODES, BLOCK_SIZE);
+    FSDataOutputStream stm = fileSys.create(name, new FsPermission((short)0777),
+        true, fileSys.getConf().getInt("io.file.buffer.size", 4096),
+        (short)NUM_OF_DATANODES, BLOCK_SIZE, null);
     stm.write(expected);
     stm.close();
   }