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 xy...@apache.org on 2017/12/12 23:59:19 UTC

[44/50] hadoop git commit: HDFS-12882. Support full open(PathHandle) contract in HDFS

HDFS-12882. Support full open(PathHandle) contract in HDFS


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

Branch: refs/heads/HDFS-7240
Commit: 693169ef34f856a27dc09d90a45fb4ec5b66ed2c
Parents: 55fc2d6
Author: Chris Douglas <cd...@apache.org>
Authored: Mon Dec 11 20:14:15 2017 -0800
Committer: Chris Douglas <cd...@apache.org>
Committed: Mon Dec 11 20:14:15 2017 -0800

----------------------------------------------------------------------
 .../fs/contract/AbstractContractOpenTest.java   | 53 ++++++++++----
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 74 +++++++++++++++++---
 .../hadoop/hdfs/DistributedFileSystem.java      | 28 +++++---
 .../hadoop/hdfs/protocol/ClientProtocol.java    | 15 ++++
 .../hadoop/hdfs/protocol/HdfsPathHandle.java    | 65 ++++++++++++-----
 .../ClientNamenodeProtocolTranslatorPB.java     | 25 ++++++-
 .../hadoop/hdfs/protocolPB/PBHelperClient.java  | 15 ----
 .../src/main/proto/ClientNamenodeProtocol.proto | 11 +++
 .../src/main/proto/hdfs.proto                   |  2 +
 ...tNamenodeProtocolServerSideTranslatorPB.java | 22 ++++++
 .../federation/router/RouterRpcServer.java      | 13 ++++
 .../hdfs/server/namenode/FSDirAppendOp.java     |  3 +-
 .../hdfs/server/namenode/FSDirSnapshotOp.java   |  5 +-
 .../server/namenode/FSDirStatAndListingOp.java  | 41 ++++++-----
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |  2 +-
 .../hdfs/server/namenode/FSNamesystem.java      | 16 +++--
 .../hdfs/server/namenode/NameNodeRpcServer.java | 24 +++++--
 .../fsdataset/impl/LazyPersistTestCase.java     |  1 -
 .../hdfs/server/namenode/NameNodeAdapter.java   |  7 +-
 .../hdfs/server/namenode/TestBackupNode.java    |  8 ++-
 .../hdfs/server/namenode/TestEditLog.java       |  3 +-
 .../hdfs/server/namenode/TestReencryption.java  |  5 +-
 .../hdfs/server/namenode/TestSaveNamespace.java |  2 +-
 .../server/namenode/ha/TestEditLogTailer.java   |  4 +-
 .../namenode/ha/TestEditLogsDuringFailover.java | 15 ++--
 .../namenode/ha/TestFailureToReadEdits.java     | 12 ++--
 .../namenode/ha/TestInitializeSharedEdits.java  |  2 +-
 .../namenode/ha/TestStandbyInProgressTail.java  | 69 +++++++++---------
 .../src/test/resources/contract/hdfs.xml        |  2 +-
 29 files changed, 387 insertions(+), 157 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/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 9b6bbd4..7528312 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
@@ -218,14 +218,18 @@ public abstract class AbstractContractOpenTest
     Path path2 = path("testopenfilebyexact2");
     byte[] file1 = dataset(TEST_FILE_LEN, 43, 255);
     createFile(getFileSystem(), path1, false, file1);
-    FileStatus stat = getFileSystem().getFileStatus(path1);
-    assertNotNull(stat);
-    assertEquals(path1, stat.getPath());
+    FileStatus stat1 = getFileSystem().getFileStatus(path1);
+    assertNotNull(stat1);
+    assertEquals(path1, stat1.getPath());
     ContractTestUtils.rename(getFileSystem(), path1, path2);
+    FileStatus stat2 = getFileSystem().getFileStatus(path2);
+    assertNotNull(stat2);
+    assertEquals(path2, stat2.getPath());
     // create identical file at same location, orig still exists at path2
     createFile(getFileSystem(), path1, false, file1);
 
-    PathHandle fd = getHandleOrSkip(stat, HandleOpt.exact());
+    PathHandle fd1 = getHandleOrSkip(stat1, HandleOpt.exact());
+    PathHandle fd2 = getHandleOrSkip(stat2, HandleOpt.exact());
 
     // verify path1, path2 contents identical
     verifyFileContents(getFileSystem(), path1, file1);
@@ -235,11 +239,15 @@ public abstract class AbstractContractOpenTest
       // the original entity exists, it has not been modified, and an
       // identical file exists at the old path. The handle would also
       // fail to resolve if path1 had been modified
-      instream = getFileSystem().open(fd, 1 << 15);
+      instream = getFileSystem().open(fd1);
       fail("Expected an exception");
     } catch (IOException e) {
       // expected
     }
