HADOOP-14430 the accessTime of FileStatus returned by SFTPFileSystem's

getFileStatus method is always 0.
Contributed by Hongyuan Li.
This commit is contained in:
Steve Loughran 2017-05-25 15:19:58 +01:00
parent 1ba9704eec
commit 8bf0e2d6b3
No known key found for this signature in database
GPG Key ID: 950CC3E032B79CA2
2 changed files with 15 additions and 1 deletions

View File

@ -278,7 +278,7 @@ public class SFTPFileSystem extends FileSystem {
// block sizes on server. The assumption could be less than ideal. // block sizes on server. The assumption could be less than ideal.
long blockSize = DEFAULT_BLOCK_SIZE; long blockSize = DEFAULT_BLOCK_SIZE;
long modTime = attr.getMTime() * 1000; // convert to milliseconds long modTime = attr.getMTime() * 1000; // convert to milliseconds
long accessTime = 0; long accessTime = attr.getATime() * 1000L;
FsPermission permission = getPermissions(sftpFile); FsPermission permission = getPermissions(sftpFile);
// not be able to get the real user group name, just use the user and group // not be able to get the real user group name, just use the user and group
// id // id

View File

@ -19,6 +19,8 @@ package org.apache.hadoop.fs.sftp;
import java.io.IOException; import java.io.IOException;
import java.net.URI; import java.net.URI;
import java.nio.file.Files;
import java.nio.file.attribute.BasicFileAttributes;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;
@ -28,6 +30,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
@ -305,4 +308,15 @@ public class TestSFTPFileSystem {
sftpFs.rename(file1, file2); sftpFs.rename(file1, file2);
} }
@Test
public void testGetAccessTime() throws IOException {
Path file = touch(localFs, name.getMethodName().toLowerCase());
LocalFileSystem local = (LocalFileSystem)localFs;
java.nio.file.Path path = (local).pathToFile(file).toPath();
long accessTime1 = Files.readAttributes(path, BasicFileAttributes.class)
.lastAccessTime().toMillis();
long accessTime2 = sftpFs.getFileStatus(file).getAccessTime();
assertEquals(accessTime1, accessTime2);
}
} }