browseBlock.jsp 15 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385
  1. <%@ page
  2. contentType="text/html; charset=UTF-8"
  3. import="javax.servlet.*"
  4. import="javax.servlet.http.*"
  5. import="java.io.*"
  6. import="java.util.*"
  7. import="java.net.*"
  8. import="org.apache.hadoop.hdfs.*"
  9. import="org.apache.hadoop.hdfs.server.namenode.*"
  10. import="org.apache.hadoop.hdfs.server.datanode.*"
  11. import="org.apache.hadoop.hdfs.protocol.*"
  12. import="org.apache.hadoop.io.*"
  13. import="org.apache.hadoop.conf.*"
  14. import="org.apache.hadoop.net.DNS"
  15. import="org.apache.hadoop.util.*"
  16. import="java.text.DateFormat"
  17. %>
  18. <%!
  19. static JspHelper jspHelper = new JspHelper();
  20. public void generateFileDetails(JspWriter out, HttpServletRequest req)
  21. throws IOException {
  22. int chunkSizeToView = 0;
  23. long startOffset = 0;
  24. int datanodePort;
  25. String blockIdStr = null;
  26. long currBlockId = 0;
  27. blockIdStr = req.getParameter("blockId");
  28. if (blockIdStr == null) {
  29. out.print("Invalid input (blockId absent)");
  30. return;
  31. }
  32. currBlockId = Long.parseLong(blockIdStr);
  33. String datanodePortStr = req.getParameter("datanodePort");
  34. if (datanodePortStr == null) {
  35. out.print("Invalid input (datanodePort absent)");
  36. return;
  37. }
  38. datanodePort = Integer.parseInt(datanodePortStr);
  39. String namenodeInfoPortStr = req.getParameter("namenodeInfoPort");
  40. int namenodeInfoPort = -1;
  41. if (namenodeInfoPortStr != null)
  42. namenodeInfoPort = Integer.parseInt(namenodeInfoPortStr);
  43. String chunkSizeToViewStr = req.getParameter("chunkSizeToView");
  44. if (chunkSizeToViewStr != null && Integer.parseInt(chunkSizeToViewStr) > 0)
  45. chunkSizeToView = Integer.parseInt(chunkSizeToViewStr);
  46. else chunkSizeToView = jspHelper.defaultChunkSizeToView;
  47. String startOffsetStr = req.getParameter("startOffset");
  48. if (startOffsetStr == null || Long.parseLong(startOffsetStr) < 0)
  49. startOffset = 0;
  50. else startOffset = Long.parseLong(startOffsetStr);
  51. String filename = req.getParameter("filename");
  52. if (filename == null || filename.length() == 0) {
  53. out.print("Invalid input");
  54. return;
  55. }
  56. String blockSizeStr = req.getParameter("blockSize");
  57. long blockSize = 0;
  58. if (blockSizeStr == null || blockSizeStr.length() == 0) {
  59. out.print("Invalid input");
  60. return;
  61. }
  62. blockSize = Long.parseLong(blockSizeStr);
  63. DFSClient dfs = new DFSClient(jspHelper.nameNodeAddr, jspHelper.conf);
  64. List<LocatedBlock> blocks =
  65. dfs.namenode.getBlockLocations(filename, 0, Long.MAX_VALUE).getLocatedBlocks();
  66. //Add the various links for looking at the file contents
  67. //URL for downloading the full file
  68. String downloadUrl = "http://" + req.getServerName() + ":" +
  69. + req.getServerPort() + "/streamFile?" + "filename=" +
  70. URLEncoder.encode(filename, "UTF-8");
  71. out.print("<a name=\"viewOptions\"></a>");
  72. out.print("<a href=\"" + downloadUrl + "\">Download this file</a><br>");
  73. DatanodeInfo chosenNode;
  74. //URL for TAIL
  75. LocatedBlock lastBlk = blocks.get(blocks.size() - 1);
  76. long blockId = lastBlk.getBlock().getBlockId();
  77. try {
  78. chosenNode = jspHelper.bestNode(lastBlk);
  79. } catch (IOException e) {
  80. out.print(e.toString());
  81. dfs.close();
  82. return;
  83. }
  84. String fqdn =
  85. InetAddress.getByName(chosenNode.getHost()).getCanonicalHostName();
  86. String tailUrl = "http://" + fqdn + ":" +
  87. chosenNode.getInfoPort() +
  88. "/tail.jsp?filename=" + URLEncoder.encode(filename, "UTF-8") +
  89. "&namenodeInfoPort=" + namenodeInfoPort +
  90. "&chunkSizeToView=" + chunkSizeToView +
  91. "&referrer=" +
  92. URLEncoder.encode(req.getRequestURL() + "?" + req.getQueryString(),
  93. "UTF-8");
  94. out.print("<a href=\"" + tailUrl + "\">Tail this file</a><br>");
  95. out.print("<form action=\"/browseBlock.jsp\" method=GET>");
  96. out.print("<b>Chunk size to view (in bytes, up to file's DFS block size): </b>");
  97. out.print("<input type=\"hidden\" name=\"blockId\" value=\"" + currBlockId +
  98. "\">");
  99. out.print("<input type=\"hidden\" name=\"blockSize\" value=\"" +
  100. blockSize + "\">");
  101. out.print("<input type=\"hidden\" name=\"startOffset\" value=\"" +
  102. startOffset + "\">");
  103. out.print("<input type=\"hidden\" name=\"filename\" value=\"" + filename +
  104. "\">");
  105. out.print("<input type=\"hidden\" name=\"datanodePort\" value=\"" +
  106. datanodePort+ "\">");
  107. out.print("<input type=\"hidden\" name=\"namenodeInfoPort\" value=\"" +
  108. namenodeInfoPort + "\">");
  109. out.print("<input type=\"text\" name=\"chunkSizeToView\" value=" +
  110. chunkSizeToView + " size=10 maxlength=10>");
  111. out.print("&nbsp;&nbsp;<input type=\"submit\" name=\"submit\" value=\"Refresh\">");
  112. out.print("</form>");
  113. out.print("<hr>");
  114. out.print("<a name=\"blockDetails\"></a>");
  115. out.print("<B>Total number of blocks: "+blocks.size()+"</B><br>");
  116. //generate a table and dump the info
  117. out.println("\n<table>");
  118. for (LocatedBlock cur : blocks) {
  119. out.print("<tr>");
  120. blockId = cur.getBlock().getBlockId();
  121. blockSize = cur.getBlock().getNumBytes();
  122. String blk = "blk_" + Long.toString(blockId);
  123. out.print("<td>"+Long.toString(blockId)+":</td>");
  124. DatanodeInfo[] locs = cur.getLocations();
  125. for(int j=0; j<locs.length; j++) {
  126. String datanodeAddr = locs[j].getName();
  127. datanodePort = Integer.parseInt(datanodeAddr.substring(
  128. datanodeAddr.indexOf(':') + 1,
  129. datanodeAddr.length()));
  130. fqdn = InetAddress.getByName(locs[j].getHost()).getCanonicalHostName();
  131. String blockUrl = "http://"+ fqdn + ":" +
  132. locs[j].getInfoPort() +
  133. "/browseBlock.jsp?blockId=" + Long.toString(blockId) +
  134. "&blockSize=" + blockSize +
  135. "&filename=" + URLEncoder.encode(filename, "UTF-8")+
  136. "&datanodePort=" + datanodePort +
  137. "&genstamp=" + cur.getBlock().getGenerationStamp() +
  138. "&namenodeInfoPort=" + namenodeInfoPort +
  139. "&chunkSizeToView=" + chunkSizeToView;
  140. out.print("<td>&nbsp</td>"
  141. + "<td><a href=\"" + blockUrl + "\">" + datanodeAddr + "</a></td>");
  142. }
  143. out.println("</tr>");
  144. }
  145. out.println("</table>");
  146. out.print("<hr>");
  147. String namenodeHost = jspHelper.nameNodeAddr.getHostName();
  148. out.print("<br><a href=\"http://" +
  149. InetAddress.getByName(namenodeHost).getCanonicalHostName() + ":" +
  150. namenodeInfoPort + "/dfshealth.jsp\">Go back to DFS home</a>");
  151. dfs.close();
  152. }
  153. public void generateFileChunks(JspWriter out, HttpServletRequest req)
  154. throws IOException {
  155. long startOffset = 0;
  156. int datanodePort = 0;
  157. int chunkSizeToView = 0;
  158. String namenodeInfoPortStr = req.getParameter("namenodeInfoPort");
  159. int namenodeInfoPort = -1;
  160. if (namenodeInfoPortStr != null)
  161. namenodeInfoPort = Integer.parseInt(namenodeInfoPortStr);
  162. String filename = req.getParameter("filename");
  163. if (filename == null) {
  164. out.print("Invalid input (filename absent)");
  165. return;
  166. }
  167. String blockIdStr = null;
  168. long blockId = 0;
  169. blockIdStr = req.getParameter("blockId");
  170. if (blockIdStr == null) {
  171. out.print("Invalid input (blockId absent)");
  172. return;
  173. }
  174. blockId = Long.parseLong(blockIdStr);
  175. String blockGenStamp = null;
  176. long genStamp = 0;
  177. blockGenStamp = req.getParameter("genstamp");
  178. if (blockGenStamp == null) {
  179. out.print("Invalid input (genstamp absent)");
  180. return;
  181. }
  182. genStamp = Long.parseLong(blockGenStamp);
  183. String blockSizeStr;
  184. long blockSize = 0;
  185. blockSizeStr = req.getParameter("blockSize");
  186. if (blockSizeStr == null) {
  187. out.print("Invalid input (blockSize absent)");
  188. return;
  189. }
  190. blockSize = Long.parseLong(blockSizeStr);
  191. String chunkSizeToViewStr = req.getParameter("chunkSizeToView");
  192. if (chunkSizeToViewStr != null && Integer.parseInt(chunkSizeToViewStr) > 0)
  193. chunkSizeToView = Integer.parseInt(chunkSizeToViewStr);
  194. else chunkSizeToView = jspHelper.defaultChunkSizeToView;
  195. String startOffsetStr = req.getParameter("startOffset");
  196. if (startOffsetStr == null || Long.parseLong(startOffsetStr) < 0)
  197. startOffset = 0;
  198. else startOffset = Long.parseLong(startOffsetStr);
  199. String datanodePortStr = req.getParameter("datanodePort");
  200. if (datanodePortStr == null) {
  201. out.print("Invalid input (datanodePort absent)");
  202. return;
  203. }
  204. datanodePort = Integer.parseInt(datanodePortStr);
  205. out.print("<h3>File: ");
  206. JspHelper.printPathWithLinks(filename, out, namenodeInfoPort);
  207. out.print("</h3><hr>");
  208. String parent = new File(filename).getParent();
  209. JspHelper.printGotoForm(out, namenodeInfoPort, parent);
  210. out.print("<hr>");
  211. out.print("<a href=\"http://" + req.getServerName() + ":" +
  212. req.getServerPort() +
  213. "/browseDirectory.jsp?dir=" +
  214. URLEncoder.encode(parent, "UTF-8") +
  215. "&namenodeInfoPort=" + namenodeInfoPort +
  216. "\"><i>Go back to dir listing</i></a><br>");
  217. out.print("<a href=\"#viewOptions\">Advanced view/download options</a><br>");
  218. out.print("<hr>");
  219. //Determine the prev & next blocks
  220. DFSClient dfs = new DFSClient(jspHelper.nameNodeAddr, jspHelper.conf);
  221. long nextStartOffset = 0;
  222. long nextBlockSize = 0;
  223. String nextBlockIdStr = null;
  224. String nextGenStamp = null;
  225. String nextHost = req.getServerName();
  226. int nextPort = req.getServerPort();
  227. int nextDatanodePort = datanodePort;
  228. //determine data for the next link
  229. if (startOffset + chunkSizeToView >= blockSize) {
  230. //we have to go to the next block from this point onwards
  231. List<LocatedBlock> blocks =
  232. dfs.namenode.getBlockLocations(filename, 0, Long.MAX_VALUE).getLocatedBlocks();
  233. for (int i = 0; i < blocks.size(); i++) {
  234. if (blocks.get(i).getBlock().getBlockId() == blockId) {
  235. if (i != blocks.size() - 1) {
  236. LocatedBlock nextBlock = blocks.get(i+1);
  237. nextBlockIdStr = Long.toString(nextBlock.getBlock().getBlockId());
  238. nextGenStamp = Long.toString(nextBlock.getBlock().getGenerationStamp());
  239. nextStartOffset = 0;
  240. nextBlockSize = nextBlock.getBlock().getNumBytes();
  241. DatanodeInfo d = jspHelper.bestNode(nextBlock);
  242. String datanodeAddr = d.getName();
  243. nextDatanodePort = Integer.parseInt(
  244. datanodeAddr.substring(
  245. datanodeAddr.indexOf(':') + 1,
  246. datanodeAddr.length()));
  247. nextHost = InetAddress.getByName(d.getHost()).getCanonicalHostName();
  248. nextPort = d.getInfoPort();
  249. }
  250. }
  251. }
  252. }
  253. else {
  254. //we are in the same block
  255. nextBlockIdStr = blockIdStr;
  256. nextStartOffset = startOffset + chunkSizeToView;
  257. nextBlockSize = blockSize;
  258. nextGenStamp = blockGenStamp;
  259. }
  260. String nextUrl = null;
  261. if (nextBlockIdStr != null) {
  262. nextUrl = "http://" + nextHost + ":" +
  263. nextPort +
  264. "/browseBlock.jsp?blockId=" + nextBlockIdStr +
  265. "&blockSize=" + nextBlockSize + "&startOffset=" +
  266. nextStartOffset +
  267. "&genstamp=" + nextGenStamp +
  268. "&filename=" + URLEncoder.encode(filename, "UTF-8") +
  269. "&chunkSizeToView=" + chunkSizeToView +
  270. "&datanodePort=" + nextDatanodePort +
  271. "&namenodeInfoPort=" + namenodeInfoPort;
  272. out.print("<a href=\"" + nextUrl + "\">View Next chunk</a>&nbsp;&nbsp;");
  273. }
  274. //determine data for the prev link
  275. String prevBlockIdStr = null;
  276. String prevGenStamp = null;
  277. long prevStartOffset = 0;
  278. long prevBlockSize = 0;
  279. String prevHost = req.getServerName();
  280. int prevPort = req.getServerPort();
  281. int prevDatanodePort = datanodePort;
  282. if (startOffset == 0) {
  283. List<LocatedBlock> blocks =
  284. dfs.namenode.getBlockLocations(filename, 0, Long.MAX_VALUE).getLocatedBlocks();
  285. for (int i = 0; i < blocks.size(); i++) {
  286. if (blocks.get(i).getBlock().getBlockId() == blockId) {
  287. if (i != 0) {
  288. LocatedBlock prevBlock = blocks.get(i-1);
  289. prevBlockIdStr = Long.toString(prevBlock.getBlock().getBlockId());
  290. prevGenStamp = Long.toString(prevBlock.getBlock().getGenerationStamp());
  291. prevStartOffset = prevBlock.getBlock().getNumBytes() - chunkSizeToView;
  292. if (prevStartOffset < 0)
  293. prevStartOffset = 0;
  294. prevBlockSize = prevBlock.getBlock().getNumBytes();
  295. DatanodeInfo d = jspHelper.bestNode(prevBlock);
  296. String datanodeAddr = d.getName();
  297. prevDatanodePort = Integer.parseInt(
  298. datanodeAddr.substring(
  299. datanodeAddr.indexOf(':') + 1,
  300. datanodeAddr.length()));
  301. prevHost = InetAddress.getByName(d.getHost()).getCanonicalHostName();
  302. prevPort = d.getInfoPort();
  303. }
  304. }
  305. }
  306. }
  307. else {
  308. //we are in the same block
  309. prevBlockIdStr = blockIdStr;
  310. prevStartOffset = startOffset - chunkSizeToView;
  311. if (prevStartOffset < 0) prevStartOffset = 0;
  312. prevBlockSize = blockSize;
  313. prevGenStamp = blockGenStamp;
  314. }
  315. String prevUrl = null;
  316. if (prevBlockIdStr != null) {
  317. prevUrl = "http://" + prevHost + ":" +
  318. prevPort +
  319. "/browseBlock.jsp?blockId=" + prevBlockIdStr +
  320. "&blockSize=" + prevBlockSize + "&startOffset=" +
  321. prevStartOffset +
  322. "&filename=" + URLEncoder.encode(filename, "UTF-8") +
  323. "&chunkSizeToView=" + chunkSizeToView +
  324. "&genstamp=" + prevGenStamp +
  325. "&datanodePort=" + prevDatanodePort +
  326. "&namenodeInfoPort=" + namenodeInfoPort;
  327. out.print("<a href=\"" + prevUrl + "\">View Prev chunk</a>&nbsp;&nbsp;");
  328. }
  329. out.print("<hr>");
  330. out.print("<textarea cols=\"100\" rows=\"25\" wrap=\"virtual\" style=\"width:100%\" READONLY>");
  331. try {
  332. jspHelper.streamBlockInAscii(
  333. new InetSocketAddress(req.getServerName(), datanodePort), blockId,
  334. genStamp, blockSize, startOffset, chunkSizeToView, out);
  335. } catch (Exception e){
  336. out.print(e);
  337. }
  338. out.print("</textarea>");
  339. dfs.close();
  340. }
  341. %>
  342. <html>
  343. <head>
  344. <%JspHelper.createTitle(out, request, request.getParameter("filename")); %>
  345. </head>
  346. <body onload="document.goto.dir.focus()">
  347. <%
  348. generateFileChunks(out,request);
  349. %>
  350. <hr>
  351. <%
  352. generateFileDetails(out,request);
  353. %>
  354. <h2>Local logs</h2>
  355. <a href="/logs/">Log</a> directory
  356. <%
  357. out.println(ServletUtil.htmlFooter());
  358. %>