|
@@ -47,8 +47,11 @@ import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.FileSystemLinkResolver;
|
|
|
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.LocatedFileStatus;
|
|
|
import org.apache.hadoop.fs.Options;
|
|
|
+import org.apache.hadoop.fs.StorageStatistics;
|
|
|
import org.apache.hadoop.fs.XAttrSetFlag;
|
|
|
import org.apache.hadoop.fs.Options.ChecksumOpt;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
@@ -65,6 +68,7 @@ import org.apache.hadoop.fs.StorageType;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
|
|
|
import org.apache.hadoop.hdfs.client.impl.CorruptFileBlockIterator;
|
|
|
+import org.apache.hadoop.hdfs.DFSOpsCountStatistics.OpType;
|
|
|
import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
|
|
|
import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
|
|
|
import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
|
|
@@ -95,7 +99,6 @@ import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension;
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.base.Preconditions;
|
|
|
|
|
|
-
|
|
|
/****************************************************************
|
|
|
* Implementation of the abstract FileSystem for the DFS system.
|
|
|
* This object is the way end-user code interacts with a Hadoop
|
|
@@ -113,6 +116,8 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
DFSClient dfs;
|
|
|
private boolean verifyChecksum = true;
|
|
|
|
|
|
+ private DFSOpsCountStatistics storageStatistics;
|
|
|
+
|
|
|
static{
|
|
|
HdfsConfiguration.init();
|
|
|
}
|
|
@@ -150,6 +155,15 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
this.dfs = new DFSClient(uri, conf, statistics);
|
|
|
this.uri = URI.create(uri.getScheme()+"://"+uri.getAuthority());
|
|
|
this.workingDir = getHomeDirectory();
|
|
|
+
|
|
|
+ storageStatistics = (DFSOpsCountStatistics) GlobalStorageStatistics.INSTANCE
|
|
|
+ .put(DFSOpsCountStatistics.NAME,
|
|
|
+ new StorageStatisticsProvider() {
|
|
|
+ @Override
|
|
|
+ public StorageStatistics provide() {
|
|
|
+ return new DFSOpsCountStatistics();
|
|
|
+ }
|
|
|
+ });
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -214,6 +228,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
public BlockLocation[] getFileBlockLocations(Path p,
|
|
|
final long start, final long len) throws IOException {
|
|
|
statistics.incrementReadOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.GET_FILE_BLOCK_LOCATIONS);
|
|
|
final Path absF = fixRelativePart(p);
|
|
|
return new FileSystemLinkResolver<BlockLocation[]>() {
|
|
|
@Override
|
|
@@ -264,6 +279,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
public FSDataInputStream open(Path f, final int bufferSize)
|
|
|
throws IOException {
|
|
|
statistics.incrementReadOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.OPEN);
|
|
|
Path absF = fixRelativePart(f);
|
|
|
return new FileSystemLinkResolver<FSDataInputStream>() {
|
|
|
@Override
|
|
@@ -300,6 +316,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
public FSDataOutputStream append(Path f, final EnumSet<CreateFlag> flag,
|
|
|
final int bufferSize, final Progressable progress) throws IOException {
|
|
|
statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.APPEND);
|
|
|
Path absF = fixRelativePart(f);
|
|
|
return new FileSystemLinkResolver<FSDataOutputStream>() {
|
|
|
@Override
|
|
@@ -332,6 +349,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
final int bufferSize, final Progressable progress,
|
|
|
final InetSocketAddress[] favoredNodes) throws IOException {
|
|
|
statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.APPEND);
|
|
|
Path absF = fixRelativePart(f);
|
|
|
return new FileSystemLinkResolver<FSDataOutputStream>() {
|
|
|
@Override
|
|
@@ -375,6 +393,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
final Progressable progress, final InetSocketAddress[] favoredNodes)
|
|
|
throws IOException {
|
|
|
statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.CREATE);
|
|
|
Path absF = fixRelativePart(f);
|
|
|
return new FileSystemLinkResolver<HdfsDataOutputStream>() {
|
|
|
@Override
|
|
@@ -408,6 +427,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
final Progressable progress, final ChecksumOpt checksumOpt)
|
|
|
throws IOException {
|
|
|
statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.CREATE);
|
|
|
Path absF = fixRelativePart(f);
|
|
|
return new FileSystemLinkResolver<FSDataOutputStream>() {
|
|
|
@Override
|
|
@@ -432,6 +452,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
short replication, long blockSize, Progressable progress,
|
|
|
ChecksumOpt checksumOpt) throws IOException {
|
|
|
statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.PRIMITIVE_CREATE);
|
|
|
final DFSOutputStream dfsos = dfs.primitiveCreate(
|
|
|
getPathName(fixRelativePart(f)),
|
|
|
absolutePermission, flag, true, replication, blockSize,
|
|
@@ -448,6 +469,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
final int bufferSize, final short replication, final long blockSize,
|
|
|
final Progressable progress) throws IOException {
|
|
|
statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.CREATE_NON_RECURSIVE);
|
|
|
if (flag.contains(CreateFlag.OVERWRITE)) {
|
|
|
flag.add(CreateFlag.CREATE);
|
|
|
}
|
|
@@ -473,6 +495,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
public boolean setReplication(Path src, final short replication)
|
|
|
throws IOException {
|
|
|
statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.SET_REPLICATION);
|
|
|
Path absF = fixRelativePart(src);
|
|
|
return new FileSystemLinkResolver<Boolean>() {
|
|
|
@Override
|
|
@@ -497,6 +520,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
public void setStoragePolicy(final Path src, final String policyName)
|
|
|
throws IOException {
|
|
|
statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.SET_STORAGE_POLICY);
|
|
|
Path absF = fixRelativePart(src);
|
|
|
new FileSystemLinkResolver<Void>() {
|
|
|
@Override
|
|
@@ -517,6 +541,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
public void unsetStoragePolicy(final Path src)
|
|
|
throws IOException {
|
|
|
statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.UNSET_STORAGE_POLICY);
|
|
|
Path absF = fixRelativePart(src);
|
|
|
new FileSystemLinkResolver<Void>() {
|
|
|
@Override
|
|
@@ -541,6 +566,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
@Override
|
|
|
public BlockStoragePolicySpi getStoragePolicy(Path path) throws IOException {
|
|
|
statistics.incrementReadOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.GET_STORAGE_POLICY);
|
|
|
Path absF = fixRelativePart(path);
|
|
|
|
|
|
return new FileSystemLinkResolver<BlockStoragePolicySpi>() {
|
|
@@ -571,6 +597,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
*/
|
|
|
public long getBytesWithFutureGenerationStamps() throws IOException {
|
|
|
statistics.incrementReadOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.GET_BYTES_WITH_FUTURE_GS);
|
|
|
return dfs.getBytesInFutureBlocks();
|
|
|
}
|
|
|
|
|
@@ -581,6 +608,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
@Deprecated
|
|
|
public BlockStoragePolicy[] getStoragePolicies() throws IOException {
|
|
|
statistics.incrementReadOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.GET_STORAGE_POLICIES);
|
|
|
return dfs.getStoragePolicies();
|
|
|
}
|
|
|
|
|
@@ -595,6 +623,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
@Override
|
|
|
public void concat(Path trg, Path [] psrcs) throws IOException {
|
|
|
statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.CONCAT);
|
|
|
// Make target absolute
|
|
|
Path absF = fixRelativePart(trg);
|
|
|
// Make all srcs absolute
|
|
@@ -639,6 +668,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
@Override
|
|
|
public boolean rename(Path src, Path dst) throws IOException {
|
|
|
statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.RENAME);
|
|
|
|
|
|
final Path absSrc = fixRelativePart(src);
|
|
|
final Path absDst = fixRelativePart(dst);
|
|
@@ -673,6 +703,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
public void rename(Path src, Path dst, final Options.Rename... options)
|
|
|
throws IOException {
|
|
|
statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.RENAME);
|
|
|
final Path absSrc = fixRelativePart(src);
|
|
|
final Path absDst = fixRelativePart(dst);
|
|
|
// Try the rename without resolving first
|
|
@@ -701,6 +732,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
@Override
|
|
|
public boolean truncate(Path f, final long newLength) throws IOException {
|
|
|
statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.TRUNCATE);
|
|
|
Path absF = fixRelativePart(f);
|
|
|
return new FileSystemLinkResolver<Boolean>() {
|
|
|
@Override
|
|
@@ -718,6 +750,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
@Override
|
|
|
public boolean delete(Path f, final boolean recursive) throws IOException {
|
|
|
statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.DELETE);
|
|
|
Path absF = fixRelativePart(f);
|
|
|
return new FileSystemLinkResolver<Boolean>() {
|
|
|
@Override
|
|
@@ -735,6 +768,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
@Override
|
|
|
public ContentSummary getContentSummary(Path f) throws IOException {
|
|
|
statistics.incrementReadOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.GET_CONTENT_SUMMARY);
|
|
|
Path absF = fixRelativePart(f);
|
|
|
return new FileSystemLinkResolver<ContentSummary>() {
|
|
|
@Override
|
|
@@ -752,6 +786,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
@Override
|
|
|
public QuotaUsage getQuotaUsage(Path f) throws IOException {
|
|
|
statistics.incrementReadOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.GET_QUOTA_USAGE);
|
|
|
Path absF = fixRelativePart(f);
|
|
|
return new FileSystemLinkResolver<QuotaUsage>() {
|
|
|
@Override
|
|
@@ -836,6 +871,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
stats[i] = partialListing[i].makeQualified(getUri(), p);
|
|
|
}
|
|
|
statistics.incrementReadOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.LIST_STATUS);
|
|
|
return stats;
|
|
|
}
|
|
|
|
|
@@ -850,6 +886,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
listing.add(fileStatus.makeQualified(getUri(), p));
|
|
|
}
|
|
|
statistics.incrementLargeReadOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.LIST_STATUS);
|
|
|
|
|
|
// now fetch more entries
|
|
|
do {
|
|
@@ -864,6 +901,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
listing.add(fileStatus.makeQualified(getUri(), p));
|
|
|
}
|
|
|
statistics.incrementLargeReadOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.LIST_STATUS);
|
|
|
} while (thisListing.hasMore());
|
|
|
|
|
|
return listing.toArray(new FileStatus[listing.size()]);
|
|
@@ -977,6 +1015,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
thisListing = dfs.listPaths(src, HdfsFileStatus.EMPTY_NAME,
|
|
|
needLocation);
|
|
|
statistics.incrementReadOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.LIST_LOCATED_STATUS);
|
|
|
if (thisListing == null) { // the directory does not exist
|
|
|
throw new FileNotFoundException("File " + p + " does not exist.");
|
|
|
}
|
|
@@ -1072,6 +1111,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
private boolean mkdirsInternal(Path f, final FsPermission permission,
|
|
|
final boolean createParent) throws IOException {
|
|
|
statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.MKDIRS);
|
|
|
Path absF = fixRelativePart(f);
|
|
|
return new FileSystemLinkResolver<Boolean>() {
|
|
|
@Override
|
|
@@ -1098,6 +1138,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
protected boolean primitiveMkdir(Path f, FsPermission absolutePermission)
|
|
|
throws IOException {
|
|
|
statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.PRIMITIVE_MKDIR);
|
|
|
return dfs.primitiveMkdir(getPathName(f), absolutePermission);
|
|
|
}
|
|
|
|
|
@@ -1126,6 +1167,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
@Override
|
|
|
public FsStatus getStatus(Path p) throws IOException {
|
|
|
statistics.incrementReadOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.GET_STATUS);
|
|
|
return dfs.getDiskStatus();
|
|
|
}
|
|
|
|
|
@@ -1317,6 +1359,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
@Override
|
|
|
public FileStatus getFileStatus(Path f) throws IOException {
|
|
|
statistics.incrementReadOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.GET_FILE_STATUS);
|
|
|
Path absF = fixRelativePart(f);
|
|
|
return new FileSystemLinkResolver<FileStatus>() {
|
|
|
@Override
|
|
@@ -1344,6 +1387,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
throw new UnsupportedOperationException("Symlinks not supported");
|
|
|
}
|
|
|
statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.CREATE_SYM_LINK);
|
|
|
final Path absF = fixRelativePart(link);
|
|
|
new FileSystemLinkResolver<Void>() {
|
|
|
@Override
|
|
@@ -1367,6 +1411,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
@Override
|
|
|
public FileStatus getFileLinkStatus(final Path f) throws IOException {
|
|
|
statistics.incrementReadOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.GET_FILE_LINK_STATUS);
|
|
|
final Path absF = fixRelativePart(f);
|
|
|
FileStatus status = new FileSystemLinkResolver<FileStatus>() {
|
|
|
@Override
|
|
@@ -1396,6 +1441,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
@Override
|
|
|
public Path getLinkTarget(final Path f) throws IOException {
|
|
|
statistics.incrementReadOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.GET_LINK_TARGET);
|
|
|
final Path absF = fixRelativePart(f);
|
|
|
return new FileSystemLinkResolver<Path>() {
|
|
|
@Override
|
|
@@ -1417,6 +1463,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
@Override
|
|
|
protected Path resolveLink(Path f) throws IOException {
|
|
|
statistics.incrementReadOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.RESOLVE_LINK);
|
|
|
String target = dfs.getLinkTarget(getPathName(fixRelativePart(f)));
|
|
|
if (target == null) {
|
|
|
throw new FileNotFoundException("File does not exist: " + f.toString());
|
|
@@ -1427,6 +1474,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
@Override
|
|
|
public FileChecksum getFileChecksum(Path f) throws IOException {
|
|
|
statistics.incrementReadOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.GET_FILE_CHECKSUM);
|
|
|
Path absF = fixRelativePart(f);
|
|
|
return new FileSystemLinkResolver<FileChecksum>() {
|
|
|
@Override
|
|
@@ -1446,6 +1494,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
public FileChecksum getFileChecksum(Path f, final long length)
|
|
|
throws IOException {
|
|
|
statistics.incrementReadOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.GET_FILE_CHECKSUM);
|
|
|
Path absF = fixRelativePart(f);
|
|
|
return new FileSystemLinkResolver<FileChecksum>() {
|
|
|
@Override
|
|
@@ -1471,6 +1520,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
public void setPermission(Path p, final FsPermission permission
|
|
|
) throws IOException {
|
|
|
statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.SET_PERMISSION);
|
|
|
Path absF = fixRelativePart(p);
|
|
|
new FileSystemLinkResolver<Void>() {
|
|
|
@Override
|
|
@@ -1495,6 +1545,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
throw new IOException("username == null && groupname == null");
|
|
|
}
|
|
|
statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.SET_OWNER);
|
|
|
Path absF = fixRelativePart(p);
|
|
|
new FileSystemLinkResolver<Void>() {
|
|
|
@Override
|
|
@@ -1516,6 +1567,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
public void setTimes(Path p, final long mtime, final long atime)
|
|
|
throws IOException {
|
|
|
statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.SET_TIMES);
|
|
|
Path absF = fixRelativePart(p);
|
|
|
new FileSystemLinkResolver<Void>() {
|
|
|
@Override
|
|
@@ -1593,6 +1645,8 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
|
|
|
/** @see HdfsAdmin#allowSnapshot(Path) */
|
|
|
public void allowSnapshot(final Path path) throws IOException {
|
|
|
+ statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.ALLOW_SNAPSHOT);
|
|
|
Path absF = fixRelativePart(path);
|
|
|
new FileSystemLinkResolver<Void>() {
|
|
|
@Override
|
|
@@ -1619,6 +1673,8 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
|
|
|
/** @see HdfsAdmin#disallowSnapshot(Path) */
|
|
|
public void disallowSnapshot(final Path path) throws IOException {
|
|
|
+ statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.DISALLOW_SNAPSHOT);
|
|
|
Path absF = fixRelativePart(path);
|
|
|
new FileSystemLinkResolver<Void>() {
|
|
|
@Override
|
|
@@ -1646,6 +1702,8 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
@Override
|
|
|
public Path createSnapshot(final Path path, final String snapshotName)
|
|
|
throws IOException {
|
|
|
+ statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.CREATE_SNAPSHOT);
|
|
|
Path absF = fixRelativePart(path);
|
|
|
return new FileSystemLinkResolver<Path>() {
|
|
|
@Override
|
|
@@ -1671,6 +1729,8 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
@Override
|
|
|
public void renameSnapshot(final Path path, final String snapshotOldName,
|
|
|
final String snapshotNewName) throws IOException {
|
|
|
+ statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.RENAME_SNAPSHOT);
|
|
|
Path absF = fixRelativePart(path);
|
|
|
new FileSystemLinkResolver<Void>() {
|
|
|
@Override
|
|
@@ -1707,6 +1767,8 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
@Override
|
|
|
public void deleteSnapshot(final Path snapshotDir, final String snapshotName)
|
|
|
throws IOException {
|
|
|
+ statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.DELETE_SNAPSHOT);
|
|
|
Path absF = fixRelativePart(snapshotDir);
|
|
|
new FileSystemLinkResolver<Void>() {
|
|
|
@Override
|
|
@@ -1954,6 +2016,8 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
@Override
|
|
|
public void modifyAclEntries(Path path, final List<AclEntry> aclSpec)
|
|
|
throws IOException {
|
|
|
+ statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.MODIFY_ACL_ENTRIES);
|
|
|
Path absF = fixRelativePart(path);
|
|
|
new FileSystemLinkResolver<Void>() {
|
|
|
@Override
|
|
@@ -1976,6 +2040,8 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
@Override
|
|
|
public void removeAclEntries(Path path, final List<AclEntry> aclSpec)
|
|
|
throws IOException {
|
|
|
+ statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.REMOVE_ACL_ENTRIES);
|
|
|
Path absF = fixRelativePart(path);
|
|
|
new FileSystemLinkResolver<Void>() {
|
|
|
@Override
|
|
@@ -1997,6 +2063,8 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
*/
|
|
|
@Override
|
|
|
public void removeDefaultAcl(Path path) throws IOException {
|
|
|
+ statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.REMOVE_DEFAULT_ACL);
|
|
|
final Path absF = fixRelativePart(path);
|
|
|
new FileSystemLinkResolver<Void>() {
|
|
|
@Override
|
|
@@ -2017,6 +2085,8 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
*/
|
|
|
@Override
|
|
|
public void removeAcl(Path path) throws IOException {
|
|
|
+ statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.REMOVE_ACL);
|
|
|
final Path absF = fixRelativePart(path);
|
|
|
new FileSystemLinkResolver<Void>() {
|
|
|
@Override
|
|
@@ -2038,6 +2108,8 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
@Override
|
|
|
public void setAcl(Path path, final List<AclEntry> aclSpec)
|
|
|
throws IOException {
|
|
|
+ statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.SET_ACL);
|
|
|
Path absF = fixRelativePart(path);
|
|
|
new FileSystemLinkResolver<Void>() {
|
|
|
@Override
|
|
@@ -2136,6 +2208,8 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
@Override
|
|
|
public void setXAttr(Path path, final String name, final byte[] value,
|
|
|
final EnumSet<XAttrSetFlag> flag) throws IOException {
|
|
|
+ statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.SET_XATTR);
|
|
|
Path absF = fixRelativePart(path);
|
|
|
new FileSystemLinkResolver<Void>() {
|
|
|
|
|
@@ -2155,6 +2229,8 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
|
|
|
@Override
|
|
|
public byte[] getXAttr(Path path, final String name) throws IOException {
|
|
|
+ statistics.incrementReadOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.GET_XATTR);
|
|
|
final Path absF = fixRelativePart(path);
|
|
|
return new FileSystemLinkResolver<byte[]>() {
|
|
|
@Override
|
|
@@ -2220,6 +2296,8 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
|
|
|
@Override
|
|
|
public void removeXAttr(Path path, final String name) throws IOException {
|
|
|
+ statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.REMOVE_XATTR);
|
|
|
Path absF = fixRelativePart(path);
|
|
|
new FileSystemLinkResolver<Void>() {
|
|
|
@Override
|
|
@@ -2450,4 +2528,5 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
Statistics getFsStatistics() {
|
|
|
return statistics;
|
|
|
}
|
|
|
+
|
|
|
}
|