+
+    // verify unchanged resolves
+    instream = getFileSystem().open(fd2);
+    verifyRead(instream, file1, 0, TEST_FILE_LEN);
   }
 
   /**
@@ -265,7 +273,7 @@ public abstract class AbstractContractOpenTest
     // obtain handle to entity from #getFileStatus call
     PathHandle fd = getHandleOrSkip(stat, HandleOpt.content());
 
-    try (FSDataInputStream in = getFileSystem().open(fd, 1 << 15)) {
+    try (FSDataInputStream in = getFileSystem().open(fd)) {
       // verify read of consistent content at new location
       verifyRead(in, file1, 0, TEST_FILE_LEN);
     }
@@ -280,7 +288,7 @@ public abstract class AbstractContractOpenTest
 
     try {
       // handle should not resolve when content changed
-      instream = getFileSystem().open(fd, 1 << 15);
+      instream = getFileSystem().open(fd);
       fail("Failed to detect change to content");
     } catch (IOException e) {
       // expected
@@ -302,25 +310,40 @@ public abstract class AbstractContractOpenTest
 
     byte[] file1 = dataset(TEST_FILE_LEN, 43, 255);
     createFile(getFileSystem(), path1, false, file1);
-    FileStatus stat = getFileSystem().getFileStatus(path1);
-    assertNotNull(stat);
-    assertEquals(path1, stat.getPath());
+    FileStatus stat1 = getFileSystem().getFileStatus(path1);
+    assertNotNull(stat1);
+    assertEquals(path1, stat1.getPath());
     ContractTestUtils.rename(getFileSystem(), path1, path2);
+    FileStatus stat2 = getFileSystem().getFileStatus(path2);
+    assertNotNull(stat2);
+    assertEquals(path2, stat2.getPath());
     // create identical file at same location, orig still exists at path2
     createFile(getFileSystem(), path1, false, file1);
 
-    PathHandle fd = getHandleOrSkip(stat, HandleOpt.path());
+    PathHandle fd1 = getHandleOrSkip(stat1, HandleOpt.path());
+    PathHandle fd2 = getHandleOrSkip(stat2, HandleOpt.path());
 
     // verify path1, path2 contents identical
     verifyFileContents(getFileSystem(), path1, file1);
     verifyFileContents(getFileSystem(), path2, file1);
     try {
       // verify attempt to resolve the handle fails
-      instream = getFileSystem().open(fd, 1 << 15);
+      instream = getFileSystem().open(fd1);
       fail("Expected an exception");
     } catch (IOException e) {
       // expected
     }
+
+    // verify content change OK
+    byte[] file2a = dataset(TEST_FILE_LEN, 44, 255);
+    ContractTestUtils.appendFile(getFileSystem(), path2, file2a);
+    byte[] file2x = Arrays.copyOf(file1, file1.length + file2a.length);
+    System.arraycopy(file2a, 0, file2x, file1.length, file2a.length);
+    // verify path2 contains contents of orig + appended bytes
+    verifyFileContents(getFileSystem(), path2, file2x);
+    // verify open by fd succeeds
+    instream = getFileSystem().open(fd2);
+    verifyRead(instream, file2x, 0, 2 * TEST_FILE_LEN);
   }
 
   /**
@@ -357,8 +380,8 @@ public abstract class AbstractContractOpenTest
     verifyFileContents(getFileSystem(), path1, file2);
 
     // verify fd contains contents of file1 + appended bytes
-    instream = getFileSystem().open(fd, 1 << 15);
-    verifyRead(instream, file1x, 0, TEST_FILE_LEN);
+    instream = getFileSystem().open(fd);
+    verifyRead(instream, file1x, 0, 2 * TEST_FILE_LEN);
   }
 
   /**
@@ -388,7 +411,7 @@ public abstract class AbstractContractOpenTest
     ByteBuffer sb = fd.bytes();
     PathHandle fdb = new RawPathHandle(sb);
 
-    instream = getFileSystem().open(fdb, 1 << 15);
+    instream = getFileSystem().open(fdb);
     // verify stat contains contents of file1
     verifyRead(instream, file1, 0, TEST_FILE_LEN);
     // verify path2 contains contents of file1

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 51dff8d..59f553b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -125,6 +125,8 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
+import org.apache.hadoop.hdfs.protocol.HdfsPathHandle;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -1015,16 +1017,46 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     //    Get block info from namenode
     try (TraceScope ignored = newPathTraceScope("newDFSInputStream", src)) {
       LocatedBlocks locatedBlocks = getLocatedBlocks(src, 0);
-      if (locatedBlocks != null) {
-        ErasureCodingPolicy ecPolicy = locatedBlocks.getErasureCodingPolicy();
-        if (ecPolicy != null) {
-          return new DFSStripedInputStream(this, src, verifyChecksum, ecPolicy,
-              locatedBlocks);
-        }
-        return new DFSInputStream(this, src, verifyChecksum, locatedBlocks);
-      } else {
-        throw new IOException("Cannot open filename " + src);
+      return openInternal(locatedBlocks, src, verifyChecksum);
+    }
+  }
+
+  /**
+   * Create an input stream from the {@link HdfsPathHandle} if the
+   * constraints encoded from {@link
+   * DistributedFileSystem#createPathHandle(FileStatus, Options.HandleOpt...)}
+   * are satisfied. Note that HDFS does not ensure that these constraints
+   * remain invariant for the life of the stream. It only checks that they
+   * still held when the stream was opened.
+   * @param fd Handle to an entity in HDFS, with constraints
+   * @param buffersize ignored
+   * @param verifyChecksum Verify checksums before returning data to client
+   * @return Data from the referent of the {@link HdfsPathHandle}.
+   * @throws IOException On I/O error
+   */
+  public DFSInputStream open(HdfsPathHandle fd, int buffersize,
+      boolean verifyChecksum) throws IOException {
+    checkOpen();
+    String src = fd.getPath();
+    try (TraceScope ignored = newPathTraceScope("newDFSInputStream", src)) {
+      HdfsLocatedFileStatus s = getLocatedFileInfo(src, true);
+      fd.verify(s); // check invariants in path handle
+      LocatedBlocks locatedBlocks = s.getLocatedBlocks();
+      return openInternal(locatedBlocks, src, verifyChecksum);
+    }
+  }
+
+  private DFSInputStream openInternal(LocatedBlocks locatedBlocks, String src,
+      boolean verifyChecksum) throws IOException {
+    if (locatedBlocks != null) {
+      ErasureCodingPolicy ecPolicy = locatedBlocks.getErasureCodingPolicy();
+      if (ecPolicy != null) {
+        return new DFSStripedInputStream(this, src, verifyChecksum, ecPolicy,
+            locatedBlocks);
       }
+      return new DFSInputStream(this, src, verifyChecksum, locatedBlocks);
+    } else {
+      throw new IOException("Cannot open filename " + src);
     }
   }
 
@@ -1648,6 +1680,30 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   }
 
   /**
+   * Get the file info for a specific file or directory.
+   * @param src The string representation of the path to the file
+   * @param needBlockToken Include block tokens in {@link LocatedBlocks}.
+   *        When block tokens are included, this call is a superset of
+   *        {@link #getBlockLocations(String, long)}.
+   * @return object containing information regarding the file
+   *         or null if file not found
+   *
+   * @see DFSClient#open(HdfsPathHandle, int, boolean)
+   * @see ClientProtocol#getFileInfo(String) for description of
+   *      exceptions
+   */
+  public HdfsLocatedFileStatus getLocatedFileInfo(String src,
+      boolean needBlockToken) throws IOException {
+    checkOpen();
+    try (TraceScope ignored = newPathTraceScope("getLocatedFileInfo", src)) {
+      return namenode.getLocatedFileInfo(src, needBlockToken);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+          FileNotFoundException.class,
+          UnresolvedPathException.class);
+    }
+  }
+  /**
    * Close status of a file
    * @return true if file is already closed
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/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 1478868..2449fb6 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
@@ -115,7 +115,7 @@ import java.util.Collection;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
-import java.util.stream.Collectors;
+import java.util.Optional;
 
 /****************************************************************
  * Implementation of the abstract FileSystem for the DFS system.
@@ -340,11 +340,14 @@ public class DistributedFileSystem extends FileSystem
   @Override
   public FSDataInputStream open(PathHandle fd, int bufferSize)
       throws IOException {
+    statistics.incrementReadOps(1);
+    storageStatistics.incrementOpCounter(OpType.OPEN);
     if (!(fd instanceof HdfsPathHandle)) {
       fd = new HdfsPathHandle(fd.bytes());
     }
     HdfsPathHandle id = (HdfsPathHandle) fd;
-    return open(DFSUtilClient.makePathFromFileId(id.getInodeId()), bufferSize);
+    final DFSInputStream dfsis = dfs.open(id, bufferSize, verifyChecksum);
+    return dfs.createWrappedInputStream(dfsis);
   }
 
   /**
@@ -358,7 +361,7 @@ public class DistributedFileSystem extends FileSystem
    * @return A handle to the file.
    */
   @Override
