hadoop-common-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From dhr...@apache.org
Subject svn commit: r937092 - in /hadoop/common/trunk: CHANGES.txt src/java/org/apache/hadoop/fs/FilterFileSystem.java src/test/core/org/apache/hadoop/fs/TestFilterFileSystem.java
Date Thu, 22 Apr 2010 22:40:24 GMT
Author: dhruba
Date: Thu Apr 22 22:40:23 2010
New Revision: 937092

URL: http://svn.apache.org/viewvc?rev=937092&view=rev
Log:
HADOOP-6690. FilterFileSystem correctly handles setTimes call.
(Rodrigo Schmidt via dhruba)


Added:
    hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/TestFilterFileSystem.java
Modified:
    hadoop/common/trunk/CHANGES.txt
    hadoop/common/trunk/src/java/org/apache/hadoop/fs/FilterFileSystem.java

Modified: hadoop/common/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/CHANGES.txt?rev=937092&r1=937091&r2=937092&view=diff
==============================================================================
--- hadoop/common/trunk/CHANGES.txt (original)
+++ hadoop/common/trunk/CHANGES.txt Thu Apr 22 22:40:23 2010
@@ -333,6 +333,9 @@ Trunk (unreleased changes)
     HADOOP-6439. Fixes handling of deprecated keys to follow order in which
     keys are defined. (V.V.Chaitanya Krishna via yhemanth)
 
+    HADOOP-6690. FilterFileSystem correctly handles setTimes call.
+    (Rodrigo Schmidt via dhruba)
+
 Release 0.21.0 - Unreleased
 
   INCOMPATIBLE CHANGES

Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/FilterFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/FilterFileSystem.java?rev=937092&r1=937091&r2=937092&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/FilterFileSystem.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/FilterFileSystem.java Thu Apr 22 22:40:23
2010
@@ -139,6 +139,23 @@ public class FilterFileSystem extends Fi
     return fs.delete(f, recursive);
   }
   
