Ver Fonte

HADOOP-12258. Need translate java.nio.file.NoSuchFileException to FileNotFoundException to avoid regression. Contributed by Zhihai Xu.

(cherry picked from commit 6cc8e38db5b26bdd02bc6bc1c9684db2593eec25)
cnauroth há 10 anos atrás
pai
commit
bf8b09d4b8
14 ficheiros alterados com 415 adições e 11 exclusões
  1. 3 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 25 11
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
  3. 61 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetFileStatusTest.java
  4. 61 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractSetTimesTest.java
  5. 12 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractOptions.java
  6. 33 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractGetFileStatus.java
  7. 33 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractSetTimes.java
  8. 33 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractGetFileStatus.java
  9. 33 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractSetTimes.java
  10. 10 0
      hadoop-common-project/hadoop-common/src/test/resources/contract/localfs.xml
  11. 10 0
      hadoop-common-project/hadoop-common/src/test/resources/contract/rawlocal.xml
  12. 46 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractGetFileStatus.java
  13. 45 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractSetTimes.java
  14. 10 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/contract/hdfs.xml

+ 3 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -558,6 +558,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12302. Fix native compilation on Windows after HADOOP-7824
     (Vinayakumar B via Colin P. McCabe)
 
+    HADOOP-12258. Need translate java.nio.file.NoSuchFileException to
+    FileNotFoundException to avoid regression. (Zhihai Xu via cnauroth)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 25 - 11
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java

@@ -34,6 +34,7 @@ import java.io.FileDescriptor;
 import java.net.URI;
 import java.nio.ByteBuffer;
 import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
 import java.nio.file.attribute.BasicFileAttributes;
 import java.nio.file.attribute.BasicFileAttributeView;
 import java.nio.file.attribute.FileTime;
@@ -637,13 +638,22 @@ public class RawLocalFileSystem extends FileSystem {
     private boolean isPermissionLoaded() {
       return !super.getOwner().isEmpty(); 
     }
-    
+
+    private static long getLastAccessTime(File f) throws IOException {
+      long accessTime;
+      try {
+        accessTime = Files.readAttributes(f.toPath(),
+            BasicFileAttributes.class).lastAccessTime().toMillis();
+      } catch (NoSuchFileException e) {
+        throw new FileNotFoundException("File " + f + " does not exist");
+      }
+      return accessTime;
+    }
+
     DeprecatedRawLocalFileStatus(File f, long defaultBlockSize, FileSystem fs)
       throws IOException {
       super(f.length(), f.isDirectory(), 1, defaultBlockSize,
-          f.lastModified(),
-          Files.readAttributes(f.toPath(),
-            BasicFileAttributes.class).lastAccessTime().toMillis(),
+          f.lastModified(), getLastAccessTime(f),
           null, null, null,
           new Path(f.getPath()).makeQualified(fs.getUri(),
             fs.getWorkingDirectory()));
@@ -760,17 +770,21 @@ public class RawLocalFileSystem extends FileSystem {
    * Sets the {@link Path}'s last modified time and last access time to
    * the given valid times.
    *
-   * @param mtime the modification time to set (only if greater than zero).
-   * @param atime the access time to set (only if greater than zero).
+   * @param mtime the modification time to set (only if no less than zero).
+   * @param atime the access time to set (only if no less than zero).
    * @throws IOException if setting the times fails.
    */
   @Override
   public void setTimes(Path p, long mtime, long atime) throws IOException {
-    BasicFileAttributeView view = Files.getFileAttributeView(
-        pathToFile(p).toPath(), BasicFileAttributeView.class);
-    FileTime fmtime = (mtime >= 0) ? FileTime.fromMillis(mtime) : null;
-    FileTime fatime = (atime >= 0) ? FileTime.fromMillis(atime) : null;
-    view.setTimes(fmtime, fatime, null);
+    try {
+      BasicFileAttributeView view = Files.getFileAttributeView(
+          pathToFile(p).toPath(), BasicFileAttributeView.class);
+      FileTime fmtime = (mtime >= 0) ? FileTime.fromMillis(mtime) : null;
+      FileTime fatime = (atime >= 0) ? FileTime.fromMillis(atime) : null;
+      view.setTimes(fmtime, fatime, null);
+    } catch (NoSuchFileException e) {
+      throw new FileNotFoundException("File " + p + " does not exist");
+    }
   }
 
   @Override

+ 61 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetFileStatusTest.java

@@ -0,0 +1,61 @@
+/*
+ * 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.contract;
+
+import java.io.FileNotFoundException;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test getFileStatus -if supported
+ */
+public abstract class AbstractContractGetFileStatusTest extends
+    AbstractFSContractTestBase {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractContractGetFileStatusTest.class);
+
+  private Path testPath;
+  private Path target;
+
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    skipIfUnsupported(SUPPORTS_GETFILESTATUS);
+
+    //delete the test directory
+    testPath = path("test");
+    target = new Path(testPath, "target");
+  }
+
+  @Test
+  public void testGetFileStatusNonexistentFile() throws Throwable {
+    try {
+      FileStatus status = getFileSystem().getFileStatus(target);
+      //got here: trouble
+      fail("expected a failure");
+    } catch (FileNotFoundException e) {
+      //expected
+      handleExpectedException(e);
+    }
+  }
+}

+ 61 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractSetTimesTest.java

@@ -0,0 +1,61 @@
+/*
+ * 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.contract;
+
+import java.io.FileNotFoundException;
+
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test setTimes -if supported
+ */
+public abstract class AbstractContractSetTimesTest extends
+    AbstractFSContractTestBase {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AbstractContractSetTimesTest.class);
+
+  private Path testPath;
+  private Path target;
+
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    skipIfUnsupported(SUPPORTS_SETTIMES);
+
+    //delete the test directory
+    testPath = path("test");
+    target = new Path(testPath, "target");
+  }
+
+  @Test
+  public void testSetTimesNonexistentFile() throws Throwable {
+    try {
+      long time = System.currentTimeMillis();
+      getFileSystem().setTimes(target, time, time);
+      //got here: trouble
+      fail("expected a failure");
+    } catch (FileNotFoundException e) {
+      //expected
+      handleExpectedException(e);
+    }
+  }
+}

