browseBlock.jsp 14 KB

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