You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ae...@apache.org on 2017/12/19 03:28:52 UTC

[43/51] [abbrv] hadoop git commit: HADOOP-15106. FileSystem::open(PathHandle) should throw a specific exception on validation failure

HADOOP-15106. FileSystem::open(PathHandle) should throw a specific exception on validation failure


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5e81f32d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5e81f32d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5e81f32d

Branch: refs/heads/HDFS-7240
Commit: 5e81f32d1155ea96c892099008cfeb50799082eb
Parents: fc7ec80
Author: Chris Douglas <cd...@apache.org>
Authored: Sat Dec 16 10:53:10 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Sat Dec 16 10:53:10 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/FileSystem.java   |  6 +++
 .../hadoop/fs/InvalidPathHandleException.java   | 46 ++++++++++++++++++++
 .../src/site/markdown/filesystem/filesystem.md  |  2 +-
 .../fs/contract/AbstractContractOpenTest.java   |  7 +--
 .../hadoop/hdfs/DistributedFileSystem.java      |  3 ++
 .../hadoop/hdfs/protocol/HdfsPathHandle.java    | 16 +++----
 6 files changed, 67 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e81f32d/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
index a364921..6b7dead 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
@@ -957,6 +957,8 @@ public abstract class FileSystem extends Configured implements Closeable {
    * resource directly and verify that the resource referenced
    * satisfies constraints specified at its construciton.
    * @param fd PathHandle object returned by the FS authority.
+   * @throws InvalidPathHandleException If {@link PathHandle} constraints are
+   *                                    not satisfied
    * @throws IOException IO failure
    * @throws UnsupportedOperationException If {@link #open(PathHandle, int)}
    *                                       not overridden by subclass
@@ -973,6 +975,8 @@ public abstract class FileSystem extends Configured implements Closeable {
    * satisfies constraints specified at its construciton.
    * @param fd PathHandle object returned by the FS authority.
    * @param bufferSize the size of the buffer to use
+   * @throws InvalidPathHandleException If {@link PathHandle} constraints are
+   *                                    not satisfied
    * @throws IOException IO failure
    * @throws UnsupportedOperationException If not overridden by subclass
    */
@@ -994,6 +998,8 @@ public abstract class FileSystem extends Configured implements Closeable {
    *         the specified constraints.
    */
   public final PathHandle getPathHandle(FileStatus stat, HandleOpt... opt) {
+    // method is final with a default so clients calling getPathHandle(stat)
+    // get the same semantics for all FileSystem implementations
     if (null == opt || 0 == opt.length) {
       return createPathHandle(stat, HandleOpt.path());
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e81f32d/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/InvalidPathHandleException.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/InvalidPathHandleException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/InvalidPathHandleException.java
new file mode 100644
index 0000000..8e26ea7
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/InvalidPathHandleException.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;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown when the constraints enoded in a {@link PathHandle} do not hold.
+ * For example, if a handle were created with the default
+ * {@link Options.HandleOpt#path()} constraints, a call to
+ * {@link FileSystem#open(PathHandle)} would succeed if the file were
+ * modified, but if a different file was at that location then it would throw
+ * this exception.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class InvalidPathHandleException extends IOException {
+  private static final long serialVersionUID = 0xcd8ac329L;
+
+  public InvalidPathHandleException(String str) {
+    super(str);
+  }
+
+  public InvalidPathHandleException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e81f32d/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
index c0e2a2c..2637f54 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/filesystem.md
@@ -790,7 +790,7 @@ Implementaions without a compliant call MUST throw `UnsupportedOperationExceptio
       (FS.Directories', FS.Files', FS.Symlinks')
       p' in FS.Files' where:
         FS.Files'[p'] = fd
-    if not exists(FS', p') : raise FileNotFoundException
+    if not exists(FS', p') : raise InvalidPathHandleException
 
 The implementation MUST resolve the referent of the `PathHandle` following
 the constraints specified at its creation by `getPathHandle(FileStatus)`.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e81f32d/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java
index 7528312..ab179eb 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractOpenTest.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.InvalidPathHandleException;
 import org.apache.hadoop.fs.Options.HandleOpt;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathHandle;
@@ -241,7 +242,7 @@ public abstract class AbstractContractOpenTest
       // fail to resolve if path1 had been modified
       instream = getFileSystem().open(fd1);
       fail("Expected an exception");
-    } catch (IOException e) {
+    } catch (InvalidPathHandleException e) {
       // expected
     }
 
@@ -290,7 +291,7 @@ public abstract class AbstractContractOpenTest
       // handle should not resolve when content changed
       instream = getFileSystem().open(fd);
       fail("Failed to detect change to content");
-    } catch (IOException e) {
+    } catch (InvalidPathHandleException e) {
       // expected
     }
   }
@@ -330,7 +331,7 @@ public abstract class AbstractContractOpenTest
       // verify attempt to resolve the handle fails
       instream = getFileSystem().open(fd1);
       fail("Expected an exception");
-    } catch (IOException e) {
+    } catch (InvalidPathHandleException e) {
       // expected
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e81f32d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 2449fb6..6b0c57a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.FsStatus;
 import org.apache.hadoop.fs.GlobalStorageStatistics;
 import org.apache.hadoop.fs.GlobalStorageStatistics.StorageStatisticsProvider;
+import org.apache.hadoop.fs.InvalidPathHandleException;
 import org.apache.hadoop.fs.PathHandle;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Options;
@@ -336,6 +337,8 @@ public class DistributedFileSystem extends FileSystem
    * the {@link PathHandle}.
    * @param fd Reference to entity in this FileSystem.
    * @param bufferSize the size of the buffer to be used.
+   * @throws InvalidPathHandleException If PathHandle constraints do not hold
+   * @throws IOException On I/O errors
    */
   @Override
   public FSDataInputStream open(PathHandle fd, int bufferSize)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e81f32d/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsPathHandle.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsPathHandle.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsPathHandle.java
index f80a067..14661ae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsPathHandle.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsPathHandle.java
@@ -23,6 +23,7 @@ import java.util.Optional;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.InvalidPathHandleException;
 import org.apache.hadoop.fs.PathHandle;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsPathHandleProto;
 
@@ -55,24 +56,21 @@ public final class HdfsPathHandle implements PathHandle {
     HdfsPathHandleProto p =
         HdfsPathHandleProto.parseFrom(ByteString.copyFrom(bytes));
     path = p.getPath();
-    mtime = p.hasMtime()
-        ? p.getMtime()
-        : null;
-    inodeId = p.hasInodeId()
-        ? p.getInodeId()
-        : null;
+    mtime   = p.hasMtime()   ? p.getMtime()   : null;
+    inodeId = p.hasInodeId() ? p.getInodeId() : null;
   }
 
   public String getPath() {
     return path;
   }
 
-  public void verify(HdfsLocatedFileStatus stat) throws IOException {
+  public void verify(HdfsLocatedFileStatus stat)
+      throws InvalidPathHandleException {
     if (mtime != null && mtime != stat.getModificationTime()) {
-      throw new IOException("Content changed");
+      throw new InvalidPathHandleException("Content changed");
     }
     if (inodeId != null && inodeId != stat.getFileId()) {
-      throw new IOException("Wrong file");
+      throw new InvalidPathHandleException("Wrong file");
     }
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org