1234567891011121314151617181920212223242526272829303132333435363738394041424344454647484950515253545556575859606162636465666768697071727374757677787980818283848586878889909192939495969798991001011021031041051061071081091101111121131141151161171181191201211221231241251261271281291301311321331341351361371381391401411421431441451461471481491501511521531541551561571581591601611621631641651661671681691701711721731741751761771781791801811821831841851861871881891901911921931941951961971981992002012022032042052062072082092102112122132142152162172182192202212222232242252262272282292302312322332342352362372382392402412422432442452462472482492502512522532542552562572582592602612622632642652662672682692702712722732742752762772782792802812822832842852862872882892902912922932942952962972982993003013023033043053063073083093103113123133143153163173183193203213223233243253263273283293303313323333343353363373383393403413423433443453463473483493503513523533543553563573583593603613623633643653663673683693703713723733743753763773783793803813823833843853863873883893903913923933943953963973983994004014024034044054064074084094104114124134144154164174184194204214224234244254264274284294304314324334344354364374384394404414424434444454464474484494504514524534544554564574584594604614624634644654664674684694704714724734744754764774784794804814824834844854864874884894904914924934944954964974984995005015025035045055065075085095105115125135145155165175185195205215225235245255265275285295305315325335345355365375385395405415425435445455465475485495505515525535545555565575585595605615625635645655665675685695705715725735745755765775785795805815825835845855865875885895905915925935945955965975985996006016026036046056066076086096106116126136146156166176186196206216226236246256266276286296306316326336346356366376386396406416426436446456466476486496506516526536546556566576586596606616626636646656666676686696706716726736746756766776786796806816826836846856866876886896906916926936946956966976986997007017027037047057067077087097107117127137147157167177187197207217227237247257267277287297307317327337347357367377387397407417427437447457467477487497507517527537547557567577587597607617627637647657667677687697707717727737747757767777787797807817827837847857867877887897907917927937947957967977987998008018028038048058068078088098108118128138148158168178188198208218228238248258268278288298308318328338348358368378388398408418428438448458468478488498508518528538548558568578588598608618628638648658668678688698708718728738748758768778788798808818828838848858868878888898908918928938948958968978988999009019029039049059069079089099109119129139149159169179189199209219229239249259269279289299309319329339349359369379389399409419429439449459469479489499509519529539549559569579589599609619629639649659669679689699709719729739749759769779789799809819829839849859869879889899909919929939949959969979989991000100110021003100410051006100710081009101010111012101310141015101610171018101910201021102210231024102510261027102810291030103110321033103410351036103710381039104010411042104310441045104610471048104910501051105210531054105510561057105810591060106110621063106410651066106710681069107010711072107310741075107610771078107910801081108210831084108510861087108810891090109110921093109410951096109710981099110011011102110311041105110611071108110911101111111211131114111511161117111811191120112111221123112411251126112711281129113011311132113311341135113611371138113911401141114211431144114511461147114811491150115111521153115411551156115711581159116011611162116311641165116611671168116911701171117211731174117511761177117811791180118111821183118411851186118711881189119011911192119311941195119611971198119912001201120212031204120512061207120812091210121112121213121412151216121712181219122012211222122312241225122612271228122912301231123212331234123512361237123812391240124112421243124412451246124712481249125012511252125312541255125612571258125912601261126212631264126512661267126812691270127112721273127412751276127712781279128012811282128312841285128612871288128912901291129212931294129512961297129812991300130113021303130413051306130713081309131013111312131313141315131613171318131913201321132213231324132513261327132813291330133113321333133413351336133713381339134013411342134313441345134613471348134913501351135213531354135513561357135813591360136113621363136413651366136713681369137013711372137313741375137613771378137913801381138213831384138513861387138813891390139113921393139413951396139713981399140014011402140314041405140614071408140914101411141214131414141514161417141814191420142114221423142414251426142714281429143014311432143314341435143614371438143914401441144214431444144514461447144814491450145114521453145414551456145714581459146014611462146314641465146614671468146914701471147214731474147514761477147814791480148114821483148414851486148714881489149014911492149314941495149614971498149915001501150215031504150515061507150815091510151115121513151415151516151715181519152015211522152315241525152615271528152915301531153215331534153515361537153815391540154115421543154415451546154715481549155015511552155315541555155615571558155915601561156215631564156515661567156815691570157115721573157415751576157715781579158015811582158315841585158615871588158915901591159215931594159515961597159815991600160116021603160416051606160716081609161016111612161316141615161616171618161916201621162216231624162516261627162816291630163116321633163416351636163716381639164016411642164316441645164616471648164916501651165216531654165516561657165816591660166116621663166416651666166716681669167016711672167316741675167616771678167916801681168216831684168516861687168816891690169116921693169416951696169716981699170017011702170317041705170617071708170917101711171217131714171517161717171817191720172117221723172417251726172717281729173017311732173317341735173617371738173917401741174217431744174517461747174817491750175117521753175417551756175717581759176017611762176317641765176617671768176917701771177217731774177517761777177817791780178117821783178417851786178717881789179017911792179317941795179617971798179918001801180218031804180518061807180818091810181118121813181418151816181718181819182018211822182318241825182618271828182918301831183218331834183518361837183818391840184118421843184418451846184718481849185018511852185318541855185618571858185918601861186218631864186518661867186818691870187118721873187418751876187718781879188018811882188318841885188618871888188918901891189218931894189518961897189818991900190119021903190419051906190719081909191019111912191319141915191619171918191919201921192219231924192519261927192819291930193119321933193419351936193719381939194019411942194319441945194619471948194919501951195219531954195519561957195819591960196119621963196419651966196719681969197019711972197319741975197619771978197919801981198219831984198519861987198819891990199119921993199419951996199719981999200020012002200320042005200620072008200920102011201220132014201520162017201820192020202120222023202420252026202720282029203020312032203320342035203620372038203920402041204220432044204520462047204820492050205120522053205420552056205720582059206020612062206320642065206620672068206920702071207220732074207520762077207820792080208120822083208420852086208720882089209020912092209320942095209620972098209921002101210221032104210521062107210821092110211121122113211421152116211721182119212021212122212321242125212621272128212921302131213221332134213521362137213821392140214121422143214421452146214721482149215021512152215321542155215621572158215921602161216221632164216521662167216821692170217121722173217421752176217721782179218021812182218321842185218621872188218921902191219221932194219521962197219821992200220122022203220422052206220722082209221022112212221322142215221622172218221922202221222222232224222522262227222822292230223122322233223422352236223722382239224022412242224322442245224622472248224922502251225222532254225522562257225822592260226122622263226422652266226722682269227022712272227322742275227622772278227922802281228222832284228522862287228822892290229122922293229422952296229722982299230023012302230323042305230623072308230923102311231223132314231523162317231823192320232123222323232423252326232723282329233023312332233323342335233623372338233923402341234223432344234523462347234823492350235123522353235423552356235723582359236023612362236323642365236623672368236923702371237223732374237523762377237823792380238123822383238423852386238723882389239023912392239323942395239623972398239924002401240224032404240524062407240824092410241124122413241424152416241724182419242024212422242324242425242624272428242924302431243224332434243524362437243824392440244124422443244424452446244724482449245024512452245324542455245624572458245924602461246224632464246524662467246824692470247124722473247424752476247724782479248024812482248324842485248624872488248924902491249224932494249524962497249824992500250125022503250425052506250725082509251025112512251325142515251625172518251925202521252225232524252525262527252825292530253125322533253425352536253725382539254025412542254325442545254625472548254925502551255225532554255525562557255825592560256125622563256425652566256725682569257025712572257325742575257625772578257925802581258225832584258525862587258825892590259125922593259425952596259725982599260026012602260326042605260626072608260926102611261226132614261526162617261826192620262126222623262426252626262726282629263026312632263326342635263626372638263926402641264226432644264526462647264826492650265126522653265426552656265726582659266026612662266326642665266626672668266926702671267226732674267526762677267826792680268126822683268426852686268726882689269026912692269326942695269626972698269927002701270227032704270527062707270827092710271127122713271427152716271727182719272027212722272327242725272627272728272927302731273227332734273527362737273827392740274127422743274427452746274727482749275027512752275327542755275627572758275927602761276227632764276527662767276827692770277127722773277427752776277727782779278027812782278327842785278627872788278927902791279227932794279527962797279827992800280128022803280428052806280728082809281028112812281328142815281628172818281928202821282228232824282528262827282828292830283128322833283428352836283728382839284028412842284328442845284628472848284928502851285228532854285528562857285828592860286128622863286428652866286728682869287028712872287328742875287628772878287928802881288228832884288528862887288828892890289128922893289428952896289728982899290029012902290329042905290629072908290929102911291229132914291529162917291829192920292129222923292429252926292729282929293029312932293329342935293629372938293929402941294229432944294529462947294829492950295129522953295429552956295729582959296029612962296329642965296629672968296929702971297229732974297529762977297829792980298129822983298429852986298729882989299029912992299329942995299629972998299930003001300230033004300530063007300830093010301130123013301430153016301730183019302030213022302330243025302630273028302930303031303230333034303530363037303830393040304130423043304430453046304730483049305030513052305330543055305630573058305930603061306230633064306530663067306830693070307130723073307430753076307730783079308030813082308330843085308630873088308930903091309230933094309530963097309830993100310131023103310431053106310731083109311031113112311331143115311631173118311931203121312231233124312531263127312831293130313131323133313431353136313731383139314031413142314331443145314631473148314931503151315231533154315531563157315831593160316131623163316431653166316731683169317031713172317331743175317631773178317931803181318231833184318531863187318831893190319131923193319431953196319731983199320032013202320332043205320632073208320932103211321232133214321532163217321832193220322132223223322432253226322732283229323032313232323332343235323632373238323932403241324232433244324532463247324832493250325132523253325432553256325732583259326032613262326332643265326632673268326932703271327232733274327532763277327832793280328132823283328432853286328732883289329032913292329332943295329632973298329933003301330233033304330533063307330833093310331133123313331433153316331733183319332033213322332333243325332633273328332933303331333233333334333533363337333833393340334133423343334433453346334733483349335033513352335333543355335633573358335933603361336233633364336533663367336833693370337133723373337433753376 |
- <!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
- <html>
- <head>
- <META http-equiv="Content-Type" content="text/html; charset=UTF-8">
- <meta content="Apache Forrest" name="Generator">
- <meta name="Forrest-version" content="0.8">
- <meta name="Forrest-skin-name" content="pelt">
- <title>Hadoop Map-Reduce Tutorial</title>
- <link type="text/css" href="skin/basic.css" rel="stylesheet">
- <link media="screen" type="text/css" href="skin/screen.css" rel="stylesheet">
- <link media="print" type="text/css" href="skin/print.css" rel="stylesheet">
- <link type="text/css" href="skin/profile.css" rel="stylesheet">
- <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>
- <link rel="shortcut icon" href="images/favicon.ico">
- </head>
- <body onload="init()">
- <script type="text/javascript">ndeSetTextSize();</script>
- <div id="top">
- <!--+
- |breadtrail
- +-->
- <div class="breadtrail">
- <a href="http://www.apache.org/">Apache</a> > <a href="http://hadoop.apache.org/">Hadoop</a> > <a href="http://hadoop.apache.org/core/">Core</a><script src="skin/breadcrumbs.js" language="JavaScript" type="text/javascript"></script>
- </div>
- <!--+
- |header
- +-->
- <div class="header">
- <!--+
- |start group logo
- +-->
- <div class="grouplogo">
- <a href="http://hadoop.apache.org/"><img class="logoImage" alt="Hadoop" src="images/hadoop-logo.jpg" title="Apache Hadoop"></a>
- </div>
- <!--+
- |end group logo
- +-->
- <!--+
- |start Project Logo
- +-->
- <div class="projectlogo">
- <a href="http://hadoop.apache.org/core/"><img class="logoImage" alt="Hadoop" src="images/core-logo.jpg" title="Scalable Computing Platform"></a>
- </div>
- <!--+
- |end Project Logo
- +-->
- <!--+
- |start Search
- +-->
- <div class="searchbox">
- <form action="http://www.google.com/search" method="get" class="roundtopsmall">
- <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">
- <input name="Search" value="Search" type="submit">
- </form>
- </div>
- <!--+
- |end search
- +-->
- <!--+
- |start Tabs
- +-->
- <ul id="tabs">
- <li>
- <a class="unselected" href="http://hadoop.apache.org/core/">Project</a>
- </li>
- <li>
- <a class="unselected" href="http://wiki.apache.org/hadoop">Wiki</a>
- </li>
- <li class="current">
- <a class="selected" href="index.html">Hadoop 0.16 Documentation</a>
- </li>
- </ul>
- <!--+
- |end Tabs
- +-->
- </div>
- </div>
- <div id="main">
- <div id="publishedStrip">
- <!--+
- |start Subtabs
- +-->
- <div id="level2tabs"></div>
- <!--+
- |end Endtabs
- +-->
- <script type="text/javascript"><!--
- document.write("Last Published: " + document.lastModified);
- // --></script>
- </div>
- <!--+
- |breadtrail
- +-->
- <div class="breadtrail">
-
- </div>
- <!--+
- |start Menu, mainarea
- +-->
- <!--+
- |start Menu
- +-->
- <div id="menu">
- <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>
- <div id="menu_selected_1.1" class="selectedmenuitemgroup" style="display: block;">
- <div class="menuitem">
- <a href="index.html">Overview</a>
- </div>
- <div class="menuitem">
- <a href="quickstart.html">Quickstart</a>
- </div>
- <div class="menuitem">
- <a href="cluster_setup.html">Cluster Setup</a>
- </div>
- <div class="menuitem">
- <a href="hdfs_design.html">HDFS Architecture</a>
- </div>
- <div class="menuitem">
- <a href="hdfs_user_guide.html">HDFS User Guide</a>
- </div>
- <div class="menuitem">
- <a href="hdfs_permissions_guide.html">HDFS Permissions Guide</a>
- </div>
- <div class="menupage">
- <div class="menupagetitle">Map-Reduce Tutorial</div>
- </div>
- <div class="menuitem">
- <a href="native_libraries.html">Native Hadoop Libraries</a>
- </div>
- <div class="menuitem">
- <a href="streaming.html">Streaming</a>
- </div>
- <div class="menuitem">
- <a href="hod.html">Hadoop On Demand</a>
- </div>
- <div class="menuitem">
- <a href="api/index.html">API Docs</a>
- </div>
- <div class="menuitem">
- <a href="http://wiki.apache.org/hadoop/">Wiki</a>
- </div>
- <div class="menuitem">
- <a href="http://wiki.apache.org/hadoop/FAQ">FAQ</a>
- </div>
- <div class="menuitem">
- <a href="http://hadoop.apache.org/core/mailing_lists.html">Mailing Lists</a>
- </div>
- </div>
- <div id="credit"></div>
- <div id="roundbottom">
- <img style="display: none" class="corner" height="15" width="15" alt="" src="skin/images/rc-b-l-15-1body-2menu-3menu.png"></div>
- <!--+
- |alternative credits
- +-->
- <div id="credit2"></div>
- </div>
- <!--+
- |end Menu
- +-->
- <!--+
- |start content
- +-->
- <div id="content">
- <div title="Portable Document Format" class="pdflink">
- <a class="dida" href="mapred_tutorial.pdf"><img alt="PDF -icon" src="skin/images/pdfdoc.gif" class="skin"><br>
- PDF</a>
- </div>
- <h1>Hadoop Map-Reduce Tutorial</h1>
- <div id="minitoc-area">
- <ul class="minitoc">
- <li>
- <a href="#Purpose">Purpose</a>
- </li>
- <li>
- <a href="#Pre-requisites">Pre-requisites</a>
- </li>
- <li>
- <a href="#Overview">Overview</a>
- </li>
- <li>
- <a href="#Inputs+and+Outputs">Inputs and Outputs</a>
- </li>
- <li>
- <a href="#Example%3A+WordCount+v1.0">Example: WordCount v1.0</a>
- <ul class="minitoc">
- <li>
- <a href="#Source+Code">Source Code</a>
- </li>
- <li>
- <a href="#Usage">Usage</a>
- </li>
- <li>
- <a href="#Walk-through">Walk-through</a>
- </li>
- </ul>
- </li>
- <li>
- <a href="#Map-Reduce+-+User+Interfaces">Map-Reduce - User Interfaces</a>
- <ul class="minitoc">
- <li>
- <a href="#Payload">Payload</a>
- <ul class="minitoc">
- <li>
- <a href="#Mapper">Mapper</a>
- </li>
- <li>
- <a href="#Reducer">Reducer</a>
- </li>
- <li>
- <a href="#Partitioner">Partitioner</a>
- </li>
- <li>
- <a href="#Reporter">Reporter</a>
- </li>
- <li>
- <a href="#OutputCollector">OutputCollector</a>
- </li>
- </ul>
- </li>
- <li>
- <a href="#Job+Configuration">Job Configuration</a>
- </li>
- <li>
- <a href="#Task+Execution+%26+Environment">Task Execution & Environment</a>
- </li>
- <li>
- <a href="#Job+Submission+and+Monitoring">Job Submission and Monitoring</a>
- <ul class="minitoc">
- <li>
- <a href="#Job+Control">Job Control</a>
- </li>
- </ul>
- </li>
- <li>
- <a href="#Job+Input">Job Input</a>
- <ul class="minitoc">
- <li>
- <a href="#InputSplit">InputSplit</a>
- </li>
- <li>
- <a href="#RecordReader">RecordReader</a>
- </li>
- </ul>
- </li>
- <li>
- <a href="#Job+Output">Job Output</a>
- <ul class="minitoc">
- <li>
- <a href="#Task+Side-Effect+Files">Task Side-Effect Files</a>
- </li>
- <li>
- <a href="#RecordWriter">RecordWriter</a>
- </li>
- </ul>
- </li>
- <li>
- <a href="#Other+Useful+Features">Other Useful Features</a>
- <ul class="minitoc">
- <li>
- <a href="#Counters">Counters</a>
- </li>
- <li>
- <a href="#DistributedCache">DistributedCache</a>
- </li>
- <li>
- <a href="#Tool">Tool</a>
- </li>
- <li>
- <a href="#IsolationRunner">IsolationRunner</a>
- </li>
- <li>
- <a href="#JobControl">JobControl</a>
- </li>
- <li>
- <a href="#Data+Compression">Data Compression</a>
- </li>
- </ul>
- </li>
- </ul>
- </li>
- <li>
- <a href="#Example%3A+WordCount+v2.0">Example: WordCount v2.0</a>
- <ul class="minitoc">
- <li>
- <a href="#Source+Code-N10BBE">Source Code</a>
- </li>
- <li>
- <a href="#Sample+Runs">Sample Runs</a>
- </li>
- <li>
- <a href="#Highlights">Highlights</a>
- </li>
- </ul>
- </li>
- </ul>
- </div>
-
-
- <a name="N1000D"></a><a name="Purpose"></a>
- <h2 class="h3">Purpose</h2>
- <div class="section">
- <p>This document comprehensively describes all user-facing facets of the
- Hadoop Map-Reduce framework and serves as a tutorial.
- </p>
- </div>
-
-
- <a name="N10017"></a><a name="Pre-requisites"></a>
- <h2 class="h3">Pre-requisites</h2>
- <div class="section">
- <p>Ensure that Hadoop is installed, configured and is running. More
- details:</p>
- <ul>
-
- <li>
- Hadoop <a href="quickstart.html">Quickstart</a> for first-time users.
- </li>
-
- <li>
- Hadoop <a href="cluster_setup.html">Cluster Setup</a> for large,
- distributed clusters.
- </li>
-
- </ul>
- </div>
-
-
- <a name="N10032"></a><a name="Overview"></a>
- <h2 class="h3">Overview</h2>
- <div class="section">
- <p>Hadoop Map-Reduce is a software framework for easily writing
- applications which process vast amounts of data (multi-terabyte data-sets)
- in-parallel on large clusters (thousands of nodes) of commodity
- hardware in a reliable, fault-tolerant manner.</p>
- <p>A Map-Reduce <em>job</em> usually splits the input data-set into
- independent chunks which are processed by the <em>map tasks</em> in a
- completely parallel manner. The framework sorts the outputs of the maps,
- which are then input to the <em>reduce tasks</em>. Typically both the
- input and the output of the job are stored in a file-system. The framework
- takes care of scheduling tasks, monitoring them and re-executes the failed
- tasks.</p>
- <p>Typically the compute nodes and the storage nodes are the same, that is,
- the Map-Reduce framework and the <a href="hdfs_design.html">Distributed
- FileSystem</a> are running on the same set of nodes. This configuration
- allows the framework to effectively schedule tasks on the nodes where data
- is already present, resulting in very high aggregate bandwidth across the
- cluster.</p>
- <p>The Map-Reduce framework consists of a single master
- <span class="codefrag">JobTracker</span> and one slave <span class="codefrag">TaskTracker</span> per
- cluster-node. The master is responsible for scheduling the jobs' component
- tasks on the slaves, monitoring them and re-executing the failed tasks. The
- slaves execute the tasks as directed by the master.</p>
- <p>Minimally, applications specify the input/output locations and supply
- <em>map</em> and <em>reduce</em> functions via implementations of
- appropriate interfaces and/or abstract-classes. These, and other job
- parameters, comprise the <em>job configuration</em>. The Hadoop
- <em>job client</em> then submits the job (jar/executable etc.) and
- configuration to the <span class="codefrag">JobTracker</span> which then assumes the
- responsibility of distributing the software/configuration to the slaves,
- scheduling tasks and monitoring them, providing status and diagnostic
- information to the job-client.</p>
- <p>Although the Hadoop framework is implemented in Java<sup>TM</sup>,
- Map-Reduce applications need not be written in Java.</p>
- <ul>
-
- <li>
-
- <a href="api/org/apache/hadoop/streaming/package-summary.html">
- Hadoop Streaming</a> is a utility which allows users to create and run
- jobs with any executables (e.g. shell utilities) as the mapper and/or
- the reducer.
- </li>
-
- <li>
-
- <a href="api/org/apache/hadoop/mapred/pipes/package-summary.html">
- Hadoop Pipes</a> is a <a href="http://www.swig.org/">SWIG</a>-
- compatible <em>C++ API</em> to implement Map-Reduce applications (non
- JNI<sup>TM</sup> based).
- </li>
-
- </ul>
- </div>
-
-
- <a name="N1008B"></a><a name="Inputs+and+Outputs"></a>
- <h2 class="h3">Inputs and Outputs</h2>
- <div class="section">
- <p>The Map-Reduce framework operates exclusively on
- <span class="codefrag"><key, value></span> pairs, that is, the framework views the
- input to the job as a set of <span class="codefrag"><key, value></span> pairs and
- produces a set of <span class="codefrag"><key, value></span> pairs as the output of
- the job, conceivably of different types.</p>
- <p>The <span class="codefrag">key</span> and <span class="codefrag">value</span> classes have to be
- serializable by the framework and hence need to implement the
- <a href="api/org/apache/hadoop/io/Writable.html">Writable</a>
- interface. Additionally, the <span class="codefrag">key</span> classes have to implement the
- <a href="api/org/apache/hadoop/io/WritableComparable.html">
- WritableComparable</a> interface to facilitate sorting by the framework.
- </p>
- <p>Input and Output types of a Map-Reduce job:</p>
- <p>
- (input) <span class="codefrag"><k1, v1></span>
- ->
- <strong>map</strong>
- ->
- <span class="codefrag"><k2, v2></span>
- ->
- <strong>combine</strong>
- ->
- <span class="codefrag"><k2, v2></span>
- ->
- <strong>reduce</strong>
- ->
- <span class="codefrag"><k3, v3></span> (output)
- </p>
- </div>
-
- <a name="N100CD"></a><a name="Example%3A+WordCount+v1.0"></a>
- <h2 class="h3">Example: WordCount v1.0</h2>
- <div class="section">
- <p>Before we jump into the details, lets walk through an example Map-Reduce
- application to get a flavour for how they work.</p>
- <p>
- <span class="codefrag">WordCount</span> is a simple application that counts the number of
- occurences of each word in a given input set.</p>
- <p>This works with a
- <a href="quickstart.html#Standalone+Operation">local-standalone</a>,
- <a href="quickstart.html#SingleNodeSetup">pseudo-distributed</a> or
- <a href="quickstart.html#Fully-Distributed+Operation">fully-distributed</a>
- Hadoop installation.</p>
- <a name="N100EA"></a><a name="Source+Code"></a>
- <h3 class="h4">Source Code</h3>
- <table class="ForrestTable" cellspacing="1" cellpadding="4">
-
- <tr>
-
- <th colspan="1" rowspan="1"></th>
- <th colspan="1" rowspan="1">WordCount.java</th>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">1.</td>
- <td colspan="1" rowspan="1">
- <span class="codefrag">package org.myorg;</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">2.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">3.</td>
- <td colspan="1" rowspan="1">
- <span class="codefrag">import java.io.IOException;</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">4.</td>
- <td colspan="1" rowspan="1">
- <span class="codefrag">import java.util.*;</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">5.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">6.</td>
- <td colspan="1" rowspan="1">
- <span class="codefrag">import org.apache.hadoop.fs.Path;</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">7.</td>
- <td colspan="1" rowspan="1">
- <span class="codefrag">import org.apache.hadoop.conf.*;</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">8.</td>
- <td colspan="1" rowspan="1">
- <span class="codefrag">import org.apache.hadoop.io.*;</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">9.</td>
- <td colspan="1" rowspan="1">
- <span class="codefrag">import org.apache.hadoop.mapred.*;</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">10.</td>
- <td colspan="1" rowspan="1">
- <span class="codefrag">import org.apache.hadoop.util.*;</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">11.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">12.</td>
- <td colspan="1" rowspan="1">
- <span class="codefrag">public class WordCount {</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">13.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">14.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- public static class Map extends MapReduceBase
- implements Mapper<LongWritable, Text, Text, IntWritable> {
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">15.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- private final static IntWritable one = new IntWritable(1);
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">16.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">private Text word = new Text();</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">17.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">18.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- public void map(LongWritable key, Text value,
- OutputCollector<Text, IntWritable> output,
- Reporter reporter) throws IOException {
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">19.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">String line = value.toString();</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">20.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">StringTokenizer tokenizer = new StringTokenizer(line);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">21.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">while (tokenizer.hasMoreTokens()) {</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">22.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">word.set(tokenizer.nextToken());</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">23.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">output.collect(word, one);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">24.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">25.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">26.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">27.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">28.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- public static class Reduce extends MapReduceBase implements
- Reducer<Text, IntWritable, Text, IntWritable> {
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">29.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- public void reduce(Text key, Iterator<IntWritable> values,
- OutputCollector<Text, IntWritable> output,
- Reporter reporter) throws IOException {
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">30.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">int sum = 0;</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">31.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">while (values.hasNext()) {</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">32.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">sum += values.next().get();</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">33.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">34.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">output.collect(key, new IntWritable(sum));</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">35.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">36.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">37.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">38.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- public static void main(String[] args) throws Exception {
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">39.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- JobConf conf = new JobConf(WordCount.class);
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">40.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">conf.setJobName("wordcount");</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">41.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">42.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">conf.setOutputKeyClass(Text.class);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">43.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">conf.setOutputValueClass(IntWritable.class);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">44.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">45.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">conf.setMapperClass(Map.class);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">46.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">conf.setCombinerClass(Reduce.class);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">47.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">conf.setReducerClass(Reduce.class);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">48.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">49.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">conf.setInputFormat(TextInputFormat.class);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">50.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">conf.setOutputFormat(TextOutputFormat.class);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">51.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">52.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">conf.setInputPath(new Path(args[0]));</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">53.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">conf.setOutputPath(new Path(args[1]));</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">54.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">55.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">JobClient.runJob(conf);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">57.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">58.</td>
- <td colspan="1" rowspan="1">
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">59.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- </table>
- <a name="N1046C"></a><a name="Usage"></a>
- <h3 class="h4">Usage</h3>
- <p>Assuming <span class="codefrag">HADOOP_HOME</span> is the root of the installation and
- <span class="codefrag">HADOOP_VERSION</span> is the Hadoop version installed, compile
- <span class="codefrag">WordCount.java</span> and create a jar:</p>
- <p>
-
- <span class="codefrag">$ mkdir wordcount_classes</span>
- <br>
-
- <span class="codefrag">
- $ javac -classpath ${HADOOP_HOME}/hadoop-${HADOOP_VERSION}-core.jar
- -d wordcount_classes WordCount.java
- </span>
- <br>
-
- <span class="codefrag">$ jar -cvf /usr/joe/wordcount.jar -C wordcount_classes/ .</span>
-
- </p>
- <p>Assuming that:</p>
- <ul>
-
- <li>
-
- <span class="codefrag">/usr/joe/wordcount/input</span> - input directory in HDFS
- </li>
-
- <li>
-
- <span class="codefrag">/usr/joe/wordcount/output</span> - output directory in HDFS
- </li>
-
- </ul>
- <p>Sample text-files as input:</p>
- <p>
-
- <span class="codefrag">$ bin/hadoop dfs -ls /usr/joe/wordcount/input/</span>
- <br>
-
- <span class="codefrag">/usr/joe/wordcount/input/file01</span>
- <br>
-
- <span class="codefrag">/usr/joe/wordcount/input/file02</span>
- <br>
-
- <br>
-
- <span class="codefrag">$ bin/hadoop dfs -cat /usr/joe/wordcount/input/file01</span>
- <br>
-
- <span class="codefrag">Hello World Bye World</span>
- <br>
-
- <br>
-
- <span class="codefrag">$ bin/hadoop dfs -cat /usr/joe/wordcount/input/file02</span>
- <br>
-
- <span class="codefrag">Hello Hadoop Goodbye Hadoop</span>
-
- </p>
- <p>Run the application:</p>
- <p>
-
- <span class="codefrag">
- $ bin/hadoop jar /usr/joe/wordcount.jar org.myorg.WordCount
- /usr/joe/wordcount/input /usr/joe/wordcount/output
- </span>
-
- </p>
- <p>Output:</p>
- <p>
-
- <span class="codefrag">
- $ bin/hadoop dfs -cat /usr/joe/wordcount/output/part-00000
- </span>
-
- <br>
-
- <span class="codefrag">Bye 1</span>
- <br>
-
- <span class="codefrag">Goodbye 1</span>
- <br>
-
- <span class="codefrag">Hadoop 2</span>
- <br>
-
- <span class="codefrag">Hello 2</span>
- <br>
-
- <span class="codefrag">World 2</span>
- <br>
-
- </p>
- <a name="N104EC"></a><a name="Walk-through"></a>
- <h3 class="h4">Walk-through</h3>
- <p>The <span class="codefrag">WordCount</span> application is quite straight-forward.</p>
- <p>The <span class="codefrag">Mapper</span> implementation (lines 14-26), via the
- <span class="codefrag">map</span> method (lines 18-25), processes one line at a time,
- as provided by the specified <span class="codefrag">TextInputFormat</span> (line 49).
- It then splits the line into tokens separated by whitespaces, via the
- <span class="codefrag">StringTokenizer</span>, and emits a key-value pair of
- <span class="codefrag">< <word>, 1></span>.</p>
- <p>
- For the given sample input the first map emits:<br>
-
- <span class="codefrag">< Hello, 1></span>
- <br>
-
- <span class="codefrag">< World, 1></span>
- <br>
-
- <span class="codefrag">< Bye, 1></span>
- <br>
-
- <span class="codefrag">< World, 1></span>
- <br>
-
- </p>
- <p>
- The second map emits:<br>
-
- <span class="codefrag">< Hello, 1></span>
- <br>
-
- <span class="codefrag">< Hadoop, 1></span>
- <br>
-
- <span class="codefrag">< Goodbye, 1></span>
- <br>
-
- <span class="codefrag">< Hadoop, 1></span>
- <br>
-
- </p>
- <p>We'll learn more about the number of maps spawned for a given job, and
- how to control them in a fine-grained manner, a bit later in the
- tutorial.</p>
- <p>
- <span class="codefrag">WordCount</span> also specifies a <span class="codefrag">combiner</span> (line
- 46). Hence, the output of each map is passed through the local combiner
- (which is same as the <span class="codefrag">Reducer</span> as per the job
- configuration) for local aggregation, after being sorted on the
- <em>key</em>s.</p>
- <p>
- The output of the first map:<br>
-
- <span class="codefrag">< Bye, 1></span>
- <br>
-
- <span class="codefrag">< Hello, 1></span>
- <br>
-
- <span class="codefrag">< World, 2></span>
- <br>
-
- </p>
- <p>
- The output of the second map:<br>
-
- <span class="codefrag">< Goodbye, 1></span>
- <br>
-
- <span class="codefrag">< Hadoop, 2></span>
- <br>
-
- <span class="codefrag">< Hello, 1></span>
- <br>
-
- </p>
- <p>The <span class="codefrag">Reducer</span> implementation (lines 28-36), via the
- <span class="codefrag">reduce</span> method (lines 29-35) just sums up the values,
- which are the occurence counts for each key (i.e. words in this example).
- </p>
- <p>
- Thus the output of the job is:<br>
-
- <span class="codefrag">< Bye, 1></span>
- <br>
-
- <span class="codefrag">< Goodbye, 1></span>
- <br>
-
- <span class="codefrag">< Hadoop, 2></span>
- <br>
-
- <span class="codefrag">< Hello, 2></span>
- <br>
-
- <span class="codefrag">< World, 2></span>
- <br>
-
- </p>
- <p>The <span class="codefrag">run</span> method specifies various facets of the job, such
- as the input/output paths (passed via the command line), key/value
- types, input/output formats etc., in the <span class="codefrag">JobConf</span>.
- It then calls the <span class="codefrag">JobClient.runJob</span> (line 55) to submit the
- and monitor its progress.</p>
- <p>We'll learn more about <span class="codefrag">JobConf</span>, <span class="codefrag">JobClient</span>,
- <span class="codefrag">Tool</span> and other interfaces and classes a bit later in the
- tutorial.</p>
- </div>
-
-
- <a name="N105A3"></a><a name="Map-Reduce+-+User+Interfaces"></a>
- <h2 class="h3">Map-Reduce - User Interfaces</h2>
- <div class="section">
- <p>This section provides a reasonable amount of detail on every user-facing
- aspect of the Map-Reduce framwork. This should help users implement,
- configure and tune their jobs in a fine-grained manner. However, please
- note that the javadoc for each class/interface remains the most
- comprehensive documentation available; this is only meant to be a tutorial.
- </p>
- <p>Let us first take the <span class="codefrag">Mapper</span> and <span class="codefrag">Reducer</span>
- interfaces. Applications typically implement them to provide the
- <span class="codefrag">map</span> and <span class="codefrag">reduce</span> methods.</p>
- <p>We will then discuss other core interfaces including
- <span class="codefrag">JobConf</span>, <span class="codefrag">JobClient</span>, <span class="codefrag">Partitioner</span>,
- <span class="codefrag">OutputCollector</span>, <span class="codefrag">Reporter</span>,
- <span class="codefrag">InputFormat</span>, <span class="codefrag">OutputFormat</span> and others.</p>
- <p>Finally, we will wrap up by discussing some useful features of the
- framework such as the <span class="codefrag">DistributedCache</span>,
- <span class="codefrag">IsolationRunner</span> etc.</p>
- <a name="N105DC"></a><a name="Payload"></a>
- <h3 class="h4">Payload</h3>
- <p>Applications typically implement the <span class="codefrag">Mapper</span> and
- <span class="codefrag">Reducer</span> interfaces to provide the <span class="codefrag">map</span> and
- <span class="codefrag">reduce</span> methods. These form the core of the job.</p>
- <a name="N105F1"></a><a name="Mapper"></a>
- <h4>Mapper</h4>
- <p>
- <a href="api/org/apache/hadoop/mapred/Mapper.html">
- Mapper</a> maps input key/value pairs to a set of intermediate
- key/value pairs.</p>
- <p>Maps are the individual tasks that transform input records into
- intermediate records. The transformed intermediate records do not need
- to be of the same type as the input records. A given input pair may
- map to zero or many output pairs.</p>
- <p>The Hadoop Map-Reduce framework spawns one map task for each
- <span class="codefrag">InputSplit</span> generated by the <span class="codefrag">InputFormat</span> for
- the job.</p>
- <p>Overall, <span class="codefrag">Mapper</span> implementations are passed the
- <span class="codefrag">JobConf</span> for the job via the
- <a href="api/org/apache/hadoop/mapred/JobConfigurable.html#configure(org.apache.hadoop.mapred.JobConf)">
- JobConfigurable.configure(JobConf)</a> method and override it to
- initialize themselves. The framework then calls
- <a href="api/org/apache/hadoop/mapred/Mapper.html#map(K1, V1, org.apache.hadoop.mapred.OutputCollector, org.apache.hadoop.mapred.Reporter)">
- map(WritableComparable, Writable, OutputCollector, Reporter)</a> for
- each key/value pair in the <span class="codefrag">InputSplit</span> for that task.
- Applications can then override the
- <a href="api/org/apache/hadoop/io/Closeable.html#close()">
- Closeable.close()</a> method to perform any required cleanup.</p>
- <p>Output pairs do not need to be of the same types as input pairs. A
- given input pair may map to zero or many output pairs. Output pairs
- are collected with calls to
- <a href="api/org/apache/hadoop/mapred/OutputCollector.html#collect(K, V)">
- OutputCollector.collect(WritableComparable,Writable)</a>.</p>
- <p>Applications can use the <span class="codefrag">Reporter</span> to report
- progress, set application-level status messages and update
- <span class="codefrag">Counters</span>, or just indicate that they are alive.</p>
- <p>All intermediate values associated with a given output key are
- subsequently grouped by the framework, and passed to the
- <span class="codefrag">Reducer</span>(s) to determine the final output. Users can
- control the grouping by specifying a <span class="codefrag">Comparator</span> via
- <a href="api/org/apache/hadoop/mapred/JobConf.html#setOutputKeyComparatorClass(java.lang.Class)">
- JobConf.setOutputKeyComparatorClass(Class)</a>.</p>
- <p>The <span class="codefrag">Mapper</span> outputs are sorted and then
- partitioned per <span class="codefrag">Reducer</span>. The total number of partitions is
- the same as the number of reduce tasks for the job. Users can control
- which keys (and hence records) go to which <span class="codefrag">Reducer</span> by
- implementing a custom <span class="codefrag">Partitioner</span>.</p>
- <p>Users can optionally specify a <span class="codefrag">combiner</span>, via
- <a href="api/org/apache/hadoop/mapred/JobConf.html#setCombinerClass(java.lang.Class)">
- JobConf.setCombinerClass(Class)</a>, to perform local aggregation of
- the intermediate outputs, which helps to cut down the amount of data
- transferred from the <span class="codefrag">Mapper</span> to the <span class="codefrag">Reducer</span>.
- </p>
- <p>The intermediate, sorted outputs are always stored in files of
- <a href="api/org/apache/hadoop/io/SequenceFile.html">
- SequenceFile</a> format. Applications can control if, and how, the
- intermediate outputs are to be compressed and the
- <a href="api/org/apache/hadoop/io/compress/CompressionCodec.html">
- CompressionCodec</a> to be used via the <span class="codefrag">JobConf</span>.
- </p>
- <a name="N1066B"></a><a name="How+Many+Maps%3F"></a>
- <h5>How Many Maps?</h5>
- <p>The number of maps is usually driven by the total size of the
- inputs, that is, the total number of blocks of the input files.</p>
- <p>The right level of parallelism for maps seems to be around 10-100
- maps per-node, although it has been set up to 300 maps for very
- cpu-light map tasks. Task setup takes awhile, so it is best if the
- maps take at least a minute to execute.</p>
- <p>Thus, if you expect 10TB of input data and have a blocksize of
- <span class="codefrag">128MB</span>, you'll end up with 82,000 maps, unless
- <a href="api/org/apache/hadoop/mapred/JobConf.html#setNumMapTasks(int)">
- setNumMapTasks(int)</a> (which only provides a hint to the framework)
- is used to set it even higher.</p>
- <a name="N10683"></a><a name="Reducer"></a>
- <h4>Reducer</h4>
- <p>
- <a href="api/org/apache/hadoop/mapred/Reducer.html">
- Reducer</a> reduces a set of intermediate values which share a key to
- a smaller set of values.</p>
- <p>The number of reduces for the job is set by the user
- via <a href="api/org/apache/hadoop/mapred/JobConf.html#setNumReduceTasks(int)">
- JobConf.setNumReduceTasks(int)</a>.</p>
- <p>Overall, <span class="codefrag">Reducer</span> implementations are passed the
- <span class="codefrag">JobConf</span> for the job via the
- <a href="api/org/apache/hadoop/mapred/JobConfigurable.html#configure(org.apache.hadoop.mapred.JobConf)">
- JobConfigurable.configure(JobConf)</a> method and can override it to
- initialize themselves. The framework then calls
- <a href="api/org/apache/hadoop/mapred/Reducer.html#reduce(K2, java.util.Iterator, org.apache.hadoop.mapred.OutputCollector, org.apache.hadoop.mapred.Reporter)">
- reduce(WritableComparable, Iterator, OutputCollector, Reporter)</a>
- method for each <span class="codefrag"><key, (list of values)></span>
- pair in the grouped inputs. Applications can then override the
- <a href="api/org/apache/hadoop/io/Closeable.html#close()">
- Closeable.close()</a> method to perform any required cleanup.</p>
- <p>
- <span class="codefrag">Reducer</span> has 3 primary phases: shuffle, sort and reduce.
- </p>
- <a name="N106B3"></a><a name="Shuffle"></a>
- <h5>Shuffle</h5>
- <p>Input to the <span class="codefrag">Reducer</span> is the sorted output of the
- mappers. In this phase the framework fetches the relevant partition
- of the output of all the mappers, via HTTP.</p>
- <a name="N106C0"></a><a name="Sort"></a>
- <h5>Sort</h5>
- <p>The framework groups <span class="codefrag">Reducer</span> inputs by keys (since
- different mappers may have output the same key) in this stage.</p>
- <p>The shuffle and sort phases occur simultaneously; while
- map-outputs are being fetched they are merged.</p>
- <a name="N106CF"></a><a name="Secondary+Sort"></a>
- <h5>Secondary Sort</h5>
- <p>If equivalence rules for grouping the intermediate keys are
- required to be different from those for grouping keys before
- reduction, then one may specify a <span class="codefrag">Comparator</span> via
- <a href="api/org/apache/hadoop/mapred/JobConf.html#setOutputValueGroupingComparator(java.lang.Class)">
- JobConf.setOutputValueGroupingComparator(Class)</a>. Since
- <a href="api/org/apache/hadoop/mapred/JobConf.html#setOutputKeyComparatorClass(java.lang.Class)">
- JobConf.setOutputKeyComparatorClass(Class)</a> can be used to
- control how intermediate keys are grouped, these can be used in
- conjunction to simulate <em>secondary sort on values</em>.</p>
- <a name="N106E8"></a><a name="Reduce"></a>
- <h5>Reduce</h5>
- <p>In this phase the
- <a href="api/org/apache/hadoop/mapred/Reducer.html#reduce(K2, java.util.Iterator, org.apache.hadoop.mapred.OutputCollector, org.apache.hadoop.mapred.Reporter)">
- reduce(WritableComparable, Iterator, OutputCollector, Reporter)</a>
- method is called for each <span class="codefrag"><key, (list of values)></span>
- pair in the grouped inputs.</p>
- <p>The output of the reduce task is typically written to the
- <a href="api/org/apache/hadoop/fs/FileSystem.html">
- FileSystem</a> via
- <a href="api/org/apache/hadoop/mapred/OutputCollector.html#collect(K, V)">
- OutputCollector.collect(WritableComparable, Writable)</a>.</p>
- <p>Applications can use the <span class="codefrag">Reporter</span> to report
- progress, set application-level status messages and update
- <span class="codefrag">Counters</span>, or just indicate that they are alive.</p>
- <p>The output of the <span class="codefrag">Reducer</span> is <em>not sorted</em>.</p>
- <a name="N10716"></a><a name="How+Many+Reduces%3F"></a>
- <h5>How Many Reduces?</h5>
- <p>The right number of reduces seems to be <span class="codefrag">0.95</span> or
- <span class="codefrag">1.75</span> multiplied by (<<em>no. of nodes</em>> *
- <span class="codefrag">mapred.tasktracker.reduce.tasks.maximum</span>).</p>
- <p>With <span class="codefrag">0.95</span> all of the reduces can launch immediately
- and start transfering map outputs as the maps finish. With
- <span class="codefrag">1.75</span> the faster nodes will finish their first round of
- reduces and launch a second wave of reduces doing a much better job
- of load balancing.</p>
- <p>Increasing the number of reduces increases the framework overhead,
- but increases load balancing and lowers the cost of failures.</p>
- <p>The scaling factors above are slightly less than whole numbers to
- reserve a few reduce slots in the framework for speculative-tasks and
- failed tasks.</p>
- <a name="N1073B"></a><a name="Reducer+NONE"></a>
- <h5>Reducer NONE</h5>
- <p>It is legal to set the number of reduce-tasks to <em>zero</em> if
- no reduction is desired.</p>
- <p>In this case the outputs of the map-tasks go directly to the
- <span class="codefrag">FileSystem</span>, into the output path set by
- <a href="api/org/apache/hadoop/mapred/JobConf.html#setOutputPath(org.apache.hadoop.fs.Path)">
- setOutputPath(Path)</a>. The framework does not sort the
- map-outputs before writing them out to the <span class="codefrag">FileSystem</span>.
- </p>
- <a name="N10756"></a><a name="Partitioner"></a>
- <h4>Partitioner</h4>
- <p>
- <a href="api/org/apache/hadoop/mapred/Partitioner.html">
- Partitioner</a> partitions the key space.</p>
- <p>Partitioner controls the partitioning of the keys of the
- intermediate map-outputs. The key (or a subset of the key) is used to
- derive the partition, typically by a <em>hash function</em>. The total
- number of partitions is the same as the number of reduce tasks for the
- job. Hence this controls which of the <span class="codefrag">m</span> reduce tasks the
- intermediate key (and hence the record) is sent to for reduction.</p>
- <p>
- <a href="api/org/apache/hadoop/mapred/lib/HashPartitioner.html">
- HashPartitioner</a> is the default <span class="codefrag">Partitioner</span>.</p>
- <a name="N10775"></a><a name="Reporter"></a>
- <h4>Reporter</h4>
- <p>
- <a href="api/org/apache/hadoop/mapred/Reporter.html">
- Reporter</a> is a facility for Map-Reduce applications to report
- progress, set application-level status messages and update
- <span class="codefrag">Counters</span>.</p>
- <p>
- <span class="codefrag">Mapper</span> and <span class="codefrag">Reducer</span> implementations can use
- the <span class="codefrag">Reporter</span> to report progress or just indicate
- that they are alive. In scenarios where the application takes a
- significant amount of time to process individual key/value pairs,
- this is crucial since the framework might assume that the task has
- timed-out and kill that task. Another way to avoid this is to
- set the configuration parameter <span class="codefrag">mapred.task.timeout</span> to a
- high-enough value (or even set it to <em>zero</em> for no time-outs).
- </p>
- <p>Applications can also update <span class="codefrag">Counters</span> using the
- <span class="codefrag">Reporter</span>.</p>
- <a name="N1079F"></a><a name="OutputCollector"></a>
- <h4>OutputCollector</h4>
- <p>
- <a href="api/org/apache/hadoop/mapred/OutputCollector.html">
- OutputCollector</a> is a generalization of the facility provided by
- the Map-Reduce framework to collect data output by the
- <span class="codefrag">Mapper</span> or the <span class="codefrag">Reducer</span> (either the
- intermediate outputs or the output of the job).</p>
- <p>Hadoop Map-Reduce comes bundled with a
- <a href="api/org/apache/hadoop/mapred/lib/package-summary.html">
- library</a> of generally useful mappers, reducers, and partitioners.</p>
- <a name="N107BA"></a><a name="Job+Configuration"></a>
- <h3 class="h4">Job Configuration</h3>
- <p>
- <a href="api/org/apache/hadoop/mapred/JobConf.html">
- JobConf</a> represents a Map-Reduce job configuration.</p>
- <p>
- <span class="codefrag">JobConf</span> is the primary interface for a user to describe
- a map-reduce job to the Hadoop framework for execution. The framework
- tries to faithfully execute the job as described by <span class="codefrag">JobConf</span>,
- however:</p>
- <ul>
-
- <li>f
- Some configuration parameters may have been marked as
- <a href="api/org/apache/hadoop/conf/Configuration.html#FinalParams">
- final</a> by administrators and hence cannot be altered.
- </li>
-
- <li>
- While some job parameters are straight-forward to set (e.g.
- <a href="api/org/apache/hadoop/mapred/JobConf.html#setNumReduceTasks(int)">
- setNumReduceTasks(int)</a>), other parameters interact subtly with
- the rest of the framework and/or job configuration and are
- more complex to set (e.g.
- <a href="api/org/apache/hadoop/mapred/JobConf.html#setNumMapTasks(int)">
- setNumMapTasks(int)</a>).
- </li>
-
- </ul>
- <p>
- <span class="codefrag">JobConf</span> is typically used to specify the
- <span class="codefrag">Mapper</span>, combiner (if any), <span class="codefrag">Partitioner</span>,
- <span class="codefrag">Reducer</span>, <span class="codefrag">InputFormat</span> and
- <span class="codefrag">OutputFormat</span> implementations. <span class="codefrag">JobConf</span> also
- indicates the set of input files
- (<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>)
- and where the output files should be written
- (<a href="api/org/apache/hadoop/mapred/JobConf.html#setOutputPath(org.apache.hadoop.fs.Path)">setOutputPath(Path)</a>).</p>
- <p>Optionally, <span class="codefrag">JobConf</span> is used to specify other advanced
- facets of the job such as the <span class="codefrag">Comparator</span> to be used, files
- to be put in the <span class="codefrag">DistributedCache</span>, whether intermediate
- and/or job outputs are to be compressed (and how), debugging via
- user-provided scripts
- (<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>)
- , whether job tasks can be executed in a <em>speculative</em> manner
- (<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>)
- , maximum number of attempts per task
- (<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>)
- , percentage of tasks failure which can be tolerated by the job
- (<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>)
- etc.</p>
- <p>Of course, users can use
- <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>
- to set/get arbitrary parameters needed by applications. However, use the
- <span class="codefrag">DistributedCache</span> for large amounts of (read-only) data.</p>
- <a name="N10844"></a><a name="Task+Execution+%26+Environment"></a>
- <h3 class="h4">Task Execution & Environment</h3>
- <p>The <span class="codefrag">TaskTracker</span> executes the <span class="codefrag">Mapper</span>/
- <span class="codefrag">Reducer</span> <em>task</em> as a child process in a separate jvm.
- </p>
- <p>The child-task inherits the environment of the parent
- <span class="codefrag">TaskTracker</span>. The user can specify additional options to the
- child-jvm via the <span class="codefrag">mapred.child.java.opts</span> configuration
- parameter in the <span class="codefrag">JobConf</span> such as non-standard paths for the
- run-time linker to search shared libraries via
- <span class="codefrag">-Djava.library.path=<></span> etc. If the
- <span class="codefrag">mapred.child.java.opts</span> contains the symbol <em>@taskid@</em>
- it is interpolated with value of <span class="codefrag">taskid</span> of the map/reduce
- task.</p>
- <p>Here is an example with multiple arguments and substitutions,
- showing jvm GC logging, and start of a passwordless JVM JMX agent so that
- it can connect with jconsole and the likes to watch child memory,
- threads and get thread dumps. It also sets the maximum heap-size of the
- child jvm to 512MB and adds an additional path to the
- <span class="codefrag">java.library.path</span> of the child-jvm.</p>
- <p>
-
- <span class="codefrag"><property></span>
- <br>
- <span class="codefrag"><name>mapred.child.java.opts</name></span>
- <br>
- <span class="codefrag"><value></span>
- <br>
- <span class="codefrag">
- -Xmx512M -Djava.library.path=/home/mycompany/lib
- -verbose:gc -Xloggc:/tmp/@taskid@.gc</span>
- <br>
- <span class="codefrag">
- -Dcom.sun.management.jmxremote.authenticate=false
- -Dcom.sun.management.jmxremote.ssl=false</span>
- <br>
- <span class="codefrag"></value></span>
- <br>
-
- <span class="codefrag"></property></span>
-
- </p>
- <p>The <a href="#DistributedCache">DistributedCache</a> can also be used
- as a rudimentary software distribution mechanism for use in the map
- and/or reduce tasks. It can be used to distribute both jars and
- native libraries. The
- <a href="api/org/apache/hadoop/filecache/DistributedCache.html#addArchiveToClassPath(org.apache.hadoop.fs.Path,%20org.apache.hadoop.conf.Configuration)">
- DistributedCache.addArchiveToClassPath(Path, Configuration)</a> or
- <a href="api/org/apache/hadoop/filecache/DistributedCache.html#addFileToClassPath(org.apache.hadoop.fs.Path,%20org.apache.hadoop.conf.Configuration)">
- DistributedCache.addFileToClassPath(Path, Configuration)</a> api can
- be used to cache files/jars and also add them to the <em>classpath</em>
- of child-jvm. Similarly the facility provided by the
- <span class="codefrag">DistributedCache</span> where-in it symlinks the cached files into
- the working directory of the task can be used to distribute native
- libraries and load them. The underlying detail is that child-jvm always
- has its <em>current working directory</em> added to the
- <span class="codefrag">java.library.path</span> and hence the cached libraries can be
- loaded via <a href="http://java.sun.com/j2se/1.5.0/docs/api/java/lang/System.html#loadLibrary(java.lang.String)">
- System.loadLibrary</a> or <a href="http://java.sun.com/j2se/1.5.0/docs/api/java/lang/System.html#load(java.lang.String)">
- System.load</a>.</p>
- <a name="N108B9"></a><a name="Job+Submission+and+Monitoring"></a>
- <h3 class="h4">Job Submission and Monitoring</h3>
- <p>
- <a href="api/org/apache/hadoop/mapred/JobClient.html">
- JobClient</a> is the primary interface by which user-job interacts
- with the <span class="codefrag">JobTracker</span>.</p>
- <p>
- <span class="codefrag">JobClient</span> provides facilities to submit jobs, track their
- progress, access component-tasks' reports/logs, get the Map-Reduce
- cluster's status information and so on.</p>
- <p>The job submission process involves:</p>
- <ol>
-
- <li>Checking the input and output specifications of the job.</li>
-
- <li>Computing the <span class="codefrag">InputSplit</span> values for the job.</li>
-
- <li>
- Setting up the requisite accounting information for the
- <span class="codefrag">DistributedCache</span> of the job, if necessary.
- </li>
-
- <li>
- Copying the job's jar and configuration to the map-reduce system
- directory on the <span class="codefrag">FileSystem</span>.
- </li>
-
- <li>
- Submitting the job to the <span class="codefrag">JobTracker</span> and optionally
- monitoring it's status.
- </li>
-
- </ol>
- <p>Normally the user creates the application, describes various facets
- of the job via <span class="codefrag">JobConf</span>, and then uses the
- <span class="codefrag">JobClient</span> to submit the job and monitor its progress.</p>
- <a name="N108F7"></a><a name="Job+Control"></a>
- <h4>Job Control</h4>
- <p>Users may need to chain map-reduce jobs to accomplish complex
- tasks which cannot be done via a single map-reduce job. This is fairly
- easy since the output of the job typically goes to distributed
- file-system, and the output, in turn, can be used as the input for the
- next job.</p>
- <p>However, this also means that the onus on ensuring jobs are
- complete (success/failure) lies squarely on the clients. In such
- cases, the various job-control options are:</p>
- <ul>
-
- <li>
-
- <a href="api/org/apache/hadoop/mapred/JobClient.html#runJob(org.apache.hadoop.mapred.JobConf)">
- runJob(JobConf)</a> : Submits the job and returns only after the
- job has completed.
- </li>
-
- <li>
-
- <a href="api/org/apache/hadoop/mapred/JobClient.html#submitJob(org.apache.hadoop.mapred.JobConf)">
- submitJob(JobConf)</a> : Only submits the job, then poll the
- returned handle to the
- <a href="api/org/apache/hadoop/mapred/RunningJob.html">
- RunningJob</a> to query status and make scheduling decisions.
- </li>
-
- <li>
-
- <a href="api/org/apache/hadoop/mapred/JobConf.html#setJobEndNotificationURI(java.lang.String)">
- JobConf.setJobEndNotificationURI(String)</a> : Sets up a
- notification upon job-completion, thus avoiding polling.
- </li>
-
- </ul>
- <a name="N10921"></a><a name="Job+Input"></a>
- <h3 class="h4">Job Input</h3>
- <p>
- <a href="api/org/apache/hadoop/mapred/InputFormat.html">
- InputFormat</a> describes the input-specification for a Map-Reduce job.
- </p>
- <p>The Map-Reduce framework relies on the <span class="codefrag">InputFormat</span> of
- the job to:</p>
- <ol>
-
- <li>Validate the input-specification of the job.</li>
-
- <li>
- Split-up the input file(s) into logical <span class="codefrag">InputSplit</span>
- instances, each of which is then assigned to an individual
- <span class="codefrag">Mapper</span>.
- </li>
-
- <li>
- Provide the <span class="codefrag">RecordReader</span> implementation used to
- glean input records from the logical <span class="codefrag">InputSplit</span> for
- processing by the <span class="codefrag">Mapper</span>.
- </li>
-
- </ol>
- <p>The default behavior of file-based <span class="codefrag">InputFormat</span>
- implementations, typically sub-classes of
- <a href="api/org/apache/hadoop/mapred/FileInputFormat.html">
- FileInputFormat</a>, is to split the input into <em>logical</em>
- <span class="codefrag">InputSplit</span> instances based on the total size, in bytes, of
- the input files. However, the <span class="codefrag">FileSystem</span> blocksize of the
- input files is treated as an upper bound for input splits. A lower bound
- on the split size can be set via <span class="codefrag">mapred.min.split.size</span>.</p>
- <p>Clearly, logical splits based on input-size is insufficient for many
- applications since record boundaries must be respected. In such cases,
- the application should implement a <span class="codefrag">RecordReader</span>, who is
- responsible for respecting record-boundaries and presents a
- record-oriented view of the logical <span class="codefrag">InputSplit</span> to the
- individual task.</p>
- <p>
- <a href="api/org/apache/hadoop/mapred/TextInputFormat.html">
- TextInputFormat</a> is the default <span class="codefrag">InputFormat</span>.</p>
- <p>If <span class="codefrag">TextInputFormat</span> is the <span class="codefrag">InputFormat</span> for a
- given job, the framework detects input-files with the <em>.gz</em> and
- <em>.lzo</em> extensions and automatically decompresses them using the
- appropriate <span class="codefrag">CompressionCodec</span>. However, it must be noted that
- compressed files with the above extensions cannot be <em>split</em> and
- each compressed file is processed in its entirety by a single mapper.</p>
- <a name="N1098B"></a><a name="InputSplit"></a>
- <h4>InputSplit</h4>
- <p>
- <a href="api/org/apache/hadoop/mapred/InputSplit.html">
- InputSplit</a> represents the data to be processed by an individual
- <span class="codefrag">Mapper</span>.</p>
- <p>Typically <span class="codefrag">InputSplit</span> presents a byte-oriented view of
- the input, and it is the responsibility of <span class="codefrag">RecordReader</span>
- to process and present a record-oriented view.</p>
- <p>
- <a href="api/org/apache/hadoop/mapred/FileSplit.html">
- FileSplit</a> is the default <span class="codefrag">InputSplit</span>. It sets
- <span class="codefrag">map.input.file</span> to the path of the input file for the
- logical split.</p>
- <a name="N109B0"></a><a name="RecordReader"></a>
- <h4>RecordReader</h4>
- <p>
- <a href="api/org/apache/hadoop/mapred/RecordReader.html">
- RecordReader</a> reads <span class="codefrag"><key, value></span> pairs from an
- <span class="codefrag">InputSplit</span>.</p>
- <p>Typically the <span class="codefrag">RecordReader</span> converts the byte-oriented
- view of the input, provided by the <span class="codefrag">InputSplit</span>, and
- presents a record-oriented to the <span class="codefrag">Mapper</span> implementations
- for processing. <span class="codefrag">RecordReader</span> thus assumes the
- responsibility of processing record boundaries and presents the tasks
- with keys and values.</p>
- <a name="N109D3"></a><a name="Job+Output"></a>
- <h3 class="h4">Job Output</h3>
- <p>
- <a href="api/org/apache/hadoop/mapred/OutputFormat.html">
- OutputFormat</a> describes the output-specification for a Map-Reduce
- job.</p>
- <p>The Map-Reduce framework relies on the <span class="codefrag">OutputFormat</span> of
- the job to:</p>
- <ol>
-
- <li>
- Validate the output-specification of the job; for example, check that
- the output directory doesn't already exist.
- </li>
-
- <li>
- Provide the <span class="codefrag">RecordWriter</span> implementation used to
- write the output files of the job. Output files are stored in a
- <span class="codefrag">FileSystem</span>.
- </li>
-
- </ol>
- <p>
- <span class="codefrag">TextOutputFormat</span> is the default
- <span class="codefrag">OutputFormat</span>.</p>
- <a name="N109FC"></a><a name="Task+Side-Effect+Files"></a>
- <h4>Task Side-Effect Files</h4>
- <p>In some applications, component tasks need to create and/or write to
- side-files, which differ from the actual job-output files.</p>
- <p>In such cases there could be issues with two instances of the same
- <span class="codefrag">Mapper</span> or <span class="codefrag">Reducer</span> running simultaneously (for
- example, speculative tasks) trying to open and/or write to the same
- file (path) on the <span class="codefrag">FileSystem</span>. Hence the
- application-writer will have to pick unique names per task-attempt
- (using the taskid, say <span class="codefrag">task_200709221812_0001_m_000000_0</span>),
- not just per task.</p>
- <p>To avoid these issues the Map-Reduce framework maintains a special
- <span class="codefrag">${mapred.output.dir}/_temporary/_${taskid}</span> sub-directory
- for each task-attempt on the <span class="codefrag">FileSystem</span> where the output
- of the task-attempt is stored. On successful completion of the
- task-attempt, the files in the
- <span class="codefrag">${mapred.output.dir}/_temporary/_${taskid}</span> (only)
- are <em>promoted</em> to <span class="codefrag">${mapred.output.dir}</span>. Of course,
- the framework discards the sub-directory of unsuccessful task-attempts.
- This process is completely transparent to the application.</p>
- <p>The application-writer can take advantage of this feature by
- creating any side-files required in <span class="codefrag">${mapred.output.dir}</span>
- during execution of a task via
- <a href="api/org/apache/hadoop/mapred/JobConf.html#getOutputPath()">
- JobConf.getOutputPath()</a>, and the framework will promote them
- similarly for succesful task-attempts, thus eliminating the need to
- pick unique paths per task-attempt.</p>
- <a name="N10A31"></a><a name="RecordWriter"></a>
- <h4>RecordWriter</h4>
- <p>
- <a href="api/org/apache/hadoop/mapred/RecordWriter.html">
- RecordWriter</a> writes the output <span class="codefrag"><key, value></span>
- pairs to an output file.</p>
- <p>RecordWriter implementations write the job outputs to the
- <span class="codefrag">FileSystem</span>.</p>
- <a name="N10A48"></a><a name="Other+Useful+Features"></a>
- <h3 class="h4">Other Useful Features</h3>
- <a name="N10A4E"></a><a name="Counters"></a>
- <h4>Counters</h4>
- <p>
- <span class="codefrag">Counters</span> represent global counters, defined either by
- the Map-Reduce framework or applications. Each <span class="codefrag">Counter</span> can
- be of any <span class="codefrag">Enum</span> type. Counters of a particular
- <span class="codefrag">Enum</span> are bunched into groups of type
- <span class="codefrag">Counters.Group</span>.</p>
- <p>Applications can define arbitrary <span class="codefrag">Counters</span> (of type
- <span class="codefrag">Enum</span>) and update them via
- <a href="api/org/apache/hadoop/mapred/Reporter.html#incrCounter(java.lang.Enum, long)">
- Reporter.incrCounter(Enum, long)</a> in the <span class="codefrag">map</span> and/or
- <span class="codefrag">reduce</span> methods. These counters are then globally
- aggregated by the framework.</p>
- <a name="N10A79"></a><a name="DistributedCache"></a>
- <h4>DistributedCache</h4>
- <p>
- <a href="api/org/apache/hadoop/filecache/DistributedCache.html">
- DistributedCache</a> distributes application-specific, large, read-only
- files efficiently.</p>
- <p>
- <span class="codefrag">DistributedCache</span> is a facility provided by the
- Map-Reduce framework to cache files (text, archives, jars and so on)
- needed by applications.</p>
- <p>Applications specify the files to be cached via urls (hdfs:// or
- http://) in the <span class="codefrag">JobConf</span>. The <span class="codefrag">DistributedCache</span>
- assumes that the files specified via hdfs:// urls are already present
- on the <span class="codefrag">FileSystem</span>.</p>
- <p>The framework will copy the necessary files to the slave node
- before any tasks for the job are executed on that node. Its
- efficiency stems from the fact that the files are only copied once
- per job and the ability to cache archives which are un-archived on
- the slaves.</p>
- <p>
- <span class="codefrag">DistributedCache</span> tracks the modification timestamps of
- the cached files. Clearly the cache files should not be modified by
- the application or externally while the job is executing.</p>
- <p>
- <span class="codefrag">DistributedCache</span> can be used to distribute simple,
- read-only data/text files and more complex types such as archives and
- jars. Archives (zip files) are <em>un-archived</em> at the slave nodes.
- Optionally users can also direct the <span class="codefrag">DistributedCache</span> to
- <em>symlink</em> the cached file(s) into the <span class="codefrag">current working
- directory</span> of the task via the
- <a href="api/org/apache/hadoop/filecache/DistributedCache.html#createSymlink(org.apache.hadoop.conf.Configuration)">
- DistributedCache.createSymlink(Path, Configuration)</a> api. Files
- have <em>execution permissions</em> set.</p>
- <a name="N10AB7"></a><a name="Tool"></a>
- <h4>Tool</h4>
- <p>The <a href="api/org/apache/hadoop/util/Tool.html">Tool</a>
- interface supports the handling of generic Hadoop command-line options.
- </p>
- <p>
- <span class="codefrag">Tool</span> is the standard for any Map-Reduce tool or
- application. The application should delegate the handling of
- standard command-line options to
- <a href="api/org/apache/hadoop/util/GenericOptionsParser.html">
- GenericOptionsParser</a> via
- <a href="api/org/apache/hadoop/util/ToolRunner.html#run(org.apache.hadoop.util.Tool, java.lang.String[])">
- ToolRunner.run(Tool, String[])</a> and only handle its custom
- arguments.</p>
- <p>
- The generic Hadoop command-line options are:<br>
-
- <span class="codefrag">
- -conf <configuration file>
- </span>
-
- <br>
-
- <span class="codefrag">
- -D <property=value>
- </span>
-
- <br>
-
- <span class="codefrag">
- -fs <local|namenode:port>
- </span>
-
- <br>
-
- <span class="codefrag">
- -jt <local|jobtracker:port>
- </span>
-
- </p>
- <a name="N10AE9"></a><a name="IsolationRunner"></a>
- <h4>IsolationRunner</h4>
- <p>
- <a href="api/org/apache/hadoop/mapred/IsolationRunner.html">
- IsolationRunner</a> is a utility to help debug Map-Reduce programs.</p>
- <p>To use the <span class="codefrag">IsolationRunner</span>, first set
- <span class="codefrag">keep.failed.tasks.files</span> to <span class="codefrag">true</span>
- (also see <span class="codefrag">keep.tasks.files.pattern</span>).</p>
- <p>
- Next, go to the node on which the failed task ran and go to the
- <span class="codefrag">TaskTracker</span>'s local directory and run the
- <span class="codefrag">IsolationRunner</span>:<br>
-
- <span class="codefrag">$ cd <local path>/taskTracker/${taskid}/work</span>
- <br>
-
- <span class="codefrag">
- $ bin/hadoop org.apache.hadoop.mapred.IsolationRunner ../job.xml
- </span>
-
- </p>
- <p>
- <span class="codefrag">IsolationRunner</span> will run the failed task in a single
- jvm, which can be in the debugger, over precisely the same input.</p>
- <a name="N10B1C"></a><a name="JobControl"></a>
- <h4>JobControl</h4>
- <p>
- <a href="api/org/apache/hadoop/mapred/jobcontrol/package-summary.html">
- JobControl</a> is a utility which encapsulates a set of Map-Reduce jobs
- and their dependencies.</p>
- <a name="N10B29"></a><a name="Data+Compression"></a>
- <h4>Data Compression</h4>
- <p>Hadoop Map-Reduce provides facilities for the application-writer to
- specify compression for both intermediate map-outputs and the
- job-outputs i.e. output of the reduces. It also comes bundled with
- <a href="api/org/apache/hadoop/io/compress/CompressionCodec.html">
- CompressionCodec</a> implementations for the
- <a href="http://www.zlib.net/">zlib</a> and <a href="http://www.oberhumer.com/opensource/lzo/">lzo</a> compression
- algorithms. The <a href="http://www.gzip.org/">gzip</a> file format is also
- supported.</p>
- <p>Hadoop also provides native implementations of the above compression
- codecs for reasons of both performance (zlib) and non-availability of
- Java libraries (lzo). More details on their usage and availability are
- available <a href="native_libraries.html">here</a>.</p>
- <a name="N10B49"></a><a name="Intermediate+Outputs"></a>
- <h5>Intermediate Outputs</h5>
- <p>Applications can control compression of intermediate map-outputs
- via the
- <a href="api/org/apache/hadoop/mapred/JobConf.html#setCompressMapOutput(boolean)">
- JobConf.setCompressMapOutput(boolean)</a> api and the
- <span class="codefrag">CompressionCodec</span> to be used via the
- <a href="api/org/apache/hadoop/mapred/JobConf.html#setMapOutputCompressorClass(java.lang.Class)">
- JobConf.setMapOutputCompressorClass(Class)</a> api. Since
- the intermediate map-outputs are always stored in the
- <a href="api/org/apache/hadoop/io/SequenceFile.html">SequenceFile</a>
- format, the
- <a href="api/org/apache/hadoop/io/SequenceFile.CompressionType.html">
- SequenceFile.CompressionType</a> (i.e.
- <a href="api/org/apache/hadoop/io/SequenceFile.CompressionType.html#RECORD">
- RECORD</a> /
- <a href="api/org/apache/hadoop/io/SequenceFile.CompressionType.html#BLOCK">
- BLOCK</a> - defaults to <span class="codefrag">RECORD</span>) can be specified via the
- <a href="api/org/apache/hadoop/mapred/JobConf.html#setMapOutputCompressionType(org.apache.hadoop.io.SequenceFile.CompressionType)">
- JobConf.setMapOutputCompressionType(SequenceFile.CompressionType)</a>
- api.</p>
- <a name="N10B75"></a><a name="Job+Outputs"></a>
- <h5>Job Outputs</h5>
- <p>Applications can control compression of job-outputs via the
- <a href="api/org/apache/hadoop/mapred/OutputFormatBase.html#setCompressOutput(org.apache.hadoop.mapred.JobConf,%20boolean)">
- OutputFormatBase.setCompressOutput(JobConf, boolean)</a> api and the
- <span class="codefrag">CompressionCodec</span> to be used can be specified via the
- <a href="api/org/apache/hadoop/mapred/OutputFormatBase.html#setOutputCompressorClass(org.apache.hadoop.mapred.JobConf,%20java.lang.Class)">
- OutputFormatBase.setOutputCompressorClass(JobConf, Class)</a> api.</p>
- <p>If the job outputs are to be stored in the
- <a href="api/org/apache/hadoop/mapred/SequenceFileOutputFormat.html">
- SequenceFileOutputFormat</a>, the required
- <span class="codefrag">SequenceFile.CompressionType</span> (i.e. <span class="codefrag">RECORD</span> /
- <span class="codefrag">BLOCK</span> - defaults to <span class="codefrag">RECORD</span>)can be specified
- via the
- <a href="api/org/apache/hadoop/mapred/SequenceFileOutputFormat.html#setOutputCompressionType(org.apache.hadoop.mapred.JobConf,%20org.apache.hadoop.io.SequenceFile.CompressionType)">
- SequenceFileOutputFormat.setOutputCompressionType(JobConf,
- SequenceFile.CompressionType)</a> api.</p>
- </div>
-
- <a name="N10BA4"></a><a name="Example%3A+WordCount+v2.0"></a>
- <h2 class="h3">Example: WordCount v2.0</h2>
- <div class="section">
- <p>Here is a more complete <span class="codefrag">WordCount</span> which uses many of the
- features provided by the Map-Reduce framework we discussed so far.</p>
- <p>This needs the HDFS to be up and running, especially for the
- <span class="codefrag">DistributedCache</span>-related features. Hence it only works with a
- <a href="quickstart.html#SingleNodeSetup">pseudo-distributed</a> or
- <a href="quickstart.html#Fully-Distributed+Operation">fully-distributed</a>
- Hadoop installation.</p>
- <a name="N10BBE"></a><a name="Source+Code-N10BBE"></a>
- <h3 class="h4">Source Code</h3>
- <table class="ForrestTable" cellspacing="1" cellpadding="4">
-
- <tr>
-
- <th colspan="1" rowspan="1"></th>
- <th colspan="1" rowspan="1">WordCount.java</th>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">1.</td>
- <td colspan="1" rowspan="1">
- <span class="codefrag">package org.myorg;</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">2.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">3.</td>
- <td colspan="1" rowspan="1">
- <span class="codefrag">import java.io.*;</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">4.</td>
- <td colspan="1" rowspan="1">
- <span class="codefrag">import java.util.*;</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">5.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">6.</td>
- <td colspan="1" rowspan="1">
- <span class="codefrag">import org.apache.hadoop.fs.Path;</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">7.</td>
- <td colspan="1" rowspan="1">
- <span class="codefrag">import org.apache.hadoop.filecache.DistributedCache;</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">8.</td>
- <td colspan="1" rowspan="1">
- <span class="codefrag">import org.apache.hadoop.conf.*;</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">9.</td>
- <td colspan="1" rowspan="1">
- <span class="codefrag">import org.apache.hadoop.io.*;</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">10.</td>
- <td colspan="1" rowspan="1">
- <span class="codefrag">import org.apache.hadoop.mapred.*;</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">11.</td>
- <td colspan="1" rowspan="1">
- <span class="codefrag">import org.apache.hadoop.util.*;</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">12.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">13.</td>
- <td colspan="1" rowspan="1">
- <span class="codefrag">public class WordCount extends Configured implements Tool {</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">14.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">15.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- public static class Map extends MapReduceBase
- implements Mapper<LongWritable, Text, Text, IntWritable> {
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">16.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">17.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- static enum Counters { INPUT_WORDS }
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">18.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">19.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- private final static IntWritable one = new IntWritable(1);
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">20.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">private Text word = new Text();</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">21.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">22.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">private boolean caseSensitive = true;</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">23.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">private Set<String> patternsToSkip = new HashSet<String>();</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">24.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">25.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">private long numRecords = 0;</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">26.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">private String inputFile;</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">27.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">28.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">public void configure(JobConf job) {</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">29.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- caseSensitive = job.getBoolean("wordcount.case.sensitive", true);
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">30.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">inputFile = job.get("map.input.file");</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">31.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">32.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">if (job.getBoolean("wordcount.skip.patterns", false)) {</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">33.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">Path[] patternsFiles = new Path[0];</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">34.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">try {</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">35.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- patternsFiles = DistributedCache.getLocalCacheFiles(job);
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">36.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">} catch (IOException ioe) {</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">37.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- System.err.println("Caught exception while getting cached files: "
- + StringUtils.stringifyException(ioe));
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">38.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">39.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">for (Path patternsFile : patternsFiles) {</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">40.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">parseSkipFile(patternsFile);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">41.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">42.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">43.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">44.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">45.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">private void parseSkipFile(Path patternsFile) {</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">46.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">try {</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">47.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- BufferedReader fis =
- new BufferedReader(new FileReader(patternsFile.toString()));
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">48.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">String pattern = null;</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">49.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">while ((pattern = fis.readLine()) != null) {</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">50.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">patternsToSkip.add(pattern);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">51.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">52.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">} catch (IOException ioe) {</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">53.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- System.err.println("Caught exception while parsing the cached file '" +
- patternsFile + "' : " +
- StringUtils.stringifyException(ioe));
-
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">54.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">55.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">56.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">57.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- public void map(LongWritable key, Text value,
- OutputCollector<Text, IntWritable> output,
- Reporter reporter) throws IOException {
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">58.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- String line =
- (caseSensitive) ? value.toString() :
- value.toString().toLowerCase();
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">59.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">60.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">for (String pattern : patternsToSkip) {</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">61.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">line = line.replaceAll(pattern, "");</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">62.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">63.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">64.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">StringTokenizer tokenizer = new StringTokenizer(line);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">65.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">while (tokenizer.hasMoreTokens()) {</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">66.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">word.set(tokenizer.nextToken());</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">67.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">output.collect(word, one);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">68.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">reporter.incrCounter(Counters.INPUT_WORDS, 1);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">69.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">70.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">71.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">if ((++numRecords % 100) == 0) {</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">72.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- reporter.setStatus("Finished processing " + numRecords +
- " records " + "from the input file: " +
- inputFile);
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">73.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">74.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">75.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">76.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">77.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- public static class Reduce extends MapReduceBase implements
- Reducer<Text, IntWritable, Text, IntWritable> {
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">78.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- public void reduce(Text key, Iterator<IntWritable> values,
- OutputCollector<Text, IntWritable> output,
- Reporter reporter) throws IOException {
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">79.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">int sum = 0;</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">80.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">while (values.hasNext()) {</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">81.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">sum += values.next().get();</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">82.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">83.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">output.collect(key, new IntWritable(sum));</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">84.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">85.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">86.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">87.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">public int run(String[] args) throws Exception {</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">88.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- JobConf conf = new JobConf(getConf(), WordCount.class);
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">89.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">conf.setJobName("wordcount");</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">90.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">91.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">conf.setOutputKeyClass(Text.class);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">92.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">conf.setOutputValueClass(IntWritable.class);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">93.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">94.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">conf.setMapperClass(Map.class);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">95.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">conf.setCombinerClass(Reduce.class);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">96.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">conf.setReducerClass(Reduce.class);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">97.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">98.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">conf.setInputFormat(TextInputFormat.class);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">99.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">conf.setOutputFormat(TextOutputFormat.class);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">100.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">101.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- List<String> other_args = new ArrayList<String>();
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">102.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">for (int i=0; i < args.length; ++i) {</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">103.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">if ("-skip".equals(args[i])) {</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">104.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- DistributedCache.addCacheFile(new Path(args[++i]).toUri(), conf);
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">105.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- conf.setBoolean("wordcount.skip.patterns", true);
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">106.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">} else {</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">107.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">other_args.add(args[i]);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">108.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">109.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">110.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">111.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">conf.setInputPath(new Path(other_args.get(0)));</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">112.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">conf.setOutputPath(new Path(other_args.get(1)));</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">113.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">114.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">JobClient.runJob(conf);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">115.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">return 0;</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">116.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">117.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">118.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- public static void main(String[] args) throws Exception {
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">119.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">
- int res = ToolRunner.run(new Configuration(), new WordCount(),
- args);
- </span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">120.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">System.exit(res);</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">121.</td>
- <td colspan="1" rowspan="1">
-
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">122.</td>
- <td colspan="1" rowspan="1">
- <span class="codefrag">}</span>
- </td>
-
- </tr>
-
- <tr>
-
- <td colspan="1" rowspan="1">123.</td>
- <td colspan="1" rowspan="1"></td>
-
- </tr>
-
- </table>
- <a name="N11320"></a><a name="Sample+Runs"></a>
- <h3 class="h4">Sample Runs</h3>
- <p>Sample text-files as input:</p>
- <p>
-
- <span class="codefrag">$ bin/hadoop dfs -ls /usr/joe/wordcount/input/</span>
- <br>
-
- <span class="codefrag">/usr/joe/wordcount/input/file01</span>
- <br>
-
- <span class="codefrag">/usr/joe/wordcount/input/file02</span>
- <br>
-
- <br>
-
- <span class="codefrag">$ bin/hadoop dfs -cat /usr/joe/wordcount/input/file01</span>
- <br>
-
- <span class="codefrag">Hello World, Bye World!</span>
- <br>
-
- <br>
-
- <span class="codefrag">$ bin/hadoop dfs -cat /usr/joe/wordcount/input/file02</span>
- <br>
-
- <span class="codefrag">Hello Hadoop, Goodbye to hadoop.</span>
-
- </p>
- <p>Run the application:</p>
- <p>
-
- <span class="codefrag">
- $ bin/hadoop jar /usr/joe/wordcount.jar org.myorg.WordCount
- /usr/joe/wordcount/input /usr/joe/wordcount/output
- </span>
-
- </p>
- <p>Output:</p>
- <p>
-
- <span class="codefrag">
- $ bin/hadoop dfs -cat /usr/joe/wordcount/output/part-00000
- </span>
-
- <br>
-
- <span class="codefrag">Bye 1</span>
- <br>
-
- <span class="codefrag">Goodbye 1</span>
- <br>
-
- <span class="codefrag">Hadoop, 1</span>
- <br>
-
- <span class="codefrag">Hello 2</span>
- <br>
-
- <span class="codefrag">World! 1</span>
- <br>
-
- <span class="codefrag">World, 1</span>
- <br>
-
- <span class="codefrag">hadoop. 1</span>
- <br>
-
- <span class="codefrag">to 1</span>
- <br>
-
- </p>
- <p>Notice that the inputs differ from the first version we looked at,
- and how they affect the outputs.</p>
- <p>Now, lets plug-in a pattern-file which lists the word-patterns to be
- ignored, via the <span class="codefrag">DistributedCache</span>.</p>
- <p>
-
- <span class="codefrag">$ hadoop dfs -cat /user/joe/wordcount/patterns.txt</span>
- <br>
-
- <span class="codefrag">\.</span>
- <br>
-
- <span class="codefrag">\,</span>
- <br>
-
- <span class="codefrag">\!</span>
- <br>
-
- <span class="codefrag">to</span>
- <br>
-
- </p>
- <p>Run it again, this time with more options:</p>
- <p>
-
- <span class="codefrag">
- $ bin/hadoop jar /usr/joe/wordcount.jar org.myorg.WordCount
- -Dwordcount.case.sensitive=true /usr/joe/wordcount/input
- /usr/joe/wordcount/output -skip /user/joe/wordcount/patterns.txt
- </span>
-
- </p>
- <p>As expected, the output:</p>
- <p>
-
- <span class="codefrag">
- $ bin/hadoop dfs -cat /usr/joe/wordcount/output/part-00000
- </span>
-
- <br>
-
- <span class="codefrag">Bye 1</span>
- <br>
-
- <span class="codefrag">Goodbye 1</span>
- <br>
-
- <span class="codefrag">Hadoop 1</span>
- <br>
-
- <span class="codefrag">Hello 2</span>
- <br>
-
- <span class="codefrag">World 2</span>
- <br>
-
- <span class="codefrag">hadoop 1</span>
- <br>
-
- </p>
- <p>Run it once more, this time switch-off case-sensitivity:</p>
- <p>
-
- <span class="codefrag">
- $ bin/hadoop jar /usr/joe/wordcount.jar org.myorg.WordCount
- -Dwordcount.case.sensitive=false /usr/joe/wordcount/input
- /usr/joe/wordcount/output -skip /user/joe/wordcount/patterns.txt
- </span>
-
- </p>
- <p>Sure enough, the output:</p>
- <p>
-
- <span class="codefrag">
- $ bin/hadoop dfs -cat /usr/joe/wordcount/output/part-00000
- </span>
-
- <br>
-
- <span class="codefrag">bye 1</span>
- <br>
-
- <span class="codefrag">goodbye 1</span>
- <br>
-
- <span class="codefrag">hadoop 2</span>
- <br>
-
- <span class="codefrag">hello 2</span>
- <br>
-
- <span class="codefrag">world 2</span>
- <br>
-
- </p>
- <a name="N113F4"></a><a name="Highlights"></a>
- <h3 class="h4">Highlights</h3>
- <p>The second version of <span class="codefrag">WordCount</span> improves upon the
- previous one by using some features offered by the Map-Reduce framework:
- </p>
- <ul>
-
- <li>
- Demonstrates how applications can access configuration parameters
- in the <span class="codefrag">configure</span> method of the <span class="codefrag">Mapper</span> (and
- <span class="codefrag">Reducer</span>) implementations (lines 28-43).
- </li>
-
- <li>
- Demonstrates how the <span class="codefrag">DistributedCache</span> can be used to
- distribute read-only data needed by the jobs. Here it allows the user
- to specify word-patterns to skip while counting (line 104).
- </li>
-
- <li>
- Demonstrates the utility of the <span class="codefrag">Tool</span> interface and the
- <span class="codefrag">GenericOptionsParser</span> to handle generic Hadoop
- command-line options (lines 87-116, 119).
- </li>
-
- <li>
- Demonstrates how applications can use <span class="codefrag">Counters</span> (line 68)
- and how they can set application-specific status information via
- the <span class="codefrag">Reporter</span> instance passed to the <span class="codefrag">map</span> (and
- <span class="codefrag">reduce</span>) method (line 72).
- </li>
-
- </ul>
- </div>
-
- <p>
-
- <em>Java and JNI are trademarks or registered trademarks of
- Sun Microsystems, Inc. in the United States and other countries.</em>
-
- </p>
-
-
- </div>
- <!--+
- |end content
- +-->
- <div class="clearboth"> </div>
- </div>
- <div id="footer">
- <!--+
- |start bottomstrip
- +-->
- <div class="lastmodified">
- <script type="text/javascript"><!--
- document.write("Last Published: " + document.lastModified);
- // --></script>
- </div>
- <div class="copyright">
- Copyright ©
- 2007 <a href="http://www.apache.org/licenses/">The Apache Software Foundation.</a>
- </div>
- <!--+
- |end bottomstrip
- +-->
- </div>
- </body>
- </html>
|