mapred_tutorial.html 114 KB

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