|
@@ -18,6 +18,7 @@
|
|
|
package org.apache.hadoop.hdfs.server.datanode;
|
|
|
|
|
|
import java.io.File;
|
|
|
+import java.io.FileNotFoundException;
|
|
|
import java.io.IOException;
|
|
|
import java.net.InetSocketAddress;
|
|
|
import java.net.URL;
|
|
@@ -35,6 +36,7 @@ import javax.servlet.jsp.JspWriter;
|
|
|
import org.apache.commons.lang.StringEscapeUtils;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.fs.UnresolvedLinkException;
|
|
|
import org.apache.hadoop.hdfs.DFSClient;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
@@ -45,16 +47,22 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
|
|
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
|
|
|
import org.apache.hadoop.hdfs.server.common.JspHelper;
|
|
|
import org.apache.hadoop.http.HtmlQuoting;
|
|
|
-import org.apache.hadoop.http.HttpConfig;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
+import org.apache.hadoop.security.AccessControlException;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
|
import org.apache.hadoop.util.ServletUtil;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
import org.apache.hadoop.util.VersionInfo;
|
|
|
|
|
|
+import com.google.common.base.Predicate;
|
|
|
+import com.google.common.collect.Iterables;
|
|
|
+
|
|
|
@InterfaceAudience.Private
|
|
|
public class DatanodeJspHelper {
|
|
|
+ private static final int PREV_BLOCK = -1;
|
|
|
+ private static final int NEXT_BLOCK = 1;
|
|
|
+
|
|
|
private static DFSClient getDFSClient(final UserGroupInformation user,
|
|
|
final String addr,
|
|
|
final Configuration conf
|
|
@@ -137,10 +145,10 @@ public class DatanodeJspHelper {
|
|
|
out.print("Empty file");
|
|
|
} else {
|
|
|
DatanodeInfo chosenNode = JspHelper.bestNode(firstBlock, conf);
|
|
|
- String fqdn = canonicalize(chosenNode.getIpAddr());
|
|
|
int datanodePort = chosenNode.getXferPort();
|
|
|
- String redirectLocation = HttpConfig.getSchemePrefix() + fqdn + ":"
|
|
|
- + chosenNode.getInfoPort() + "/browseBlock.jsp?blockId="
|
|
|
+ String redirectLocation = JspHelper.Url.url(req.getScheme(),
|
|
|
+ chosenNode)
|
|
|
+ + "/browseBlock.jsp?blockId="
|
|
|
+ firstBlock.getBlock().getBlockId() + "&blockSize="
|
|
|
+ firstBlock.getBlock().getNumBytes() + "&genstamp="
|
|
|
+ firstBlock.getBlock().getGenerationStamp() + "&filename="
|
|
@@ -312,8 +320,8 @@ public class DatanodeJspHelper {
|
|
|
dfs.close();
|
|
|
return;
|
|
|
}
|
|
|
- String fqdn = canonicalize(chosenNode.getIpAddr());
|
|
|
- String tailUrl = "///" + fqdn + ":" + chosenNode.getInfoPort()
|
|
|
+
|
|
|
+ String tailUrl = "///" + JspHelper.Url.authority(req.getScheme(), chosenNode)
|
|
|
+ "/tail.jsp?filename=" + URLEncoder.encode(filename, "UTF-8")
|
|
|
+ "&namenodeInfoPort=" + namenodeInfoPort
|
|
|
+ "&chunkSizeToView=" + chunkSizeToView
|
|
@@ -361,8 +369,7 @@ public class DatanodeJspHelper {
|
|
|
for (int j = 0; j < locs.length; j++) {
|
|
|
String datanodeAddr = locs[j].getXferAddr();
|
|
|
datanodePort = locs[j].getXferPort();
|
|
|
- fqdn = canonicalize(locs[j].getIpAddr());
|
|
|
- String blockUrl = "///" + fqdn + ":" + locs[j].getInfoPort()
|
|
|
+ String blockUrl = "///" + JspHelper.Url.authority(req.getScheme(), locs[j])
|
|
|
+ "/browseBlock.jsp?blockId=" + blockidstring
|
|
|
+ "&blockSize=" + blockSize
|
|
|
+ "&filename=" + URLEncoder.encode(filename, "UTF-8")
|
|
@@ -492,112 +499,23 @@ public class DatanodeJspHelper {
|
|
|
out.print("<a href=\"#viewOptions\">Advanced view/download options</a><br>");
|
|
|
out.print("<hr>");
|
|
|
|
|
|
- // Determine the prev & next blocks
|
|
|
- long nextStartOffset = 0;
|
|
|
- long nextBlockSize = 0;
|
|
|
- String nextBlockIdStr = null;
|
|
|
- String nextGenStamp = null;
|
|
|
- String nextHost = req.getServerName();
|
|
|
- int nextPort = req.getServerPort();
|
|
|
- int nextDatanodePort = datanodePort;
|
|
|
- // determine data for the next link
|
|
|
- if (startOffset + chunkSizeToView >= blockSize) {
|
|
|
- // we have to go to the next block from this point onwards
|
|
|
- List<LocatedBlock> blocks = dfs.getNamenode().getBlockLocations(filename, 0,
|
|
|
- Long.MAX_VALUE).getLocatedBlocks();
|
|
|
- for (int i = 0; i < blocks.size(); i++) {
|
|
|
- if (blocks.get(i).getBlock().getBlockId() == blockId) {
|
|
|
- if (i != blocks.size() - 1) {
|
|
|
- LocatedBlock nextBlock = blocks.get(i + 1);
|
|
|
- nextBlockIdStr = Long.toString(nextBlock.getBlock().getBlockId());
|
|
|
- nextGenStamp = Long.toString(nextBlock.getBlock()
|
|
|
- .getGenerationStamp());
|
|
|
- nextStartOffset = 0;
|
|
|
- nextBlockSize = nextBlock.getBlock().getNumBytes();
|
|
|
- DatanodeInfo d = JspHelper.bestNode(nextBlock, conf);
|
|
|
- nextDatanodePort = d.getXferPort();
|
|
|
- nextHost = d.getIpAddr();
|
|
|
- nextPort = d.getInfoPort();
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- } else {
|
|
|
- // we are in the same block
|
|
|
- nextBlockIdStr = blockId.toString();
|
|
|
- nextStartOffset = startOffset + chunkSizeToView;
|
|
|
- nextBlockSize = blockSize;
|
|
|
- nextGenStamp = genStamp.toString();
|
|
|
- }
|
|
|
- String nextUrl = null;
|
|
|
- if (nextBlockIdStr != null) {
|
|
|
- nextUrl = "///" + canonicalize(nextHost) + ":" + nextPort
|
|
|
- + "/browseBlock.jsp?blockId=" + nextBlockIdStr
|
|
|
- + "&blockSize=" + nextBlockSize
|
|
|
- + "&startOffset=" + nextStartOffset
|
|
|
- + "&genstamp=" + nextGenStamp
|
|
|
- + "&filename=" + URLEncoder.encode(filename, "UTF-8")
|
|
|
- + "&chunkSizeToView=" + chunkSizeToView
|
|
|
- + "&datanodePort=" + nextDatanodePort
|
|
|
- + "&namenodeInfoPort=" + namenodeInfoPort
|
|
|
- + JspHelper.getDelegationTokenUrlParam(tokenString)
|
|
|
- + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnAddr);
|
|
|
+ String authority = req.getServerName() + ":" + req.getServerPort();
|
|
|
+ String nextUrl = generateLinksForAdjacentBlock(NEXT_BLOCK, authority,
|
|
|
+ datanodePort, startOffset, chunkSizeToView, blockSize, blockId,
|
|
|
+ genStamp, dfs, filename, conf, req.getScheme(), tokenString,
|
|
|
+ namenodeInfoPort, nnAddr);
|
|
|
+ if (nextUrl != null) {
|
|
|
out.print("<a href=\"" + nextUrl + "\">View Next chunk</a> ");
|
|
|
}
|
|
|
- // determine data for the prev link
|
|
|
- String prevBlockIdStr = null;
|
|
|
- String prevGenStamp = null;
|
|
|
- long prevStartOffset = 0;
|
|
|
- long prevBlockSize = 0;
|
|
|
- String prevHost = req.getServerName();
|
|
|
- int prevPort = req.getServerPort();
|
|
|
- int prevDatanodePort = datanodePort;
|
|
|
- if (startOffset == 0) {
|
|
|
- List<LocatedBlock> blocks = dfs.getNamenode().getBlockLocations(filename, 0,
|
|
|
- Long.MAX_VALUE).getLocatedBlocks();
|
|
|
- for (int i = 0; i < blocks.size(); i++) {
|
|
|
- if (blocks.get(i).getBlock().getBlockId() == blockId) {
|
|
|
- if (i != 0) {
|
|
|
- LocatedBlock prevBlock = blocks.get(i - 1);
|
|
|
- prevBlockIdStr = Long.toString(prevBlock.getBlock().getBlockId());
|
|
|
- prevGenStamp = Long.toString(prevBlock.getBlock()
|
|
|
- .getGenerationStamp());
|
|
|
- prevStartOffset = prevBlock.getBlock().getNumBytes()
|
|
|
- - chunkSizeToView;
|
|
|
- if (prevStartOffset < 0)
|
|
|
- prevStartOffset = 0;
|
|
|
- prevBlockSize = prevBlock.getBlock().getNumBytes();
|
|
|
- DatanodeInfo d = JspHelper.bestNode(prevBlock, conf);
|
|
|
- prevDatanodePort = d.getXferPort();
|
|
|
- prevHost = d.getIpAddr();
|
|
|
- prevPort = d.getInfoPort();
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- } else {
|
|
|
- // we are in the same block
|
|
|
- prevBlockIdStr = blockId.toString();
|
|
|
- prevStartOffset = startOffset - chunkSizeToView;
|
|
|
- if (prevStartOffset < 0)
|
|
|
- prevStartOffset = 0;
|
|
|
- prevBlockSize = blockSize;
|
|
|
- prevGenStamp = genStamp.toString();
|
|
|
- }
|
|
|
|
|
|
- String prevUrl = null;
|
|
|
- if (prevBlockIdStr != null) {
|
|
|
- prevUrl = "///" + canonicalize(prevHost) + ":" + prevPort
|
|
|
- + "/browseBlock.jsp?blockId=" + prevBlockIdStr
|
|
|
- + "&blockSize=" + prevBlockSize
|
|
|
- + "&startOffset=" + prevStartOffset
|
|
|
- + "&filename=" + URLEncoder.encode(filename, "UTF-8")
|
|
|
- + "&chunkSizeToView=" + chunkSizeToView
|
|
|
- + "&genstamp=" + prevGenStamp
|
|
|
- + "&datanodePort=" + prevDatanodePort
|
|
|
- + "&namenodeInfoPort=" + namenodeInfoPort
|
|
|
- + JspHelper.getDelegationTokenUrlParam(tokenString)
|
|
|
- + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnAddr);
|
|
|
+ String prevUrl = generateLinksForAdjacentBlock(PREV_BLOCK, authority,
|
|
|
+ datanodePort, startOffset, chunkSizeToView, blockSize, blockId,
|
|
|
+ genStamp, dfs, filename, conf, req.getScheme(), tokenString,
|
|
|
+ namenodeInfoPort, nnAddr);
|
|
|
+ if (prevUrl != null) {
|
|
|
out.print("<a href=\"" + prevUrl + "\">View Prev chunk</a> ");
|
|
|
}
|
|
|
+
|
|
|
out.print("<hr>");
|
|
|
out.print("<textarea cols=\"100\" rows=\"25\" wrap=\"virtual\" style=\"width:100%\" READONLY>");
|
|
|
try {
|
|
@@ -612,6 +530,71 @@ public class DatanodeJspHelper {
|
|
|
dfs.close();
|
|
|
}
|
|
|
|
|
|
+ private static String generateLinksForAdjacentBlock(final int direction,
|
|
|
+ String authority, int datanodePort, long startOffset,
|
|
|
+ int chunkSizeToView, long blockSize, long blockId, Long genStamp,
|
|
|
+ final DFSClient dfs, final String filename, final Configuration conf,
|
|
|
+ final String scheme, final String tokenString,
|
|
|
+ final int namenodeInfoPort, final String nnAddr)
|
|
|
+ throws AccessControlException, FileNotFoundException,
|
|
|
+ UnresolvedLinkException, IOException {
|
|
|
+
|
|
|
+ boolean found = false;
|
|
|
+ if ((direction == NEXT_BLOCK && startOffset + chunkSizeToView < blockSize)
|
|
|
+ || (direction == PREV_BLOCK && startOffset != 0)) {
|
|
|
+ // we are in the same block
|
|
|
+ found = true;
|
|
|
+
|
|
|
+ if (direction == NEXT_BLOCK) {
|
|
|
+ startOffset = startOffset + chunkSizeToView;
|
|
|
+ } else {
|
|
|
+ startOffset = Math.max(0, startOffset - chunkSizeToView);
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ List<LocatedBlock> blocks = dfs.getNamenode().getBlockLocations(filename, 0,
|
|
|
+ Long.MAX_VALUE).getLocatedBlocks();
|
|
|
+
|
|
|
+ final long curBlockId = blockId;
|
|
|
+ int curBlockIdx = Iterables.indexOf(blocks, new Predicate<LocatedBlock>() {
|
|
|
+ @Override
|
|
|
+ public boolean apply(LocatedBlock b) {
|
|
|
+ return b.getBlock().getBlockId() == curBlockId;
|
|
|
+ }
|
|
|
+ });
|
|
|
+ found = curBlockIdx != -1 &&
|
|
|
+ ((direction == NEXT_BLOCK && curBlockIdx < blocks.size() - 1)
|
|
|
+ || (direction == PREV_BLOCK && curBlockIdx > 0));
|
|
|
+
|
|
|
+ if (found) {
|
|
|
+ LocatedBlock nextBlock = blocks.get(curBlockIdx + direction);
|
|
|
+
|
|
|
+ blockId = nextBlock.getBlock().getBlockId();
|
|
|
+ genStamp = nextBlock.getBlock().getGenerationStamp();
|
|
|
+ startOffset = 0;
|
|
|
+ blockSize = nextBlock.getBlock().getNumBytes();
|
|
|
+ DatanodeInfo d = JspHelper.bestNode(nextBlock, conf);
|
|
|
+ datanodePort = d.getXferPort();
|
|
|
+ authority = JspHelper.Url.authority(scheme, d);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ if (found) {
|
|
|
+ return "///" + authority
|
|
|
+ + "/browseBlock.jsp?blockId=" + blockId
|
|
|
+ + "&blockSize=" + blockSize
|
|
|
+ + "&startOffset=" + startOffset
|
|
|
+ + "&genstamp=" + genStamp
|
|
|
+ + "&filename=" + URLEncoder.encode(filename, "UTF-8")
|
|
|
+ + "&chunkSizeToView=" + chunkSizeToView
|
|
|
+ + "&datanodePort=" + datanodePort
|
|
|
+ + "&namenodeInfoPort=" + namenodeInfoPort
|
|
|
+ + JspHelper.getDelegationTokenUrlParam(tokenString)
|
|
|
+ + JspHelper.getUrlParam(JspHelper.NAMENODE_ADDRESS, nnAddr);
|
|
|
+ } else {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
static void generateFileChunksForTail(JspWriter out, HttpServletRequest req,
|
|
|
Configuration conf
|
|
|
) throws IOException,
|