mapred_tutorial.html 120 KB

12345678910111213141516171819202122232425262728293031323334353637383940414243444546474849505152535455565758596061626364656667686970717273747576777879808182838485868788899091929394959697989910010110210310410510610710810911011111211311411511611711811912012112212312412512612712812913013113213313413513613713813914014114214314414514614714814915015115215315415515615715815916016116216316416516616716816917017117217317417517617717817918018118218318418518618718818919019119219319419519619719819920020120220320420520620720820921021121221321421521621721821922022122222322422522622722822923023123223323423523623723823924024124224324424524624724824925025125225325425525625725825926026126226326426526626726826927027127227327427527627727827928028128228328428528628728828929029129229329429529629729829930030130230330430530630730830931031131231331431531631731831932032132232332432532632732832933033133233333433533633733833934034134234334434534634734834935035135235335435535635735835936036136236336436536636736836937037137237337437537637737837938038138238338438538638738838939039139239339439539639739839940040140240340440540640740840941041141241341441541641741841942042142242342442542642742842943043143243343443543643743843944044144244344444544644744844945045145245345445545645745845946046146246346446546646746846947047147247347447547647747847948048148248348448548648748848949049149249349449549649749849950050150250350450550650750850951051151251351451551651751851952052152252352452552652752852953053153253353453553653753853954054154254354454554654754854955055155255355455555655755855956056156256356456556656756856957057157257357457557657757857958058158258358458558658758858959059159259359459559659759859960060160260360460560660760860961061161261361461561661761861962062162262362462562662762862963063163263363463563663763863964064164264364464564664764864965065165265365465565665765865966066166266366466566666766866967067167267367467567667767867968068168268368468568668768868969069169269369469569669769869970070170270370470570670770870971071171271371471571671771871972072172272372472572672772872973073173273373473573673773873974074174274374474574674774874975075175275375475575675775875976076176276376476576676776876977077177277377477577677777877978078178278378478578678778878979079179279379479579679779879980080180280380480580680780880981081181281381481581681781881982082182282382482582682782882983083183283383483583683783883984084184284384484584684784884985085185285385485585685785885986086186286386486586686786886987087187287387487587687787887988088188288388488588688788888989089189289389489589689789889990090190290390490590690790890991091191291391491591691791891992092192292392492592692792892993093193293393493593693793893994094194294394494594694794894995095195295395495595695795895996096196296396496596696796896997097197297397497597697797897998098198298398498598698798898999099199299399499599699799899910001001100210031004100510061007100810091010101110121013101410151016101710181019102010211022102310241025102610271028102910301031103210331034103510361037103810391040104110421043104410451046104710481049105010511052105310541055105610571058105910601061106210631064106510661067106810691070107110721073107410751076107710781079108010811082108310841085108610871088108910901091109210931094109510961097109810991100110111021103110411051106110711081109111011111112111311141115111611171118111911201121112211231124112511261127112811291130113111321133113411351136113711381139114011411142114311441145114611471148114911501151115211531154115511561157115811591160116111621163116411651166116711681169117011711172117311741175117611771178117911801181118211831184118511861187118811891190119111921193119411951196119711981199120012011202120312041205120612071208120912101211121212131214121512161217121812191220122112221223122412251226122712281229123012311232123312341235123612371238123912401241124212431244124512461247124812491250125112521253125412551256125712581259126012611262126312641265126612671268126912701271127212731274127512761277127812791280128112821283128412851286128712881289129012911292129312941295129612971298129913001301130213031304130513061307130813091310131113121313131413151316131713181319132013211322132313241325132613271328132913301331133213331334133513361337133813391340134113421343134413451346134713481349135013511352135313541355135613571358135913601361136213631364136513661367136813691370137113721373137413751376137713781379138013811382138313841385138613871388138913901391139213931394139513961397139813991400140114021403140414051406140714081409141014111412141314141415141614171418141914201421142214231424142514261427142814291430143114321433143414351436143714381439144014411442144314441445144614471448144914501451145214531454145514561457145814591460146114621463146414651466146714681469147014711472147314741475147614771478147914801481148214831484148514861487148814891490149114921493149414951496149714981499150015011502150315041505150615071508150915101511151215131514151515161517151815191520152115221523152415251526152715281529153015311532153315341535153615371538153915401541154215431544154515461547154815491550155115521553155415551556155715581559156015611562156315641565156615671568156915701571157215731574157515761577157815791580158115821583158415851586158715881589159015911592159315941595159615971598159916001601160216031604160516061607160816091610161116121613161416151616161716181619162016211622162316241625162616271628162916301631163216331634163516361637163816391640164116421643164416451646164716481649165016511652165316541655165616571658165916601661166216631664166516661667166816691670167116721673167416751676167716781679168016811682168316841685168616871688168916901691169216931694169516961697169816991700170117021703170417051706170717081709171017111712171317141715171617171718171917201721172217231724172517261727172817291730173117321733173417351736173717381739174017411742174317441745174617471748174917501751175217531754175517561757175817591760176117621763176417651766176717681769177017711772177317741775177617771778177917801781178217831784178517861787178817891790179117921793179417951796179717981799180018011802180318041805180618071808180918101811181218131814181518161817181818191820182118221823182418251826182718281829183018311832183318341835183618371838183918401841184218431844184518461847184818491850185118521853185418551856185718581859186018611862186318641865186618671868186918701871187218731874187518761877187818791880188118821883188418851886188718881889189018911892189318941895189618971898189919001901190219031904190519061907190819091910191119121913191419151916191719181919192019211922192319241925192619271928192919301931193219331934193519361937193819391940194119421943194419451946194719481949195019511952195319541955195619571958195919601961196219631964196519661967196819691970197119721973197419751976197719781979198019811982198319841985198619871988198919901991199219931994199519961997199819992000200120022003200420052006200720082009201020112012201320142015201620172018201920202021202220232024202520262027202820292030203120322033203420352036203720382039204020412042204320442045204620472048204920502051205220532054205520562057205820592060206120622063206420652066206720682069207020712072207320742075207620772078207920802081208220832084208520862087208820892090209120922093209420952096209720982099210021012102210321042105210621072108210921102111211221132114211521162117211821192120212121222123212421252126212721282129213021312132213321342135213621372138213921402141214221432144214521462147214821492150215121522153215421552156215721582159216021612162216321642165216621672168216921702171217221732174217521762177217821792180218121822183218421852186218721882189219021912192219321942195219621972198219922002201220222032204220522062207220822092210221122122213221422152216221722182219222022212222222322242225222622272228222922302231223222332234223522362237223822392240224122422243224422452246224722482249225022512252225322542255225622572258225922602261226222632264226522662267226822692270227122722273227422752276227722782279228022812282228322842285228622872288228922902291229222932294229522962297229822992300230123022303230423052306230723082309231023112312231323142315231623172318231923202321232223232324232523262327232823292330233123322333233423352336233723382339234023412342234323442345234623472348234923502351235223532354235523562357235823592360236123622363236423652366236723682369237023712372237323742375237623772378237923802381238223832384238523862387238823892390239123922393239423952396239723982399240024012402240324042405240624072408240924102411241224132414241524162417241824192420242124222423242424252426242724282429243024312432243324342435243624372438243924402441244224432444244524462447244824492450245124522453245424552456245724582459246024612462246324642465246624672468246924702471247224732474247524762477247824792480248124822483248424852486248724882489249024912492249324942495249624972498249925002501250225032504250525062507250825092510251125122513251425152516251725182519252025212522252325242525252625272528252925302531253225332534253525362537253825392540254125422543254425452546254725482549255025512552255325542555255625572558255925602561256225632564256525662567256825692570257125722573257425752576257725782579258025812582258325842585258625872588258925902591259225932594259525962597259825992600260126022603260426052606260726082609261026112612261326142615261626172618261926202621262226232624262526262627262826292630263126322633263426352636263726382639264026412642264326442645264626472648264926502651265226532654265526562657265826592660266126622663266426652666266726682669267026712672267326742675267626772678267926802681268226832684268526862687268826892690269126922693269426952696269726982699270027012702270327042705270627072708270927102711271227132714271527162717271827192720272127222723272427252726272727282729273027312732273327342735273627372738273927402741274227432744274527462747274827492750275127522753275427552756275727582759276027612762276327642765276627672768276927702771277227732774277527762777277827792780278127822783278427852786278727882789279027912792279327942795279627972798279928002801280228032804280528062807280828092810281128122813281428152816281728182819282028212822282328242825282628272828282928302831283228332834283528362837283828392840284128422843284428452846284728482849285028512852285328542855285628572858285928602861286228632864286528662867286828692870287128722873287428752876287728782879288028812882288328842885288628872888288928902891289228932894289528962897289828992900290129022903290429052906290729082909291029112912291329142915291629172918291929202921292229232924292529262927292829292930293129322933293429352936293729382939294029412942294329442945294629472948294929502951295229532954295529562957295829592960296129622963296429652966296729682969297029712972297329742975297629772978297929802981298229832984298529862987298829892990299129922993299429952996299729982999300030013002300330043005300630073008300930103011301230133014301530163017301830193020302130223023302430253026302730283029303030313032303330343035303630373038303930403041304230433044304530463047304830493050305130523053305430553056305730583059306030613062306330643065306630673068306930703071307230733074307530763077307830793080308130823083308430853086308730883089309030913092309330943095309630973098309931003101310231033104310531063107310831093110311131123113311431153116311731183119312031213122312331243125312631273128312931303131313231333134313531363137313831393140314131423143314431453146314731483149315031513152315331543155315631573158315931603161316231633164316531663167316831693170317131723173317431753176317731783179318031813182318331843185318631873188318931903191319231933194319531963197319831993200320132023203320432053206320732083209321032113212321332143215321632173218321932203221322232233224322532263227322832293230323132323233323432353236323732383239324032413242324332443245324632473248324932503251325232533254325532563257325832593260326132623263326432653266326732683269327032713272327332743275327632773278327932803281328232833284328532863287328832893290329132923293329432953296329732983299330033013302330333043305330633073308330933103311331233133314331533163317331833193320332133223323332433253326332733283329333033313332333333343335333633373338333933403341334233433344334533463347334833493350335133523353335433553356335733583359336033613362336333643365336633673368336933703371337233733374337533763377337833793380338133823383338433853386338733883389339033913392339333943395339633973398339934003401340234033404340534063407340834093410341134123413341434153416341734183419342034213422342334243425342634273428342934303431343234333434343534363437343834393440344134423443344434453446344734483449345034513452345334543455345634573458345934603461346234633464346534663467346834693470347134723473347434753476347734783479348034813482348334843485348634873488348934903491349234933494349534963497349834993500350135023503350435053506350735083509351035113512351335143515351635173518351935203521352235233524352535263527352835293530353135323533353435353536
  1. <!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
  2. <html>
  3. <head>
  4. <META http-equiv="Content-Type" content="text/html; charset=UTF-8">
  5. <meta content="Apache Forrest" name="Generator">
  6. <meta name="Forrest-version" content="0.8">
  7. <meta name="Forrest-skin-name" content="pelt">
  8. <title>Hadoop Map-Reduce Tutorial</title>
  9. <link type="text/css" href="skin/basic.css" rel="stylesheet">
  10. <link media="screen" type="text/css" href="skin/screen.css" rel="stylesheet">
  11. <link media="print" type="text/css" href="skin/print.css" rel="stylesheet">
  12. <link type="text/css" href="skin/profile.css" rel="stylesheet">
  13. <script src="skin/getBlank.js" language="javascript" type="text/javascript"></script><script src="skin/getMenu.js" language="javascript" type="text/javascript"></script><script src="skin/fontsize.js" language="javascript" type="text/javascript"></script>
  14. <link rel="shortcut icon" href="images/favicon.ico">
  15. </head>
  16. <body onload="init()">
  17. <script type="text/javascript">ndeSetTextSize();</script>
  18. <div id="top">
  19. <!--+
  20. |breadtrail
  21. +-->
  22. <div class="breadtrail">
  23. <a href="http://www.apache.org/">Apache</a> &gt; <a href="http://hadoop.apache.org/">Hadoop</a> &gt; <a href="http://hadoop.apache.org/core/">Core</a><script src="skin/breadcrumbs.js" language="JavaScript" type="text/javascript"></script>
  24. </div>
  25. <!--+
  26. |header
  27. +-->
  28. <div class="header">
  29. <!--+
  30. |start group logo
  31. +-->
  32. <div class="grouplogo">
  33. <a href="http://hadoop.apache.org/"><img class="logoImage" alt="Hadoop" src="images/hadoop-logo.jpg" title="Apache Hadoop"></a>
  34. </div>
  35. <!--+
  36. |end group logo
  37. +-->
  38. <!--+
  39. |start Project Logo
  40. +-->
  41. <div class="projectlogo">
  42. <a href="http://hadoop.apache.org/core/"><img class="logoImage" alt="Hadoop" src="images/core-logo.gif" title="Scalable Computing Platform"></a>
  43. </div>
  44. <!--+
  45. |end Project Logo
  46. +-->
  47. <!--+
  48. |start Search
  49. +-->
  50. <div class="searchbox">
  51. <form action="http://www.google.com/search" method="get" class="roundtopsmall">
  52. <input value="hadoop.apache.org" name="sitesearch" type="hidden"><input onFocus="getBlank (this, 'Search the site with google');" size="25" name="q" id="query" type="text" value="Search the site with google">&nbsp;
  53. <input name="Search" value="Search" type="submit">
  54. </form>
  55. </div>
  56. <!--+
  57. |end search
  58. +-->
  59. <!--+
  60. |start Tabs
  61. +-->
  62. <ul id="tabs">
  63. <li>
  64. <a class="unselected" href="http://hadoop.apache.org/core/">Project</a>
  65. </li>
  66. <li>
  67. <a class="unselected" href="http://wiki.apache.org/hadoop">Wiki</a>
  68. </li>
  69. <li class="current">
  70. <a class="selected" href="index.html">Hadoop 0.18 Documentation</a>
  71. </li>
  72. </ul>
  73. <!--+
  74. |end Tabs
  75. +-->
  76. </div>
  77. </div>
  78. <div id="main">
  79. <div id="publishedStrip">
  80. <!--+
  81. |start Subtabs
  82. +-->
  83. <div id="level2tabs"></div>
  84. <!--+
  85. |end Endtabs
  86. +-->
  87. <script type="text/javascript"><!--
  88. document.write("Last Published: " + document.lastModified);
  89. // --></script>
  90. </div>
  91. <!--+
  92. |breadtrail
  93. +-->
  94. <div class="breadtrail">
  95. &nbsp;
  96. </div>
  97. <!--+
  98. |start Menu, mainarea
  99. +-->
  100. <!--+
  101. |start Menu
  102. +-->
  103. <div id="menu">
  104. <div onclick="SwitchMenu('menu_selected_1.1', 'skin/')" id="menu_selected_1.1Title" class="menutitle" style="background-image: url('skin/images/chapter_open.gif');">Documentation</div>
  105. <div id="menu_selected_1.1" class="selectedmenuitemgroup" style="display: block;">
  106. <div class="menuitem">
  107. <a href="index.html">Overview</a>
  108. </div>
  109. <div class="menuitem">
  110. <a href="quickstart.html">Quickstart</a>
  111. </div>
  112. <div class="menuitem">
  113. <a href="cluster_setup.html">Cluster Setup</a>
  114. </div>
  115. <div class="menuitem">
  116. <a href="hdfs_design.html">HDFS Architecture</a>
  117. </div>
  118. <div class="menuitem">
  119. <a href="hdfs_user_guide.html">HDFS User Guide</a>
  120. </div>
  121. <div class="menuitem">
  122. <a href="hdfs_permissions_guide.html">HDFS Permissions Guide</a>
  123. </div>
  124. <div class="menuitem">
  125. <a href="hdfs_quota_admin_guide.html">HDFS Quotas Administrator Guide</a>
  126. </div>
  127. <div class="menuitem">
  128. <a href="hdfs_shell.html">FS Shell Guide</a>
  129. </div>
  130. <div class="menupage">
  131. <div class="menupagetitle">Map-Reduce Tutorial</div>
  132. </div>
  133. <div class="menuitem">
  134. <a href="native_libraries.html">Native Hadoop Libraries</a>
  135. </div>
  136. <div class="menuitem">
  137. <a href="streaming.html">Streaming</a>
  138. </div>
  139. <div class="menuitem">
  140. <a href="hadoop_archives.html">Hadoop Archives</a>
  141. </div>
  142. <div class="menuitem">
  143. <a href="hod.html">Hadoop On Demand</a>
  144. </div>
  145. <div class="menuitem">
  146. <a href="api/index.html">API Docs</a>
  147. </div>
  148. <div class="menuitem">
  149. <a href="http://wiki.apache.org/hadoop/">Wiki</a>
  150. </div>
  151. <div class="menuitem">
  152. <a href="http://wiki.apache.org/hadoop/FAQ">FAQ</a>
  153. </div>
  154. <div class="menuitem">
  155. <a href="http://hadoop.apache.org/core/mailing_lists.html">Mailing Lists</a>
  156. </div>
  157. <div class="menuitem">
  158. <a href="releasenotes.html">Release Notes</a>
  159. </div>
  160. <div class="menuitem">
  161. <a href="changes.html">All Changes</a>
  162. </div>
  163. </div>
  164. <div id="credit"></div>
  165. <div id="roundbottom">
  166. <img style="display: none" class="corner" height="15" width="15" alt="" src="skin/images/rc-b-l-15-1body-2menu-3menu.png"></div>
  167. <!--+
  168. |alternative credits
  169. +-->
  170. <div id="credit2"></div>
  171. </div>
  172. <!--+
  173. |end Menu
  174. +-->
  175. <!--+
  176. |start content
  177. +-->
  178. <div id="content">
  179. <div title="Portable Document Format" class="pdflink">
  180. <a class="dida" href="mapred_tutorial.pdf"><img alt="PDF -icon" src="skin/images/pdfdoc.gif" class="skin"><br>
  181. PDF</a>
  182. </div>
  183. <h1>Hadoop Map-Reduce Tutorial</h1>
  184. <div id="minitoc-area">
  185. <ul class="minitoc">
  186. <li>
  187. <a href="#Purpose">Purpose</a>
  188. </li>
  189. <li>
  190. <a href="#Pre-requisites">Pre-requisites</a>
  191. </li>
  192. <li>
  193. <a href="#Overview">Overview</a>
  194. </li>
  195. <li>
  196. <a href="#Inputs+and+Outputs">Inputs and Outputs</a>
  197. </li>
  198. <li>
  199. <a href="#Example%3A+WordCount+v1.0">Example: WordCount v1.0</a>
  200. <ul class="minitoc">
  201. <li>
  202. <a href="#Source+Code">Source Code</a>
  203. </li>
  204. <li>
  205. <a href="#Usage">Usage</a>
  206. </li>
  207. <li>
  208. <a href="#Walk-through">Walk-through</a>
  209. </li>
  210. </ul>
  211. </li>
  212. <li>
  213. <a href="#Map-Reduce+-+User+Interfaces">Map-Reduce - User Interfaces</a>
  214. <ul class="minitoc">
  215. <li>
  216. <a href="#Payload">Payload</a>
  217. <ul class="minitoc">
  218. <li>
  219. <a href="#Mapper">Mapper</a>
  220. </li>
  221. <li>
  222. <a href="#Reducer">Reducer</a>
  223. </li>
  224. <li>
  225. <a href="#Partitioner">Partitioner</a>
  226. </li>
  227. <li>
  228. <a href="#Reporter">Reporter</a>
  229. </li>
  230. <li>
  231. <a href="#OutputCollector">OutputCollector</a>
  232. </li>
  233. </ul>
  234. </li>
  235. <li>
  236. <a href="#Job+Configuration">Job Configuration</a>
  237. </li>
  238. <li>
  239. <a href="#Task+Execution+%26+Environment">Task Execution &amp; Environment</a>
  240. </li>
  241. <li>
  242. <a href="#Job+Submission+and+Monitoring">Job Submission and Monitoring</a>
  243. <ul class="minitoc">
  244. <li>
  245. <a href="#Job+Control">Job Control</a>
  246. </li>
  247. </ul>
  248. </li>
  249. <li>
  250. <a href="#Job+Input">Job Input</a>
  251. <ul class="minitoc">
  252. <li>
  253. <a href="#InputSplit">InputSplit</a>
  254. </li>
  255. <li>
  256. <a href="#RecordReader">RecordReader</a>
  257. </li>
  258. </ul>
  259. </li>
  260. <li>
  261. <a href="#Job+Output">Job Output</a>
  262. <ul class="minitoc">
  263. <li>
  264. <a href="#Task+Side-Effect+Files">Task Side-Effect Files</a>
  265. </li>
  266. <li>
  267. <a href="#RecordWriter">RecordWriter</a>
  268. </li>
  269. </ul>
  270. </li>
  271. <li>
  272. <a href="#Other+Useful+Features">Other Useful Features</a>
  273. <ul class="minitoc">
  274. <li>
  275. <a href="#Counters">Counters</a>
  276. </li>
  277. <li>
  278. <a href="#DistributedCache">DistributedCache</a>
  279. </li>
  280. <li>
  281. <a href="#Tool">Tool</a>
  282. </li>
  283. <li>
  284. <a href="#IsolationRunner">IsolationRunner</a>
  285. </li>
  286. <li>
  287. <a href="#Debugging">Debugging</a>
  288. </li>
  289. <li>
  290. <a href="#JobControl">JobControl</a>
  291. </li>
  292. <li>
  293. <a href="#Data+Compression">Data Compression</a>
  294. </li>
  295. </ul>
  296. </li>
  297. </ul>
  298. </li>
  299. <li>
  300. <a href="#Example%3A+WordCount+v2.0">Example: WordCount v2.0</a>
  301. <ul class="minitoc">
  302. <li>
  303. <a href="#Source+Code-N10C87">Source Code</a>
  304. </li>
  305. <li>
  306. <a href="#Sample+Runs">Sample Runs</a>
  307. </li>
  308. <li>
  309. <a href="#Highlights">Highlights</a>
  310. </li>
  311. </ul>
  312. </li>
  313. </ul>
  314. </div>
  315. <a name="N1000D"></a><a name="Purpose"></a>
  316. <h2 class="h3">Purpose</h2>
  317. <div class="section">
  318. <p>This document comprehensively describes all user-facing facets of the
  319. Hadoop Map-Reduce framework and serves as a tutorial.
  320. </p>
  321. </div>
  322. <a name="N10017"></a><a name="Pre-requisites"></a>
  323. <h2 class="h3">Pre-requisites</h2>
  324. <div class="section">
  325. <p>Ensure that Hadoop is installed, configured and is running. More
  326. details:</p>
  327. <ul>
  328. <li>
  329. Hadoop <a href="quickstart.html">Quickstart</a> for first-time users.
  330. </li>
  331. <li>
  332. Hadoop <a href="cluster_setup.html">Cluster Setup</a> for large,
  333. distributed clusters.
  334. </li>
  335. </ul>
  336. </div>
  337. <a name="N10032"></a><a name="Overview"></a>
  338. <h2 class="h3">Overview</h2>
  339. <div class="section">
  340. <p>Hadoop Map-Reduce is a software framework for easily writing
  341. applications which process vast amounts of data (multi-terabyte data-sets)
  342. in-parallel on large clusters (thousands of nodes) of commodity
  343. hardware in a reliable, fault-tolerant manner.</p>
  344. <p>A Map-Reduce <em>job</em> usually splits the input data-set into
  345. independent chunks which are processed by the <em>map tasks</em> in a
  346. completely parallel manner. The framework sorts the outputs of the maps,
  347. which are then input to the <em>reduce tasks</em>. Typically both the
  348. input and the output of the job are stored in a file-system. The framework
  349. takes care of scheduling tasks, monitoring them and re-executes the failed
  350. tasks.</p>
  351. <p>Typically the compute nodes and the storage nodes are the same, that is,
  352. the Map-Reduce framework and the <a href="hdfs_design.html">Distributed
  353. FileSystem</a> are running on the same set of nodes. This configuration
  354. allows the framework to effectively schedule tasks on the nodes where data
  355. is already present, resulting in very high aggregate bandwidth across the
  356. cluster.</p>
  357. <p>The Map-Reduce framework consists of a single master
  358. <span class="codefrag">JobTracker</span> and one slave <span class="codefrag">TaskTracker</span> per
  359. cluster-node. The master is responsible for scheduling the jobs' component
  360. tasks on the slaves, monitoring them and re-executing the failed tasks. The
  361. slaves execute the tasks as directed by the master.</p>
  362. <p>Minimally, applications specify the input/output locations and supply
  363. <em>map</em> and <em>reduce</em> functions via implementations of
  364. appropriate interfaces and/or abstract-classes. These, and other job
  365. parameters, comprise the <em>job configuration</em>. The Hadoop
  366. <em>job client</em> then submits the job (jar/executable etc.) and
  367. configuration to the <span class="codefrag">JobTracker</span> which then assumes the
  368. responsibility of distributing the software/configuration to the slaves,
  369. scheduling tasks and monitoring them, providing status and diagnostic
  370. information to the job-client.</p>
  371. <p>Although the Hadoop framework is implemented in Java<sup>TM</sup>,
  372. Map-Reduce applications need not be written in Java.</p>
  373. <ul>
  374. <li>
  375. <a href="api/org/apache/hadoop/streaming/package-summary.html">
  376. Hadoop Streaming</a> is a utility which allows users to create and run
  377. jobs with any executables (e.g. shell utilities) as the mapper and/or
  378. the reducer.
  379. </li>
  380. <li>
  381. <a href="api/org/apache/hadoop/mapred/pipes/package-summary.html">
  382. Hadoop Pipes</a> is a <a href="http://www.swig.org/">SWIG</a>-
  383. compatible <em>C++ API</em> to implement Map-Reduce applications (non
  384. JNI<sup>TM</sup> based).
  385. </li>
  386. </ul>
  387. </div>
  388. <a name="N1008B"></a><a name="Inputs+and+Outputs"></a>
  389. <h2 class="h3">Inputs and Outputs</h2>
  390. <div class="section">
  391. <p>The Map-Reduce framework operates exclusively on
  392. <span class="codefrag">&lt;key, value&gt;</span> pairs, that is, the framework views the
  393. input to the job as a set of <span class="codefrag">&lt;key, value&gt;</span> pairs and
  394. produces a set of <span class="codefrag">&lt;key, value&gt;</span> pairs as the output of
  395. the job, conceivably of different types.</p>
  396. <p>The <span class="codefrag">key</span> and <span class="codefrag">value</span> classes have to be
  397. serializable by the framework and hence need to implement the
  398. <a href="api/org/apache/hadoop/io/Writable.html">Writable</a>
  399. interface. Additionally, the <span class="codefrag">key</span> classes have to implement the
  400. <a href="api/org/apache/hadoop/io/WritableComparable.html">
  401. WritableComparable</a> interface to facilitate sorting by the framework.
  402. </p>
  403. <p>Input and Output types of a Map-Reduce job:</p>
  404. <p>
  405. (input) <span class="codefrag">&lt;k1, v1&gt;</span>
  406. -&gt;
  407. <strong>map</strong>
  408. -&gt;
  409. <span class="codefrag">&lt;k2, v2&gt;</span>
  410. -&gt;
  411. <strong>combine</strong>
  412. -&gt;
  413. <span class="codefrag">&lt;k2, v2&gt;</span>
  414. -&gt;
  415. <strong>reduce</strong>
  416. -&gt;
  417. <span class="codefrag">&lt;k3, v3&gt;</span> (output)
  418. </p>
  419. </div>
  420. <a name="N100CD"></a><a name="Example%3A+WordCount+v1.0"></a>
  421. <h2 class="h3">Example: WordCount v1.0</h2>
  422. <div class="section">
  423. <p>Before we jump into the details, lets walk through an example Map-Reduce
  424. application to get a flavour for how they work.</p>
  425. <p>
  426. <span class="codefrag">WordCount</span> is a simple application that counts the number of
  427. occurences of each word in a given input set.</p>
  428. <p>This works with a
  429. <a href="quickstart.html#Standalone+Operation">local-standalone</a>,
  430. <a href="quickstart.html#SingleNodeSetup">pseudo-distributed</a> or
  431. <a href="quickstart.html#Fully-Distributed+Operation">fully-distributed</a>
  432. Hadoop installation.</p>
  433. <a name="N100EA"></a><a name="Source+Code"></a>
  434. <h3 class="h4">Source Code</h3>
  435. <table class="ForrestTable" cellspacing="1" cellpadding="4">
  436. <tr>
  437. <th colspan="1" rowspan="1"></th>
  438. <th colspan="1" rowspan="1">WordCount.java</th>
  439. </tr>
  440. <tr>
  441. <td colspan="1" rowspan="1">1.</td>
  442. <td colspan="1" rowspan="1">
  443. <span class="codefrag">package org.myorg;</span>
  444. </td>
  445. </tr>
  446. <tr>
  447. <td colspan="1" rowspan="1">2.</td>
  448. <td colspan="1" rowspan="1"></td>
  449. </tr>
  450. <tr>
  451. <td colspan="1" rowspan="1">3.</td>
  452. <td colspan="1" rowspan="1">
  453. <span class="codefrag">import java.io.IOException;</span>
  454. </td>
  455. </tr>
  456. <tr>
  457. <td colspan="1" rowspan="1">4.</td>
  458. <td colspan="1" rowspan="1">
  459. <span class="codefrag">import java.util.*;</span>
  460. </td>
  461. </tr>
  462. <tr>
  463. <td colspan="1" rowspan="1">5.</td>
  464. <td colspan="1" rowspan="1"></td>
  465. </tr>
  466. <tr>
  467. <td colspan="1" rowspan="1">6.</td>
  468. <td colspan="1" rowspan="1">
  469. <span class="codefrag">import org.apache.hadoop.fs.Path;</span>
  470. </td>
  471. </tr>
  472. <tr>
  473. <td colspan="1" rowspan="1">7.</td>
  474. <td colspan="1" rowspan="1">
  475. <span class="codefrag">import org.apache.hadoop.conf.*;</span>
  476. </td>
  477. </tr>
  478. <tr>
  479. <td colspan="1" rowspan="1">8.</td>
  480. <td colspan="1" rowspan="1">
  481. <span class="codefrag">import org.apache.hadoop.io.*;</span>
  482. </td>
  483. </tr>
  484. <tr>
  485. <td colspan="1" rowspan="1">9.</td>
  486. <td colspan="1" rowspan="1">
  487. <span class="codefrag">import org.apache.hadoop.mapred.*;</span>
  488. </td>
  489. </tr>
  490. <tr>
  491. <td colspan="1" rowspan="1">10.</td>
  492. <td colspan="1" rowspan="1">
  493. <span class="codefrag">import org.apache.hadoop.util.*;</span>
  494. </td>
  495. </tr>
  496. <tr>
  497. <td colspan="1" rowspan="1">11.</td>
  498. <td colspan="1" rowspan="1"></td>
  499. </tr>
  500. <tr>
  501. <td colspan="1" rowspan="1">12.</td>
  502. <td colspan="1" rowspan="1">
  503. <span class="codefrag">public class WordCount {</span>
  504. </td>
  505. </tr>
  506. <tr>
  507. <td colspan="1" rowspan="1">13.</td>
  508. <td colspan="1" rowspan="1"></td>
  509. </tr>
  510. <tr>
  511. <td colspan="1" rowspan="1">14.</td>
  512. <td colspan="1" rowspan="1">
  513. &nbsp;&nbsp;
  514. <span class="codefrag">
  515. public static class Map extends MapReduceBase
  516. implements Mapper&lt;LongWritable, Text, Text, IntWritable&gt; {
  517. </span>
  518. </td>
  519. </tr>
  520. <tr>
  521. <td colspan="1" rowspan="1">15.</td>
  522. <td colspan="1" rowspan="1">
  523. &nbsp;&nbsp;&nbsp;&nbsp;
  524. <span class="codefrag">
  525. private final static IntWritable one = new IntWritable(1);
  526. </span>
  527. </td>
  528. </tr>
  529. <tr>
  530. <td colspan="1" rowspan="1">16.</td>
  531. <td colspan="1" rowspan="1">
  532. &nbsp;&nbsp;&nbsp;&nbsp;
  533. <span class="codefrag">private Text word = new Text();</span>
  534. </td>
  535. </tr>
  536. <tr>
  537. <td colspan="1" rowspan="1">17.</td>
  538. <td colspan="1" rowspan="1"></td>
  539. </tr>
  540. <tr>
  541. <td colspan="1" rowspan="1">18.</td>
  542. <td colspan="1" rowspan="1">
  543. &nbsp;&nbsp;&nbsp;&nbsp;
  544. <span class="codefrag">
  545. public void map(LongWritable key, Text value,
  546. OutputCollector&lt;Text, IntWritable&gt; output,
  547. Reporter reporter) throws IOException {
  548. </span>
  549. </td>
  550. </tr>
  551. <tr>
  552. <td colspan="1" rowspan="1">19.</td>
  553. <td colspan="1" rowspan="1">
  554. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  555. <span class="codefrag">String line = value.toString();</span>
  556. </td>
  557. </tr>
  558. <tr>
  559. <td colspan="1" rowspan="1">20.</td>
  560. <td colspan="1" rowspan="1">
  561. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  562. <span class="codefrag">StringTokenizer tokenizer = new StringTokenizer(line);</span>
  563. </td>
  564. </tr>
  565. <tr>
  566. <td colspan="1" rowspan="1">21.</td>
  567. <td colspan="1" rowspan="1">
  568. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  569. <span class="codefrag">while (tokenizer.hasMoreTokens()) {</span>
  570. </td>
  571. </tr>
  572. <tr>
  573. <td colspan="1" rowspan="1">22.</td>
  574. <td colspan="1" rowspan="1">
  575. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  576. <span class="codefrag">word.set(tokenizer.nextToken());</span>
  577. </td>
  578. </tr>
  579. <tr>
  580. <td colspan="1" rowspan="1">23.</td>
  581. <td colspan="1" rowspan="1">
  582. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  583. <span class="codefrag">output.collect(word, one);</span>
  584. </td>
  585. </tr>
  586. <tr>
  587. <td colspan="1" rowspan="1">24.</td>
  588. <td colspan="1" rowspan="1">
  589. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  590. <span class="codefrag">}</span>
  591. </td>
  592. </tr>
  593. <tr>
  594. <td colspan="1" rowspan="1">25.</td>
  595. <td colspan="1" rowspan="1">
  596. &nbsp;&nbsp;&nbsp;&nbsp;
  597. <span class="codefrag">}</span>
  598. </td>
  599. </tr>
  600. <tr>
  601. <td colspan="1" rowspan="1">26.</td>
  602. <td colspan="1" rowspan="1">
  603. &nbsp;&nbsp;
  604. <span class="codefrag">}</span>
  605. </td>
  606. </tr>
  607. <tr>
  608. <td colspan="1" rowspan="1">27.</td>
  609. <td colspan="1" rowspan="1"></td>
  610. </tr>
  611. <tr>
  612. <td colspan="1" rowspan="1">28.</td>
  613. <td colspan="1" rowspan="1">
  614. &nbsp;&nbsp;
  615. <span class="codefrag">
  616. public static class Reduce extends MapReduceBase implements
  617. Reducer&lt;Text, IntWritable, Text, IntWritable&gt; {
  618. </span>
  619. </td>
  620. </tr>
  621. <tr>
  622. <td colspan="1" rowspan="1">29.</td>
  623. <td colspan="1" rowspan="1">
  624. &nbsp;&nbsp;&nbsp;&nbsp;
  625. <span class="codefrag">
  626. public void reduce(Text key, Iterator&lt;IntWritable&gt; values,
  627. OutputCollector&lt;Text, IntWritable&gt; output,
  628. Reporter reporter) throws IOException {
  629. </span>
  630. </td>
  631. </tr>
  632. <tr>
  633. <td colspan="1" rowspan="1">30.</td>
  634. <td colspan="1" rowspan="1">
  635. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  636. <span class="codefrag">int sum = 0;</span>
  637. </td>
  638. </tr>
  639. <tr>
  640. <td colspan="1" rowspan="1">31.</td>
  641. <td colspan="1" rowspan="1">
  642. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  643. <span class="codefrag">while (values.hasNext()) {</span>
  644. </td>
  645. </tr>
  646. <tr>
  647. <td colspan="1" rowspan="1">32.</td>
  648. <td colspan="1" rowspan="1">
  649. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  650. <span class="codefrag">sum += values.next().get();</span>
  651. </td>
  652. </tr>
  653. <tr>
  654. <td colspan="1" rowspan="1">33.</td>
  655. <td colspan="1" rowspan="1">
  656. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  657. <span class="codefrag">}</span>
  658. </td>
  659. </tr>
  660. <tr>
  661. <td colspan="1" rowspan="1">34.</td>
  662. <td colspan="1" rowspan="1">
  663. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  664. <span class="codefrag">output.collect(key, new IntWritable(sum));</span>
  665. </td>
  666. </tr>
  667. <tr>
  668. <td colspan="1" rowspan="1">35.</td>
  669. <td colspan="1" rowspan="1">
  670. &nbsp;&nbsp;&nbsp;&nbsp;
  671. <span class="codefrag">}</span>
  672. </td>
  673. </tr>
  674. <tr>
  675. <td colspan="1" rowspan="1">36.</td>
  676. <td colspan="1" rowspan="1">
  677. &nbsp;&nbsp;
  678. <span class="codefrag">}</span>
  679. </td>
  680. </tr>
  681. <tr>
  682. <td colspan="1" rowspan="1">37.</td>
  683. <td colspan="1" rowspan="1"></td>
  684. </tr>
  685. <tr>
  686. <td colspan="1" rowspan="1">38.</td>
  687. <td colspan="1" rowspan="1">
  688. &nbsp;&nbsp;
  689. <span class="codefrag">
  690. public static void main(String[] args) throws Exception {
  691. </span>
  692. </td>
  693. </tr>
  694. <tr>
  695. <td colspan="1" rowspan="1">39.</td>
  696. <td colspan="1" rowspan="1">
  697. &nbsp;&nbsp;&nbsp;&nbsp;
  698. <span class="codefrag">
  699. JobConf conf = new JobConf(WordCount.class);
  700. </span>
  701. </td>
  702. </tr>
  703. <tr>
  704. <td colspan="1" rowspan="1">40.</td>
  705. <td colspan="1" rowspan="1">
  706. &nbsp;&nbsp;&nbsp;&nbsp;
  707. <span class="codefrag">conf.setJobName("wordcount");</span>
  708. </td>
  709. </tr>
  710. <tr>
  711. <td colspan="1" rowspan="1">41.</td>
  712. <td colspan="1" rowspan="1"></td>
  713. </tr>
  714. <tr>
  715. <td colspan="1" rowspan="1">42.</td>
  716. <td colspan="1" rowspan="1">
  717. &nbsp;&nbsp;&nbsp;&nbsp;
  718. <span class="codefrag">conf.setOutputKeyClass(Text.class);</span>
  719. </td>
  720. </tr>
  721. <tr>
  722. <td colspan="1" rowspan="1">43.</td>
  723. <td colspan="1" rowspan="1">
  724. &nbsp;&nbsp;&nbsp;&nbsp;
  725. <span class="codefrag">conf.setOutputValueClass(IntWritable.class);</span>
  726. </td>
  727. </tr>
  728. <tr>
  729. <td colspan="1" rowspan="1">44.</td>
  730. <td colspan="1" rowspan="1"></td>
  731. </tr>
  732. <tr>
  733. <td colspan="1" rowspan="1">45.</td>
  734. <td colspan="1" rowspan="1">
  735. &nbsp;&nbsp;&nbsp;&nbsp;
  736. <span class="codefrag">conf.setMapperClass(Map.class);</span>
  737. </td>
  738. </tr>
  739. <tr>
  740. <td colspan="1" rowspan="1">46.</td>
  741. <td colspan="1" rowspan="1">
  742. &nbsp;&nbsp;&nbsp;&nbsp;
  743. <span class="codefrag">conf.setCombinerClass(Reduce.class);</span>
  744. </td>
  745. </tr>
  746. <tr>
  747. <td colspan="1" rowspan="1">47.</td>
  748. <td colspan="1" rowspan="1">
  749. &nbsp;&nbsp;&nbsp;&nbsp;
  750. <span class="codefrag">conf.setReducerClass(Reduce.class);</span>
  751. </td>
  752. </tr>
  753. <tr>
  754. <td colspan="1" rowspan="1">48.</td>
  755. <td colspan="1" rowspan="1"></td>
  756. </tr>
  757. <tr>
  758. <td colspan="1" rowspan="1">49.</td>
  759. <td colspan="1" rowspan="1">
  760. &nbsp;&nbsp;&nbsp;&nbsp;
  761. <span class="codefrag">conf.setInputFormat(TextInputFormat.class);</span>
  762. </td>
  763. </tr>
  764. <tr>
  765. <td colspan="1" rowspan="1">50.</td>
  766. <td colspan="1" rowspan="1">
  767. &nbsp;&nbsp;&nbsp;&nbsp;
  768. <span class="codefrag">conf.setOutputFormat(TextOutputFormat.class);</span>
  769. </td>
  770. </tr>
  771. <tr>
  772. <td colspan="1" rowspan="1">51.</td>
  773. <td colspan="1" rowspan="1"></td>
  774. </tr>
  775. <tr>
  776. <td colspan="1" rowspan="1">52.</td>
  777. <td colspan="1" rowspan="1">
  778. &nbsp;&nbsp;&nbsp;&nbsp;
  779. <span class="codefrag">FileInputFormat.setInputPaths(conf, new Path(args[0]));</span>
  780. </td>
  781. </tr>
  782. <tr>
  783. <td colspan="1" rowspan="1">53.</td>
  784. <td colspan="1" rowspan="1">
  785. &nbsp;&nbsp;&nbsp;&nbsp;
  786. <span class="codefrag">FileOutputFormat.setOutputPath(conf, new Path(args[1]));</span>
  787. </td>
  788. </tr>
  789. <tr>
  790. <td colspan="1" rowspan="1">54.</td>
  791. <td colspan="1" rowspan="1"></td>
  792. </tr>
  793. <tr>
  794. <td colspan="1" rowspan="1">55.</td>
  795. <td colspan="1" rowspan="1">
  796. &nbsp;&nbsp;&nbsp;&nbsp;
  797. <span class="codefrag">JobClient.runJob(conf);</span>
  798. </td>
  799. </tr>
  800. <tr>
  801. <td colspan="1" rowspan="1">57.</td>
  802. <td colspan="1" rowspan="1">
  803. &nbsp;&nbsp;
  804. <span class="codefrag">}</span>
  805. </td>
  806. </tr>
  807. <tr>
  808. <td colspan="1" rowspan="1">58.</td>
  809. <td colspan="1" rowspan="1">
  810. <span class="codefrag">}</span>
  811. </td>
  812. </tr>
  813. <tr>
  814. <td colspan="1" rowspan="1">59.</td>
  815. <td colspan="1" rowspan="1"></td>
  816. </tr>
  817. </table>
  818. <a name="N1046C"></a><a name="Usage"></a>
  819. <h3 class="h4">Usage</h3>
  820. <p>Assuming <span class="codefrag">HADOOP_HOME</span> is the root of the installation and
  821. <span class="codefrag">HADOOP_VERSION</span> is the Hadoop version installed, compile
  822. <span class="codefrag">WordCount.java</span> and create a jar:</p>
  823. <p>
  824. <span class="codefrag">$ mkdir wordcount_classes</span>
  825. <br>
  826. <span class="codefrag">
  827. $ javac -classpath ${HADOOP_HOME}/hadoop-${HADOOP_VERSION}-core.jar
  828. -d wordcount_classes WordCount.java
  829. </span>
  830. <br>
  831. <span class="codefrag">$ jar -cvf /usr/joe/wordcount.jar -C wordcount_classes/ .</span>
  832. </p>
  833. <p>Assuming that:</p>
  834. <ul>
  835. <li>
  836. <span class="codefrag">/usr/joe/wordcount/input</span> - input directory in HDFS
  837. </li>
  838. <li>
  839. <span class="codefrag">/usr/joe/wordcount/output</span> - output directory in HDFS
  840. </li>
  841. </ul>
  842. <p>Sample text-files as input:</p>
  843. <p>
  844. <span class="codefrag">$ bin/hadoop dfs -ls /usr/joe/wordcount/input/</span>
  845. <br>
  846. <span class="codefrag">/usr/joe/wordcount/input/file01</span>
  847. <br>
  848. <span class="codefrag">/usr/joe/wordcount/input/file02</span>
  849. <br>
  850. <br>
  851. <span class="codefrag">$ bin/hadoop dfs -cat /usr/joe/wordcount/input/file01</span>
  852. <br>
  853. <span class="codefrag">Hello World Bye World</span>
  854. <br>
  855. <br>
  856. <span class="codefrag">$ bin/hadoop dfs -cat /usr/joe/wordcount/input/file02</span>
  857. <br>
  858. <span class="codefrag">Hello Hadoop Goodbye Hadoop</span>
  859. </p>
  860. <p>Run the application:</p>
  861. <p>
  862. <span class="codefrag">
  863. $ bin/hadoop jar /usr/joe/wordcount.jar org.myorg.WordCount
  864. /usr/joe/wordcount/input /usr/joe/wordcount/output
  865. </span>
  866. </p>
  867. <p>Output:</p>
  868. <p>
  869. <span class="codefrag">
  870. $ bin/hadoop dfs -cat /usr/joe/wordcount/output/part-00000
  871. </span>
  872. <br>
  873. <span class="codefrag">Bye 1</span>
  874. <br>
  875. <span class="codefrag">Goodbye 1</span>
  876. <br>
  877. <span class="codefrag">Hadoop 2</span>
  878. <br>
  879. <span class="codefrag">Hello 2</span>
  880. <br>
  881. <span class="codefrag">World 2</span>
  882. <br>
  883. </p>
  884. <a name="N104EC"></a><a name="Walk-through"></a>
  885. <h3 class="h4">Walk-through</h3>
  886. <p>The <span class="codefrag">WordCount</span> application is quite straight-forward.</p>
  887. <p>The <span class="codefrag">Mapper</span> implementation (lines 14-26), via the
  888. <span class="codefrag">map</span> method (lines 18-25), processes one line at a time,
  889. as provided by the specified <span class="codefrag">TextInputFormat</span> (line 49).
  890. It then splits the line into tokens separated by whitespaces, via the
  891. <span class="codefrag">StringTokenizer</span>, and emits a key-value pair of
  892. <span class="codefrag">&lt; &lt;word&gt;, 1&gt;</span>.</p>
  893. <p>
  894. For the given sample input the first map emits:<br>
  895. <span class="codefrag">&lt; Hello, 1&gt;</span>
  896. <br>
  897. <span class="codefrag">&lt; World, 1&gt;</span>
  898. <br>
  899. <span class="codefrag">&lt; Bye, 1&gt;</span>
  900. <br>
  901. <span class="codefrag">&lt; World, 1&gt;</span>
  902. <br>
  903. </p>
  904. <p>
  905. The second map emits:<br>
  906. <span class="codefrag">&lt; Hello, 1&gt;</span>
  907. <br>
  908. <span class="codefrag">&lt; Hadoop, 1&gt;</span>
  909. <br>
  910. <span class="codefrag">&lt; Goodbye, 1&gt;</span>
  911. <br>
  912. <span class="codefrag">&lt; Hadoop, 1&gt;</span>
  913. <br>
  914. </p>
  915. <p>We'll learn more about the number of maps spawned for a given job, and
  916. how to control them in a fine-grained manner, a bit later in the
  917. tutorial.</p>
  918. <p>
  919. <span class="codefrag">WordCount</span> also specifies a <span class="codefrag">combiner</span> (line
  920. 46). Hence, the output of each map is passed through the local combiner
  921. (which is same as the <span class="codefrag">Reducer</span> as per the job
  922. configuration) for local aggregation, after being sorted on the
  923. <em>key</em>s.</p>
  924. <p>
  925. The output of the first map:<br>
  926. <span class="codefrag">&lt; Bye, 1&gt;</span>
  927. <br>
  928. <span class="codefrag">&lt; Hello, 1&gt;</span>
  929. <br>
  930. <span class="codefrag">&lt; World, 2&gt;</span>
  931. <br>
  932. </p>
  933. <p>
  934. The output of the second map:<br>
  935. <span class="codefrag">&lt; Goodbye, 1&gt;</span>
  936. <br>
  937. <span class="codefrag">&lt; Hadoop, 2&gt;</span>
  938. <br>
  939. <span class="codefrag">&lt; Hello, 1&gt;</span>
  940. <br>
  941. </p>
  942. <p>The <span class="codefrag">Reducer</span> implementation (lines 28-36), via the
  943. <span class="codefrag">reduce</span> method (lines 29-35) just sums up the values,
  944. which are the occurence counts for each key (i.e. words in this example).
  945. </p>
  946. <p>
  947. Thus the output of the job is:<br>
  948. <span class="codefrag">&lt; Bye, 1&gt;</span>
  949. <br>
  950. <span class="codefrag">&lt; Goodbye, 1&gt;</span>
  951. <br>
  952. <span class="codefrag">&lt; Hadoop, 2&gt;</span>
  953. <br>
  954. <span class="codefrag">&lt; Hello, 2&gt;</span>
  955. <br>
  956. <span class="codefrag">&lt; World, 2&gt;</span>
  957. <br>
  958. </p>
  959. <p>The <span class="codefrag">run</span> method specifies various facets of the job, such
  960. as the input/output paths (passed via the command line), key/value
  961. types, input/output formats etc., in the <span class="codefrag">JobConf</span>.
  962. It then calls the <span class="codefrag">JobClient.runJob</span> (line 55) to submit the
  963. and monitor its progress.</p>
  964. <p>We'll learn more about <span class="codefrag">JobConf</span>, <span class="codefrag">JobClient</span>,
  965. <span class="codefrag">Tool</span> and other interfaces and classes a bit later in the
  966. tutorial.</p>
  967. </div>
  968. <a name="N105A3"></a><a name="Map-Reduce+-+User+Interfaces"></a>
  969. <h2 class="h3">Map-Reduce - User Interfaces</h2>
  970. <div class="section">
  971. <p>This section provides a reasonable amount of detail on every user-facing
  972. aspect of the Map-Reduce framwork. This should help users implement,
  973. configure and tune their jobs in a fine-grained manner. However, please
  974. note that the javadoc for each class/interface remains the most
  975. comprehensive documentation available; this is only meant to be a tutorial.
  976. </p>
  977. <p>Let us first take the <span class="codefrag">Mapper</span> and <span class="codefrag">Reducer</span>
  978. interfaces. Applications typically implement them to provide the
  979. <span class="codefrag">map</span> and <span class="codefrag">reduce</span> methods.</p>
  980. <p>We will then discuss other core interfaces including
  981. <span class="codefrag">JobConf</span>, <span class="codefrag">JobClient</span>, <span class="codefrag">Partitioner</span>,
  982. <span class="codefrag">OutputCollector</span>, <span class="codefrag">Reporter</span>,
  983. <span class="codefrag">InputFormat</span>, <span class="codefrag">OutputFormat</span> and others.</p>
  984. <p>Finally, we will wrap up by discussing some useful features of the
  985. framework such as the <span class="codefrag">DistributedCache</span>,
  986. <span class="codefrag">IsolationRunner</span> etc.</p>
  987. <a name="N105DC"></a><a name="Payload"></a>
  988. <h3 class="h4">Payload</h3>
  989. <p>Applications typically implement the <span class="codefrag">Mapper</span> and
  990. <span class="codefrag">Reducer</span> interfaces to provide the <span class="codefrag">map</span> and
  991. <span class="codefrag">reduce</span> methods. These form the core of the job.</p>
  992. <a name="N105F1"></a><a name="Mapper"></a>
  993. <h4>Mapper</h4>
  994. <p>
  995. <a href="api/org/apache/hadoop/mapred/Mapper.html">
  996. Mapper</a> maps input key/value pairs to a set of intermediate
  997. key/value pairs.</p>
  998. <p>Maps are the individual tasks that transform input records into
  999. intermediate records. The transformed intermediate records do not need
  1000. to be of the same type as the input records. A given input pair may
  1001. map to zero or many output pairs.</p>
  1002. <p>The Hadoop Map-Reduce framework spawns one map task for each
  1003. <span class="codefrag">InputSplit</span> generated by the <span class="codefrag">InputFormat</span> for
  1004. the job.</p>
  1005. <p>Overall, <span class="codefrag">Mapper</span> implementations are passed the
  1006. <span class="codefrag">JobConf</span> for the job via the
  1007. <a href="api/org/apache/hadoop/mapred/JobConfigurable.html#configure(org.apache.hadoop.mapred.JobConf)">
  1008. JobConfigurable.configure(JobConf)</a> method and override it to
  1009. initialize themselves. The framework then calls
  1010. <a href="api/org/apache/hadoop/mapred/Mapper.html#map(K1, V1, org.apache.hadoop.mapred.OutputCollector, org.apache.hadoop.mapred.Reporter)">
  1011. map(WritableComparable, Writable, OutputCollector, Reporter)</a> for
  1012. each key/value pair in the <span class="codefrag">InputSplit</span> for that task.
  1013. Applications can then override the
  1014. <a href="api/org/apache/hadoop/io/Closeable.html#close()">
  1015. Closeable.close()</a> method to perform any required cleanup.</p>
  1016. <p>Output pairs do not need to be of the same types as input pairs. A
  1017. given input pair may map to zero or many output pairs. Output pairs
  1018. are collected with calls to
  1019. <a href="api/org/apache/hadoop/mapred/OutputCollector.html#collect(K, V)">
  1020. OutputCollector.collect(WritableComparable,Writable)</a>.</p>
  1021. <p>Applications can use the <span class="codefrag">Reporter</span> to report
  1022. progress, set application-level status messages and update
  1023. <span class="codefrag">Counters</span>, or just indicate that they are alive.</p>
  1024. <p>All intermediate values associated with a given output key are
  1025. subsequently grouped by the framework, and passed to the
  1026. <span class="codefrag">Reducer</span>(s) to determine the final output. Users can
  1027. control the grouping by specifying a <span class="codefrag">Comparator</span> via
  1028. <a href="api/org/apache/hadoop/mapred/JobConf.html#setOutputKeyComparatorClass(java.lang.Class)">
  1029. JobConf.setOutputKeyComparatorClass(Class)</a>.</p>
  1030. <p>The <span class="codefrag">Mapper</span> outputs are sorted and then
  1031. partitioned per <span class="codefrag">Reducer</span>. The total number of partitions is
  1032. the same as the number of reduce tasks for the job. Users can control
  1033. which keys (and hence records) go to which <span class="codefrag">Reducer</span> by
  1034. implementing a custom <span class="codefrag">Partitioner</span>.</p>
  1035. <p>Users can optionally specify a <span class="codefrag">combiner</span>, via
  1036. <a href="api/org/apache/hadoop/mapred/JobConf.html#setCombinerClass(java.lang.Class)">
  1037. JobConf.setCombinerClass(Class)</a>, to perform local aggregation of
  1038. the intermediate outputs, which helps to cut down the amount of data
  1039. transferred from the <span class="codefrag">Mapper</span> to the <span class="codefrag">Reducer</span>.
  1040. </p>
  1041. <p>The intermediate, sorted outputs are always stored in files of
  1042. <a href="api/org/apache/hadoop/io/SequenceFile.html">
  1043. SequenceFile</a> format. Applications can control if, and how, the
  1044. intermediate outputs are to be compressed and the
  1045. <a href="api/org/apache/hadoop/io/compress/CompressionCodec.html">
  1046. CompressionCodec</a> to be used via the <span class="codefrag">JobConf</span>.
  1047. </p>
  1048. <a name="N1066B"></a><a name="How+Many+Maps%3F"></a>
  1049. <h5>How Many Maps?</h5>
  1050. <p>The number of maps is usually driven by the total size of the
  1051. inputs, that is, the total number of blocks of the input files.</p>
  1052. <p>The right level of parallelism for maps seems to be around 10-100
  1053. maps per-node, although it has been set up to 300 maps for very
  1054. cpu-light map tasks. Task setup takes awhile, so it is best if the
  1055. maps take at least a minute to execute.</p>
  1056. <p>Thus, if you expect 10TB of input data and have a blocksize of
  1057. <span class="codefrag">128MB</span>, you'll end up with 82,000 maps, unless
  1058. <a href="api/org/apache/hadoop/mapred/JobConf.html#setNumMapTasks(int)">
  1059. setNumMapTasks(int)</a> (which only provides a hint to the framework)
  1060. is used to set it even higher.</p>
  1061. <a name="N10683"></a><a name="Reducer"></a>
  1062. <h4>Reducer</h4>
  1063. <p>
  1064. <a href="api/org/apache/hadoop/mapred/Reducer.html">
  1065. Reducer</a> reduces a set of intermediate values which share a key to
  1066. a smaller set of values.</p>
  1067. <p>The number of reduces for the job is set by the user
  1068. via <a href="api/org/apache/hadoop/mapred/JobConf.html#setNumReduceTasks(int)">
  1069. JobConf.setNumReduceTasks(int)</a>.</p>
  1070. <p>Overall, <span class="codefrag">Reducer</span> implementations are passed the
  1071. <span class="codefrag">JobConf</span> for the job via the
  1072. <a href="api/org/apache/hadoop/mapred/JobConfigurable.html#configure(org.apache.hadoop.mapred.JobConf)">
  1073. JobConfigurable.configure(JobConf)</a> method and can override it to
  1074. initialize themselves. The framework then calls
  1075. <a href="api/org/apache/hadoop/mapred/Reducer.html#reduce(K2, java.util.Iterator, org.apache.hadoop.mapred.OutputCollector, org.apache.hadoop.mapred.Reporter)">
  1076. reduce(WritableComparable, Iterator, OutputCollector, Reporter)</a>
  1077. method for each <span class="codefrag">&lt;key, (list of values)&gt;</span>
  1078. pair in the grouped inputs. Applications can then override the
  1079. <a href="api/org/apache/hadoop/io/Closeable.html#close()">
  1080. Closeable.close()</a> method to perform any required cleanup.</p>
  1081. <p>
  1082. <span class="codefrag">Reducer</span> has 3 primary phases: shuffle, sort and reduce.
  1083. </p>
  1084. <a name="N106B3"></a><a name="Shuffle"></a>
  1085. <h5>Shuffle</h5>
  1086. <p>Input to the <span class="codefrag">Reducer</span> is the sorted output of the
  1087. mappers. In this phase the framework fetches the relevant partition
  1088. of the output of all the mappers, via HTTP.</p>
  1089. <a name="N106C0"></a><a name="Sort"></a>
  1090. <h5>Sort</h5>
  1091. <p>The framework groups <span class="codefrag">Reducer</span> inputs by keys (since
  1092. different mappers may have output the same key) in this stage.</p>
  1093. <p>The shuffle and sort phases occur simultaneously; while
  1094. map-outputs are being fetched they are merged.</p>
  1095. <a name="N106CF"></a><a name="Secondary+Sort"></a>
  1096. <h5>Secondary Sort</h5>
  1097. <p>If equivalence rules for grouping the intermediate keys are
  1098. required to be different from those for grouping keys before
  1099. reduction, then one may specify a <span class="codefrag">Comparator</span> via
  1100. <a href="api/org/apache/hadoop/mapred/JobConf.html#setOutputValueGroupingComparator(java.lang.Class)">
  1101. JobConf.setOutputValueGroupingComparator(Class)</a>. Since
  1102. <a href="api/org/apache/hadoop/mapred/JobConf.html#setOutputKeyComparatorClass(java.lang.Class)">
  1103. JobConf.setOutputKeyComparatorClass(Class)</a> can be used to
  1104. control how intermediate keys are grouped, these can be used in
  1105. conjunction to simulate <em>secondary sort on values</em>.</p>
  1106. <a name="N106E8"></a><a name="Reduce"></a>
  1107. <h5>Reduce</h5>
  1108. <p>In this phase the
  1109. <a href="api/org/apache/hadoop/mapred/Reducer.html#reduce(K2, java.util.Iterator, org.apache.hadoop.mapred.OutputCollector, org.apache.hadoop.mapred.Reporter)">
  1110. reduce(WritableComparable, Iterator, OutputCollector, Reporter)</a>
  1111. method is called for each <span class="codefrag">&lt;key, (list of values)&gt;</span>
  1112. pair in the grouped inputs.</p>
  1113. <p>The output of the reduce task is typically written to the
  1114. <a href="api/org/apache/hadoop/fs/FileSystem.html">
  1115. FileSystem</a> via
  1116. <a href="api/org/apache/hadoop/mapred/OutputCollector.html#collect(K, V)">
  1117. OutputCollector.collect(WritableComparable, Writable)</a>.</p>
  1118. <p>Applications can use the <span class="codefrag">Reporter</span> to report
  1119. progress, set application-level status messages and update
  1120. <span class="codefrag">Counters</span>, or just indicate that they are alive.</p>
  1121. <p>The output of the <span class="codefrag">Reducer</span> is <em>not sorted</em>.</p>
  1122. <a name="N10716"></a><a name="How+Many+Reduces%3F"></a>
  1123. <h5>How Many Reduces?</h5>
  1124. <p>The right number of reduces seems to be <span class="codefrag">0.95</span> or
  1125. <span class="codefrag">1.75</span> multiplied by (&lt;<em>no. of nodes</em>&gt; *
  1126. <span class="codefrag">mapred.tasktracker.reduce.tasks.maximum</span>).</p>
  1127. <p>With <span class="codefrag">0.95</span> all of the reduces can launch immediately
  1128. and start transfering map outputs as the maps finish. With
  1129. <span class="codefrag">1.75</span> the faster nodes will finish their first round of
  1130. reduces and launch a second wave of reduces doing a much better job
  1131. of load balancing.</p>
  1132. <p>Increasing the number of reduces increases the framework overhead,
  1133. but increases load balancing and lowers the cost of failures.</p>
  1134. <p>The scaling factors above are slightly less than whole numbers to
  1135. reserve a few reduce slots in the framework for speculative-tasks and
  1136. failed tasks.</p>
  1137. <a name="N1073B"></a><a name="Reducer+NONE"></a>
  1138. <h5>Reducer NONE</h5>
  1139. <p>It is legal to set the number of reduce-tasks to <em>zero</em> if
  1140. no reduction is desired.</p>
  1141. <p>In this case the outputs of the map-tasks go directly to the
  1142. <span class="codefrag">FileSystem</span>, into the output path set by
  1143. <a href="api/org/apache/hadoop/mapred/FileInputFormat.html#setOutputPath(org.apache.hadoop.mapred.JobConf,%20org.apache.hadoop.fs.Path)">
  1144. setOutputPath(Path)</a>. The framework does not sort the
  1145. map-outputs before writing them out to the <span class="codefrag">FileSystem</span>.
  1146. </p>
  1147. <a name="N10756"></a><a name="Partitioner"></a>
  1148. <h4>Partitioner</h4>
  1149. <p>
  1150. <a href="api/org/apache/hadoop/mapred/Partitioner.html">
  1151. Partitioner</a> partitions the key space.</p>
  1152. <p>Partitioner controls the partitioning of the keys of the
  1153. intermediate map-outputs. The key (or a subset of the key) is used to
  1154. derive the partition, typically by a <em>hash function</em>. The total
  1155. number of partitions is the same as the number of reduce tasks for the
  1156. job. Hence this controls which of the <span class="codefrag">m</span> reduce tasks the
  1157. intermediate key (and hence the record) is sent to for reduction.</p>
  1158. <p>
  1159. <a href="api/org/apache/hadoop/mapred/lib/HashPartitioner.html">
  1160. HashPartitioner</a> is the default <span class="codefrag">Partitioner</span>.</p>
  1161. <a name="N10775"></a><a name="Reporter"></a>
  1162. <h4>Reporter</h4>
  1163. <p>
  1164. <a href="api/org/apache/hadoop/mapred/Reporter.html">
  1165. Reporter</a> is a facility for Map-Reduce applications to report
  1166. progress, set application-level status messages and update
  1167. <span class="codefrag">Counters</span>.</p>
  1168. <p>
  1169. <span class="codefrag">Mapper</span> and <span class="codefrag">Reducer</span> implementations can use
  1170. the <span class="codefrag">Reporter</span> to report progress or just indicate
  1171. that they are alive. In scenarios where the application takes a
  1172. significant amount of time to process individual key/value pairs,
  1173. this is crucial since the framework might assume that the task has
  1174. timed-out and kill that task. Another way to avoid this is to
  1175. set the configuration parameter <span class="codefrag">mapred.task.timeout</span> to a
  1176. high-enough value (or even set it to <em>zero</em> for no time-outs).
  1177. </p>
  1178. <p>Applications can also update <span class="codefrag">Counters</span> using the
  1179. <span class="codefrag">Reporter</span>.</p>
  1180. <a name="N1079F"></a><a name="OutputCollector"></a>
  1181. <h4>OutputCollector</h4>
  1182. <p>
  1183. <a href="api/org/apache/hadoop/mapred/OutputCollector.html">
  1184. OutputCollector</a> is a generalization of the facility provided by
  1185. the Map-Reduce framework to collect data output by the
  1186. <span class="codefrag">Mapper</span> or the <span class="codefrag">Reducer</span> (either the
  1187. intermediate outputs or the output of the job).</p>
  1188. <p>Hadoop Map-Reduce comes bundled with a
  1189. <a href="api/org/apache/hadoop/mapred/lib/package-summary.html">
  1190. library</a> of generally useful mappers, reducers, and partitioners.</p>
  1191. <a name="N107BA"></a><a name="Job+Configuration"></a>
  1192. <h3 class="h4">Job Configuration</h3>
  1193. <p>
  1194. <a href="api/org/apache/hadoop/mapred/JobConf.html">
  1195. JobConf</a> represents a Map-Reduce job configuration.</p>
  1196. <p>
  1197. <span class="codefrag">JobConf</span> is the primary interface for a user to describe
  1198. a map-reduce job to the Hadoop framework for execution. The framework
  1199. tries to faithfully execute the job as described by <span class="codefrag">JobConf</span>,
  1200. however:</p>
  1201. <ul>
  1202. <li>f
  1203. Some configuration parameters may have been marked as
  1204. <a href="api/org/apache/hadoop/conf/Configuration.html#FinalParams">
  1205. final</a> by administrators and hence cannot be altered.
  1206. </li>
  1207. <li>
  1208. While some job parameters are straight-forward to set (e.g.
  1209. <a href="api/org/apache/hadoop/mapred/JobConf.html#setNumReduceTasks(int)">
  1210. setNumReduceTasks(int)</a>), other parameters interact subtly with
  1211. the rest of the framework and/or job configuration and are
  1212. more complex to set (e.g.
  1213. <a href="api/org/apache/hadoop/mapred/JobConf.html#setNumMapTasks(int)">
  1214. setNumMapTasks(int)</a>).
  1215. </li>
  1216. </ul>
  1217. <p>
  1218. <span class="codefrag">JobConf</span> is typically used to specify the
  1219. <span class="codefrag">Mapper</span>, combiner (if any), <span class="codefrag">Partitioner</span>,
  1220. <span class="codefrag">Reducer</span>, <span class="codefrag">InputFormat</span> and
  1221. <span class="codefrag">OutputFormat</span> implementations. <span class="codefrag">JobConf</span> also
  1222. indicates the set of input files
  1223. (<a href="api/org/apache/hadoop/mapred/FileInputFormat.html#setInputPaths(org.apache.hadoop.mapred.JobConf,%20org.apache.hadoop.fs.Path[])">setInputPaths(JobConf, Path...)</a>
  1224. /<a href="api/org/apache/hadoop/mapred/FileInputFormat.html#addInputPath(org.apache.hadoop.mapred.JobConf,%20org.apache.hadoop.fs.Path)">addInputPath(JobConf, Path)</a>)
  1225. and (<a href="api/org/apache/hadoop/mapred/FileInputFormat.html#setInputPaths(org.apache.hadoop.mapred.JobConf,%20java.lang.String)">setInputPaths(JobConf, String)</a>
  1226. /<a href="api/org/apache/hadoop/mapred/FileInputFormat.html#addInputPath(org.apache.hadoop.mapred.JobConf,%20java.lang.String)">addInputPaths(JobConf, String)</a>)
  1227. and where the output files should be written
  1228. (<a href="api/org/apache/hadoop/mapred/FileInputFormat.html#setOutputPath(org.apache.hadoop.mapred.JobConf,%20org.apache.hadoop.fs.Path)">setOutputPath(Path)</a>).</p>
  1229. <p>Optionally, <span class="codefrag">JobConf</span> is used to specify other advanced
  1230. facets of the job such as the <span class="codefrag">Comparator</span> to be used, files
  1231. to be put in the <span class="codefrag">DistributedCache</span>, whether intermediate
  1232. and/or job outputs are to be compressed (and how), debugging via
  1233. user-provided scripts
  1234. (<a href="api/org/apache/hadoop/mapred/JobConf.html#setMapDebugScript(java.lang.String)">setMapDebugScript(String)</a>/<a href="api/org/apache/hadoop/mapred/JobConf.html#setReduceDebugScript(java.lang.String)">setReduceDebugScript(String)</a>)
  1235. , whether job tasks can be executed in a <em>speculative</em> manner
  1236. (<a href="api/org/apache/hadoop/mapred/JobConf.html#setMapSpeculativeExecution(boolean)">setMapSpeculativeExecution(boolean)</a>)/(<a href="api/org/apache/hadoop/mapred/JobConf.html#setReduceSpeculativeExecution(boolean)">setReduceSpeculativeExecution(boolean)</a>)
  1237. , maximum number of attempts per task
  1238. (<a href="api/org/apache/hadoop/mapred/JobConf.html#setMaxMapAttempts(int)">setMaxMapAttempts(int)</a>/<a href="api/org/apache/hadoop/mapred/JobConf.html#setMaxReduceAttempts(int)">setMaxReduceAttempts(int)</a>)
  1239. , percentage of tasks failure which can be tolerated by the job
  1240. (<a href="api/org/apache/hadoop/mapred/JobConf.html#setMaxMapTaskFailuresPercent(int)">setMaxMapTaskFailuresPercent(int)</a>/<a href="api/org/apache/hadoop/mapred/JobConf.html#setMaxReduceTaskFailuresPercent(int)">setMaxReduceTaskFailuresPercent(int)</a>)
  1241. etc.</p>
  1242. <p>Of course, users can use
  1243. <a href="api/org/apache/hadoop/conf/Configuration.html#set(java.lang.String, java.lang.String)">set(String, String)</a>/<a href="api/org/apache/hadoop/conf/Configuration.html#get(java.lang.String, java.lang.String)">get(String, String)</a>
  1244. to set/get arbitrary parameters needed by applications. However, use the
  1245. <span class="codefrag">DistributedCache</span> for large amounts of (read-only) data.</p>
  1246. <a name="N1084C"></a><a name="Task+Execution+%26+Environment"></a>
  1247. <h3 class="h4">Task Execution &amp; Environment</h3>
  1248. <p>The <span class="codefrag">TaskTracker</span> executes the <span class="codefrag">Mapper</span>/
  1249. <span class="codefrag">Reducer</span> <em>task</em> as a child process in a separate jvm.
  1250. </p>
  1251. <p>The child-task inherits the environment of the parent
  1252. <span class="codefrag">TaskTracker</span>. The user can specify additional options to the
  1253. child-jvm via the <span class="codefrag">mapred.child.java.opts</span> configuration
  1254. parameter in the <span class="codefrag">JobConf</span> such as non-standard paths for the
  1255. run-time linker to search shared libraries via
  1256. <span class="codefrag">-Djava.library.path=&lt;&gt;</span> etc. If the
  1257. <span class="codefrag">mapred.child.java.opts</span> contains the symbol <em>@taskid@</em>
  1258. it is interpolated with value of <span class="codefrag">taskid</span> of the map/reduce
  1259. task.</p>
  1260. <p>Here is an example with multiple arguments and substitutions,
  1261. showing jvm GC logging, and start of a passwordless JVM JMX agent so that
  1262. it can connect with jconsole and the likes to watch child memory,
  1263. threads and get thread dumps. It also sets the maximum heap-size of the
  1264. child jvm to 512MB and adds an additional path to the
  1265. <span class="codefrag">java.library.path</span> of the child-jvm.</p>
  1266. <p>
  1267. <span class="codefrag">&lt;property&gt;</span>
  1268. <br>
  1269. &nbsp;&nbsp;<span class="codefrag">&lt;name&gt;mapred.child.java.opts&lt;/name&gt;</span>
  1270. <br>
  1271. &nbsp;&nbsp;<span class="codefrag">&lt;value&gt;</span>
  1272. <br>
  1273. &nbsp;&nbsp;&nbsp;&nbsp;<span class="codefrag">
  1274. -Xmx512M -Djava.library.path=/home/mycompany/lib
  1275. -verbose:gc -Xloggc:/tmp/@taskid@.gc</span>
  1276. <br>
  1277. &nbsp;&nbsp;&nbsp;&nbsp;<span class="codefrag">
  1278. -Dcom.sun.management.jmxremote.authenticate=false
  1279. -Dcom.sun.management.jmxremote.ssl=false</span>
  1280. <br>
  1281. &nbsp;&nbsp;<span class="codefrag">&lt;/value&gt;</span>
  1282. <br>
  1283. <span class="codefrag">&lt;/property&gt;</span>
  1284. </p>
  1285. <p>Users/admins can also specify the maximum virtual memory
  1286. of the launched child-task using <span class="codefrag">mapred.child.ulimit</span>.</p>
  1287. <p>When the job starts, the localized job directory
  1288. <span class="codefrag"> ${mapred.local.dir}/taskTracker/jobcache/$jobid/</span>
  1289. has the following directories: </p>
  1290. <ul>
  1291. <li> A job-specific shared directory, created at location
  1292. <span class="codefrag">${mapred.local.dir}/taskTracker/jobcache/$jobid/work/ </span>.
  1293. This directory is exposed to the users through
  1294. <span class="codefrag">job.local.dir </span>. The tasks can use this space as scratch
  1295. space and share files among them. The directory can accessed through
  1296. api <a href="api/org/apache/hadoop/mapred/JobConf.html#getJobLocalDir()">
  1297. JobConf.getJobLocalDir()</a>. It is available as System property also.
  1298. So,users can call <span class="codefrag">System.getProperty("job.local.dir")</span>;
  1299. </li>
  1300. <li>A jars directory, which has the job jar file and expanded jar </li>
  1301. <li>A job.xml file, the generic job configuration </li>
  1302. <li>Each task has directory <span class="codefrag">task-id</span> which again has the
  1303. following structure
  1304. <ul>
  1305. <li>A job.xml file, task localized job configuration </li>
  1306. <li>A directory for intermediate output files</li>
  1307. <li>The working directory of the task.
  1308. And work directory has a temporary directory
  1309. to create temporary files</li>
  1310. </ul>
  1311. </li>
  1312. </ul>
  1313. <p>The <a href="#DistributedCache">DistributedCache</a> can also be used
  1314. as a rudimentary software distribution mechanism for use in the map
  1315. and/or reduce tasks. It can be used to distribute both jars and
  1316. native libraries. The
  1317. <a href="api/org/apache/hadoop/filecache/DistributedCache.html#addArchiveToClassPath(org.apache.hadoop.fs.Path,%20org.apache.hadoop.conf.Configuration)">
  1318. DistributedCache.addArchiveToClassPath(Path, Configuration)</a> or
  1319. <a href="api/org/apache/hadoop/filecache/DistributedCache.html#addFileToClassPath(org.apache.hadoop.fs.Path,%20org.apache.hadoop.conf.Configuration)">
  1320. DistributedCache.addFileToClassPath(Path, Configuration)</a> api can
  1321. be used to cache files/jars and also add them to the <em>classpath</em>
  1322. of child-jvm. Similarly the facility provided by the
  1323. <span class="codefrag">DistributedCache</span> where-in it symlinks the cached files into
  1324. the working directory of the task can be used to distribute native
  1325. libraries and load them. The underlying detail is that child-jvm always
  1326. has its <em>current working directory</em> added to the
  1327. <span class="codefrag">java.library.path</span> and <span class="codefrag">LD_LIBRARY_PATH</span>.
  1328. And hence the cached libraries can be
  1329. loaded via <a href="http://java.sun.com/j2se/1.5.0/docs/api/java/lang/System.html#loadLibrary(java.lang.String)">
  1330. System.loadLibrary</a> or <a href="http://java.sun.com/j2se/1.5.0/docs/api/java/lang/System.html#load(java.lang.String)">
  1331. System.load</a>.</p>
  1332. <a name="N108FB"></a><a name="Job+Submission+and+Monitoring"></a>
  1333. <h3 class="h4">Job Submission and Monitoring</h3>
  1334. <p>
  1335. <a href="api/org/apache/hadoop/mapred/JobClient.html">
  1336. JobClient</a> is the primary interface by which user-job interacts
  1337. with the <span class="codefrag">JobTracker</span>.</p>
  1338. <p>
  1339. <span class="codefrag">JobClient</span> provides facilities to submit jobs, track their
  1340. progress, access component-tasks' reports/logs, get the Map-Reduce
  1341. cluster's status information and so on.</p>
  1342. <p>The job submission process involves:</p>
  1343. <ol>
  1344. <li>Checking the input and output specifications of the job.</li>
  1345. <li>Computing the <span class="codefrag">InputSplit</span> values for the job.</li>
  1346. <li>
  1347. Setting up the requisite accounting information for the
  1348. <span class="codefrag">DistributedCache</span> of the job, if necessary.
  1349. </li>
  1350. <li>
  1351. Copying the job's jar and configuration to the map-reduce system
  1352. directory on the <span class="codefrag">FileSystem</span>.
  1353. </li>
  1354. <li>
  1355. Submitting the job to the <span class="codefrag">JobTracker</span> and optionally
  1356. monitoring it's status.
  1357. </li>
  1358. </ol>
  1359. <p> Job history files are also logged to user specified directory
  1360. <span class="codefrag">hadoop.job.history.user.location</span>
  1361. which defaults to job output directory. The files are stored in
  1362. "_logs/history/" in the specified directory. Hence, by default they
  1363. will be in mapred.output.dir/_logs/history. User can stop
  1364. logging by giving the value <span class="codefrag">none</span> for
  1365. <span class="codefrag">hadoop.job.history.user.location</span>
  1366. </p>
  1367. <p> User can view the history logs summary in specified directory
  1368. using the following command <br>
  1369. <span class="codefrag">$ bin/hadoop job -history output-dir</span>
  1370. <br>
  1371. This command will print job details, failed and killed tip
  1372. details. <br>
  1373. More details about the job such as successful tasks and
  1374. task attempts made for each task can be viewed using the
  1375. following command <br>
  1376. <span class="codefrag">$ bin/hadoop job -history all output-dir</span>
  1377. <br>
  1378. </p>
  1379. <p> User can use
  1380. <a href="api/org/apache/hadoop/mapred/OutputLogFilter.html">OutputLogFilter</a>
  1381. to filter log files from the output directory listing. </p>
  1382. <p>Normally the user creates the application, describes various facets
  1383. of the job via <span class="codefrag">JobConf</span>, and then uses the
  1384. <span class="codefrag">JobClient</span> to submit the job and monitor its progress.</p>
  1385. <a name="N1095B"></a><a name="Job+Control"></a>
  1386. <h4>Job Control</h4>
  1387. <p>Users may need to chain map-reduce jobs to accomplish complex
  1388. tasks which cannot be done via a single map-reduce job. This is fairly
  1389. easy since the output of the job typically goes to distributed
  1390. file-system, and the output, in turn, can be used as the input for the
  1391. next job.</p>
  1392. <p>However, this also means that the onus on ensuring jobs are
  1393. complete (success/failure) lies squarely on the clients. In such
  1394. cases, the various job-control options are:</p>
  1395. <ul>
  1396. <li>
  1397. <a href="api/org/apache/hadoop/mapred/JobClient.html#runJob(org.apache.hadoop.mapred.JobConf)">
  1398. runJob(JobConf)</a> : Submits the job and returns only after the
  1399. job has completed.
  1400. </li>
  1401. <li>
  1402. <a href="api/org/apache/hadoop/mapred/JobClient.html#submitJob(org.apache.hadoop.mapred.JobConf)">
  1403. submitJob(JobConf)</a> : Only submits the job, then poll the
  1404. returned handle to the
  1405. <a href="api/org/apache/hadoop/mapred/RunningJob.html">
  1406. RunningJob</a> to query status and make scheduling decisions.
  1407. </li>
  1408. <li>
  1409. <a href="api/org/apache/hadoop/mapred/JobConf.html#setJobEndNotificationURI(java.lang.String)">
  1410. JobConf.setJobEndNotificationURI(String)</a> : Sets up a
  1411. notification upon job-completion, thus avoiding polling.
  1412. </li>
  1413. </ul>
  1414. <a name="N10985"></a><a name="Job+Input"></a>
  1415. <h3 class="h4">Job Input</h3>
  1416. <p>
  1417. <a href="api/org/apache/hadoop/mapred/InputFormat.html">
  1418. InputFormat</a> describes the input-specification for a Map-Reduce job.
  1419. </p>
  1420. <p>The Map-Reduce framework relies on the <span class="codefrag">InputFormat</span> of
  1421. the job to:</p>
  1422. <ol>
  1423. <li>Validate the input-specification of the job.</li>
  1424. <li>
  1425. Split-up the input file(s) into logical <span class="codefrag">InputSplit</span>
  1426. instances, each of which is then assigned to an individual
  1427. <span class="codefrag">Mapper</span>.
  1428. </li>
  1429. <li>
  1430. Provide the <span class="codefrag">RecordReader</span> implementation used to
  1431. glean input records from the logical <span class="codefrag">InputSplit</span> for
  1432. processing by the <span class="codefrag">Mapper</span>.
  1433. </li>
  1434. </ol>
  1435. <p>The default behavior of file-based <span class="codefrag">InputFormat</span>
  1436. implementations, typically sub-classes of
  1437. <a href="api/org/apache/hadoop/mapred/FileInputFormat.html">
  1438. FileInputFormat</a>, is to split the input into <em>logical</em>
  1439. <span class="codefrag">InputSplit</span> instances based on the total size, in bytes, of
  1440. the input files. However, the <span class="codefrag">FileSystem</span> blocksize of the
  1441. input files is treated as an upper bound for input splits. A lower bound
  1442. on the split size can be set via <span class="codefrag">mapred.min.split.size</span>.</p>
  1443. <p>Clearly, logical splits based on input-size is insufficient for many
  1444. applications since record boundaries must be respected. In such cases,
  1445. the application should implement a <span class="codefrag">RecordReader</span>, who is
  1446. responsible for respecting record-boundaries and presents a
  1447. record-oriented view of the logical <span class="codefrag">InputSplit</span> to the
  1448. individual task.</p>
  1449. <p>
  1450. <a href="api/org/apache/hadoop/mapred/TextInputFormat.html">
  1451. TextInputFormat</a> is the default <span class="codefrag">InputFormat</span>.</p>
  1452. <p>If <span class="codefrag">TextInputFormat</span> is the <span class="codefrag">InputFormat</span> for a
  1453. given job, the framework detects input-files with the <em>.gz</em> and
  1454. <em>.lzo</em> extensions and automatically decompresses them using the
  1455. appropriate <span class="codefrag">CompressionCodec</span>. However, it must be noted that
  1456. compressed files with the above extensions cannot be <em>split</em> and
  1457. each compressed file is processed in its entirety by a single mapper.</p>
  1458. <a name="N109EF"></a><a name="InputSplit"></a>
  1459. <h4>InputSplit</h4>
  1460. <p>
  1461. <a href="api/org/apache/hadoop/mapred/InputSplit.html">
  1462. InputSplit</a> represents the data to be processed by an individual
  1463. <span class="codefrag">Mapper</span>.</p>
  1464. <p>Typically <span class="codefrag">InputSplit</span> presents a byte-oriented view of
  1465. the input, and it is the responsibility of <span class="codefrag">RecordReader</span>
  1466. to process and present a record-oriented view.</p>
  1467. <p>
  1468. <a href="api/org/apache/hadoop/mapred/FileSplit.html">
  1469. FileSplit</a> is the default <span class="codefrag">InputSplit</span>. It sets
  1470. <span class="codefrag">map.input.file</span> to the path of the input file for the
  1471. logical split.</p>
  1472. <a name="N10A14"></a><a name="RecordReader"></a>
  1473. <h4>RecordReader</h4>
  1474. <p>
  1475. <a href="api/org/apache/hadoop/mapred/RecordReader.html">
  1476. RecordReader</a> reads <span class="codefrag">&lt;key, value&gt;</span> pairs from an
  1477. <span class="codefrag">InputSplit</span>.</p>
  1478. <p>Typically the <span class="codefrag">RecordReader</span> converts the byte-oriented
  1479. view of the input, provided by the <span class="codefrag">InputSplit</span>, and
  1480. presents a record-oriented to the <span class="codefrag">Mapper</span> implementations
  1481. for processing. <span class="codefrag">RecordReader</span> thus assumes the
  1482. responsibility of processing record boundaries and presents the tasks
  1483. with keys and values.</p>
  1484. <a name="N10A37"></a><a name="Job+Output"></a>
  1485. <h3 class="h4">Job Output</h3>
  1486. <p>
  1487. <a href="api/org/apache/hadoop/mapred/OutputFormat.html">
  1488. OutputFormat</a> describes the output-specification for a Map-Reduce
  1489. job.</p>
  1490. <p>The Map-Reduce framework relies on the <span class="codefrag">OutputFormat</span> of
  1491. the job to:</p>
  1492. <ol>
  1493. <li>
  1494. Validate the output-specification of the job; for example, check that
  1495. the output directory doesn't already exist.
  1496. </li>
  1497. <li>
  1498. Provide the <span class="codefrag">RecordWriter</span> implementation used to
  1499. write the output files of the job. Output files are stored in a
  1500. <span class="codefrag">FileSystem</span>.
  1501. </li>
  1502. </ol>
  1503. <p>
  1504. <span class="codefrag">TextOutputFormat</span> is the default
  1505. <span class="codefrag">OutputFormat</span>.</p>
  1506. <a name="N10A60"></a><a name="Task+Side-Effect+Files"></a>
  1507. <h4>Task Side-Effect Files</h4>
  1508. <p>In some applications, component tasks need to create and/or write to
  1509. side-files, which differ from the actual job-output files.</p>
  1510. <p>In such cases there could be issues with two instances of the same
  1511. <span class="codefrag">Mapper</span> or <span class="codefrag">Reducer</span> running simultaneously (for
  1512. example, speculative tasks) trying to open and/or write to the same
  1513. file (path) on the <span class="codefrag">FileSystem</span>. Hence the
  1514. application-writer will have to pick unique names per task-attempt
  1515. (using the attemptid, say <span class="codefrag">attempt_200709221812_0001_m_000000_0</span>),
  1516. not just per task.</p>
  1517. <p>To avoid these issues the Map-Reduce framework maintains a special
  1518. <span class="codefrag">${mapred.output.dir}/_temporary/_${taskid}</span> sub-directory
  1519. accessible via <span class="codefrag">${mapred.work.output.dir}</span>
  1520. for each task-attempt on the <span class="codefrag">FileSystem</span> where the output
  1521. of the task-attempt is stored. On successful completion of the
  1522. task-attempt, the files in the
  1523. <span class="codefrag">${mapred.output.dir}/_temporary/_${taskid}</span> (only)
  1524. are <em>promoted</em> to <span class="codefrag">${mapred.output.dir}</span>. Of course,
  1525. the framework discards the sub-directory of unsuccessful task-attempts.
  1526. This process is completely transparent to the application.</p>
  1527. <p>The application-writer can take advantage of this feature by
  1528. creating any side-files required in <span class="codefrag">${mapred.work.output.dir}</span>
  1529. during execution of a task via
  1530. <a href="api/org/apache/hadoop/mapred/FileInputFormat.html#getWorkOutputPath(org.apache.hadoop.mapred.JobConf)">
  1531. FileOutputFormat.getWorkOutputPath()</a>, and the framework will promote them
  1532. similarly for succesful task-attempts, thus eliminating the need to
  1533. pick unique paths per task-attempt.</p>
  1534. <p>Note: The value of <span class="codefrag">${mapred.work.output.dir}</span> during
  1535. execution of a particular task-attempt is actually
  1536. <span class="codefrag">${mapred.output.dir}/_temporary/_{$taskid}</span>, and this value is
  1537. set by the map-reduce framework. So, just create any side-files in the
  1538. path returned by
  1539. <a href="api/org/apache/hadoop/mapred/FileInputFormat.html#getWorkOutputPath(org.apache.hadoop.mapred.JobConf)">
  1540. FileOutputFormat.getWorkOutputPath() </a>from map/reduce
  1541. task to take advantage of this feature.</p>
  1542. <p>The entire discussion holds true for maps of jobs with
  1543. reducer=NONE (i.e. 0 reduces) since output of the map, in that case,
  1544. goes directly to HDFS.</p>
  1545. <a name="N10AA8"></a><a name="RecordWriter"></a>
  1546. <h4>RecordWriter</h4>
  1547. <p>
  1548. <a href="api/org/apache/hadoop/mapred/RecordWriter.html">
  1549. RecordWriter</a> writes the output <span class="codefrag">&lt;key, value&gt;</span>
  1550. pairs to an output file.</p>
  1551. <p>RecordWriter implementations write the job outputs to the
  1552. <span class="codefrag">FileSystem</span>.</p>
  1553. <a name="N10ABF"></a><a name="Other+Useful+Features"></a>
  1554. <h3 class="h4">Other Useful Features</h3>
  1555. <a name="N10AC5"></a><a name="Counters"></a>
  1556. <h4>Counters</h4>
  1557. <p>
  1558. <span class="codefrag">Counters</span> represent global counters, defined either by
  1559. the Map-Reduce framework or applications. Each <span class="codefrag">Counter</span> can
  1560. be of any <span class="codefrag">Enum</span> type. Counters of a particular
  1561. <span class="codefrag">Enum</span> are bunched into groups of type
  1562. <span class="codefrag">Counters.Group</span>.</p>
  1563. <p>Applications can define arbitrary <span class="codefrag">Counters</span> (of type
  1564. <span class="codefrag">Enum</span>) and update them via
  1565. <a href="api/org/apache/hadoop/mapred/Reporter.html#incrCounter(java.lang.Enum, long)">
  1566. Reporter.incrCounter(Enum, long)</a> in the <span class="codefrag">map</span> and/or
  1567. <span class="codefrag">reduce</span> methods. These counters are then globally
  1568. aggregated by the framework.</p>
  1569. <a name="N10AF0"></a><a name="DistributedCache"></a>
  1570. <h4>DistributedCache</h4>
  1571. <p>
  1572. <a href="api/org/apache/hadoop/filecache/DistributedCache.html">
  1573. DistributedCache</a> distributes application-specific, large, read-only
  1574. files efficiently.</p>
  1575. <p>
  1576. <span class="codefrag">DistributedCache</span> is a facility provided by the
  1577. Map-Reduce framework to cache files (text, archives, jars and so on)
  1578. needed by applications.</p>
  1579. <p>Applications specify the files to be cached via urls (hdfs:// or
  1580. http://) in the <span class="codefrag">JobConf</span>. The <span class="codefrag">DistributedCache</span>
  1581. assumes that the files specified via hdfs:// urls are already present
  1582. on the <span class="codefrag">FileSystem</span>.</p>
  1583. <p>The framework will copy the necessary files to the slave node
  1584. before any tasks for the job are executed on that node. Its
  1585. efficiency stems from the fact that the files are only copied once
  1586. per job and the ability to cache archives which are un-archived on
  1587. the slaves.</p>
  1588. <p>
  1589. <span class="codefrag">DistributedCache</span> tracks the modification timestamps of
  1590. the cached files. Clearly the cache files should not be modified by
  1591. the application or externally while the job is executing.</p>
  1592. <p>
  1593. <span class="codefrag">DistributedCache</span> can be used to distribute simple,
  1594. read-only data/text files and more complex types such as archives and
  1595. jars. Archives (zip, tar, tgz and tar.gz files) are
  1596. <em>un-archived</em> at the slave nodes.
  1597. Optionally users can also direct the <span class="codefrag">DistributedCache</span> to
  1598. <em>symlink</em> the cached file(s) into the <span class="codefrag">current working
  1599. directory</span> of the task via the
  1600. <a href="api/org/apache/hadoop/filecache/DistributedCache.html#createSymlink(org.apache.hadoop.conf.Configuration)">
  1601. DistributedCache.createSymlink(Configuration)</a> api. Files
  1602. have <em>execution permissions</em> set.</p>
  1603. <a name="N10B2E"></a><a name="Tool"></a>
  1604. <h4>Tool</h4>
  1605. <p>The <a href="api/org/apache/hadoop/util/Tool.html">Tool</a>
  1606. interface supports the handling of generic Hadoop command-line options.
  1607. </p>
  1608. <p>
  1609. <span class="codefrag">Tool</span> is the standard for any Map-Reduce tool or
  1610. application. The application should delegate the handling of
  1611. standard command-line options to
  1612. <a href="api/org/apache/hadoop/util/GenericOptionsParser.html">
  1613. GenericOptionsParser</a> via
  1614. <a href="api/org/apache/hadoop/util/ToolRunner.html#run(org.apache.hadoop.util.Tool, java.lang.String[])">
  1615. ToolRunner.run(Tool, String[])</a> and only handle its custom
  1616. arguments.</p>
  1617. <p>
  1618. The generic Hadoop command-line options are:<br>
  1619. <span class="codefrag">
  1620. -conf &lt;configuration file&gt;
  1621. </span>
  1622. <br>
  1623. <span class="codefrag">
  1624. -D &lt;property=value&gt;
  1625. </span>
  1626. <br>
  1627. <span class="codefrag">
  1628. -fs &lt;local|namenode:port&gt;
  1629. </span>
  1630. <br>
  1631. <span class="codefrag">
  1632. -jt &lt;local|jobtracker:port&gt;
  1633. </span>
  1634. </p>
  1635. <a name="N10B60"></a><a name="IsolationRunner"></a>
  1636. <h4>IsolationRunner</h4>
  1637. <p>
  1638. <a href="api/org/apache/hadoop/mapred/IsolationRunner.html">
  1639. IsolationRunner</a> is a utility to help debug Map-Reduce programs.</p>
  1640. <p>To use the <span class="codefrag">IsolationRunner</span>, first set
  1641. <span class="codefrag">keep.failed.tasks.files</span> to <span class="codefrag">true</span>
  1642. (also see <span class="codefrag">keep.tasks.files.pattern</span>).</p>
  1643. <p>
  1644. Next, go to the node on which the failed task ran and go to the
  1645. <span class="codefrag">TaskTracker</span>'s local directory and run the
  1646. <span class="codefrag">IsolationRunner</span>:<br>
  1647. <span class="codefrag">$ cd &lt;local path&gt;/taskTracker/${taskid}/work</span>
  1648. <br>
  1649. <span class="codefrag">
  1650. $ bin/hadoop org.apache.hadoop.mapred.IsolationRunner ../job.xml
  1651. </span>
  1652. </p>
  1653. <p>
  1654. <span class="codefrag">IsolationRunner</span> will run the failed task in a single
  1655. jvm, which can be in the debugger, over precisely the same input.</p>
  1656. <a name="N10B93"></a><a name="Debugging"></a>
  1657. <h4>Debugging</h4>
  1658. <p>Map/Reduce framework provides a facility to run user-provided
  1659. scripts for debugging. When map/reduce task fails, user can run
  1660. script for doing post-processing on task logs i.e task's stdout,
  1661. stderr, syslog and jobconf. The stdout and stderr of the
  1662. user-provided debug script are printed on the diagnostics.
  1663. These outputs are also displayed on job UI on demand. </p>
  1664. <p> In the following sections we discuss how to submit debug script
  1665. along with the job. For submitting debug script, first it has to
  1666. distributed. Then the script has to supplied in Configuration. </p>
  1667. <a name="N10B9F"></a><a name="How+to+distribute+script+file%3A"></a>
  1668. <h5> How to distribute script file: </h5>
  1669. <p>
  1670. To distribute the debug script file, first copy the file to the dfs.
  1671. The file can be distributed by setting the property
  1672. "mapred.cache.files" with value "path"#"script-name".
  1673. If more than one file has to be distributed, the files can be added
  1674. as comma separated paths. This property can also be set by APIs
  1675. <a href="api/org/apache/hadoop/filecache/DistributedCache.html#addCacheFile(java.net.URI,%20org.apache.hadoop.conf.Configuration)">
  1676. DistributedCache.addCacheFile(URI,conf) </a> and
  1677. <a href="api/org/apache/hadoop/filecache/DistributedCache.html#setCacheFiles(java.net.URI[],%20org.apache.hadoop.conf.Configuration)">
  1678. DistributedCache.setCacheFiles(URIs,conf) </a> where URI is of
  1679. the form "hdfs://host:port/'absolutepath'#'script-name'".
  1680. For Streaming, the file can be added through
  1681. command line option -cacheFile.
  1682. </p>
  1683. <p>
  1684. The files has to be symlinked in the current working directory of
  1685. of the task. To create symlink for the file, the property
  1686. "mapred.create.symlink" is set to "yes". This can also be set by
  1687. <a href="api/org/apache/hadoop/filecache/DistributedCache.html#createSymlink(org.apache.hadoop.conf.Configuration)">
  1688. DistributedCache.createSymLink(Configuration) </a> api.
  1689. </p>
  1690. <a name="N10BB8"></a><a name="How+to+submit+script%3A"></a>
  1691. <h5> How to submit script: </h5>
  1692. <p> A quick way to submit debug script is to set values for the
  1693. properties "mapred.map.task.debug.script" and
  1694. "mapred.reduce.task.debug.script" for debugging map task and reduce
  1695. task respectively. These properties can also be set by using APIs
  1696. <a href="api/org/apache/hadoop/mapred/JobConf.html#setMapDebugScript(java.lang.String)">
  1697. JobConf.setMapDebugScript(String) </a> and
  1698. <a href="api/org/apache/hadoop/mapred/JobConf.html#setReduceDebugScript(java.lang.String)">
  1699. JobConf.setReduceDebugScript(String) </a>. For streaming, debug
  1700. script can be submitted with command-line options -mapdebug,
  1701. -reducedebug for debugging mapper and reducer respectively.</p>
  1702. <p>The arguments of the script are task's stdout, stderr,
  1703. syslog and jobconf files. The debug command, run on the node where
  1704. the map/reduce failed, is: <br>
  1705. <span class="codefrag"> $script $stdout $stderr $syslog $jobconf </span>
  1706. </p>
  1707. <p> Pipes programs have the c++ program name as a fifth argument
  1708. for the command. Thus for the pipes programs the command is <br>
  1709. <span class="codefrag">$script $stdout $stderr $syslog $jobconf $program </span>
  1710. </p>
  1711. <a name="N10BDA"></a><a name="Default+Behavior%3A"></a>
  1712. <h5> Default Behavior: </h5>
  1713. <p> For pipes, a default script is run to process core dumps under
  1714. gdb, prints stack trace and gives info about running threads. </p>
  1715. <a name="N10BE5"></a><a name="JobControl"></a>
  1716. <h4>JobControl</h4>
  1717. <p>
  1718. <a href="api/org/apache/hadoop/mapred/jobcontrol/package-summary.html">
  1719. JobControl</a> is a utility which encapsulates a set of Map-Reduce jobs
  1720. and their dependencies.</p>
  1721. <a name="N10BF2"></a><a name="Data+Compression"></a>
  1722. <h4>Data Compression</h4>
  1723. <p>Hadoop Map-Reduce provides facilities for the application-writer to
  1724. specify compression for both intermediate map-outputs and the
  1725. job-outputs i.e. output of the reduces. It also comes bundled with
  1726. <a href="api/org/apache/hadoop/io/compress/CompressionCodec.html">
  1727. CompressionCodec</a> implementations for the
  1728. <a href="http://www.zlib.net/">zlib</a> and <a href="http://www.oberhumer.com/opensource/lzo/">lzo</a> compression
  1729. algorithms. The <a href="http://www.gzip.org/">gzip</a> file format is also
  1730. supported.</p>
  1731. <p>Hadoop also provides native implementations of the above compression
  1732. codecs for reasons of both performance (zlib) and non-availability of
  1733. Java libraries (lzo). More details on their usage and availability are
  1734. available <a href="native_libraries.html">here</a>.</p>
  1735. <a name="N10C12"></a><a name="Intermediate+Outputs"></a>
  1736. <h5>Intermediate Outputs</h5>
  1737. <p>Applications can control compression of intermediate map-outputs
  1738. via the
  1739. <a href="api/org/apache/hadoop/mapred/JobConf.html#setCompressMapOutput(boolean)">
  1740. JobConf.setCompressMapOutput(boolean)</a> api and the
  1741. <span class="codefrag">CompressionCodec</span> to be used via the
  1742. <a href="api/org/apache/hadoop/mapred/JobConf.html#setMapOutputCompressorClass(java.lang.Class)">
  1743. JobConf.setMapOutputCompressorClass(Class)</a> api. Since
  1744. the intermediate map-outputs are always stored in the
  1745. <a href="api/org/apache/hadoop/io/SequenceFile.html">SequenceFile</a>
  1746. format, the
  1747. <a href="api/org/apache/hadoop/io/SequenceFile.CompressionType.html">
  1748. SequenceFile.CompressionType</a> (i.e.
  1749. <a href="api/org/apache/hadoop/io/SequenceFile.CompressionType.html#RECORD">
  1750. RECORD</a> /
  1751. <a href="api/org/apache/hadoop/io/SequenceFile.CompressionType.html#BLOCK">
  1752. BLOCK</a> - defaults to <span class="codefrag">RECORD</span>) can be specified via the
  1753. <a href="api/org/apache/hadoop/mapred/JobConf.html#setMapOutputCompressionType(org.apache.hadoop.io.SequenceFile.CompressionType)">
  1754. JobConf.setMapOutputCompressionType(SequenceFile.CompressionType)</a>
  1755. api.</p>
  1756. <a name="N10C3E"></a><a name="Job+Outputs"></a>
  1757. <h5>Job Outputs</h5>
  1758. <p>Applications can control compression of job-outputs via the
  1759. <a href="api/org/apache/hadoop/mapred/OutputFormatBase.html#setCompressOutput(org.apache.hadoop.mapred.JobConf,%20boolean)">
  1760. OutputFormatBase.setCompressOutput(JobConf, boolean)</a> api and the
  1761. <span class="codefrag">CompressionCodec</span> to be used can be specified via the
  1762. <a href="api/org/apache/hadoop/mapred/OutputFormatBase.html#setOutputCompressorClass(org.apache.hadoop.mapred.JobConf,%20java.lang.Class)">
  1763. OutputFormatBase.setOutputCompressorClass(JobConf, Class)</a> api.</p>
  1764. <p>If the job outputs are to be stored in the
  1765. <a href="api/org/apache/hadoop/mapred/SequenceFileOutputFormat.html">
  1766. SequenceFileOutputFormat</a>, the required
  1767. <span class="codefrag">SequenceFile.CompressionType</span> (i.e. <span class="codefrag">RECORD</span> /
  1768. <span class="codefrag">BLOCK</span> - defaults to <span class="codefrag">RECORD</span>)can be specified
  1769. via the
  1770. <a href="api/org/apache/hadoop/mapred/SequenceFileOutputFormat.html#setOutputCompressionType(org.apache.hadoop.mapred.JobConf,%20org.apache.hadoop.io.SequenceFile.CompressionType)">
  1771. SequenceFileOutputFormat.setOutputCompressionType(JobConf,
  1772. SequenceFile.CompressionType)</a> api.</p>
  1773. </div>
  1774. <a name="N10C6D"></a><a name="Example%3A+WordCount+v2.0"></a>
  1775. <h2 class="h3">Example: WordCount v2.0</h2>
  1776. <div class="section">
  1777. <p>Here is a more complete <span class="codefrag">WordCount</span> which uses many of the
  1778. features provided by the Map-Reduce framework we discussed so far.</p>
  1779. <p>This needs the HDFS to be up and running, especially for the
  1780. <span class="codefrag">DistributedCache</span>-related features. Hence it only works with a
  1781. <a href="quickstart.html#SingleNodeSetup">pseudo-distributed</a> or
  1782. <a href="quickstart.html#Fully-Distributed+Operation">fully-distributed</a>
  1783. Hadoop installation.</p>
  1784. <a name="N10C87"></a><a name="Source+Code-N10C87"></a>
  1785. <h3 class="h4">Source Code</h3>
  1786. <table class="ForrestTable" cellspacing="1" cellpadding="4">
  1787. <tr>
  1788. <th colspan="1" rowspan="1"></th>
  1789. <th colspan="1" rowspan="1">WordCount.java</th>
  1790. </tr>
  1791. <tr>
  1792. <td colspan="1" rowspan="1">1.</td>
  1793. <td colspan="1" rowspan="1">
  1794. <span class="codefrag">package org.myorg;</span>
  1795. </td>
  1796. </tr>
  1797. <tr>
  1798. <td colspan="1" rowspan="1">2.</td>
  1799. <td colspan="1" rowspan="1"></td>
  1800. </tr>
  1801. <tr>
  1802. <td colspan="1" rowspan="1">3.</td>
  1803. <td colspan="1" rowspan="1">
  1804. <span class="codefrag">import java.io.*;</span>
  1805. </td>
  1806. </tr>
  1807. <tr>
  1808. <td colspan="1" rowspan="1">4.</td>
  1809. <td colspan="1" rowspan="1">
  1810. <span class="codefrag">import java.util.*;</span>
  1811. </td>
  1812. </tr>
  1813. <tr>
  1814. <td colspan="1" rowspan="1">5.</td>
  1815. <td colspan="1" rowspan="1"></td>
  1816. </tr>
  1817. <tr>
  1818. <td colspan="1" rowspan="1">6.</td>
  1819. <td colspan="1" rowspan="1">
  1820. <span class="codefrag">import org.apache.hadoop.fs.Path;</span>
  1821. </td>
  1822. </tr>
  1823. <tr>
  1824. <td colspan="1" rowspan="1">7.</td>
  1825. <td colspan="1" rowspan="1">
  1826. <span class="codefrag">import org.apache.hadoop.filecache.DistributedCache;</span>
  1827. </td>
  1828. </tr>
  1829. <tr>
  1830. <td colspan="1" rowspan="1">8.</td>
  1831. <td colspan="1" rowspan="1">
  1832. <span class="codefrag">import org.apache.hadoop.conf.*;</span>
  1833. </td>
  1834. </tr>
  1835. <tr>
  1836. <td colspan="1" rowspan="1">9.</td>
  1837. <td colspan="1" rowspan="1">
  1838. <span class="codefrag">import org.apache.hadoop.io.*;</span>
  1839. </td>
  1840. </tr>
  1841. <tr>
  1842. <td colspan="1" rowspan="1">10.</td>
  1843. <td colspan="1" rowspan="1">
  1844. <span class="codefrag">import org.apache.hadoop.mapred.*;</span>
  1845. </td>
  1846. </tr>
  1847. <tr>
  1848. <td colspan="1" rowspan="1">11.</td>
  1849. <td colspan="1" rowspan="1">
  1850. <span class="codefrag">import org.apache.hadoop.util.*;</span>
  1851. </td>
  1852. </tr>
  1853. <tr>
  1854. <td colspan="1" rowspan="1">12.</td>
  1855. <td colspan="1" rowspan="1"></td>
  1856. </tr>
  1857. <tr>
  1858. <td colspan="1" rowspan="1">13.</td>
  1859. <td colspan="1" rowspan="1">
  1860. <span class="codefrag">public class WordCount extends Configured implements Tool {</span>
  1861. </td>
  1862. </tr>
  1863. <tr>
  1864. <td colspan="1" rowspan="1">14.</td>
  1865. <td colspan="1" rowspan="1"></td>
  1866. </tr>
  1867. <tr>
  1868. <td colspan="1" rowspan="1">15.</td>
  1869. <td colspan="1" rowspan="1">
  1870. &nbsp;&nbsp;
  1871. <span class="codefrag">
  1872. public static class Map extends MapReduceBase
  1873. implements Mapper&lt;LongWritable, Text, Text, IntWritable&gt; {
  1874. </span>
  1875. </td>
  1876. </tr>
  1877. <tr>
  1878. <td colspan="1" rowspan="1">16.</td>
  1879. <td colspan="1" rowspan="1"></td>
  1880. </tr>
  1881. <tr>
  1882. <td colspan="1" rowspan="1">17.</td>
  1883. <td colspan="1" rowspan="1">
  1884. &nbsp;&nbsp;&nbsp;&nbsp;
  1885. <span class="codefrag">
  1886. static enum Counters { INPUT_WORDS }
  1887. </span>
  1888. </td>
  1889. </tr>
  1890. <tr>
  1891. <td colspan="1" rowspan="1">18.</td>
  1892. <td colspan="1" rowspan="1"></td>
  1893. </tr>
  1894. <tr>
  1895. <td colspan="1" rowspan="1">19.</td>
  1896. <td colspan="1" rowspan="1">
  1897. &nbsp;&nbsp;&nbsp;&nbsp;
  1898. <span class="codefrag">
  1899. private final static IntWritable one = new IntWritable(1);
  1900. </span>
  1901. </td>
  1902. </tr>
  1903. <tr>
  1904. <td colspan="1" rowspan="1">20.</td>
  1905. <td colspan="1" rowspan="1">
  1906. &nbsp;&nbsp;&nbsp;&nbsp;
  1907. <span class="codefrag">private Text word = new Text();</span>
  1908. </td>
  1909. </tr>
  1910. <tr>
  1911. <td colspan="1" rowspan="1">21.</td>
  1912. <td colspan="1" rowspan="1"></td>
  1913. </tr>
  1914. <tr>
  1915. <td colspan="1" rowspan="1">22.</td>
  1916. <td colspan="1" rowspan="1">
  1917. &nbsp;&nbsp;&nbsp;&nbsp;
  1918. <span class="codefrag">private boolean caseSensitive = true;</span>
  1919. </td>
  1920. </tr>
  1921. <tr>
  1922. <td colspan="1" rowspan="1">23.</td>
  1923. <td colspan="1" rowspan="1">
  1924. &nbsp;&nbsp;&nbsp;&nbsp;
  1925. <span class="codefrag">private Set&lt;String&gt; patternsToSkip = new HashSet&lt;String&gt;();</span>
  1926. </td>
  1927. </tr>
  1928. <tr>
  1929. <td colspan="1" rowspan="1">24.</td>
  1930. <td colspan="1" rowspan="1"></td>
  1931. </tr>
  1932. <tr>
  1933. <td colspan="1" rowspan="1">25.</td>
  1934. <td colspan="1" rowspan="1">
  1935. &nbsp;&nbsp;&nbsp;&nbsp;
  1936. <span class="codefrag">private long numRecords = 0;</span>
  1937. </td>
  1938. </tr>
  1939. <tr>
  1940. <td colspan="1" rowspan="1">26.</td>
  1941. <td colspan="1" rowspan="1">
  1942. &nbsp;&nbsp;&nbsp;&nbsp;
  1943. <span class="codefrag">private String inputFile;</span>
  1944. </td>
  1945. </tr>
  1946. <tr>
  1947. <td colspan="1" rowspan="1">27.</td>
  1948. <td colspan="1" rowspan="1"></td>
  1949. </tr>
  1950. <tr>
  1951. <td colspan="1" rowspan="1">28.</td>
  1952. <td colspan="1" rowspan="1">
  1953. &nbsp;&nbsp;&nbsp;&nbsp;
  1954. <span class="codefrag">public void configure(JobConf job) {</span>
  1955. </td>
  1956. </tr>
  1957. <tr>
  1958. <td colspan="1" rowspan="1">29.</td>
  1959. <td colspan="1" rowspan="1">
  1960. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  1961. <span class="codefrag">
  1962. caseSensitive = job.getBoolean("wordcount.case.sensitive", true);
  1963. </span>
  1964. </td>
  1965. </tr>
  1966. <tr>
  1967. <td colspan="1" rowspan="1">30.</td>
  1968. <td colspan="1" rowspan="1">
  1969. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  1970. <span class="codefrag">inputFile = job.get("map.input.file");</span>
  1971. </td>
  1972. </tr>
  1973. <tr>
  1974. <td colspan="1" rowspan="1">31.</td>
  1975. <td colspan="1" rowspan="1"></td>
  1976. </tr>
  1977. <tr>
  1978. <td colspan="1" rowspan="1">32.</td>
  1979. <td colspan="1" rowspan="1">
  1980. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  1981. <span class="codefrag">if (job.getBoolean("wordcount.skip.patterns", false)) {</span>
  1982. </td>
  1983. </tr>
  1984. <tr>
  1985. <td colspan="1" rowspan="1">33.</td>
  1986. <td colspan="1" rowspan="1">
  1987. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  1988. <span class="codefrag">Path[] patternsFiles = new Path[0];</span>
  1989. </td>
  1990. </tr>
  1991. <tr>
  1992. <td colspan="1" rowspan="1">34.</td>
  1993. <td colspan="1" rowspan="1">
  1994. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  1995. <span class="codefrag">try {</span>
  1996. </td>
  1997. </tr>
  1998. <tr>
  1999. <td colspan="1" rowspan="1">35.</td>
  2000. <td colspan="1" rowspan="1">
  2001. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2002. <span class="codefrag">
  2003. patternsFiles = DistributedCache.getLocalCacheFiles(job);
  2004. </span>
  2005. </td>
  2006. </tr>
  2007. <tr>
  2008. <td colspan="1" rowspan="1">36.</td>
  2009. <td colspan="1" rowspan="1">
  2010. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2011. <span class="codefrag">} catch (IOException ioe) {</span>
  2012. </td>
  2013. </tr>
  2014. <tr>
  2015. <td colspan="1" rowspan="1">37.</td>
  2016. <td colspan="1" rowspan="1">
  2017. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2018. <span class="codefrag">
  2019. System.err.println("Caught exception while getting cached files: "
  2020. + StringUtils.stringifyException(ioe));
  2021. </span>
  2022. </td>
  2023. </tr>
  2024. <tr>
  2025. <td colspan="1" rowspan="1">38.</td>
  2026. <td colspan="1" rowspan="1">
  2027. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2028. <span class="codefrag">}</span>
  2029. </td>
  2030. </tr>
  2031. <tr>
  2032. <td colspan="1" rowspan="1">39.</td>
  2033. <td colspan="1" rowspan="1">
  2034. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2035. <span class="codefrag">for (Path patternsFile : patternsFiles) {</span>
  2036. </td>
  2037. </tr>
  2038. <tr>
  2039. <td colspan="1" rowspan="1">40.</td>
  2040. <td colspan="1" rowspan="1">
  2041. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2042. <span class="codefrag">parseSkipFile(patternsFile);</span>
  2043. </td>
  2044. </tr>
  2045. <tr>
  2046. <td colspan="1" rowspan="1">41.</td>
  2047. <td colspan="1" rowspan="1">
  2048. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2049. <span class="codefrag">}</span>
  2050. </td>
  2051. </tr>
  2052. <tr>
  2053. <td colspan="1" rowspan="1">42.</td>
  2054. <td colspan="1" rowspan="1">
  2055. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2056. <span class="codefrag">}</span>
  2057. </td>
  2058. </tr>
  2059. <tr>
  2060. <td colspan="1" rowspan="1">43.</td>
  2061. <td colspan="1" rowspan="1">
  2062. &nbsp;&nbsp;&nbsp;&nbsp;
  2063. <span class="codefrag">}</span>
  2064. </td>
  2065. </tr>
  2066. <tr>
  2067. <td colspan="1" rowspan="1">44.</td>
  2068. <td colspan="1" rowspan="1"></td>
  2069. </tr>
  2070. <tr>
  2071. <td colspan="1" rowspan="1">45.</td>
  2072. <td colspan="1" rowspan="1">
  2073. &nbsp;&nbsp;&nbsp;&nbsp;
  2074. <span class="codefrag">private void parseSkipFile(Path patternsFile) {</span>
  2075. </td>
  2076. </tr>
  2077. <tr>
  2078. <td colspan="1" rowspan="1">46.</td>
  2079. <td colspan="1" rowspan="1">
  2080. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2081. <span class="codefrag">try {</span>
  2082. </td>
  2083. </tr>
  2084. <tr>
  2085. <td colspan="1" rowspan="1">47.</td>
  2086. <td colspan="1" rowspan="1">
  2087. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2088. <span class="codefrag">
  2089. BufferedReader fis =
  2090. new BufferedReader(new FileReader(patternsFile.toString()));
  2091. </span>
  2092. </td>
  2093. </tr>
  2094. <tr>
  2095. <td colspan="1" rowspan="1">48.</td>
  2096. <td colspan="1" rowspan="1">
  2097. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2098. <span class="codefrag">String pattern = null;</span>
  2099. </td>
  2100. </tr>
  2101. <tr>
  2102. <td colspan="1" rowspan="1">49.</td>
  2103. <td colspan="1" rowspan="1">
  2104. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2105. <span class="codefrag">while ((pattern = fis.readLine()) != null) {</span>
  2106. </td>
  2107. </tr>
  2108. <tr>
  2109. <td colspan="1" rowspan="1">50.</td>
  2110. <td colspan="1" rowspan="1">
  2111. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2112. <span class="codefrag">patternsToSkip.add(pattern);</span>
  2113. </td>
  2114. </tr>
  2115. <tr>
  2116. <td colspan="1" rowspan="1">51.</td>
  2117. <td colspan="1" rowspan="1">
  2118. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2119. <span class="codefrag">}</span>
  2120. </td>
  2121. </tr>
  2122. <tr>
  2123. <td colspan="1" rowspan="1">52.</td>
  2124. <td colspan="1" rowspan="1">
  2125. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2126. <span class="codefrag">} catch (IOException ioe) {</span>
  2127. </td>
  2128. </tr>
  2129. <tr>
  2130. <td colspan="1" rowspan="1">53.</td>
  2131. <td colspan="1" rowspan="1">
  2132. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2133. <span class="codefrag">
  2134. System.err.println("Caught exception while parsing the cached file '" +
  2135. patternsFile + "' : " +
  2136. StringUtils.stringifyException(ioe));
  2137. </span>
  2138. </td>
  2139. </tr>
  2140. <tr>
  2141. <td colspan="1" rowspan="1">54.</td>
  2142. <td colspan="1" rowspan="1">
  2143. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2144. <span class="codefrag">}</span>
  2145. </td>
  2146. </tr>
  2147. <tr>
  2148. <td colspan="1" rowspan="1">55.</td>
  2149. <td colspan="1" rowspan="1">
  2150. &nbsp;&nbsp;&nbsp;&nbsp;
  2151. <span class="codefrag">}</span>
  2152. </td>
  2153. </tr>
  2154. <tr>
  2155. <td colspan="1" rowspan="1">56.</td>
  2156. <td colspan="1" rowspan="1"></td>
  2157. </tr>
  2158. <tr>
  2159. <td colspan="1" rowspan="1">57.</td>
  2160. <td colspan="1" rowspan="1">
  2161. &nbsp;&nbsp;&nbsp;&nbsp;
  2162. <span class="codefrag">
  2163. public void map(LongWritable key, Text value,
  2164. OutputCollector&lt;Text, IntWritable&gt; output,
  2165. Reporter reporter) throws IOException {
  2166. </span>
  2167. </td>
  2168. </tr>
  2169. <tr>
  2170. <td colspan="1" rowspan="1">58.</td>
  2171. <td colspan="1" rowspan="1">
  2172. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2173. <span class="codefrag">
  2174. String line =
  2175. (caseSensitive) ? value.toString() :
  2176. value.toString().toLowerCase();
  2177. </span>
  2178. </td>
  2179. </tr>
  2180. <tr>
  2181. <td colspan="1" rowspan="1">59.</td>
  2182. <td colspan="1" rowspan="1"></td>
  2183. </tr>
  2184. <tr>
  2185. <td colspan="1" rowspan="1">60.</td>
  2186. <td colspan="1" rowspan="1">
  2187. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2188. <span class="codefrag">for (String pattern : patternsToSkip) {</span>
  2189. </td>
  2190. </tr>
  2191. <tr>
  2192. <td colspan="1" rowspan="1">61.</td>
  2193. <td colspan="1" rowspan="1">
  2194. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2195. <span class="codefrag">line = line.replaceAll(pattern, "");</span>
  2196. </td>
  2197. </tr>
  2198. <tr>
  2199. <td colspan="1" rowspan="1">62.</td>
  2200. <td colspan="1" rowspan="1">
  2201. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2202. <span class="codefrag">}</span>
  2203. </td>
  2204. </tr>
  2205. <tr>
  2206. <td colspan="1" rowspan="1">63.</td>
  2207. <td colspan="1" rowspan="1"></td>
  2208. </tr>
  2209. <tr>
  2210. <td colspan="1" rowspan="1">64.</td>
  2211. <td colspan="1" rowspan="1">
  2212. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2213. <span class="codefrag">StringTokenizer tokenizer = new StringTokenizer(line);</span>
  2214. </td>
  2215. </tr>
  2216. <tr>
  2217. <td colspan="1" rowspan="1">65.</td>
  2218. <td colspan="1" rowspan="1">
  2219. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2220. <span class="codefrag">while (tokenizer.hasMoreTokens()) {</span>
  2221. </td>
  2222. </tr>
  2223. <tr>
  2224. <td colspan="1" rowspan="1">66.</td>
  2225. <td colspan="1" rowspan="1">
  2226. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2227. <span class="codefrag">word.set(tokenizer.nextToken());</span>
  2228. </td>
  2229. </tr>
  2230. <tr>
  2231. <td colspan="1" rowspan="1">67.</td>
  2232. <td colspan="1" rowspan="1">
  2233. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2234. <span class="codefrag">output.collect(word, one);</span>
  2235. </td>
  2236. </tr>
  2237. <tr>
  2238. <td colspan="1" rowspan="1">68.</td>
  2239. <td colspan="1" rowspan="1">
  2240. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2241. <span class="codefrag">reporter.incrCounter(Counters.INPUT_WORDS, 1);</span>
  2242. </td>
  2243. </tr>
  2244. <tr>
  2245. <td colspan="1" rowspan="1">69.</td>
  2246. <td colspan="1" rowspan="1">
  2247. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2248. <span class="codefrag">}</span>
  2249. </td>
  2250. </tr>
  2251. <tr>
  2252. <td colspan="1" rowspan="1">70.</td>
  2253. <td colspan="1" rowspan="1"></td>
  2254. </tr>
  2255. <tr>
  2256. <td colspan="1" rowspan="1">71.</td>
  2257. <td colspan="1" rowspan="1">
  2258. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2259. <span class="codefrag">if ((++numRecords % 100) == 0) {</span>
  2260. </td>
  2261. </tr>
  2262. <tr>
  2263. <td colspan="1" rowspan="1">72.</td>
  2264. <td colspan="1" rowspan="1">
  2265. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2266. <span class="codefrag">
  2267. reporter.setStatus("Finished processing " + numRecords +
  2268. " records " + "from the input file: " +
  2269. inputFile);
  2270. </span>
  2271. </td>
  2272. </tr>
  2273. <tr>
  2274. <td colspan="1" rowspan="1">73.</td>
  2275. <td colspan="1" rowspan="1">
  2276. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2277. <span class="codefrag">}</span>
  2278. </td>
  2279. </tr>
  2280. <tr>
  2281. <td colspan="1" rowspan="1">74.</td>
  2282. <td colspan="1" rowspan="1">
  2283. &nbsp;&nbsp;&nbsp;&nbsp;
  2284. <span class="codefrag">}</span>
  2285. </td>
  2286. </tr>
  2287. <tr>
  2288. <td colspan="1" rowspan="1">75.</td>
  2289. <td colspan="1" rowspan="1">
  2290. &nbsp;&nbsp;
  2291. <span class="codefrag">}</span>
  2292. </td>
  2293. </tr>
  2294. <tr>
  2295. <td colspan="1" rowspan="1">76.</td>
  2296. <td colspan="1" rowspan="1"></td>
  2297. </tr>
  2298. <tr>
  2299. <td colspan="1" rowspan="1">77.</td>
  2300. <td colspan="1" rowspan="1">
  2301. &nbsp;&nbsp;
  2302. <span class="codefrag">
  2303. public static class Reduce extends MapReduceBase implements
  2304. Reducer&lt;Text, IntWritable, Text, IntWritable&gt; {
  2305. </span>
  2306. </td>
  2307. </tr>
  2308. <tr>
  2309. <td colspan="1" rowspan="1">78.</td>
  2310. <td colspan="1" rowspan="1">
  2311. &nbsp;&nbsp;&nbsp;&nbsp;
  2312. <span class="codefrag">
  2313. public void reduce(Text key, Iterator&lt;IntWritable&gt; values,
  2314. OutputCollector&lt;Text, IntWritable&gt; output,
  2315. Reporter reporter) throws IOException {
  2316. </span>
  2317. </td>
  2318. </tr>
  2319. <tr>
  2320. <td colspan="1" rowspan="1">79.</td>
  2321. <td colspan="1" rowspan="1">
  2322. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2323. <span class="codefrag">int sum = 0;</span>
  2324. </td>
  2325. </tr>
  2326. <tr>
  2327. <td colspan="1" rowspan="1">80.</td>
  2328. <td colspan="1" rowspan="1">
  2329. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2330. <span class="codefrag">while (values.hasNext()) {</span>
  2331. </td>
  2332. </tr>
  2333. <tr>
  2334. <td colspan="1" rowspan="1">81.</td>
  2335. <td colspan="1" rowspan="1">
  2336. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2337. <span class="codefrag">sum += values.next().get();</span>
  2338. </td>
  2339. </tr>
  2340. <tr>
  2341. <td colspan="1" rowspan="1">82.</td>
  2342. <td colspan="1" rowspan="1">
  2343. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2344. <span class="codefrag">}</span>
  2345. </td>
  2346. </tr>
  2347. <tr>
  2348. <td colspan="1" rowspan="1">83.</td>
  2349. <td colspan="1" rowspan="1">
  2350. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2351. <span class="codefrag">output.collect(key, new IntWritable(sum));</span>
  2352. </td>
  2353. </tr>
  2354. <tr>
  2355. <td colspan="1" rowspan="1">84.</td>
  2356. <td colspan="1" rowspan="1">
  2357. &nbsp;&nbsp;&nbsp;&nbsp;
  2358. <span class="codefrag">}</span>
  2359. </td>
  2360. </tr>
  2361. <tr>
  2362. <td colspan="1" rowspan="1">85.</td>
  2363. <td colspan="1" rowspan="1">
  2364. &nbsp;&nbsp;
  2365. <span class="codefrag">}</span>
  2366. </td>
  2367. </tr>
  2368. <tr>
  2369. <td colspan="1" rowspan="1">86.</td>
  2370. <td colspan="1" rowspan="1"></td>
  2371. </tr>
  2372. <tr>
  2373. <td colspan="1" rowspan="1">87.</td>
  2374. <td colspan="1" rowspan="1">
  2375. &nbsp;&nbsp;
  2376. <span class="codefrag">public int run(String[] args) throws Exception {</span>
  2377. </td>
  2378. </tr>
  2379. <tr>
  2380. <td colspan="1" rowspan="1">88.</td>
  2381. <td colspan="1" rowspan="1">
  2382. &nbsp;&nbsp;&nbsp;&nbsp;
  2383. <span class="codefrag">
  2384. JobConf conf = new JobConf(getConf(), WordCount.class);
  2385. </span>
  2386. </td>
  2387. </tr>
  2388. <tr>
  2389. <td colspan="1" rowspan="1">89.</td>
  2390. <td colspan="1" rowspan="1">
  2391. &nbsp;&nbsp;&nbsp;&nbsp;
  2392. <span class="codefrag">conf.setJobName("wordcount");</span>
  2393. </td>
  2394. </tr>
  2395. <tr>
  2396. <td colspan="1" rowspan="1">90.</td>
  2397. <td colspan="1" rowspan="1"></td>
  2398. </tr>
  2399. <tr>
  2400. <td colspan="1" rowspan="1">91.</td>
  2401. <td colspan="1" rowspan="1">
  2402. &nbsp;&nbsp;&nbsp;&nbsp;
  2403. <span class="codefrag">conf.setOutputKeyClass(Text.class);</span>
  2404. </td>
  2405. </tr>
  2406. <tr>
  2407. <td colspan="1" rowspan="1">92.</td>
  2408. <td colspan="1" rowspan="1">
  2409. &nbsp;&nbsp;&nbsp;&nbsp;
  2410. <span class="codefrag">conf.setOutputValueClass(IntWritable.class);</span>
  2411. </td>
  2412. </tr>
  2413. <tr>
  2414. <td colspan="1" rowspan="1">93.</td>
  2415. <td colspan="1" rowspan="1"></td>
  2416. </tr>
  2417. <tr>
  2418. <td colspan="1" rowspan="1">94.</td>
  2419. <td colspan="1" rowspan="1">
  2420. &nbsp;&nbsp;&nbsp;&nbsp;
  2421. <span class="codefrag">conf.setMapperClass(Map.class);</span>
  2422. </td>
  2423. </tr>
  2424. <tr>
  2425. <td colspan="1" rowspan="1">95.</td>
  2426. <td colspan="1" rowspan="1">
  2427. &nbsp;&nbsp;&nbsp;&nbsp;
  2428. <span class="codefrag">conf.setCombinerClass(Reduce.class);</span>
  2429. </td>
  2430. </tr>
  2431. <tr>
  2432. <td colspan="1" rowspan="1">96.</td>
  2433. <td colspan="1" rowspan="1">
  2434. &nbsp;&nbsp;&nbsp;&nbsp;
  2435. <span class="codefrag">conf.setReducerClass(Reduce.class);</span>
  2436. </td>
  2437. </tr>
  2438. <tr>
  2439. <td colspan="1" rowspan="1">97.</td>
  2440. <td colspan="1" rowspan="1"></td>
  2441. </tr>
  2442. <tr>
  2443. <td colspan="1" rowspan="1">98.</td>
  2444. <td colspan="1" rowspan="1">
  2445. &nbsp;&nbsp;&nbsp;&nbsp;
  2446. <span class="codefrag">conf.setInputFormat(TextInputFormat.class);</span>
  2447. </td>
  2448. </tr>
  2449. <tr>
  2450. <td colspan="1" rowspan="1">99.</td>
  2451. <td colspan="1" rowspan="1">
  2452. &nbsp;&nbsp;&nbsp;&nbsp;
  2453. <span class="codefrag">conf.setOutputFormat(TextOutputFormat.class);</span>
  2454. </td>
  2455. </tr>
  2456. <tr>
  2457. <td colspan="1" rowspan="1">100.</td>
  2458. <td colspan="1" rowspan="1"></td>
  2459. </tr>
  2460. <tr>
  2461. <td colspan="1" rowspan="1">101.</td>
  2462. <td colspan="1" rowspan="1">
  2463. &nbsp;&nbsp;&nbsp;&nbsp;
  2464. <span class="codefrag">
  2465. List&lt;String&gt; other_args = new ArrayList&lt;String&gt;();
  2466. </span>
  2467. </td>
  2468. </tr>
  2469. <tr>
  2470. <td colspan="1" rowspan="1">102.</td>
  2471. <td colspan="1" rowspan="1">
  2472. &nbsp;&nbsp;&nbsp;&nbsp;
  2473. <span class="codefrag">for (int i=0; i &lt; args.length; ++i) {</span>
  2474. </td>
  2475. </tr>
  2476. <tr>
  2477. <td colspan="1" rowspan="1">103.</td>
  2478. <td colspan="1" rowspan="1">
  2479. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2480. <span class="codefrag">if ("-skip".equals(args[i])) {</span>
  2481. </td>
  2482. </tr>
  2483. <tr>
  2484. <td colspan="1" rowspan="1">104.</td>
  2485. <td colspan="1" rowspan="1">
  2486. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2487. <span class="codefrag">
  2488. DistributedCache.addCacheFile(new Path(args[++i]).toUri(), conf);
  2489. </span>
  2490. </td>
  2491. </tr>
  2492. <tr>
  2493. <td colspan="1" rowspan="1">105.</td>
  2494. <td colspan="1" rowspan="1">
  2495. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2496. <span class="codefrag">
  2497. conf.setBoolean("wordcount.skip.patterns", true);
  2498. </span>
  2499. </td>
  2500. </tr>
  2501. <tr>
  2502. <td colspan="1" rowspan="1">106.</td>
  2503. <td colspan="1" rowspan="1">
  2504. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2505. <span class="codefrag">} else {</span>
  2506. </td>
  2507. </tr>
  2508. <tr>
  2509. <td colspan="1" rowspan="1">107.</td>
  2510. <td colspan="1" rowspan="1">
  2511. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2512. <span class="codefrag">other_args.add(args[i]);</span>
  2513. </td>
  2514. </tr>
  2515. <tr>
  2516. <td colspan="1" rowspan="1">108.</td>
  2517. <td colspan="1" rowspan="1">
  2518. &nbsp;&nbsp;&nbsp;&nbsp;&nbsp;&nbsp;
  2519. <span class="codefrag">}</span>
  2520. </td>
  2521. </tr>
  2522. <tr>
  2523. <td colspan="1" rowspan="1">109.</td>
  2524. <td colspan="1" rowspan="1">
  2525. &nbsp;&nbsp;&nbsp;&nbsp;
  2526. <span class="codefrag">}</span>
  2527. </td>
  2528. </tr>
  2529. <tr>
  2530. <td colspan="1" rowspan="1">110.</td>
  2531. <td colspan="1" rowspan="1"></td>
  2532. </tr>
  2533. <tr>
  2534. <td colspan="1" rowspan="1">111.</td>
  2535. <td colspan="1" rowspan="1">
  2536. &nbsp;&nbsp;&nbsp;&nbsp;
  2537. <span class="codefrag">FileInputFormat.setInputPaths(conf, new Path(other_args.get(0)));</span>
  2538. </td>
  2539. </tr>
  2540. <tr>
  2541. <td colspan="1" rowspan="1">112.</td>
  2542. <td colspan="1" rowspan="1">
  2543. &nbsp;&nbsp;&nbsp;&nbsp;
  2544. <span class="codefrag">FileOutputFormat.setOutputPath(conf, new Path(other_args.get(1)));</span>
  2545. </td>
  2546. </tr>
  2547. <tr>
  2548. <td colspan="1" rowspan="1">113.</td>
  2549. <td colspan="1" rowspan="1"></td>
  2550. </tr>
  2551. <tr>
  2552. <td colspan="1" rowspan="1">114.</td>
  2553. <td colspan="1" rowspan="1">
  2554. &nbsp;&nbsp;&nbsp;&nbsp;
  2555. <span class="codefrag">JobClient.runJob(conf);</span>
  2556. </td>
  2557. </tr>
  2558. <tr>
  2559. <td colspan="1" rowspan="1">115.</td>
  2560. <td colspan="1" rowspan="1">
  2561. &nbsp;&nbsp;&nbsp;&nbsp;
  2562. <span class="codefrag">return 0;</span>
  2563. </td>
  2564. </tr>
  2565. <tr>
  2566. <td colspan="1" rowspan="1">116.</td>
  2567. <td colspan="1" rowspan="1">
  2568. &nbsp;&nbsp;
  2569. <span class="codefrag">}</span>
  2570. </td>
  2571. </tr>
  2572. <tr>
  2573. <td colspan="1" rowspan="1">117.</td>
  2574. <td colspan="1" rowspan="1"></td>
  2575. </tr>
  2576. <tr>
  2577. <td colspan="1" rowspan="1">118.</td>
  2578. <td colspan="1" rowspan="1">
  2579. &nbsp;&nbsp;
  2580. <span class="codefrag">
  2581. public static void main(String[] args) throws Exception {
  2582. </span>
  2583. </td>
  2584. </tr>
  2585. <tr>
  2586. <td colspan="1" rowspan="1">119.</td>
  2587. <td colspan="1" rowspan="1">
  2588. &nbsp;&nbsp;&nbsp;&nbsp;
  2589. <span class="codefrag">
  2590. int res = ToolRunner.run(new Configuration(), new WordCount(),
  2591. args);
  2592. </span>
  2593. </td>
  2594. </tr>
  2595. <tr>
  2596. <td colspan="1" rowspan="1">120.</td>
  2597. <td colspan="1" rowspan="1">
  2598. &nbsp;&nbsp;&nbsp;&nbsp;
  2599. <span class="codefrag">System.exit(res);</span>
  2600. </td>
  2601. </tr>
  2602. <tr>
  2603. <td colspan="1" rowspan="1">121.</td>
  2604. <td colspan="1" rowspan="1">
  2605. &nbsp;&nbsp;
  2606. <span class="codefrag">}</span>
  2607. </td>
  2608. </tr>
  2609. <tr>
  2610. <td colspan="1" rowspan="1">122.</td>
  2611. <td colspan="1" rowspan="1">
  2612. <span class="codefrag">}</span>
  2613. </td>
  2614. </tr>
  2615. <tr>
  2616. <td colspan="1" rowspan="1">123.</td>
  2617. <td colspan="1" rowspan="1"></td>
  2618. </tr>
  2619. </table>
  2620. <a name="N113E9"></a><a name="Sample+Runs"></a>
  2621. <h3 class="h4">Sample Runs</h3>
  2622. <p>Sample text-files as input:</p>
  2623. <p>
  2624. <span class="codefrag">$ bin/hadoop dfs -ls /usr/joe/wordcount/input/</span>
  2625. <br>
  2626. <span class="codefrag">/usr/joe/wordcount/input/file01</span>
  2627. <br>
  2628. <span class="codefrag">/usr/joe/wordcount/input/file02</span>
  2629. <br>
  2630. <br>
  2631. <span class="codefrag">$ bin/hadoop dfs -cat /usr/joe/wordcount/input/file01</span>
  2632. <br>
  2633. <span class="codefrag">Hello World, Bye World!</span>
  2634. <br>
  2635. <br>
  2636. <span class="codefrag">$ bin/hadoop dfs -cat /usr/joe/wordcount/input/file02</span>
  2637. <br>
  2638. <span class="codefrag">Hello Hadoop, Goodbye to hadoop.</span>
  2639. </p>
  2640. <p>Run the application:</p>
  2641. <p>
  2642. <span class="codefrag">
  2643. $ bin/hadoop jar /usr/joe/wordcount.jar org.myorg.WordCount
  2644. /usr/joe/wordcount/input /usr/joe/wordcount/output
  2645. </span>
  2646. </p>
  2647. <p>Output:</p>
  2648. <p>
  2649. <span class="codefrag">
  2650. $ bin/hadoop dfs -cat /usr/joe/wordcount/output/part-00000
  2651. </span>
  2652. <br>
  2653. <span class="codefrag">Bye 1</span>
  2654. <br>
  2655. <span class="codefrag">Goodbye 1</span>
  2656. <br>
  2657. <span class="codefrag">Hadoop, 1</span>
  2658. <br>
  2659. <span class="codefrag">Hello 2</span>
  2660. <br>
  2661. <span class="codefrag">World! 1</span>
  2662. <br>
  2663. <span class="codefrag">World, 1</span>
  2664. <br>
  2665. <span class="codefrag">hadoop. 1</span>
  2666. <br>
  2667. <span class="codefrag">to 1</span>
  2668. <br>
  2669. </p>
  2670. <p>Notice that the inputs differ from the first version we looked at,
  2671. and how they affect the outputs.</p>
  2672. <p>Now, lets plug-in a pattern-file which lists the word-patterns to be
  2673. ignored, via the <span class="codefrag">DistributedCache</span>.</p>
  2674. <p>
  2675. <span class="codefrag">$ hadoop dfs -cat /user/joe/wordcount/patterns.txt</span>
  2676. <br>
  2677. <span class="codefrag">\.</span>
  2678. <br>
  2679. <span class="codefrag">\,</span>
  2680. <br>
  2681. <span class="codefrag">\!</span>
  2682. <br>
  2683. <span class="codefrag">to</span>
  2684. <br>
  2685. </p>
  2686. <p>Run it again, this time with more options:</p>
  2687. <p>
  2688. <span class="codefrag">
  2689. $ bin/hadoop jar /usr/joe/wordcount.jar org.myorg.WordCount
  2690. -Dwordcount.case.sensitive=true /usr/joe/wordcount/input
  2691. /usr/joe/wordcount/output -skip /user/joe/wordcount/patterns.txt
  2692. </span>
  2693. </p>
  2694. <p>As expected, the output:</p>
  2695. <p>
  2696. <span class="codefrag">
  2697. $ bin/hadoop dfs -cat /usr/joe/wordcount/output/part-00000
  2698. </span>
  2699. <br>
  2700. <span class="codefrag">Bye 1</span>
  2701. <br>
  2702. <span class="codefrag">Goodbye 1</span>
  2703. <br>
  2704. <span class="codefrag">Hadoop 1</span>
  2705. <br>
  2706. <span class="codefrag">Hello 2</span>
  2707. <br>
  2708. <span class="codefrag">World 2</span>
  2709. <br>
  2710. <span class="codefrag">hadoop 1</span>
  2711. <br>
  2712. </p>
  2713. <p>Run it once more, this time switch-off case-sensitivity:</p>
  2714. <p>
  2715. <span class="codefrag">
  2716. $ bin/hadoop jar /usr/joe/wordcount.jar org.myorg.WordCount
  2717. -Dwordcount.case.sensitive=false /usr/joe/wordcount/input
  2718. /usr/joe/wordcount/output -skip /user/joe/wordcount/patterns.txt
  2719. </span>
  2720. </p>
  2721. <p>Sure enough, the output:</p>
  2722. <p>
  2723. <span class="codefrag">
  2724. $ bin/hadoop dfs -cat /usr/joe/wordcount/output/part-00000
  2725. </span>
  2726. <br>
  2727. <span class="codefrag">bye 1</span>
  2728. <br>
  2729. <span class="codefrag">goodbye 1</span>
  2730. <br>
  2731. <span class="codefrag">hadoop 2</span>
  2732. <br>
  2733. <span class="codefrag">hello 2</span>
  2734. <br>
  2735. <span class="codefrag">world 2</span>
  2736. <br>
  2737. </p>
  2738. <a name="N114BD"></a><a name="Highlights"></a>
  2739. <h3 class="h4">Highlights</h3>
  2740. <p>The second version of <span class="codefrag">WordCount</span> improves upon the
  2741. previous one by using some features offered by the Map-Reduce framework:
  2742. </p>
  2743. <ul>
  2744. <li>
  2745. Demonstrates how applications can access configuration parameters
  2746. in the <span class="codefrag">configure</span> method of the <span class="codefrag">Mapper</span> (and
  2747. <span class="codefrag">Reducer</span>) implementations (lines 28-43).
  2748. </li>
  2749. <li>
  2750. Demonstrates how the <span class="codefrag">DistributedCache</span> can be used to
  2751. distribute read-only data needed by the jobs. Here it allows the user
  2752. to specify word-patterns to skip while counting (line 104).
  2753. </li>
  2754. <li>
  2755. Demonstrates the utility of the <span class="codefrag">Tool</span> interface and the
  2756. <span class="codefrag">GenericOptionsParser</span> to handle generic Hadoop
  2757. command-line options (lines 87-116, 119).
  2758. </li>
  2759. <li>
  2760. Demonstrates how applications can use <span class="codefrag">Counters</span> (line 68)
  2761. and how they can set application-specific status information via
  2762. the <span class="codefrag">Reporter</span> instance passed to the <span class="codefrag">map</span> (and
  2763. <span class="codefrag">reduce</span>) method (line 72).
  2764. </li>
  2765. </ul>
  2766. </div>
  2767. <p>
  2768. <em>Java and JNI are trademarks or registered trademarks of
  2769. Sun Microsystems, Inc. in the United States and other countries.</em>
  2770. </p>
  2771. </div>
  2772. <!--+
  2773. |end content
  2774. +-->
  2775. <div class="clearboth">&nbsp;</div>
  2776. </div>
  2777. <div id="footer">
  2778. <!--+
  2779. |start bottomstrip
  2780. +-->
  2781. <div class="lastmodified">
  2782. <script type="text/javascript"><!--
  2783. document.write("Last Published: " + document.lastModified);
  2784. // --></script>
  2785. </div>
  2786. <div class="copyright">
  2787. Copyright &copy;
  2788. 2007 <a href="http://www.apache.org/licenses/">The Apache Software Foundation.</a>
  2789. </div>
  2790. <!--+
  2791. |end bottomstrip
  2792. +-->
  2793. </div>
  2794. </body>
  2795. </html>