+ 12 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractOptions.java

@@ -92,6 +92,18 @@ public interface ContractOptions {
    */
   String SUPPORTS_APPEND = "supports-append";
 
+  /**
+   * Flag to indicate that setTimes is supported.
+   * @{value}
+   */
+  String SUPPORTS_SETTIMES = "supports-settimes";
+
+  /**
+   * Flag to indicate that getFileStatus is supported.
+   * @{value}
+   */
+  String SUPPORTS_GETFILESTATUS = "supports-getfilestatus";
+
   /**
    * Flag to indicate that renames are atomic
    * @{value}

+ 33 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractGetFileStatus.java

@@ -0,0 +1,33 @@
+/*
+ * 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.contract.localfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestLocalFSContractGetFileStatus extends
+    AbstractContractGetFileStatusTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new LocalFSContract(conf);
+  }
+
+}

+ 33 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractSetTimes.java

@@ -0,0 +1,33 @@
+/*
+ * 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.contract.localfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractSetTimesTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestLocalFSContractSetTimes extends
+    AbstractContractSetTimesTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new LocalFSContract(conf);
+  }
+
+}

+ 33 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractGetFileStatus.java

@@ -0,0 +1,33 @@
+/*
+ * 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.contract.rawlocal;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestRawlocalContractGetFileStatus extends
+    AbstractContractGetFileStatusTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RawlocalFSContract(conf);
+  }
+
+}

+ 33 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractSetTimes.java

@@ -0,0 +1,33 @@
+/*
+ * 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.contract.rawlocal;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractSetTimesTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+public class TestRawlocalContractSetTimes extends
+    AbstractContractSetTimesTest {
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new RawlocalFSContract(conf);
+  }
+
+}

+ 10 - 0
hadoop-common-project/hadoop-common/src/test/resources/contract/localfs.xml

@@ -111,4 +111,14 @@ case sensitivity and permission options are determined at run time from OS type
     <value>false</value>
   </property>
 
+  <property>
+    <name>fs.contract.supports-settimes</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-getfilestatus</name>
+    <value>true</value>
+  </property>
+
 </configuration>

+ 10 - 0
hadoop-common-project/hadoop-common/src/test/resources/contract/rawlocal.xml

@@ -98,4 +98,14 @@
     <value>true</value>
   </property>
 
+  <property>
+    <name>fs.contract.supports-settimes</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-getfilestatus</name>
+    <value>true</value>
+  </property>
+
 </configuration>

+ 46 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractGetFileStatus.java

@@ -0,0 +1,46 @@
+/*
+ * 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.contract.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+public class TestHDFSContractGetFileStatus extends
+    AbstractContractGetFileStatusTest {
+
+  @BeforeClass
+  public static void createCluster() throws IOException {
+    HDFSContract.createCluster();
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    HDFSContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new HDFSContract(conf);
+  }
+}

+ 45 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractSetTimes.java

@@ -0,0 +1,45 @@
+/*
+ * 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.contract.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractSetTimesTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.IOException;
+
+public class TestHDFSContractSetTimes extends AbstractContractSetTimesTest {
+
+  @BeforeClass
+  public static void createCluster() throws IOException {
+    HDFSContract.createCluster();
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    HDFSContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new HDFSContract(conf);
+  }
+}

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/contract/hdfs.xml

@@ -91,4 +91,14 @@
     <value>true</value>
   </property>
 
+  <property>
+    <name>fs.contract.supports-settimes</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>fs.contract.supports-getfilestatus</name>
+    <value>true</value>
+  </property>
+
 </configuration>