Pārlūkot izejas kodu

HADOOP-7363. TestRawLocalFileSystemContract is needed. Contributed by Andras Bokor.

Anu Engineer 8 gadi atpakaļ
vecāks
revīzija
e1ad598cef

+ 32 - 14
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java

@@ -172,22 +172,39 @@ public abstract class FileSystemContractBaseTest extends TestCase {
   }
 
   public void testMkdirsWithUmask() throws Exception {
-    if (fs.getScheme().equals("s3n")) {
-      // skip permission tests for S3FileSystem until HDFS-1333 is fixed.
-      return;
+    if (!isS3(fs)) {
+      Configuration conf = fs.getConf();
+      String oldUmask = conf.get(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY);
+      try {
+        conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, TEST_UMASK);
+        final Path dir = path("/test/newDir");
+        assertTrue(fs.mkdirs(dir, new FsPermission((short) 0777)));
+        FileStatus status = fs.getFileStatus(dir);
+        assertTrue(status.isDirectory());
+        assertEquals((short) 0715, status.getPermission().toShort());
+      } finally {
+        conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, oldUmask);
+      }
     }
-    Configuration conf = fs.getConf();
-    String oldUmask = conf.get(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY);
+  }
+
+  /**
+   * Skip permission tests for S3FileSystem until HDFS-1333 is fixed.
+   * Classes that do not implement {@link FileSystem#getScheme()} method
+   * (e.g {@link RawLocalFileSystem}) will throw an
+   * {@link UnsupportedOperationException}.
+   * @param fileSystem FileSystem object to determine if it is S3 or not
+   * @return true if S3 false in any other case
+   */
+  private boolean isS3(FileSystem fileSystem) {
     try {
-      conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, TEST_UMASK);
-      final Path dir = new Path("/test/newDir");
-      assertTrue(fs.mkdirs(dir, new FsPermission((short)0777)));
-      FileStatus status = fs.getFileStatus(dir);
-      assertTrue(status.isDirectory());
-      assertEquals((short)0715, status.getPermission().toShort());
-    } finally {
-      conf.set(CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY, oldUmask);
+      if (fileSystem.getScheme().equals("s3n")) {
+        return true;
+      }
+    } catch (UnsupportedOperationException e) {
+      LOG.warn("Unable to determine the schema of filesystem.");
     }
+    return false;
   }
 
   public void testGetFileStatusThrowsExceptionForNonExistentFile() 
@@ -480,7 +497,8 @@ public abstract class FileSystemContractBaseTest extends TestCase {
   }
   
   protected Path path(String pathString) {
-    return new Path(pathString).makeQualified(fs);
+    return new Path(pathString).makeQualified(fs.getUri(),
+        fs.getWorkingDirectory());
   }
   
   protected void createFile(Path path) throws IOException {

+ 75 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestRawLocalFileSystemContract.java

@@ -0,0 +1,75 @@
+/**
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.util.Shell;
+import org.junit.Before;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test filesystem contracts with {@link RawLocalFileSystem}.
+ * Root directory related tests from super class will work into target
+ * directory since we have no permission to write / on local filesystem.
+ */
+public class TestRawLocalFileSystemContract extends FileSystemContractBaseTest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestRawLocalFileSystemContract.class);
+
+  @Before
+  public void setUp() throws Exception {
+    Configuration conf = new Configuration();
+    fs = FileSystem.getLocal(conf).getRawFileSystem();
+  }
+
+  /**
+   * Actually rename is supported in RawLocalFileSystem but
+   * it works different as the other filesystems. Short term we do not test it.
+   * Please check HADOOP-13082.
+   * @return true if rename supported so rename related tests will run
+   */
+  @Override
+  protected boolean renameSupported() {
+    return false;
+  }
+
+  @Override
+  public String getDefaultWorkingDirectory() {
+    return fs.getWorkingDirectory().toUri().getPath();
+  }
+
+  @Override
+  protected Path path(String pathString) {
+    // For testWorkingDirectory
+    if (pathString.equals(getDefaultWorkingDirectory()) ||
+        pathString.equals(".") || pathString.equals("..")) {
+      return super.path(pathString);
+    }
+
+    return new Path(GenericTestUtils.getTempPath(pathString)).
+        makeQualified(fs.getUri(), fs.getWorkingDirectory());
+  }
+
+  @Override
+  protected boolean filesystemIsCaseSensitive() {
+    return !(Shell.WINDOWS || Shell.MAC);
+  }
+}