|
@@ -18,7 +18,6 @@
|
|
|
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,7 +34,6 @@ 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;
|
|
@@ -46,21 +44,15 @@ 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 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
|
|
@@ -143,10 +135,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 = JspHelper.Url.url(req.getScheme(),
|
|
|
- chosenNode)
|
|
|
- + "/browseBlock.jsp?blockId="
|
|
|
+ String redirectLocation = HttpConfig.getSchemePrefix() + fqdn + ":"
|
|
|
+ + chosenNode.getInfoPort() + "/browseBlock.jsp?blockId="
|
|
|
+ firstBlock.getBlock().getBlockId() + "&blockSize="
|
|
|
+ firstBlock.getBlock().getNumBytes() + "&genstamp="
|
|
|
+ firstBlock.getBlock().getGenerationStamp() + "&filename="
|
|
@@ -318,8 +310,8 @@ public class DatanodeJspHelper {
|
|
|
dfs.close();
|
|
|
return;
|
|
|
}
|
|
|
-
|
|
|
- String tailUrl = "///" + JspHelper.Url.authority(req.getScheme(), chosenNode)
|
|
|
+ String fqdn = canonicalize(chosenNode.getIpAddr());
|
|
|
+ String tailUrl = "///" + fqdn + ":" + chosenNode.getInfoPort()
|
|
|
+ "/tail.jsp?filename=" + URLEncoder.encode(filename, "UTF-8")
|
|
|
+ "&namenodeInfoPort=" + namenodeInfoPort
|
|
|
+ "&chunkSizeToView=" + chunkSizeToView
|
|
@@ -367,7 +359,8 @@ public class DatanodeJspHelper {
|
|
|
for (int j = 0; j < locs.length; j++) {
|
|
|
String datanodeAddr = locs[j].getXferAddr();
|
|
|
datanodePort = locs[j].getXferPort();
|
|
|
- String blockUrl = "///" + JspHelper.Url.authority(req.getScheme(), locs[j])
|
|
|
+ fqdn = canonicalize(locs[j].getIpAddr());
|
|
|
+ String blockUrl = "///" + fqdn + ":" + locs[j].getInfoPort()
|
|
|
+ "/browseBlock.jsp?blockId=" + blockidstring
|
|
|
+ "&blockSize=" + blockSize
|
|
|
+ "&filename=" + URLEncoder.encode(filename, "UTF-8")
|
|
@@ -497,23 +490,112 @@ public class DatanodeJspHelper {
|
|
|
out.print("<a href=\"#viewOptions\">Advanced view/download options</a><br>");
|
|
|
out.print("<hr>");
|
|
|
|
|
|
- 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) {
|
|
|
+ // 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);
|
|
|
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 = generateLinksForAdjacentBlock(PREV_BLOCK, authority,
|
|
|
- datanodePort, startOffset, chunkSizeToView, blockSize, blockId,
|
|
|
- genStamp, dfs, filename, conf, req.getScheme(), tokenString,
|
|
|
- namenodeInfoPort, nnAddr);
|
|
|
- if (prevUrl != null) {
|
|
|
+ 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);
|
|
|
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 {
|
|
@@ -528,71 +610,6 @@ 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,
|