|
@@ -18,6 +18,7 @@
|
|
|
package org.apache.hadoop.fs.http.server;
|
|
|
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.BlockStoragePolicySpi;
|
|
|
import org.apache.hadoop.fs.ContentSummary;
|
|
|
import org.apache.hadoop.fs.FileChecksum;
|
|
@@ -47,7 +48,6 @@ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
|
|
|
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
|
|
|
import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
|
|
|
import org.apache.hadoop.hdfs.web.JsonUtil;
|
|
|
-import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.lib.service.FileSystemAccess;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
import org.json.simple.JSONArray;
|
|
@@ -73,7 +73,22 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.HTTP_BUFFER_SIZE_DEFAULT;
|
|
|
* FileSystem operation executors used by {@link HttpFSServer}.
|
|
|
*/
|
|
|
@InterfaceAudience.Private
|
|
|
-public class FSOperations {
|
|
|
+public final class FSOperations {
|
|
|
+
|
|
|
+ private static int bufferSize = 4096;
|
|
|
+
|
|
|
+ private FSOperations() {
|
|
|
+ // not called
|
|
|
+ }
|
|
|
+ /**
|
|
|
+ * Set the buffer size. The size is set during the initialization of
|
|
|
+ * HttpFSServerWebApp.
|
|
|
+ * @param conf the configuration to get the bufferSize
|
|
|
+ */
|
|
|
+ public static void setBufferSize(Configuration conf) {
|
|
|
+ bufferSize = conf.getInt(HTTPFS_BUFFER_SIZE_KEY,
|
|
|
+ HTTP_BUFFER_SIZE_DEFAULT);
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
|
* @param fileStatus a FileStatus object
|
|
@@ -436,10 +451,9 @@ public class FSOperations {
|
|
|
*/
|
|
|
@Override
|
|
|
public Void execute(FileSystem fs) throws IOException {
|
|
|
- int bufferSize = fs.getConf().getInt("httpfs.buffer.size", 4096);
|
|
|
OutputStream os = fs.append(path, bufferSize);
|
|
|
- IOUtils.copyBytes(is, os, bufferSize, true);
|
|
|
- os.close();
|
|
|
+ long bytes = copyBytes(is, os);
|
|
|
+ HttpFSServerWebApp.get().getMetrics().incrBytesWritten(bytes);
|
|
|
return null;
|
|
|
}
|
|
|
|
|
@@ -522,6 +536,7 @@ public class FSOperations {
|
|
|
@Override
|
|
|
public JSONObject execute(FileSystem fs) throws IOException {
|
|
|
boolean result = fs.truncate(path, newLength);
|
|
|
+ HttpFSServerWebApp.get().getMetrics().incrOpsTruncate();
|
|
|
return toJSON(
|
|
|
StringUtils.toLowerCase(HttpFSFileSystem.TRUNCATE_JSON), result);
|
|
|
}
|
|
@@ -638,16 +653,65 @@ public class FSOperations {
|
|
|
fsPermission = FsCreateModes.create(fsPermission,
|
|
|
new FsPermission(unmaskedPermission));
|
|
|
}
|
|
|
- int bufferSize = fs.getConf().getInt(HTTPFS_BUFFER_SIZE_KEY,
|
|
|
- HTTP_BUFFER_SIZE_DEFAULT);
|
|
|
OutputStream os = fs.create(path, fsPermission, override, bufferSize, replication, blockSize, null);
|
|
|
- IOUtils.copyBytes(is, os, bufferSize, true);
|
|
|
- os.close();
|
|
|
+ long bytes = copyBytes(is, os);
|
|
|
+ HttpFSServerWebApp.get().getMetrics().incrBytesWritten(bytes);
|
|
|
return null;
|
|
|
}
|
|
|
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * These copyBytes methods combines the two different flavors used originally.
|
|
|
+ * One with length and another one with buffer size.
|
|
|
+ * In this impl, buffer size is determined internally, which is a singleton
|
|
|
+ * normally set during initialization.
|
|
|
+ * @param in the inputStream
|
|
|
+ * @param out the outputStream
|
|
|
+ * @return the totalBytes
|
|
|
+ * @throws IOException the exception to be thrown.
|
|
|
+ */
|
|
|
+ public static long copyBytes(InputStream in, OutputStream out)
|
|
|
+ throws IOException {
|
|
|
+ return copyBytes(in, out, Long.MAX_VALUE);
|
|
|
+ }
|
|
|
+
|
|
|
+ public static long copyBytes(InputStream in, OutputStream out, long count)
|
|
|
+ throws IOException {
|
|
|
+ long totalBytes = 0;
|
|
|
+
|
|
|
+ // If bufferSize is not initialized use 4k. This will not happen
|
|
|
+ // if all callers check and set it.
|
|
|
+ byte[] buf = new byte[bufferSize];
|
|
|
+ long bytesRemaining = count;
|
|
|
+ int bytesRead;
|
|
|
+
|
|
|
+ try {
|
|
|
+ while (bytesRemaining > 0) {
|
|
|
+ int bytesToRead = (int)
|
|
|
+ (bytesRemaining < buf.length ? bytesRemaining : buf.length);
|
|
|
+
|
|
|
+ bytesRead = in.read(buf, 0, bytesToRead);
|
|
|
+ if (bytesRead == -1) {
|
|
|
+ break;
|
|
|
+ }
|
|
|
+
|
|
|
+ out.write(buf, 0, bytesRead);
|
|
|
+ bytesRemaining -= bytesRead;
|
|
|
+ totalBytes += bytesRead;
|
|
|
+ }
|
|
|
+ return totalBytes;
|
|
|
+ } finally {
|
|
|
+ // Originally IOUtils.copyBytes() were called with close=true. So we are
|
|
|
+ // implementing the same behavior here.
|
|
|
+ try {
|
|
|
+ in.close();
|
|
|
+ } finally {
|
|
|
+ out.close();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Executor that performs a delete FileSystemAccess files system operation.
|
|
|
*/
|
|
@@ -680,6 +744,7 @@ public class FSOperations {
|
|
|
@Override
|
|
|
public JSONObject execute(FileSystem fs) throws IOException {
|
|
|
boolean deleted = fs.delete(path, recursive);
|
|
|
+ HttpFSServerWebApp.get().getMetrics().incrOpsDelete();
|
|
|
return toJSON(
|
|
|
StringUtils.toLowerCase(HttpFSFileSystem.DELETE_JSON), deleted);
|
|
|
}
|
|
@@ -748,6 +813,7 @@ public class FSOperations {
|
|
|
@Override
|
|
|
public Map execute(FileSystem fs) throws IOException {
|
|
|
FileStatus status = fs.getFileStatus(path);
|
|
|
+ HttpFSServerWebApp.get().getMetrics().incrOpsStat();
|
|
|
return toJson(status);
|
|
|
}
|
|
|
|
|
@@ -776,7 +842,6 @@ public class FSOperations {
|
|
|
json.put(HttpFSFileSystem.HOME_DIR_JSON, homeDir.toUri().getPath());
|
|
|
return json;
|
|
|
}
|
|
|
-
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -814,6 +879,7 @@ public class FSOperations {
|
|
|
@Override
|
|
|
public Map execute(FileSystem fs) throws IOException {
|
|
|
FileStatus[] fileStatuses = fs.listStatus(path, filter);
|
|
|
+ HttpFSServerWebApp.get().getMetrics().incrOpsListing();
|
|
|
return toJson(fileStatuses, fs.getFileStatus(path).isFile());
|
|
|
}
|
|
|
|
|
@@ -905,6 +971,7 @@ public class FSOperations {
|
|
|
new FsPermission(unmaskedPermission));
|
|
|
}
|
|
|
boolean mkdirs = fs.mkdirs(path, fsPermission);
|
|
|
+ HttpFSServerWebApp.get().getMetrics().incrOpsMkdir();
|
|
|
return toJSON(HttpFSFileSystem.MKDIRS_JSON, mkdirs);
|
|
|
}
|
|
|
|
|
@@ -937,8 +1004,8 @@ public class FSOperations {
|
|
|
*/
|
|
|
@Override
|
|
|
public InputStream execute(FileSystem fs) throws IOException {
|
|
|
- int bufferSize = HttpFSServerWebApp.get().getConfig().getInt(
|
|
|
- HTTPFS_BUFFER_SIZE_KEY, HTTP_BUFFER_SIZE_DEFAULT);
|
|
|
+ // Only updating ops count. bytesRead is updated in InputStreamEntity
|
|
|
+ HttpFSServerWebApp.get().getMetrics().incrOpsOpen();
|
|
|
return fs.open(path, bufferSize);
|
|
|
}
|
|
|
|
|
@@ -976,6 +1043,7 @@ public class FSOperations {
|
|
|
@Override
|
|
|
public JSONObject execute(FileSystem fs) throws IOException {
|
|
|
boolean renamed = fs.rename(path, toPath);
|
|
|
+ HttpFSServerWebApp.get().getMetrics().incrOpsRename();
|
|
|
return toJSON(HttpFSFileSystem.RENAME_JSON, renamed);
|
|
|
}
|
|
|
|
|
@@ -1944,6 +2012,7 @@ public class FSOperations {
|
|
|
if (fs instanceof DistributedFileSystem) {
|
|
|
DistributedFileSystem dfs = (DistributedFileSystem) fs;
|
|
|
dfs.access(path, mode);
|
|
|
+ HttpFSServerWebApp.get().getMetrics().incrOpsCheckAccess();
|
|
|
} else {
|
|
|
throw new UnsupportedOperationException("checkaccess is "
|
|
|
+ "not supported for HttpFs on " + fs.getClass()
|