+  /**
+   * Mark a path to be deleted when FileSystem is closed.
+   * When the JVM shuts down,
+   * all FileSystem objects will be closed automatically.
+   * Then,
+   * the marked path will be deleted as a result of closing the FileSystem.
+   *
+   * The path has to exist in the file system.
+   * 
+   * @param f the path to delete.
+   * @return  true if deleteOnExit is successful, otherwise false.
+   * @throws IOException
+   */
+  public boolean deleteOnExit(Path f) throws IOException {
+    return fs.deleteOnExit(f);
+  }    
+
   /** List files in a directory. */
   public FileStatus[] listStatus(Path f) throws IOException {
     return fs.listStatus(f);
@@ -195,6 +212,28 @@ public class FilterFileSystem extends Fi
   }
   
   /**
+   * The src files are on the local disk.  Add it to FS at
+   * the given dst name.
+   * delSrc indicates if the source should be removed
+   */
+  public void copyFromLocalFile(boolean delSrc, boolean overwrite, 
+                                Path[] srcs, Path dst)
+    throws IOException {
+    fs.copyFromLocalFile(delSrc, overwrite, srcs, dst);
+  }
+  
+  /**
+   * The src file is on the local disk.  Add it to FS at
+   * the given dst name.
+   * delSrc indicates if the source should be removed
+   */
+  public void copyFromLocalFile(boolean delSrc, boolean overwrite, 
+                                Path src, Path dst)
+    throws IOException {
+    fs.copyFromLocalFile(delSrc, overwrite, src, dst);
+  }
+
+  /**
    * The src file is under FS, and the dst is on the local disk.
    * Copy it from FS control to the local dst name.
    * delSrc indicates if the src will be removed or not.
@@ -226,6 +265,11 @@ public class FilterFileSystem extends Fi
     fs.completeLocalOutput(fsOutputFile, tmpLocalFile);
   }
 
+  /** Return the total size of all files in the filesystem.*/
+  public long getUsed() throws IOException{
+    return fs.getUsed();
+  }
+  
   /** Return the number of bytes that large input files should be optimally
    * be split into to minimize i/o time. */
   public long getDefaultBlockSize() {
@@ -276,6 +320,13 @@ public class FilterFileSystem extends Fi
 
   /** {@inheritDoc} */
   @Override
+  public void setTimes(Path p, long mtime, long atime
+      ) throws IOException {
+    fs.setTimes(p, mtime, atime);
+  }
+
+  /** {@inheritDoc} */
+  @Override
   public void setPermission(Path p, FsPermission permission
       ) throws IOException {
     fs.setPermission(p, permission);

Added: hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/TestFilterFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/TestFilterFileSystem.java?rev=937092&view=auto
==============================================================================
--- hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/TestFilterFileSystem.java (added)
+++ hadoop/common/trunk/src/test/core/org/apache/hadoop/fs/TestFilterFileSystem.java Thu Apr
22 22:40:23 2010
@@ -0,0 +1,144 @@
+/**
+ * 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.fs;
+
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.EnumSet;
+
+import junit.framework.TestCase;
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.Options.CreateOpts;
+import org.apache.hadoop.fs.Options.Rename;
+import org.apache.hadoop.util.Progressable;
+
+public class TestFilterFileSystem extends TestCase {
+
+  private static final Log LOG = FileSystem.LOG;
+
+  public static class DontCheck {
+    public BlockLocation[] getFileBlockLocations(Path p, 
+        long start, long len) { return null; }
+    public FsServerDefaults getServerDefaults() { return null; }
+    public long getLength(Path f) { return 0; }
+    public FSDataOutputStream append(Path f) { return null; }
+    public FSDataOutputStream append(Path f, int bufferSize) { return null; }
+    public void rename(final Path src, final Path dst, final Rename... options) { }
+    public boolean exists(Path f) { return false; }
+    public boolean isDirectory(Path f) { return false; }
+    public boolean isFile(Path f) { return false; }
+    public boolean createNewFile(Path f) { return false; }
+    public boolean mkdirs(Path f) { return false; }
+    public FSDataInputStream open(Path f) { return null; }
+    public FSDataOutputStream create(Path f) { return null; }
+    public FSDataOutputStream create(Path f, boolean overwrite) { return null; }
+    public FSDataOutputStream create(Path f, Progressable progress) {
+      return null;
+    }
+    public FSDataOutputStream create(Path f, short replication) {
+      return null;
+    }
+    public FSDataOutputStream create(Path f, short replication, 
+        Progressable progress) {
+      return null;
+    }
+    public FSDataOutputStream create(Path f, 
+        boolean overwrite,
+        int bufferSize) {
+      return null;
+    }
+    public FSDataOutputStream create(Path f, 
+        boolean overwrite,
+        int bufferSize,
+        Progressable progress) {
+      return null;
+    }
+    public FSDataOutputStream create(Path f, 
+        boolean overwrite,
+        int bufferSize,
+        short replication,
+        long blockSize) {
+      return null;
+    }
+    public FSDataOutputStream create(Path f,
+        boolean overwrite,
+        int bufferSize,
+        short replication,
+        long blockSize,
+        Progressable progress) {
+      return null;
+    }
+    public FSDataOutputStream create(Path f,
+        FsPermission permission,
+        boolean overwrite,
+        int bufferSize,
+        short replication,
+        long blockSize,
+        Progressable progress) {
+      return null;
+    }
+    public short getReplication(Path src) { return 0 ; }
+    public void processDeleteOnExit() { }
+    public ContentSummary getContentSummary(Path f) { return null; }
+    public FsStatus getStatus() { return null; }
+    public FileStatus[] listStatus(Path f, PathFilter filter) { return null; }
+    public FileStatus[] listStatus(Path[] files) { return null; }
+    public FileStatus[] listStatus(Path[] files, PathFilter filter) { return null; }
+    public FileStatus[] globStatus(Path pathPattern) { return null; }
+    public FileStatus[] globStatus(Path pathPattern, PathFilter filter) {
+      return null;
+    }
+    public void copyFromLocalFile(Path src, Path dst) { }
+    public void moveFromLocalFile(Path[] srcs, Path dst) { }
+    public void moveFromLocalFile(Path src, Path dst) { }
+    public void copyToLocalFile(Path src, Path dst) { }
+    public void moveToLocalFile(Path src, Path dst) { }
+    public long getBlockSize(Path f) { return 0; }
+    public FSDataOutputStream primitiveCreate(final Path f,
+        final EnumSet<CreateFlag> createFlag,
+        CreateOpts... opts) { return null; }
+    public void primitiveMkdir(Path f, FsPermission absolutePermission, 
+                      boolean createParent) { }
+  } 
+  
+  public void testFilterFileSystem() throws Exception {
+    for (Method m : FileSystem.class.getDeclaredMethods()) {
+      if (Modifier.isStatic(m.getModifiers()))
+        continue;
+      if (Modifier.isPrivate(m.getModifiers()))
+        continue;
+      
+      try {
+        DontCheck.class.getMethod(m.getName(), m.getParameterTypes());
+        LOG.info("Skipping " + m);
+      } catch (NoSuchMethodException exc) {
+        LOG.info("Testing " + m);
+        try{
+          FilterFileSystem.class.getDeclaredMethod(m.getName(), m.getParameterTypes());
+        }
+        catch(NoSuchMethodException exc2){
+          LOG.error("FilterFileSystem doesn't implement " + m);
+          throw exc2;
+        }
+      }
+    }
+  }
+  
+}



Mime
View raw message