-  protected PathHandle createPathHandle(FileStatus st, HandleOpt... opts) {
+  protected HdfsPathHandle createPathHandle(FileStatus st, HandleOpt... opts) {
     if (!(st instanceof HdfsFileStatus)) {
       throw new IllegalArgumentException("Invalid FileStatus "
           + st.getClass().getSimpleName());
@@ -373,12 +376,21 @@ public class DistributedFileSystem extends FileSystem
         .orElse(HandleOpt.changed(false));
     HandleOpt.Location loc = HandleOpt.getOpt(HandleOpt.Location.class, opts)
         .orElse(HandleOpt.moved(false));
-    if (!data.allowChange() || !loc.allowChange()) {
-      throw new UnsupportedOperationException("Unsupported opts "
-          + Arrays.stream(opts)
-                  .map(HandleOpt::toString).collect(Collectors.joining(",")));
+
+    HdfsFileStatus hst = (HdfsFileStatus) st;
+    final Path p;
+    final Optional<Long> inodeId;
+    if (loc.allowChange()) {
+      p = DFSUtilClient.makePathFromFileId(hst.getFileId());
+      inodeId = Optional.empty();
+    } else {
+      p = hst.getPath();
+      inodeId = Optional.of(hst.getFileId());
     }
-    return new HdfsPathHandle((HdfsFileStatus)st);
+    final Optional<Long> mtime = !data.allowChange()
+        ? Optional.of(hst.getModificationTime())
+        : Optional.empty();
+    return new HdfsPathHandle(getPathName(p), inodeId, mtime);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index eb2e11c..e8a33dd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -1025,6 +1025,21 @@ public interface ClientProtocol {
   HdfsFileStatus getFileLinkInfo(String src) throws IOException;
 
   /**
+   * Get the file info for a specific file or directory with
+   * {@link LocatedBlocks}.
+   * @param src The string representation of the path to the file
+   * @param needBlockToken Generate block tokens for {@link LocatedBlocks}
+   * @return object containing information regarding the file
+   *         or null if file not found
+   * @throws org.apache.hadoop.security.AccessControlException permission denied
+   * @throws java.io.FileNotFoundException file <code>src</code> is not found
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  HdfsLocatedFileStatus getLocatedFileInfo(String src, boolean needBlockToken)
+      throws IOException;
+
+  /**
    * Get {@link ContentSummary} rooted at the specified directory.
    * @param path The string representation of the path
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/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 2d8dcca..f80a067 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
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.protocol;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.Optional;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -34,16 +35,17 @@ import com.google.protobuf.ByteString;
 @InterfaceStability.Unstable
 public final class HdfsPathHandle implements PathHandle {
 
-  private static final long serialVersionUID = 0xc5308795428L;
+  private static final long serialVersionUID = 0xc53087a5428L;
 
-  private final long inodeId;
+  private final String path;
+  private final Long mtime;
+  private final Long inodeId;
 
-  public HdfsPathHandle(HdfsFileStatus hstat) {
-    this(hstat.getFileId());
-  }
-
-  public HdfsPathHandle(long inodeId) {
-    this.inodeId = inodeId;
+  public HdfsPathHandle(String path,
+      Optional<Long> inodeId, Optional<Long> mtime) {
+    this.path = path;
+    this.mtime = mtime.orElse(null);
+    this.inodeId = inodeId.orElse(null);
   }
 
   public HdfsPathHandle(ByteBuffer bytes) throws IOException {
@@ -52,20 +54,39 @@ public final class HdfsPathHandle implements PathHandle {
     }
     HdfsPathHandleProto p =
         HdfsPathHandleProto.parseFrom(ByteString.copyFrom(bytes));
-    inodeId = p.getInodeId();
+    path = p.getPath();
+    mtime = p.hasMtime()
+        ? p.getMtime()
+        : null;
+    inodeId = p.hasInodeId()
+        ? p.getInodeId()
+        : null;
   }
 
-  public long getInodeId() {
-    return inodeId;
+  public String getPath() {
+    return path;
+  }
+
+  public void verify(HdfsLocatedFileStatus stat) throws IOException {
+    if (mtime != null && mtime != stat.getModificationTime()) {
+      throw new IOException("Content changed");
+    }
+    if (inodeId != null && inodeId != stat.getFileId()) {
+      throw new IOException("Wrong file");
+    }
   }
 
   @Override
   public ByteBuffer bytes() {
-    return HdfsPathHandleProto.newBuilder()
-      .setInodeId(getInodeId())
-      .build()
-      .toByteString()
-      .asReadOnlyByteBuffer();
+    HdfsPathHandleProto.Builder b = HdfsPathHandleProto.newBuilder();
+    b.setPath(path);
+    if (inodeId != null) {
+      b.setInodeId(inodeId);
+    }
+    if (mtime != null) {
+      b.setMtime(mtime);
+    }
+    return b.build().toByteString().asReadOnlyByteBuffer();
   }
 
   @Override
@@ -78,19 +99,25 @@ public final class HdfsPathHandle implements PathHandle {
       return false;
     }
     HdfsPathHandle o = (HdfsPathHandle)other;
-    return getInodeId() == o.getInodeId();
+    return getPath().equals(o.getPath());
   }
 
   @Override
   public int hashCode() {
-    return Long.hashCode(inodeId);
+    return path.hashCode();
   }
 
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder();
     sb.append("{ ");
-    sb.append("inodeId : ").append(Long.toString(getInodeId()));
+    sb.append("\"path\" : \"").append(path).append("\"");
+    if (inodeId != null) {
+      sb.append(",\"inodeId\" : ").append(inodeId);
+    }
+    if (mtime != null) {
+      sb.append(",\"mtime\" : ").append(mtime);
+    }
     sb.append(" }");
     return sb.toString();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index 38dc44b..9ccc2fa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -71,6 +71,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -129,6 +130,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLin
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLinkTargetResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLocatedFileInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLocatedFileInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPreferredBlockSizeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetQuotaUsageRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetServerDefaultsRequestProto;
@@ -872,7 +875,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   public HdfsFileStatus getFileInfo(String src) throws IOException {
     GetFileInfoRequestProto req = GetFileInfoRequestProto.newBuilder()
-        .setSrc(src).build();
+        .setSrc(src)
+        .build();
     try {
       GetFileInfoResponseProto res = rpcProxy.getFileInfo(null, req);
       return res.hasFs() ? PBHelperClient.convert(res.getFs()) : null;
@@ -882,6 +886,25 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
+  public HdfsLocatedFileStatus getLocatedFileInfo(String src,
+      boolean needBlockToken) throws IOException {
+    GetLocatedFileInfoRequestProto req =
+        GetLocatedFileInfoRequestProto.newBuilder()
+            .setSrc(src)
+            .setNeedBlockToken(needBlockToken)
+            .build();
+    try {
+      GetLocatedFileInfoResponseProto res =
+          rpcProxy.getLocatedFileInfo(null, req);
+      return (HdfsLocatedFileStatus) (res.hasFs()
+          ? PBHelperClient.convert(res.getFs())
+          : null);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
   public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
     GetFileLinkInfoRequestProto req = GetFileLinkInfoRequestProto.newBuilder()
         .setSrc(src).build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
index fbc6dbf..73c5215 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
@@ -91,7 +91,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
-import org.apache.hadoop.hdfs.protocol.HdfsPathHandle;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
@@ -164,7 +163,6 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsServerDefaultsProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto.FileType;
-import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsPathHandleProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto.Builder;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlocksProto;
@@ -1624,19 +1622,6 @@ public class PBHelperClient {
     return FsPermissionProto.newBuilder().setPerm(p.toShort()).build();
   }
 
-  public static HdfsPathHandle convert(HdfsPathHandleProto fd) {
-    if (null == fd) {
-      return null;
-    }
-    return new HdfsPathHandle(fd.getInodeId());
-  }
-
-  public static HdfsPathHandleProto convert(HdfsPathHandle fd) {
-    return HdfsPathHandleProto.newBuilder()
-        .setInodeId(fd.getInodeId())
-        .build();
-  }
-
   public static HdfsFileStatus convert(HdfsFileStatusProto fs) {
     if (fs == null) {
       return null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
index eb6da25..b33462b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto
@@ -495,6 +495,15 @@ message GetFileInfoResponseProto {
   optional HdfsFileStatusProto fs = 1;
 }
 
+message GetLocatedFileInfoRequestProto {
+  optional string src = 1;
+  optional bool needBlockToken = 2 [default = false];
+}
+
+message GetLocatedFileInfoResponseProto {
+  optional HdfsFileStatusProto fs = 1;
+}
+
 message IsFileClosedRequestProto {
   required string src = 1;
 }
@@ -868,6 +877,8 @@ service ClientNamenodeProtocol {
       returns(ListCorruptFileBlocksResponseProto);
   rpc metaSave(MetaSaveRequestProto) returns(MetaSaveResponseProto);
   rpc getFileInfo(GetFileInfoRequestProto) returns(GetFileInfoResponseProto);
+  rpc getLocatedFileInfo(GetLocatedFileInfoRequestProto)
+      returns(GetLocatedFileInfoResponseProto);
   rpc addCacheDirective(AddCacheDirectiveRequestProto)
       returns (AddCacheDirectiveResponseProto);
   rpc modifyCacheDirective(ModifyCacheDirectiveRequestProto)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
index a423a4b..fca1e0e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
@@ -401,6 +401,8 @@ message AddErasureCodingPolicyResponseProto {
  */
 message HdfsPathHandleProto {
   optional uint64 inodeId = 1;
+  optional uint64 mtime = 2;
+  optional string path = 3;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index 2ae41e4..d63460b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -136,6 +136,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLin
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLinkTargetResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetListingResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLocatedFileInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetLocatedFileInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPreferredBlockSizeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetPreferredBlockSizeResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetQuotaUsageRequestProto;
@@ -344,6 +346,10 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   private static final GetFileInfoResponseProto VOID_GETFILEINFO_RESPONSE = 
   GetFileInfoResponseProto.newBuilder().build();
 
+  private static final GetLocatedFileInfoResponseProto
+      VOID_GETLOCATEDFILEINFO_RESPONSE =
+          GetLocatedFileInfoResponseProto.newBuilder().build();
+
   private static final GetFileLinkInfoResponseProto VOID_GETFILELINKINFO_RESPONSE = 
   GetFileLinkInfoResponseProto.newBuilder().build();
 
@@ -952,7 +958,23 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
+  public GetLocatedFileInfoResponseProto getLocatedFileInfo(
+      RpcController controller, GetLocatedFileInfoRequestProto req)
+      throws ServiceException {
+    try {
+      HdfsFileStatus result = server.getLocatedFileInfo(req.getSrc(),
+          req.getNeedBlockToken());
+      if (result != null) {
+        return GetLocatedFileInfoResponseProto.newBuilder().setFs(
+            PBHelperClient.convert(result)).build();
+      }
+      return VOID_GETLOCATEDFILEINFO_RESPONSE;
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
 
+  @Override
   public GetFileLinkInfoResponseProto getFileLinkInfo(RpcController controller,
       GetFileLinkInfoRequestProto req) throws ServiceException {
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
index d401e69..c6cd595 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java
@@ -85,6 +85,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -1071,6 +1072,18 @@ public class RouterRpcServer extends AbstractService implements ClientProtocol {
         locations, method, HdfsFileStatus.class, null);
   }
 
+  @Override
+  public HdfsLocatedFileStatus getLocatedFileInfo(String src,
+      boolean needBlockToken) throws IOException {
+    checkOperation(OperationCategory.READ);
+    final List<RemoteLocation> locations = getLocationsForPath(src, false);
+    RemoteMethod method = new RemoteMethod("getLocatedFileInfo",
+        new Class<?>[] {String.class, boolean.class}, new RemoteParam(),
+        Boolean.valueOf(needBlockToken));
+    return (HdfsLocatedFileStatus) rpcClient.invokeSequential(
+        locations, method, HdfsFileStatus.class, null);
+  }
+
   @Override // ClientProtocol
   public long[] getStats() throws IOException {
     checkOperation(OperationCategory.UNCHECKED);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java
index 9926ee0..be272d2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java
@@ -148,7 +148,8 @@ final class FSDirAppendOp {
       fsd.writeUnlock();
     }
 
-    HdfsFileStatus stat = FSDirStatAndListingOp.getFileInfo(fsd, iip);
+    HdfsFileStatus stat =
+        FSDirStatAndListingOp.getFileInfo(fsd, iip, false, false);
     if (lb != null) {
       NameNode.stateChangeLog.debug(
           "DIR* NameSystem.appendFile: file {} for {} at {} block {} block"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java
index 1842707..4dacbf2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.FSLimitException;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
@@ -213,7 +214,9 @@ class FSDirSnapshotOp {
           snapname += Path.SEPARATOR;
         }
         snapname += file.substring(file.indexOf(dirName) + dirName.length());
-        if (fsd.getFSNamesystem().getFileInfo(snapname, true) != null) {
+        HdfsFileStatus stat =
+            fsd.getFSNamesystem().getFileInfo(snapname, true, false, false);
+        if (stat != null) {
           snaps.add(snapname);
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
index 4da8a28..8b77034 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
@@ -90,11 +90,13 @@ class FSDirStatAndListingOp {
    * @param resolveLink whether to throw UnresolvedLinkException
    *        if src refers to a symlink
    *
+   * @param needLocation Include {@link LocatedBlocks} in result.
+   * @param needBlockToken Include block tokens in {@link LocatedBlocks}.
    * @return object containing information regarding the file
    *         or null if file not found
    */
-  static HdfsFileStatus getFileInfo(
-      FSDirectory fsd, String srcArg, boolean resolveLink)
+  static HdfsFileStatus getFileInfo(FSDirectory fsd, String srcArg,
+      boolean resolveLink, boolean needLocation, boolean needBlockToken)
       throws IOException {
     DirOp dirOp = resolveLink ? DirOp.READ : DirOp.READ_LINK;
     FSPermissionChecker pc = fsd.getPermissionChecker();
@@ -111,7 +113,7 @@ class FSDirStatAndListingOp {
     } else {
       iip = fsd.resolvePath(pc, srcArg, dirOp);
     }
-    return getFileInfo(fsd, iip);
+    return getFileInfo(fsd, iip, needLocation, needBlockToken);
   }
 
   /**
@@ -234,7 +236,7 @@ class FSDirStatAndListingOp {
         // target INode
         return new DirectoryListing(
             new HdfsFileStatus[]{ createFileStatus(
-                fsd, iip, null, parentStoragePolicy, needLocation)
+                fsd, iip, null, parentStoragePolicy, needLocation, false)
             }, 0);
       }
 
@@ -253,8 +255,8 @@ class FSDirStatAndListingOp {
             ? getStoragePolicyID(child.getLocalStoragePolicyID(),
                                  parentStoragePolicy)
             : parentStoragePolicy;
-        listing[i] =
-            createFileStatus(fsd, iip, child, childStoragePolicy, needLocation);
+        listing[i] = createFileStatus(fsd, iip, child, childStoragePolicy,
+            needLocation, false);
         listingCnt++;
         if (listing[i] instanceof HdfsLocatedFileStatus) {
             // Once we  hit lsLimit locations, stop.
@@ -305,7 +307,7 @@ class FSDirStatAndListingOp {
     for (int i = 0; i < numOfListing; i++) {
       Snapshot.Root sRoot = snapshots.get(i + skipSize).getRoot();
       listing[i] = createFileStatus(fsd, iip, sRoot,
-          HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, false);
+          HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, false, false);
     }
     return new DirectoryListing(
         listing, snapshots.size() - skipSize - numOfListing);
@@ -324,11 +326,14 @@ class FSDirStatAndListingOp {
    * @param fsd FSDirectory
    * @param iip The path to the file, the file is included
    * @param includeStoragePolicy whether to include storage policy
+   * @param needLocation Include {@link LocatedBlocks} in response
+   * @param needBlockToken Generate block tokens for {@link LocatedBlocks}
    * @return object containing information regarding the file
    *         or null if file not found
    */
-  static HdfsFileStatus getFileInfo(FSDirectory fsd,
-      INodesInPath iip, boolean includeStoragePolicy) throws IOException {
+  static HdfsFileStatus getFileInfo(FSDirectory fsd, INodesInPath iip,
+      boolean includeStoragePolicy, boolean needLocation,
+      boolean needBlockToken) throws IOException {
     fsd.readLock();
     try {
       final INode node = iip.getLastINode();
@@ -338,14 +343,15 @@ class FSDirStatAndListingOp {
       byte policy = (includeStoragePolicy && !node.isSymlink())
           ? node.getStoragePolicyID()
           : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
-      return createFileStatus(fsd, iip, null, policy, false);
+      return createFileStatus(fsd, iip, null, policy, needLocation,
+          needBlockToken);
     } finally {
       fsd.readUnlock();
     }
   }
 
-  static HdfsFileStatus getFileInfo(FSDirectory fsd, INodesInPath iip)
-    throws IOException {
+  static HdfsFileStatus getFileInfo(FSDirectory fsd, INodesInPath iip,
+      boolean needLocation, boolean needBlockToken) throws IOException {
     fsd.readLock();
     try {
       HdfsFileStatus status = null;
@@ -356,7 +362,7 @@ class FSDirStatAndListingOp {
           status = FSDirectory.DOT_SNAPSHOT_DIR_STATUS;
         }
       } else {
-        status = getFileInfo(fsd, iip, true);
+        status = getFileInfo(fsd, iip, true, needLocation, needBlockToken);
       }
       return status;
     } finally {
@@ -373,7 +379,7 @@ class FSDirStatAndListingOp {
   static HdfsFileStatus createFileStatusForEditLog(
       FSDirectory fsd, INodesInPath iip) throws IOException {
     return createFileStatus(fsd, iip,
-        null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, false);
+        null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, false, false);
   }
 
   /**
@@ -384,12 +390,13 @@ class FSDirStatAndListingOp {
    * @param child for a directory listing of the iip, else null
    * @param storagePolicy for the path or closest ancestor
    * @param needLocation if block locations need to be included or not
+   * @param needBlockToken
    * @return a file status
    * @throws java.io.IOException if any error occurs
    */
   private static HdfsFileStatus createFileStatus(
       FSDirectory fsd, INodesInPath iip, INode child, byte storagePolicy,
-      boolean needLocation) throws IOException {
+      boolean needLocation, boolean needBlockToken) throws IOException {
     assert fsd.hasReadLock();
     // only directory listing sets the status name.
     byte[] name = HdfsFileStatus.EMPTY_NAME;
@@ -429,8 +436,8 @@ class FSDirStatAndListingOp {
         final long fileSize = !inSnapshot && isUc
             ? fileNode.computeFileSizeNotIncludingLastUcBlock() : size;
         loc = fsd.getBlockManager().createLocatedBlocks(
-            fileNode.getBlocks(snapshot), fileSize, isUc, 0L, size, false,
-            inSnapshot, feInfo, ecPolicy);
+            fileNode.getBlocks(snapshot), fileSize, isUc, 0L, size,
+            needBlockToken, inSnapshot, feInfo, ecPolicy);
         if (loc == null) {
           loc = new LocatedBlocks();
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index c4041a3..8f34e1c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -408,7 +408,7 @@ class FSDirWriteFileOp {
       NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: added " +
           src + " inode " + newNode.getId() + " " + holder);
     }
-    return FSDirStatAndListingOp.getFileInfo(fsd, iip);
+    return FSDirStatAndListingOp.getFileInfo(fsd, iip, false, false);
   }
 
   static INodeFile addFileForEditLog(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 6a890e2..dedb737 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -2990,6 +2990,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @param resolveLink whether to throw UnresolvedLinkException
    *        if src refers to a symlink
    *
+   * @param needLocation Include {@link LocatedBlocks} in result.
+   * @param needBlockToken Include block tokens in {@link LocatedBlocks}
    * @throws AccessControlException if access is denied
    * @throws UnresolvedLinkException if a symlink is encountered.
    *
@@ -2997,15 +2999,19 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    *         or null if file not found
    * @throws StandbyException
    */
-  HdfsFileStatus getFileInfo(final String src, boolean resolveLink)
-    throws IOException {
-    final String operationName = "getfileinfo";
+  HdfsFileStatus getFileInfo(final String src, boolean resolveLink,
+      boolean needLocation, boolean needBlockToken) throws IOException {
+    // if the client requests block tokens, then it can read data blocks
+    // and should appear in the audit log as if getBlockLocations had been
+    // called
+    final String operationName = needBlockToken ? "open" : "getfileinfo";
     checkOperation(OperationCategory.READ);
     HdfsFileStatus stat = null;
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      stat = FSDirStatAndListingOp.getFileInfo(dir, src, resolveLink);
+      stat = FSDirStatAndListingOp.getFileInfo(
+          dir, src, resolveLink, needLocation, needBlockToken);
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
@@ -6158,7 +6164,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
 
     for (CorruptFileBlockInfo c : corruptFileBlocks) {
-      if (getFileInfo(c.path, true) != null) {
+      if (getFileInfo(c.path, true, false, false) != null) {
         list.add(c.toString());
       }
       final Collection<String> snaps = FSDirSnapshotOp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 36d33a6..94bd15f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -104,6 +104,7 @@ import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FSLimitException;
+import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
@@ -1138,12 +1139,25 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ClientProtocol
-  public HdfsFileStatus getFileInfo(String src)  throws IOException {
+  public HdfsFileStatus getFileInfo(String src) throws IOException {
     checkNNStartup();
     metrics.incrFileInfoOps();
-    return namesystem.getFileInfo(src, true);
+    return namesystem.getFileInfo(src, true, false, false);
   }
-  
+
+  @Override // ClientProtocol
+  public HdfsLocatedFileStatus getLocatedFileInfo(String src,
+      boolean needBlockToken) throws IOException {
+    checkNNStartup();
+    if (needBlockToken) {
+      metrics.incrGetBlockLocations();
+    } else {
+      metrics.incrFileInfoOps();
+    }
+    return (HdfsLocatedFileStatus)
+        namesystem.getFileInfo(src, true, true, needBlockToken);
+  }
+
   @Override // ClientProtocol
   public boolean isFileClosed(String src) throws IOException{
     checkNNStartup();
@@ -1154,7 +1168,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
     checkNNStartup();
     metrics.incrFileInfoOps();
-    return namesystem.getFileInfo(src, false);
+    return namesystem.getFileInfo(src, false, false, false);
   }
   
   @Override // ClientProtocol
@@ -1429,7 +1443,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     metrics.incrGetLinkTargetOps();
     HdfsFileStatus stat = null;
     try {
-      stat = namesystem.getFileInfo(path, false);
+      stat = namesystem.getFileInfo(path, false, false, false);
     } catch (UnresolvedPathException e) {
       return e.getResolvedPath().toString();
     } catch (UnresolvedLinkException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
index 799d5d1..c412dad 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/LazyPersistTestCase.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 import com.google.common.base.Supplier;
-import org.apache.commons.lang.UnhandledException;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 
 import static org.apache.hadoop.fs.CreateFlag.CREATE;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
index 242e8f5..11d7959 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
@@ -72,12 +72,13 @@ public class NameNodeAdapter {
   }
   
   public static HdfsFileStatus getFileInfo(NameNode namenode, String src,
-      boolean resolveLink) throws AccessControlException, UnresolvedLinkException,
-        StandbyException, IOException {
+      boolean resolveLink, boolean needLocation, boolean needBlockToken)
+      throws AccessControlException, UnresolvedLinkException, StandbyException,
+      IOException {
     namenode.getNamesystem().readLock();
     try {
       return FSDirStatAndListingOp.getFileInfo(namenode.getNamesystem()
-          .getFSDirectory(), src, resolveLink);
+          .getFSDirectory(), src, resolveLink, needLocation, needBlockToken);
     } finally {
       namenode.getNamesystem().readUnlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
index 10d9f11..2678964 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
@@ -274,7 +274,8 @@ public class TestBackupNode {
       backup = startBackupNode(conf, StartupOption.BACKUP, 1);
 
       testBNInSync(cluster, backup, 4);
-      assertNotNull(backup.getNamesystem().getFileInfo("/edit-while-bn-down", false));
+      assertNotNull(backup.getNamesystem()
+          .getFileInfo("/edit-while-bn-down", false, false, false));
       
       // Trigger an unclean shutdown of the backup node. Backup node will not
       // unregister from the active when this is done simulating a node crash.
@@ -314,7 +315,8 @@ public class TestBackupNode {
         public Boolean get() {
           LOG.info("Checking for " + src + " on BN");
           try {
-            boolean hasFile = backup.getNamesystem().getFileInfo(src, false) != null;
+            boolean hasFile = backup.getNamesystem()
+                .getFileInfo(src, false, false, false) != null;
             boolean txnIdMatch =
               backup.getRpcServer().getTransactionID() ==
               nn.getRpcServer().getTransactionID();
@@ -465,7 +467,7 @@ public class TestBackupNode {
       assertTrue("file3 does not exist on BackupNode",
           op != StartupOption.BACKUP ||
           backup.getNamesystem().getFileInfo(
-              file3.toUri().getPath(), false) != null);
+              file3.toUri().getPath(), false, false, false) != null);
 
     } catch(IOException e) {
       LOG.error("Error in TestBackupNode:", e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
index 8bc8a77..ce52221 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
@@ -288,7 +288,8 @@ public class TestEditLog {
       long numEdits = testLoad(HADOOP20_SOME_EDITS, namesystem);
       assertEquals(3, numEdits);
       // Sanity check the edit
-      HdfsFileStatus fileInfo = namesystem.getFileInfo("/myfile", false);
+      HdfsFileStatus fileInfo =
+          namesystem.getFileInfo("/myfile", false, false, false);
       assertEquals("supergroup", fileInfo.getGroup());
       assertEquals(3, fileInfo.getReplication());
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java
index aca9a73..f0d6834 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java
@@ -163,7 +163,8 @@ public class TestReencryption {
   }
 
   private FileEncryptionInfo getFileEncryptionInfo(Path path) throws Exception {
-    return fsn.getFileInfo(path.toString(), false).getFileEncryptionInfo();
+    return fsn.getFileInfo(path.toString(), false, false, false)
+        .getFileEncryptionInfo();
   }
 
   @Test
@@ -1954,4 +1955,4 @@ public class TestReencryption {
     // after NN  restart consistent.
     dfsAdmin.getKeyProvider().flush();
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
index 0c86ef4..5602954 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
@@ -745,7 +745,7 @@ public class TestSaveNamespace {
 
   private void checkEditExists(FSNamesystem fsn, int id) throws IOException {
     // Make sure the image loaded including our edit.
-    assertNotNull(fsn.getFileInfo("/test" + id, false));
+    assertNotNull(fsn.getFileInfo("/test" + id, false, false, false));
   }
 
   private Configuration getConf() throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
index 9d218c2..61f890c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
@@ -124,7 +124,7 @@ public class TestEditLogTailer {
       
       for (int i = 0; i < DIRS_TO_MAKE / 2; i++) {
         assertTrue(NameNodeAdapter.getFileInfo(nn2,
-            getDirPath(i), false).isDirectory());
+            getDirPath(i), false, false, false).isDirectory());
       }
       
       for (int i = DIRS_TO_MAKE / 2; i < DIRS_TO_MAKE; i++) {
@@ -137,7 +137,7 @@ public class TestEditLogTailer {
       
       for (int i = DIRS_TO_MAKE / 2; i < DIRS_TO_MAKE; i++) {
         assertTrue(NameNodeAdapter.getFileInfo(nn2,
-            getDirPath(i), false).isDirectory());
+            getDirPath(i), false, false, false).isDirectory());
       }
     } finally {
       cluster.shutdown();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogsDuringFailover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogsDuringFailover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogsDuringFailover.java
index a8d350d..4405c5b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogsDuringFailover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogsDuringFailover.java
@@ -38,9 +38,10 @@ import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage;
-import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
+import static org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter.getFileInfo;
+
 import org.junit.Test;
 
 import com.google.common.base.Joiner;
@@ -110,7 +111,8 @@ public class TestEditLogsDuringFailover {
       // the current log segment, and on the next roll, it would have to
       // either replay starting in the middle of the segment (not allowed)
       // or double-replay the edits (incorrect).
-      assertNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1), "/test", true));
+      assertNull(getFileInfo(cluster.getNameNode(1), "/test",
+          true, false, false));
       
       cluster.getNameNode(0).getRpcServer().mkdirs("/test2",
           FsPermission.createImmutable((short)0755), true);
@@ -122,8 +124,10 @@ public class TestEditLogsDuringFailover {
 
       // NN1 should have both the edits that came before its restart, and the edits that
       // came after its restart.
-      assertNotNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1), "/test", true));
-      assertNotNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1), "/test2", true));
+      assertNotNull(getFileInfo(cluster.getNameNode(1), "/test",
+          true, false, false));
+      assertNotNull(getFileInfo(cluster.getNameNode(1), "/test2",
+          true, false, false));
     } finally {
       cluster.shutdown();
     }
@@ -165,7 +169,8 @@ public class TestEditLogsDuringFailover {
       // In the transition to active, it should have read the log -- and
       // hence see one of the dirs we made in the fake log.
       String testPath = "/dir" + NUM_DIRS_IN_LOG;
-      assertNotNull(cluster.getNameNode(0).getRpcServer().getFileInfo(testPath));
+      assertNotNull(cluster.getNameNode(0).getRpcServer()
+          .getFileInfo(testPath));
       
       // It also should have finalized that log in the shared directory and started
       // writing to a new one at the next txid.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
index a37631f..284e283 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
@@ -213,13 +213,13 @@ public class TestFailureToReadEdits {
     
     // Null because it was deleted.
     assertNull(NameNodeAdapter.getFileInfo(nn1,
-        TEST_DIR1, false));
+        TEST_DIR1, false, false, false));
     // Should have been successfully created.
     assertTrue(NameNodeAdapter.getFileInfo(nn1,
-        TEST_DIR2, false).isDirectory());
+        TEST_DIR2, false, false, false).isDirectory());
     // Null because it hasn't been created yet.
     assertNull(NameNodeAdapter.getFileInfo(nn1,
-        TEST_DIR3, false));
+        TEST_DIR3, false, false, false));
 
     // Now let the standby read ALL the edits.
     answer.setThrowExceptionOnRead(false);
@@ -227,13 +227,13 @@ public class TestFailureToReadEdits {
     
     // Null because it was deleted.
     assertNull(NameNodeAdapter.getFileInfo(nn1,
-        TEST_DIR1, false));
+        TEST_DIR1, false, false, false));
     // Should have been successfully created.
     assertTrue(NameNodeAdapter.getFileInfo(nn1,
-        TEST_DIR2, false).isDirectory());
+        TEST_DIR2, false, false, false).isDirectory());
     // Should now have been successfully created.
     assertTrue(NameNodeAdapter.getFileInfo(nn1,
-        TEST_DIR3, false).isDirectory());
+        TEST_DIR3, false, false, false).isDirectory());
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java
index 8eeb853..5015f9e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java
@@ -128,7 +128,7 @@ public class TestInitializeSharedEdits {
       HATestUtil.waitForStandbyToCatchUp(cluster.getNameNode(0),
           cluster.getNameNode(1));
       assertTrue(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
-          newPath.toString(), false).isDirectory());
+          newPath.toString(), false, false, false).isDirectory());
     } finally {
       if (fs != null) {
         fs.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyInProgressTail.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyInProgressTail.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyInProgressTail.java
index b1cd037..5da8178 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyInProgressTail.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyInProgressTail.java
@@ -35,8 +35,9 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.test.GenericTestUtils;
+import static org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter.getFileInfo;
+
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -116,8 +117,8 @@ public class TestStandbyInProgressTail {
       cluster.getNameNode(1).getNamesystem().getEditLogTailer().doTailEdits();
 
       // StandbyNameNode should not finish tailing in-progress logs
-      assertNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
-              "/test", true));
+      assertNull(getFileInfo(cluster.getNameNode(1),
+              "/test", true, false, false));
 
       // Restarting the standby should not finalize any edits files
       // in the shared directory when it starts up!
@@ -132,8 +133,8 @@ public class TestStandbyInProgressTail {
       // the current log segment, and on the next roll, it would have to
       // either replay starting in the middle of the segment (not allowed)
       // or double-replay the edits (incorrect).
-      assertNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
-              "/test", true));
+      assertNull(getFileInfo(cluster.getNameNode(1),
+              "/test", true, false, false));
 
       cluster.getNameNode(0).getRpcServer().mkdirs("/test2",
               FsPermission.createImmutable((short) 0755), true);
@@ -145,10 +146,10 @@ public class TestStandbyInProgressTail {
 
       // NN1 should have both the edits that came before its restart,
       // and the edits that came after its restart.
-      assertNotNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
-              "/test", true));
-      assertNotNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
-              "/test2", true));
+      assertNotNull(getFileInfo(cluster.getNameNode(1),
+              "/test", true, false, false));
+      assertNotNull(getFileInfo(cluster.getNameNode(1),
+              "/test2", true, false, false));
     } finally {
       if (qjmhaCluster != null) {
         qjmhaCluster.shutdown();
@@ -182,8 +183,8 @@ public class TestStandbyInProgressTail {
 
     // After waiting for 5 seconds, StandbyNameNode should finish tailing
     // in-progress logs
-    assertNotNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
-            "/test", true));
+    assertNotNull(getFileInfo(cluster.getNameNode(1),
+            "/test", true, false, false));
 
     // Restarting the standby should not finalize any edits files
     // in the shared directory when it starts up!
@@ -194,8 +195,8 @@ public class TestStandbyInProgressTail {
     assertNoEditFiles(cluster.getNameDirs(1));
 
     // Because we're using in-progress tailer, this should not be null
-    assertNotNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
-            "/test", true));
+    assertNotNull(getFileInfo(cluster.getNameNode(1),
+            "/test", true, false, false));
 
     cluster.getNameNode(0).getRpcServer().mkdirs("/test2",
             FsPermission.createImmutable((short) 0755), true);
@@ -207,10 +208,10 @@ public class TestStandbyInProgressTail {
 
     // NN1 should have both the edits that came before its restart,
     // and the edits that came after its restart.
-    assertNotNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
-            "/test", true));
-    assertNotNull(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
-            "/test2", true));
+    assertNotNull(getFileInfo(cluster.getNameNode(1),
+            "/test", true, false, false));
+    assertNotNull(getFileInfo(cluster.getNameNode(1),
+            "/test2", true, false, false));
   }
 
   @Test
@@ -229,7 +230,7 @@ public class TestStandbyInProgressTail {
     nn1.getNamesystem().getEditLogTailer().doTailEdits();
 
     // StandbyNameNode should tail the in-progress edit
-    assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test", true));
+    assertNotNull(getFileInfo(nn1, "/test", true, false, false));
 
     // Create a new edit and finalized it
     cluster.getNameNode(0).getRpcServer().mkdirs("/test2",
@@ -237,7 +238,7 @@ public class TestStandbyInProgressTail {
     nn0.getRpcServer().rollEditLog();
 
     // StandbyNameNode shouldn't tail the edit since we do not call the method
-    assertNull(NameNodeAdapter.getFileInfo(nn1, "/test2", true));
+    assertNull(getFileInfo(nn1, "/test2", true, false, false));
 
     // Create a new in-progress edit and let SBNN do the tail
     cluster.getNameNode(0).getRpcServer().mkdirs("/test3",
@@ -245,9 +246,9 @@ public class TestStandbyInProgressTail {
     nn1.getNamesystem().getEditLogTailer().doTailEdits();
 
     // StandbyNameNode should tail the finalized edit and the new in-progress
-    assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test", true));
-    assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test2", true));
-    assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test3", true));
+    assertNotNull(getFileInfo(nn1, "/test", true, false, false));
+    assertNotNull(getFileInfo(nn1, "/test2", true, false, false));
+    assertNotNull(getFileInfo(nn1, "/test3", true, false, false));
   }
 
   @Test
@@ -270,16 +271,16 @@ public class TestStandbyInProgressTail {
     cluster.getNameNode(0).getRpcServer().mkdirs("/test3",
             FsPermission.createImmutable((short) 0755), true);
 
-    assertNull(NameNodeAdapter.getFileInfo(nn1, "/test", true));
-    assertNull(NameNodeAdapter.getFileInfo(nn1, "/test2", true));
-    assertNull(NameNodeAdapter.getFileInfo(nn1, "/test3", true));
+    assertNull(getFileInfo(nn1, "/test", true, false, false));
+    assertNull(getFileInfo(nn1, "/test2", true, false, false));
+    assertNull(getFileInfo(nn1, "/test3", true, false, false));
 
     nn1.getNamesystem().getEditLogTailer().doTailEdits();
 
     // StandbyNameNode shoudl tail the finalized edit and the new in-progress
-    assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test", true));
-    assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test2", true));
-    assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test3", true));
+    assertNotNull(getFileInfo(nn1, "/test", true, false, false));
+    assertNotNull(getFileInfo(nn1, "/test2", true, false, false));
+    assertNotNull(getFileInfo(nn1, "/test3", true, false, false));
   }
 
   @Test
@@ -296,17 +297,17 @@ public class TestStandbyInProgressTail {
             FsPermission.createImmutable((short) 0755), true);
     nn1.getNamesystem().getEditLogTailer().doTailEdits();
     nn0.getRpcServer().rollEditLog();
-    assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test", true));
-    assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test2", true));
+    assertNotNull(getFileInfo(nn1, "/test", true, false, false));
+    assertNotNull(getFileInfo(nn1, "/test2", true, false, false));
 
     cluster.getNameNode(0).getRpcServer().mkdirs("/test3",
             FsPermission.createImmutable((short) 0755), true);
     nn1.getNamesystem().getEditLogTailer().doTailEdits();
 
     // StandbyNameNode shoudl tail the finalized edit and the new in-progress
-    assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test", true));
-    assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test2", true));
-    assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test3", true));
+    assertNotNull(getFileInfo(nn1, "/test", true, false, false));
+    assertNotNull(getFileInfo(nn1, "/test2", true, false, false));
+    assertNotNull(getFileInfo(nn1, "/test3", true, false, false));
   }
 
   @Test
@@ -325,7 +326,7 @@ public class TestStandbyInProgressTail {
     cluster.getNameNode(0).getRpcServer().rollEdits();
 
     cluster.getNameNode(1).getNamesystem().getEditLogTailer().doTailEdits();
-    assertNotNull(NameNodeAdapter.getFileInfo(nn1, "/test", true));
+    assertNotNull(getFileInfo(nn1, "/test", true, false, false));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/693169ef/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/contract/hdfs.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/contract/hdfs.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/contract/hdfs.xml
index d38d3d8..261d4ba 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/contract/hdfs.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/contract/hdfs.xml
@@ -108,7 +108,7 @@
 
   <property>
     <name>fs.contract.supports-content-check</name>
-    <value>false</value>
+    <value>true</value>
   </property>
 
 </configuration>


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