瀏覽代碼

HADOOP-12726. Unsupported FS operations should throw UnsupportedOperationException. Contributed by Daniel Templeton.

Chris Douglas 8 年之前
父節點
當前提交
c37346d0e3

+ 4 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java

@@ -356,12 +356,14 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
   @Override
   public FSDataOutputStream append(Path f, int bufferSize,
       Progressable progress) throws IOException {
-    throw new IOException("Not supported");
+    throw new UnsupportedOperationException("Append is not supported "
+        + "by ChecksumFileSystem");
   }
 
   @Override
   public boolean truncate(Path f, long newLength) throws IOException {
-    throw new IOException("Not supported");
+    throw new UnsupportedOperationException("Truncate is not supported "
+        + "by ChecksumFileSystem");
   }
 
   /**

+ 2 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFs.java

@@ -297,7 +297,8 @@ public abstract class ChecksumFs extends FilterFs {
 
   @Override
   public boolean truncate(Path f, long newLength) throws IOException {
-    throw new IOException("Not supported");
+    throw new UnsupportedOperationException("Truncate is not supported "
+        + "by ChecksumFs");
   }
 
   /**

+ 2 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ftp/FTPFileSystem.java

@@ -288,7 +288,8 @@ public class FTPFileSystem extends FileSystem {
   @Override
   public FSDataOutputStream append(Path f, int bufferSize,
       Progressable progress) throws IOException {
-    throw new IOException("Not supported");
+    throw new UnsupportedOperationException("Append is not supported "
+        + "by FTPFileSystem");
   }
   
   /**

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPFileSystem.java

@@ -77,7 +77,6 @@ public class SFTPFileSystem extends FileSystem {
       "Can't make directory for path \"%s\" under \"%s\".";
   public static final String E_DIR_NOTEMPTY = "Directory: %s is not empty.";
   public static final String E_FILE_CHECK_FAILED = "File check failed";
-  public static final String E_NOT_SUPPORTED = "Not supported";
   public static final String E_SPATH_NOTEXIST = "Source path %s does not exist";
   public static final String E_DPATH_EXIST =
       "Destination path %s already exist, cannot rename!";
@@ -578,7 +577,8 @@ public class SFTPFileSystem extends FileSystem {
   public FSDataOutputStream append(Path f, int bufferSize,
       Progressable progress)
       throws IOException {
-    throw new IOException(E_NOT_SUPPORTED);
+    throw new UnsupportedOperationException("Append is not supported "
+        + "by SFTPFileSystem");
   }
 
   /*

+ 3 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/RollingFileSystemSink.java

@@ -485,10 +485,10 @@ public class RollingFileSystemSink implements MetricsSink, Closeable {
 
     try {
       fs.append(basePath);
+    } catch (UnsupportedOperationException ex) {
+      canAppend = false;
     } catch (IOException ex) {
-      if (ex.getMessage().equals("Not supported")) {
-        canAppend = false;
-      }
+      // Ignore. The operation is supported.
     }
 
     return canAppend;

+ 4 - 4
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md

@@ -611,7 +611,7 @@ this precondition fails.
 
 ### `FSDataOutputStream append(Path p, int bufferSize, Progressable progress)`
 
-Implementations MAY throw `UnsupportedOperationException`.
+Implementations without a compliant call SHOULD throw `UnsupportedOperationException`.
 
 #### Preconditions
 
@@ -630,7 +630,7 @@ by appending data to the existing list.
 
 ### `FSDataInputStream open(Path f, int bufferSize)`
 
-Implementations MAY throw `UnsupportedOperationException`.
+Implementations without a compliant call SHOULD throw `UnsupportedOperationException`.
 
 #### Preconditions
 
@@ -944,7 +944,7 @@ to the `DFSFileSystem` implementation is an ongoing matter for debate.
 Joins multiple blocks together to create a single file. This
 is a little-used operation currently implemented only by HDFS.
 
-Implementations MAY throw `UnsupportedOperationException`
+Implementations without a compliant call SHOULD throw `UnsupportedOperationException`.
 
 #### Preconditions
 
@@ -989,7 +989,7 @@ from specification.
 
 Truncate file `p` to the specified `newLength`.
 
-Implementations MAY throw `UnsupportedOperationException`.
+Implementations without a compliant call SHOULD throw `UnsupportedOperationException`.
 
 #### Preconditions
 

+ 2 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java

@@ -633,7 +633,8 @@ public class S3AFileSystem extends FileSystem {
    */
   public FSDataOutputStream append(Path f, int bufferSize,
       Progressable progress) throws IOException {
-    throw new IOException("Not supported");
+    throw new UnsupportedOperationException("Append is not supported "
+        + "by S3AFileSystem");
   }
 
 

+ 2 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java

@@ -421,7 +421,8 @@ public class NativeS3FileSystem extends FileSystem {
   @Override
   public FSDataOutputStream append(Path f, int bufferSize,
       Progressable progress) throws IOException {
-    throw new IOException("Not supported");
+    throw new UnsupportedOperationException("Append is not supported "
+        + "by NativeS3FileSystem");
   }
   
   @Override