1 | <!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd"> |
---|
2 | <html> |
---|
3 | <head> |
---|
4 | <META http-equiv="Content-Type" content="text/html; charset=UTF-8"> |
---|
5 | <meta content="Apache Forrest" name="Generator"> |
---|
6 | <meta name="Forrest-version" content="0.8"> |
---|
7 | <meta name="Forrest-skin-name" content="pelt"> |
---|
8 | <title>Map/Reduce Tutorial</title> |
---|
9 | <link type="text/css" href="skin/basic.css" rel="stylesheet"> |
---|
10 | <link media="screen" type="text/css" href="skin/screen.css" rel="stylesheet"> |
---|
11 | <link media="print" type="text/css" href="skin/print.css" rel="stylesheet"> |
---|
12 | <link type="text/css" href="skin/profile.css" rel="stylesheet"> |
---|
13 | <script src="skin/getBlank.js" language="javascript" type="text/javascript"></script><script src="skin/getMenu.js" language="javascript" type="text/javascript"></script><script src="skin/fontsize.js" language="javascript" type="text/javascript"></script> |
---|
14 | <link rel="shortcut icon" href="images/favicon.ico"> |
---|
15 | </head> |
---|
16 | <body onload="init()"> |
---|
17 | <script type="text/javascript">ndeSetTextSize();</script> |
---|
18 | <div id="top"> |
---|
19 | <!--+ |
---|
20 | |breadtrail |
---|
21 | +--> |
---|
22 | <div class="breadtrail"> |
---|
23 | <a href="http://www.apache.org/">Apache</a> > <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> |
---|
24 | </div> |
---|
25 | <!--+ |
---|
26 | |header |
---|
27 | +--> |
---|
28 | <div class="header"> |
---|
29 | <!--+ |
---|
30 | |start group logo |
---|
31 | +--> |
---|
32 | <div class="grouplogo"> |
---|
33 | <a href="http://hadoop.apache.org/"><img class="logoImage" alt="Hadoop" src="images/hadoop-logo.jpg" title="Apache Hadoop"></a> |
---|
34 | </div> |
---|
35 | <!--+ |
---|
36 | |end group logo |
---|
37 | +--> |
---|
38 | <!--+ |
---|
39 | |start Project Logo |
---|
40 | +--> |
---|
41 | <div class="projectlogo"> |
---|
42 | <a href="http://hadoop.apache.org/core/"><img class="logoImage" alt="Hadoop" src="images/core-logo.gif" title="Scalable Computing Platform"></a> |
---|
43 | </div> |
---|
44 | <!--+ |
---|
45 | |end Project Logo |
---|
46 | +--> |
---|
47 | <!--+ |
---|
48 | |start Search |
---|
49 | +--> |
---|
50 | <div class="searchbox"> |
---|
51 | <form action="http://www.google.com/search" method="get" class="roundtopsmall"> |
---|
52 | <input value="hadoop.apache.org" name="sitesearch" type="hidden"><input onFocus="getBlank (this, 'Search the site with google');" size="25" name="q" id="query" type="text" value="Search the site with google"> |
---|
53 | <input name="Search" value="Search" type="submit"> |
---|
54 | </form> |
---|
55 | </div> |
---|
56 | <!--+ |
---|
57 | |end search |
---|
58 | +--> |
---|
59 | <!--+ |
---|
60 | |start Tabs |
---|
61 | +--> |
---|
62 | <ul id="tabs"> |
---|
63 | <li> |
---|
64 | <a class="unselected" href="http://hadoop.apache.org/core/">Project</a> |
---|
65 | </li> |
---|
66 | <li> |
---|
67 | <a class="unselected" href="http://wiki.apache.org/hadoop">Wiki</a> |
---|
68 | </li> |
---|
69 | <li class="current"> |
---|
70 | <a class="selected" href="index.html">Hadoop 0.20 Documentation</a> |
---|
71 | </li> |
---|
72 | </ul> |
---|
73 | <!--+ |
---|
74 | |end Tabs |
---|
75 | +--> |
---|
76 | </div> |
---|
77 | </div> |
---|
78 | <div id="main"> |
---|
79 | <div id="publishedStrip"> |
---|
80 | <!--+ |
---|
81 | |start Subtabs |
---|
82 | +--> |
---|
83 | <div id="level2tabs"></div> |
---|
84 | <!--+ |
---|
85 | |end Endtabs |
---|
86 | +--> |
---|
87 | <script type="text/javascript"><!-- |
---|
88 | document.write("Last Published: " + document.lastModified); |
---|
89 | // --></script> |
---|
90 | </div> |
---|
91 | <!--+ |
---|
92 | |breadtrail |
---|
93 | +--> |
---|
94 | <div class="breadtrail"> |
---|
95 | |
---|
96 | |
---|
97 | </div> |
---|
98 | <!--+ |
---|
99 | |start Menu, mainarea |
---|
100 | +--> |
---|
101 | <!--+ |
---|
102 | |start Menu |
---|
103 | +--> |
---|
104 | <div id="menu"> |
---|
105 | <div onclick="SwitchMenu('menu_selected_1.1', 'skin/')" id="menu_selected_1.1Title" class="menutitle" style="background-image: url('skin/images/chapter_open.gif');">Getting Started</div> |
---|
106 | <div id="menu_selected_1.1" class="selectedmenuitemgroup" style="display: block;"> |
---|
107 | <div class="menuitem"> |
---|
108 | <a href="index.html">Overview</a> |
---|
109 | </div> |
---|
110 | <div class="menuitem"> |
---|
111 | <a href="quickstart.html">Quick Start</a> |
---|
112 | </div> |
---|
113 | <div class="menuitem"> |
---|
114 | <a href="cluster_setup.html">Cluster Setup</a> |
---|
115 | </div> |
---|
116 | <div class="menupage"> |
---|
117 | <div class="menupagetitle">Map/Reduce Tutorial</div> |
---|
118 | </div> |
---|
119 | </div> |
---|
120 | <div onclick="SwitchMenu('menu_1.2', 'skin/')" id="menu_1.2Title" class="menutitle">Programming Guides</div> |
---|
121 | <div id="menu_1.2" class="menuitemgroup"> |
---|
122 | <div class="menuitem"> |
---|
123 | <a href="commands_manual.html">Commands</a> |
---|
124 | </div> |
---|
125 | <div class="menuitem"> |
---|
126 | <a href="distcp.html">DistCp</a> |
---|
127 | </div> |
---|
128 | <div class="menuitem"> |
---|
129 | <a href="native_libraries.html">Native Libraries</a> |
---|
130 | </div> |
---|
131 | <div class="menuitem"> |
---|
132 | <a href="streaming.html">Streaming</a> |
---|
133 | </div> |
---|
134 | <div class="menuitem"> |
---|
135 | <a href="fair_scheduler.html">Fair Scheduler</a> |
---|
136 | </div> |
---|
137 | <div class="menuitem"> |
---|
138 | <a href="capacity_scheduler.html">Capacity Scheduler</a> |
---|
139 | </div> |
---|
140 | <div class="menuitem"> |
---|
141 | <a href="service_level_auth.html">Service Level Authorization</a> |
---|
142 | </div> |
---|
143 | <div class="menuitem"> |
---|
144 | <a href="vaidya.html">Vaidya</a> |
---|
145 | </div> |
---|
146 | <div class="menuitem"> |
---|
147 | <a href="hadoop_archives.html">Archives</a> |
---|
148 | </div> |
---|
149 | </div> |
---|
150 | <div onclick="SwitchMenu('menu_1.3', 'skin/')" id="menu_1.3Title" class="menutitle">HDFS</div> |
---|
151 | <div id="menu_1.3" class="menuitemgroup"> |
---|
152 | <div class="menuitem"> |
---|
153 | <a href="hdfs_user_guide.html">User Guide</a> |
---|
154 | </div> |
---|
155 | <div class="menuitem"> |
---|
156 | <a href="hdfs_design.html">Architecture</a> |
---|
157 | </div> |
---|
158 | <div class="menuitem"> |
---|
159 | <a href="hdfs_shell.html">File System Shell Guide</a> |
---|
160 | </div> |
---|
161 | <div class="menuitem"> |
---|
162 | <a href="hdfs_permissions_guide.html">Permissions Guide</a> |
---|
163 | </div> |
---|
164 | <div class="menuitem"> |
---|
165 | <a href="hdfs_quota_admin_guide.html">Quotas Guide</a> |
---|
166 | </div> |
---|
167 | <div class="menuitem"> |
---|
168 | <a href="SLG_user_guide.html">Synthetic Load Generator Guide</a> |
---|
169 | </div> |
---|
170 | <div class="menuitem"> |
---|
171 | <a href="libhdfs.html">C API libhdfs</a> |
---|
172 | </div> |
---|
173 | </div> |
---|
174 | <div onclick="SwitchMenu('menu_1.4', 'skin/')" id="menu_1.4Title" class="menutitle">HOD</div> |
---|
175 | <div id="menu_1.4" class="menuitemgroup"> |
---|
176 | <div class="menuitem"> |
---|
177 | <a href="hod_user_guide.html">User Guide</a> |
---|
178 | </div> |
---|
179 | <div class="menuitem"> |
---|
180 | <a href="hod_admin_guide.html">Admin Guide</a> |
---|
181 | </div> |
---|
182 | <div class="menuitem"> |
---|
183 | <a href="hod_config_guide.html">Config Guide</a> |
---|
184 | </div> |
---|
185 | </div> |
---|
186 | <div onclick="SwitchMenu('menu_1.5', 'skin/')" id="menu_1.5Title" class="menutitle">Miscellaneous</div> |
---|
187 | <div id="menu_1.5" class="menuitemgroup"> |
---|
188 | <div class="menuitem"> |
---|
189 | <a href="api/index.html">API Docs</a> |
---|
190 | </div> |
---|
191 | <div class="menuitem"> |
---|
192 | <a href="jdiff/changes.html">API Changes</a> |
---|
193 | </div> |
---|
194 | <div class="menuitem"> |
---|
195 | <a href="http://wiki.apache.org/hadoop/">Wiki</a> |
---|
196 | </div> |
---|
197 | <div class="menuitem"> |
---|
198 | <a href="http://wiki.apache.org/hadoop/FAQ">FAQ</a> |
---|
199 | </div> |
---|
200 | <div class="menuitem"> |
---|
201 | <a href="releasenotes.html">Release Notes</a> |
---|
202 | </div> |
---|
203 | <div class="menuitem"> |
---|
204 | <a href="changes.html">Change Log</a> |
---|
205 | </div> |
---|
206 | </div> |
---|
207 | <div id="credit"></div> |
---|
208 | <div id="roundbottom"> |
---|
209 | <img style="display: none" class="corner" height="15" width="15" alt="" src="skin/images/rc-b-l-15-1body-2menu-3menu.png"></div> |
---|
210 | <!--+ |
---|
211 | |alternative credits |
---|
212 | +--> |
---|
213 | <div id="credit2"></div> |
---|
214 | </div> |
---|
215 | <!--+ |
---|
216 | |end Menu |
---|
217 | +--> |
---|
218 | <!--+ |
---|
219 | |start content |
---|
220 | +--> |
---|
221 | <div id="content"> |
---|
222 | <div title="Portable Document Format" class="pdflink"> |
---|
223 | <a class="dida" href="mapred_tutorial.pdf"><img alt="PDF -icon" src="skin/images/pdfdoc.gif" class="skin"><br> |
---|
224 | PDF</a> |
---|
225 | </div> |
---|
226 | <h1>Map/Reduce Tutorial</h1> |
---|
227 | <div id="minitoc-area"> |
---|
228 | <ul class="minitoc"> |
---|
229 | <li> |
---|
230 | <a href="#Purpose">Purpose</a> |
---|
231 | </li> |
---|
232 | <li> |
---|
233 | <a href="#Pre-requisites">Pre-requisites</a> |
---|
234 | </li> |
---|
235 | <li> |
---|
236 | <a href="#Overview">Overview</a> |
---|
237 | </li> |
---|
238 | <li> |
---|
239 | <a href="#Inputs+and+Outputs">Inputs and Outputs</a> |
---|
240 | </li> |
---|
241 | <li> |
---|
242 | <a href="#Example%3A+WordCount+v1.0">Example: WordCount v1.0</a> |
---|
243 | <ul class="minitoc"> |
---|
244 | <li> |
---|
245 | <a href="#Source+Code">Source Code</a> |
---|
246 | </li> |
---|
247 | <li> |
---|
248 | <a href="#Usage">Usage</a> |
---|
249 | </li> |
---|
250 | <li> |
---|
251 | <a href="#Walk-through">Walk-through</a> |
---|
252 | </li> |
---|
253 | </ul> |
---|
254 | </li> |
---|
255 | <li> |
---|
256 | <a href="#Map%2FReduce+-+User+Interfaces">Map/Reduce - User Interfaces</a> |
---|
257 | <ul class="minitoc"> |
---|
258 | <li> |
---|
259 | <a href="#Payload">Payload</a> |
---|
260 | <ul class="minitoc"> |
---|
261 | <li> |
---|
262 | <a href="#Mapper">Mapper</a> |
---|
263 | </li> |
---|
264 | <li> |
---|
265 | <a href="#Reducer">Reducer</a> |
---|
266 | </li> |
---|
267 | <li> |
---|
268 | <a href="#Partitioner">Partitioner</a> |
---|
269 | </li> |
---|
270 | <li> |
---|
271 | <a href="#Reporter">Reporter</a> |
---|
272 | </li> |
---|
273 | <li> |
---|
274 | <a href="#OutputCollector">OutputCollector</a> |
---|
275 | </li> |
---|
276 | </ul> |
---|
277 | </li> |
---|
278 | <li> |
---|
279 | <a href="#Job+Configuration">Job Configuration</a> |
---|
280 | </li> |
---|
281 | <li> |
---|
282 | <a href="#Task+Execution+%26+Environment">Task Execution & Environment</a> |
---|
283 | <ul class="minitoc"> |
---|
284 | <li> |
---|
285 | <a href="#Memory+management"> Memory management</a> |
---|
286 | </li> |
---|
287 | <li> |
---|
288 | <a href="#Map+Parameters">Map Parameters</a> |
---|
289 | </li> |
---|
290 | <li> |
---|
291 | <a href="#Shuffle%2FReduce+Parameters">Shuffle/Reduce Parameters</a> |
---|
292 | </li> |
---|
293 | <li> |
---|
294 | <a href="#Directory+Structure"> Directory Structure </a> |
---|
295 | </li> |
---|
296 | <li> |
---|
297 | <a href="#Task+JVM+Reuse">Task JVM Reuse</a> |
---|
298 | </li> |
---|
299 | </ul> |
---|
300 | </li> |
---|
301 | <li> |
---|
302 | <a href="#Job+Submission+and+Monitoring">Job Submission and Monitoring</a> |
---|
303 | <ul class="minitoc"> |
---|
304 | <li> |
---|
305 | <a href="#Job+Control">Job Control</a> |
---|
306 | </li> |
---|
307 | </ul> |
---|
308 | </li> |
---|
309 | <li> |
---|
310 | <a href="#Job+Input">Job Input</a> |
---|
311 | <ul class="minitoc"> |
---|
312 | <li> |
---|
313 | <a href="#InputSplit">InputSplit</a> |
---|
314 | </li> |
---|
315 | <li> |
---|
316 | <a href="#RecordReader">RecordReader</a> |
---|
317 | </li> |
---|
318 | </ul> |
---|
319 | </li> |
---|
320 | <li> |
---|
321 | <a href="#Job+Output">Job Output</a> |
---|
322 | <ul class="minitoc"> |
---|
323 | <li> |
---|
324 | <a href="#OutputCommitter">OutputCommitter</a> |
---|
325 | </li> |
---|
326 | <li> |
---|
327 | <a href="#Task+Side-Effect+Files">Task Side-Effect Files</a> |
---|
328 | </li> |
---|
329 | <li> |
---|
330 | <a href="#RecordWriter">RecordWriter</a> |
---|
331 | </li> |
---|
332 | </ul> |
---|
333 | </li> |
---|
334 | <li> |
---|
335 | <a href="#Other+Useful+Features">Other Useful Features</a> |
---|
336 | <ul class="minitoc"> |
---|
337 | <li> |
---|
338 | <a href="#Submitting+Jobs+to+Queues">Submitting Jobs to Queues</a> |
---|
339 | </li> |
---|
340 | <li> |
---|
341 | <a href="#Counters">Counters</a> |
---|
342 | </li> |
---|
343 | <li> |
---|
344 | <a href="#DistributedCache">DistributedCache</a> |
---|
345 | </li> |
---|
346 | <li> |
---|
347 | <a href="#Tool">Tool</a> |
---|
348 | </li> |
---|
349 | <li> |
---|
350 | <a href="#IsolationRunner">IsolationRunner</a> |
---|
351 | </li> |
---|
352 | <li> |
---|
353 | <a href="#Profiling">Profiling</a> |
---|
354 | </li> |
---|
355 | <li> |
---|
356 | <a href="#Debugging">Debugging</a> |
---|
357 | </li> |
---|
358 | <li> |
---|
359 | <a href="#JobControl">JobControl</a> |
---|
360 | </li> |
---|
361 | <li> |
---|
362 | <a href="#Data+Compression">Data Compression</a> |
---|
363 | </li> |
---|
364 | <li> |
---|
365 | <a href="#Skipping+Bad+Records">Skipping Bad Records</a> |
---|
366 | </li> |
---|
367 | </ul> |
---|
368 | </li> |
---|
369 | </ul> |
---|
370 | </li> |
---|
371 | <li> |
---|
372 | <a href="#Example%3A+WordCount+v2.0">Example: WordCount v2.0</a> |
---|
373 | <ul class="minitoc"> |
---|
374 | <li> |
---|
375 | <a href="#Source+Code-N10FD0">Source Code</a> |
---|
376 | </li> |
---|
377 | <li> |
---|
378 | <a href="#Sample+Runs">Sample Runs</a> |
---|
379 | </li> |
---|
380 | <li> |
---|
381 | <a href="#Highlights">Highlights</a> |
---|
382 | </li> |
---|
383 | </ul> |
---|
384 | </li> |
---|
385 | </ul> |
---|
386 | </div> |
---|
387 | |
---|
388 | |
---|
389 | <a name="N1000D"></a><a name="Purpose"></a> |
---|
390 | <h2 class="h3">Purpose</h2> |
---|
391 | <div class="section"> |
---|
392 | <p>This document comprehensively describes all user-facing facets of the |
---|
393 | Hadoop Map/Reduce framework and serves as a tutorial. |
---|
394 | </p> |
---|
395 | </div> |
---|
396 | |
---|
397 | |
---|
398 | <a name="N10017"></a><a name="Pre-requisites"></a> |
---|
399 | <h2 class="h3">Pre-requisites</h2> |
---|
400 | <div class="section"> |
---|
401 | <p>Ensure that Hadoop is installed, configured and is running. More |
---|
402 | details:</p> |
---|
403 | <ul> |
---|
404 | |
---|
405 | <li> |
---|
406 | |
---|
407 | <a href="quickstart.html">Hadoop Quick Start</a> for first-time users. |
---|
408 | </li> |
---|
409 | |
---|
410 | <li> |
---|
411 | |
---|
412 | <a href="cluster_setup.html">Hadoop Cluster Setup</a> for large, |
---|
413 | distributed clusters. |
---|
414 | </li> |
---|
415 | |
---|
416 | </ul> |
---|
417 | </div> |
---|
418 | |
---|
419 | |
---|
420 | <a name="N10032"></a><a name="Overview"></a> |
---|
421 | <h2 class="h3">Overview</h2> |
---|
422 | <div class="section"> |
---|
423 | <p>Hadoop Map/Reduce is a software framework for easily writing |
---|
424 | applications which process vast amounts of data (multi-terabyte data-sets) |
---|
425 | in-parallel on large clusters (thousands of nodes) of commodity |
---|
426 | hardware in a reliable, fault-tolerant manner.</p> |
---|
427 | <p>A Map/Reduce <em>job</em> usually splits the input data-set into |
---|
428 | independent chunks which are processed by the <em>map tasks</em> in a |
---|
429 | completely parallel manner. The framework sorts the outputs of the maps, |
---|
430 | which are then input to the <em>reduce tasks</em>. Typically both the |
---|
431 | input and the output of the job are stored in a file-system. The framework |
---|
432 | takes care of scheduling tasks, monitoring them and re-executes the failed |
---|
433 | tasks.</p> |
---|
434 | <p>Typically the compute nodes and the storage nodes are the same, that is, |
---|
435 | the Map/Reduce framework and the Hadoop Distributed File System (see <a href="hdfs_design.html">HDFS Architecture </a>) |
---|
436 | are running on the same set of nodes. This configuration |
---|
437 | allows the framework to effectively schedule tasks on the nodes where data |
---|
438 | is already present, resulting in very high aggregate bandwidth across the |
---|
439 | cluster.</p> |
---|
440 | <p>The Map/Reduce framework consists of a single master |
---|
441 | <span class="codefrag">JobTracker</span> and one slave <span class="codefrag">TaskTracker</span> per |
---|
442 | cluster-node. The master is responsible for scheduling the jobs' component |
---|
443 | tasks on the slaves, monitoring them and re-executing the failed tasks. The |
---|
444 | slaves execute the tasks as directed by the master.</p> |
---|
445 | <p>Minimally, applications specify the input/output locations and supply |
---|
446 | <em>map</em> and <em>reduce</em> functions via implementations of |
---|
447 | appropriate interfaces and/or abstract-classes. These, and other job |
---|
448 | parameters, comprise the <em>job configuration</em>. The Hadoop |
---|
449 | <em>job client</em> then submits the job (jar/executable etc.) and |
---|
450 | configuration to the <span class="codefrag">JobTracker</span> which then assumes the |
---|
451 | responsibility of distributing the software/configuration to the slaves, |
---|
452 | scheduling tasks and monitoring them, providing status and diagnostic |
---|
453 | information to the job-client.</p> |
---|
454 | <p>Although the Hadoop framework is implemented in Java<sup>TM</sup>, |
---|
455 | Map/Reduce applications need not be written in Java.</p> |
---|
456 | <ul> |
---|
457 | |
---|
458 | <li> |
---|
459 | |
---|
460 | <a href="api/org/apache/hadoop/streaming/package-summary.html"> |
---|
461 | Hadoop Streaming</a> is a utility which allows users to create and run |
---|
462 | jobs with any executables (e.g. shell utilities) as the mapper and/or |
---|
463 | the reducer. |
---|
464 | </li> |
---|
465 | |
---|
466 | <li> |
---|
467 | |
---|
468 | <a href="api/org/apache/hadoop/mapred/pipes/package-summary.html"> |
---|
469 | Hadoop Pipes</a> is a <a href="http://www.swig.org/">SWIG</a>- |
---|
470 | compatible <em>C++ API</em> to implement Map/Reduce applications (non |
---|
471 | JNI<sup>TM</sup> based). |
---|
472 | </li> |
---|
473 | |
---|
474 | </ul> |
---|
475 | </div> |
---|
476 | |
---|
477 | |
---|
478 | <a name="N1008B"></a><a name="Inputs+and+Outputs"></a> |
---|
479 | <h2 class="h3">Inputs and Outputs</h2> |
---|
480 | <div class="section"> |
---|
481 | <p>The Map/Reduce framework operates exclusively on |
---|
482 | <span class="codefrag"><key, value></span> pairs, that is, the framework views the |
---|
483 | input to the job as a set of <span class="codefrag"><key, value></span> pairs and |
---|
484 | produces a set of <span class="codefrag"><key, value></span> pairs as the output of |
---|
485 | the job, conceivably of different types.</p> |
---|
486 | <p>The <span class="codefrag">key</span> and <span class="codefrag">value</span> classes have to be |
---|
487 | serializable by the framework and hence need to implement the |
---|
488 | <a href="api/org/apache/hadoop/io/Writable.html">Writable</a> |
---|
489 | interface. Additionally, the <span class="codefrag">key</span> classes have to implement the |
---|
490 | <a href="api/org/apache/hadoop/io/WritableComparable.html"> |
---|
491 | WritableComparable</a> interface to facilitate sorting by the framework. |
---|
492 | </p> |
---|
493 | <p>Input and Output types of a Map/Reduce job:</p> |
---|
494 | <p> |
---|
495 | (input) <span class="codefrag"><k1, v1></span> |
---|
496 | -> |
---|
497 | <strong>map</strong> |
---|
498 | -> |
---|
499 | <span class="codefrag"><k2, v2></span> |
---|
500 | -> |
---|
501 | <strong>combine</strong> |
---|
502 | -> |
---|
503 | <span class="codefrag"><k2, v2></span> |
---|
504 | -> |
---|
505 | <strong>reduce</strong> |
---|
506 | -> |
---|
507 | <span class="codefrag"><k3, v3></span> (output) |
---|
508 | </p> |
---|
509 | </div> |
---|
510 | |
---|
511 | |
---|
512 | <a name="N100CD"></a><a name="Example%3A+WordCount+v1.0"></a> |
---|
513 | <h2 class="h3">Example: WordCount v1.0</h2> |
---|
514 | <div class="section"> |
---|
515 | <p>Before we jump into the details, lets walk through an example Map/Reduce |
---|
516 | application to get a flavour for how they work.</p> |
---|
517 | <p> |
---|
518 | <span class="codefrag">WordCount</span> is a simple application that counts the number of |
---|
519 | occurences of each word in a given input set.</p> |
---|
520 | <p>This works with a local-standalone, pseudo-distributed or fully-distributed |
---|
521 | Hadoop installation(see <a href="quickstart.html"> Hadoop Quick Start</a>).</p> |
---|
522 | <a name="N100E2"></a><a name="Source+Code"></a> |
---|
523 | <h3 class="h4">Source Code</h3> |
---|
524 | <table class="ForrestTable" cellspacing="1" cellpadding="4"> |
---|
525 | |
---|
526 | <tr> |
---|
527 | |
---|
528 | <th colspan="1" rowspan="1"></th> |
---|
529 | <th colspan="1" rowspan="1">WordCount.java</th> |
---|
530 | |
---|
531 | </tr> |
---|
532 | |
---|
533 | <tr> |
---|
534 | |
---|
535 | <td colspan="1" rowspan="1">1.</td> |
---|
536 | <td colspan="1" rowspan="1"> |
---|
537 | <span class="codefrag">package org.myorg;</span> |
---|
538 | </td> |
---|
539 | |
---|
540 | </tr> |
---|
541 | |
---|
542 | <tr> |
---|
543 | |
---|
544 | <td colspan="1" rowspan="1">2.</td> |
---|
545 | <td colspan="1" rowspan="1"></td> |
---|
546 | |
---|
547 | </tr> |
---|
548 | |
---|
549 | <tr> |
---|
550 | |
---|
551 | <td colspan="1" rowspan="1">3.</td> |
---|
552 | <td colspan="1" rowspan="1"> |
---|
553 | <span class="codefrag">import java.io.IOException;</span> |
---|
554 | </td> |
---|
555 | |
---|
556 | </tr> |
---|
557 | |
---|
558 | <tr> |
---|
559 | |
---|
560 | <td colspan="1" rowspan="1">4.</td> |
---|
561 | <td colspan="1" rowspan="1"> |
---|
562 | <span class="codefrag">import java.util.*;</span> |
---|
563 | </td> |
---|
564 | |
---|
565 | </tr> |
---|
566 | |
---|
567 | <tr> |
---|
568 | |
---|
569 | <td colspan="1" rowspan="1">5.</td> |
---|
570 | <td colspan="1" rowspan="1"></td> |
---|
571 | |
---|
572 | </tr> |
---|
573 | |
---|
574 | <tr> |
---|
575 | |
---|
576 | <td colspan="1" rowspan="1">6.</td> |
---|
577 | <td colspan="1" rowspan="1"> |
---|
578 | <span class="codefrag">import org.apache.hadoop.fs.Path;</span> |
---|
579 | </td> |
---|
580 | |
---|
581 | </tr> |
---|
582 | |
---|
583 | <tr> |
---|
584 | |
---|
585 | <td colspan="1" rowspan="1">7.</td> |
---|
586 | <td colspan="1" rowspan="1"> |
---|
587 | <span class="codefrag">import org.apache.hadoop.conf.*;</span> |
---|
588 | </td> |
---|
589 | |
---|
590 | </tr> |
---|
591 | |
---|
592 | <tr> |
---|
593 | |
---|
594 | <td colspan="1" rowspan="1">8.</td> |
---|
595 | <td colspan="1" rowspan="1"> |
---|
596 | <span class="codefrag">import org.apache.hadoop.io.*;</span> |
---|
597 | </td> |
---|
598 | |
---|
599 | </tr> |
---|
600 | |
---|
601 | <tr> |
---|
602 | |
---|
603 | <td colspan="1" rowspan="1">9.</td> |
---|
604 | <td colspan="1" rowspan="1"> |
---|
605 | <span class="codefrag">import org.apache.hadoop.mapred.*;</span> |
---|
606 | </td> |
---|
607 | |
---|
608 | </tr> |
---|
609 | |
---|
610 | <tr> |
---|
611 | |
---|
612 | <td colspan="1" rowspan="1">10.</td> |
---|
613 | <td colspan="1" rowspan="1"> |
---|
614 | <span class="codefrag">import org.apache.hadoop.util.*;</span> |
---|
615 | </td> |
---|
616 | |
---|
617 | </tr> |
---|
618 | |
---|
619 | <tr> |
---|
620 | |
---|
621 | <td colspan="1" rowspan="1">11.</td> |
---|
622 | <td colspan="1" rowspan="1"></td> |
---|
623 | |
---|
624 | </tr> |
---|
625 | |
---|
626 | <tr> |
---|
627 | |
---|
628 | <td colspan="1" rowspan="1">12.</td> |
---|
629 | <td colspan="1" rowspan="1"> |
---|
630 | <span class="codefrag">public class WordCount {</span> |
---|
631 | </td> |
---|
632 | |
---|
633 | </tr> |
---|
634 | |
---|
635 | <tr> |
---|
636 | |
---|
637 | <td colspan="1" rowspan="1">13.</td> |
---|
638 | <td colspan="1" rowspan="1"></td> |
---|
639 | |
---|
640 | </tr> |
---|
641 | |
---|
642 | <tr> |
---|
643 | |
---|
644 | <td colspan="1" rowspan="1">14.</td> |
---|
645 | <td colspan="1" rowspan="1"> |
---|
646 | |
---|
647 | <span class="codefrag"> |
---|
648 | public static class Map extends MapReduceBase |
---|
649 | implements Mapper<LongWritable, Text, Text, IntWritable> { |
---|
650 | </span> |
---|
651 | </td> |
---|
652 | |
---|
653 | </tr> |
---|
654 | |
---|
655 | <tr> |
---|
656 | |
---|
657 | <td colspan="1" rowspan="1">15.</td> |
---|
658 | <td colspan="1" rowspan="1"> |
---|
659 | |
---|
660 | <span class="codefrag"> |
---|
661 | private final static IntWritable one = new IntWritable(1); |
---|
662 | </span> |
---|
663 | </td> |
---|
664 | |
---|
665 | </tr> |
---|
666 | |
---|
667 | <tr> |
---|
668 | |
---|
669 | <td colspan="1" rowspan="1">16.</td> |
---|
670 | <td colspan="1" rowspan="1"> |
---|
671 | |
---|
672 | <span class="codefrag">private Text word = new Text();</span> |
---|
673 | </td> |
---|
674 | |
---|
675 | </tr> |
---|
676 | |
---|
677 | <tr> |
---|
678 | |
---|
679 | <td colspan="1" rowspan="1">17.</td> |
---|
680 | <td colspan="1" rowspan="1"></td> |
---|
681 | |
---|
682 | </tr> |
---|
683 | |
---|
684 | <tr> |
---|
685 | |
---|
686 | <td colspan="1" rowspan="1">18.</td> |
---|
687 | <td colspan="1" rowspan="1"> |
---|
688 | |
---|
689 | <span class="codefrag"> |
---|
690 | public void map(LongWritable key, Text value, |
---|
691 | OutputCollector<Text, IntWritable> output, |
---|
692 | Reporter reporter) throws IOException { |
---|
693 | </span> |
---|
694 | </td> |
---|
695 | |
---|
696 | </tr> |
---|
697 | |
---|
698 | <tr> |
---|
699 | |
---|
700 | <td colspan="1" rowspan="1">19.</td> |
---|
701 | <td colspan="1" rowspan="1"> |
---|
702 | |
---|
703 | <span class="codefrag">String line = value.toString();</span> |
---|
704 | </td> |
---|
705 | |
---|
706 | </tr> |
---|
707 | |
---|
708 | <tr> |
---|
709 | |
---|
710 | <td colspan="1" rowspan="1">20.</td> |
---|
711 | <td colspan="1" rowspan="1"> |
---|
712 | |
---|
713 | <span class="codefrag">StringTokenizer tokenizer = new StringTokenizer(line);</span> |
---|
714 | </td> |
---|
715 | |
---|
716 | </tr> |
---|
717 | |
---|
718 | <tr> |
---|
719 | |
---|
720 | <td colspan="1" rowspan="1">21.</td> |
---|
721 | <td colspan="1" rowspan="1"> |
---|
722 | |
---|
723 | <span class="codefrag">while (tokenizer.hasMoreTokens()) {</span> |
---|
724 | </td> |
---|
725 | |
---|
726 | </tr> |
---|
727 | |
---|
728 | <tr> |
---|
729 | |
---|
730 | <td colspan="1" rowspan="1">22.</td> |
---|
731 | <td colspan="1" rowspan="1"> |
---|
732 | |
---|
733 | <span class="codefrag">word.set(tokenizer.nextToken());</span> |
---|
734 | </td> |
---|
735 | |
---|
736 | </tr> |
---|
737 | |
---|
738 | <tr> |
---|
739 | |
---|
740 | <td colspan="1" rowspan="1">23.</td> |
---|
741 | <td colspan="1" rowspan="1"> |
---|
742 | |
---|
743 | <span class="codefrag">output.collect(word, one);</span> |
---|
744 | </td> |
---|
745 | |
---|
746 | </tr> |
---|
747 | |
---|
748 | <tr> |
---|
749 | |
---|
750 | <td colspan="1" rowspan="1">24.</td> |
---|
751 | <td colspan="1" rowspan="1"> |
---|
752 | |
---|
753 | <span class="codefrag">}</span> |
---|
754 | </td> |
---|
755 | |
---|
756 | </tr> |
---|
757 | |
---|
758 | <tr> |
---|
759 | |
---|
760 | <td colspan="1" rowspan="1">25.</td> |
---|
761 | <td colspan="1" rowspan="1"> |
---|
762 | |
---|
763 | <span class="codefrag">}</span> |
---|
764 | </td> |
---|
765 | |
---|
766 | </tr> |
---|
767 | |
---|
768 | <tr> |
---|
769 | |
---|
770 | <td colspan="1" rowspan="1">26.</td> |
---|
771 | <td colspan="1" rowspan="1"> |
---|
772 | |
---|
773 | <span class="codefrag">}</span> |
---|
774 | </td> |
---|
775 | |
---|
776 | </tr> |
---|
777 | |
---|
778 | <tr> |
---|
779 | |
---|
780 | <td colspan="1" rowspan="1">27.</td> |
---|
781 | <td colspan="1" rowspan="1"></td> |
---|
782 | |
---|
783 | </tr> |
---|
784 | |
---|
785 | <tr> |
---|
786 | |
---|
787 | <td colspan="1" rowspan="1">28.</td> |
---|
788 | <td colspan="1" rowspan="1"> |
---|
789 | |
---|
790 | <span class="codefrag"> |
---|
791 | public static class Reduce extends MapReduceBase implements |
---|
792 | Reducer<Text, IntWritable, Text, IntWritable> { |
---|
793 | </span> |
---|
794 | </td> |
---|
795 | |
---|
796 | </tr> |
---|
797 | |
---|
798 | <tr> |
---|
799 | |
---|
800 | <td colspan="1" rowspan="1">29.</td> |
---|
801 | <td colspan="1" rowspan="1"> |
---|
802 | |
---|
803 | <span class="codefrag"> |
---|
804 | public void reduce(Text key, Iterator<IntWritable> values, |
---|
805 | OutputCollector<Text, IntWritable> output, |
---|
806 | Reporter reporter) throws IOException { |
---|
807 | </span> |
---|
808 | </td> |
---|
809 | |
---|
810 | </tr> |
---|
811 | |
---|
812 | <tr> |
---|
813 | |
---|
814 | <td colspan="1" rowspan="1">30.</td> |
---|
815 | <td colspan="1" rowspan="1"> |
---|
816 | |
---|
817 | <span class="codefrag">int sum = 0;</span> |
---|
818 | </td> |
---|
819 | |
---|
820 | </tr> |
---|
821 | |
---|
822 | <tr> |
---|
823 | |
---|
824 | <td colspan="1" rowspan="1">31.</td> |
---|
825 | <td colspan="1" rowspan="1"> |
---|
826 | |
---|
827 | <span class="codefrag">while (values.hasNext()) {</span> |
---|
828 | </td> |
---|
829 | |
---|
830 | </tr> |
---|
831 | |
---|
832 | <tr> |
---|
833 | |
---|
834 | <td colspan="1" rowspan="1">32.</td> |
---|
835 | <td colspan="1" rowspan="1"> |
---|
836 | |
---|
837 | <span class="codefrag">sum += values.next().get();</span> |
---|
838 | </td> |
---|
839 | |
---|
840 | </tr> |
---|
841 | |
---|
842 | <tr> |
---|
843 | |
---|
844 | <td colspan="1" rowspan="1">33.</td> |
---|
845 | <td colspan="1" rowspan="1"> |
---|
846 | |
---|
847 | <span class="codefrag">}</span> |
---|
848 | </td> |
---|
849 | |
---|
850 | </tr> |
---|
851 | |
---|
852 | <tr> |
---|
853 | |
---|
854 | <td colspan="1" rowspan="1">34.</td> |
---|
855 | <td colspan="1" rowspan="1"> |
---|
856 | |
---|
857 | <span class="codefrag">output.collect(key, new IntWritable(sum));</span> |
---|
858 | </td> |
---|
859 | |
---|
860 | </tr> |
---|
861 | |
---|
862 | <tr> |
---|
863 | |
---|
864 | <td colspan="1" rowspan="1">35.</td> |
---|
865 | <td colspan="1" rowspan="1"> |
---|
866 | |
---|
867 | <span class="codefrag">}</span> |
---|
868 | </td> |
---|
869 | |
---|
870 | </tr> |
---|
871 | |
---|
872 | <tr> |
---|
873 | |
---|
874 | <td colspan="1" rowspan="1">36.</td> |
---|
875 | <td colspan="1" rowspan="1"> |
---|
876 | |
---|
877 | <span class="codefrag">}</span> |
---|
878 | </td> |
---|
879 | |
---|
880 | </tr> |
---|
881 | |
---|
882 | <tr> |
---|
883 | |
---|
884 | <td colspan="1" rowspan="1">37.</td> |
---|
885 | <td colspan="1" rowspan="1"></td> |
---|
886 | |
---|
887 | </tr> |
---|
888 | |
---|
889 | <tr> |
---|
890 | |
---|
891 | <td colspan="1" rowspan="1">38.</td> |
---|
892 | <td colspan="1" rowspan="1"> |
---|
893 | |
---|
894 | <span class="codefrag"> |
---|
895 | public static void main(String[] args) throws Exception { |
---|
896 | </span> |
---|
897 | </td> |
---|
898 | |
---|
899 | </tr> |
---|
900 | |
---|
901 | <tr> |
---|
902 | |
---|
903 | <td colspan="1" rowspan="1">39.</td> |
---|
904 | <td colspan="1" rowspan="1"> |
---|
905 | |
---|
906 | <span class="codefrag"> |
---|
907 | JobConf conf = new JobConf(WordCount.class); |
---|
908 | </span> |
---|
909 | </td> |
---|
910 | |
---|
911 | </tr> |
---|
912 | |
---|
913 | <tr> |
---|
914 | |
---|
915 | <td colspan="1" rowspan="1">40.</td> |
---|
916 | <td colspan="1" rowspan="1"> |
---|
917 | |
---|
918 | <span class="codefrag">conf.setJobName("wordcount");</span> |
---|
919 | </td> |
---|
920 | |
---|
921 | </tr> |
---|
922 | |
---|
923 | <tr> |
---|
924 | |
---|
925 | <td colspan="1" rowspan="1">41.</td> |
---|
926 | <td colspan="1" rowspan="1"></td> |
---|
927 | |
---|
928 | </tr> |
---|
929 | |
---|
930 | <tr> |
---|
931 | |
---|
932 | <td colspan="1" rowspan="1">42.</td> |
---|
933 | <td colspan="1" rowspan="1"> |
---|
934 | |
---|
935 | <span class="codefrag">conf.setOutputKeyClass(Text.class);</span> |
---|
936 | </td> |
---|
937 | |
---|
938 | </tr> |
---|
939 | |
---|
940 | <tr> |
---|
941 | |
---|
942 | <td colspan="1" rowspan="1">43.</td> |
---|
943 | <td colspan="1" rowspan="1"> |
---|
944 | |
---|
945 | <span class="codefrag">conf.setOutputValueClass(IntWritable.class);</span> |
---|
946 | </td> |
---|
947 | |
---|
948 | </tr> |
---|
949 | |
---|
950 | <tr> |
---|
951 | |
---|
952 | <td colspan="1" rowspan="1">44.</td> |
---|
953 | <td colspan="1" rowspan="1"></td> |
---|
954 | |
---|
955 | </tr> |
---|
956 | |
---|
957 | <tr> |
---|
958 | |
---|
959 | <td colspan="1" rowspan="1">45.</td> |
---|
960 | <td colspan="1" rowspan="1"> |
---|
961 | |
---|
962 | <span class="codefrag">conf.setMapperClass(Map.class);</span> |
---|
963 | </td> |
---|
964 | |
---|
965 | </tr> |
---|
966 | |
---|
967 | <tr> |
---|
968 | |
---|
969 | <td colspan="1" rowspan="1">46.</td> |
---|
970 | <td colspan="1" rowspan="1"> |
---|
971 | |
---|
972 | <span class="codefrag">conf.setCombinerClass(Reduce.class);</span> |
---|
973 | </td> |
---|
974 | |
---|
975 | </tr> |
---|
976 | |
---|
977 | <tr> |
---|
978 | |
---|
979 | <td colspan="1" rowspan="1">47.</td> |
---|
980 | <td colspan="1" rowspan="1"> |
---|
981 | |
---|
982 | <span class="codefrag">conf.setReducerClass(Reduce.class);</span> |
---|
983 | </td> |
---|
984 | |
---|
985 | </tr> |
---|
986 | |
---|
987 | <tr> |
---|
988 | |
---|
989 | <td colspan="1" rowspan="1">48.</td> |
---|
990 | <td colspan="1" rowspan="1"></td> |
---|
991 | |
---|
992 | </tr> |
---|
993 | |
---|
994 | <tr> |
---|
995 | |
---|
996 | <td colspan="1" rowspan="1">49.</td> |
---|
997 | <td colspan="1" rowspan="1"> |
---|
998 | |
---|
999 | <span class="codefrag">conf.setInputFormat(TextInputFormat.class);</span> |
---|
1000 | </td> |
---|
1001 | |
---|
1002 | </tr> |
---|
1003 | |
---|
1004 | <tr> |
---|
1005 | |
---|
1006 | <td colspan="1" rowspan="1">50.</td> |
---|
1007 | <td colspan="1" rowspan="1"> |
---|
1008 | |
---|
1009 | <span class="codefrag">conf.setOutputFormat(TextOutputFormat.class);</span> |
---|
1010 | </td> |
---|
1011 | |
---|
1012 | </tr> |
---|
1013 | |
---|
1014 | <tr> |
---|
1015 | |
---|
1016 | <td colspan="1" rowspan="1">51.</td> |
---|
1017 | <td colspan="1" rowspan="1"></td> |
---|
1018 | |
---|
1019 | </tr> |
---|
1020 | |
---|
1021 | <tr> |
---|
1022 | |
---|
1023 | <td colspan="1" rowspan="1">52.</td> |
---|
1024 | <td colspan="1" rowspan="1"> |
---|
1025 | |
---|
1026 | <span class="codefrag">FileInputFormat.setInputPaths(conf, new Path(args[0]));</span> |
---|
1027 | </td> |
---|
1028 | |
---|
1029 | </tr> |
---|
1030 | |
---|
1031 | <tr> |
---|
1032 | |
---|
1033 | <td colspan="1" rowspan="1">53.</td> |
---|
1034 | <td colspan="1" rowspan="1"> |
---|
1035 | |
---|
1036 | <span class="codefrag">FileOutputFormat.setOutputPath(conf, new Path(args[1]));</span> |
---|
1037 | </td> |
---|
1038 | |
---|
1039 | </tr> |
---|
1040 | |
---|
1041 | <tr> |
---|
1042 | |
---|
1043 | <td colspan="1" rowspan="1">54.</td> |
---|
1044 | <td colspan="1" rowspan="1"></td> |
---|
1045 | |
---|
1046 | </tr> |
---|
1047 | |
---|
1048 | <tr> |
---|
1049 | |
---|
1050 | <td colspan="1" rowspan="1">55.</td> |
---|
1051 | <td colspan="1" rowspan="1"> |
---|
1052 | |
---|
1053 | <span class="codefrag">JobClient.runJob(conf);</span> |
---|
1054 | </td> |
---|
1055 | |
---|
1056 | </tr> |
---|
1057 | |
---|
1058 | <tr> |
---|
1059 | |
---|
1060 | <td colspan="1" rowspan="1">57.</td> |
---|
1061 | <td colspan="1" rowspan="1"> |
---|
1062 | |
---|
1063 | <span class="codefrag">}</span> |
---|
1064 | </td> |
---|
1065 | |
---|
1066 | </tr> |
---|
1067 | |
---|
1068 | <tr> |
---|
1069 | |
---|
1070 | <td colspan="1" rowspan="1">58.</td> |
---|
1071 | <td colspan="1" rowspan="1"> |
---|
1072 | <span class="codefrag">}</span> |
---|
1073 | </td> |
---|
1074 | |
---|
1075 | </tr> |
---|
1076 | |
---|
1077 | <tr> |
---|
1078 | |
---|
1079 | <td colspan="1" rowspan="1">59.</td> |
---|
1080 | <td colspan="1" rowspan="1"></td> |
---|
1081 | |
---|
1082 | </tr> |
---|
1083 | |
---|
1084 | </table> |
---|
1085 | <a name="N10464"></a><a name="Usage"></a> |
---|
1086 | <h3 class="h4">Usage</h3> |
---|
1087 | <p>Assuming <span class="codefrag">HADOOP_HOME</span> is the root of the installation and |
---|
1088 | <span class="codefrag">HADOOP_VERSION</span> is the Hadoop version installed, compile |
---|
1089 | <span class="codefrag">WordCount.java</span> and create a jar:</p> |
---|
1090 | <p> |
---|
1091 | |
---|
1092 | <span class="codefrag">$ mkdir wordcount_classes</span> |
---|
1093 | <br> |
---|
1094 | |
---|
1095 | <span class="codefrag"> |
---|
1096 | $ javac -classpath ${HADOOP_HOME}/hadoop-${HADOOP_VERSION}-core.jar |
---|
1097 | -d wordcount_classes WordCount.java |
---|
1098 | </span> |
---|
1099 | <br> |
---|
1100 | |
---|
1101 | <span class="codefrag">$ jar -cvf /usr/joe/wordcount.jar -C wordcount_classes/ .</span> |
---|
1102 | |
---|
1103 | </p> |
---|
1104 | <p>Assuming that:</p> |
---|
1105 | <ul> |
---|
1106 | |
---|
1107 | <li> |
---|
1108 | |
---|
1109 | <span class="codefrag">/usr/joe/wordcount/input</span> - input directory in HDFS |
---|
1110 | </li> |
---|
1111 | |
---|
1112 | <li> |
---|
1113 | |
---|
1114 | <span class="codefrag">/usr/joe/wordcount/output</span> - output directory in HDFS |
---|
1115 | </li> |
---|
1116 | |
---|
1117 | </ul> |
---|
1118 | <p>Sample text-files as input:</p> |
---|
1119 | <p> |
---|
1120 | |
---|
1121 | <span class="codefrag">$ bin/hadoop dfs -ls /usr/joe/wordcount/input/</span> |
---|
1122 | <br> |
---|
1123 | |
---|
1124 | <span class="codefrag">/usr/joe/wordcount/input/file01</span> |
---|
1125 | <br> |
---|
1126 | |
---|
1127 | <span class="codefrag">/usr/joe/wordcount/input/file02</span> |
---|
1128 | <br> |
---|
1129 | |
---|
1130 | <br> |
---|
1131 | |
---|
1132 | <span class="codefrag">$ bin/hadoop dfs -cat /usr/joe/wordcount/input/file01</span> |
---|
1133 | <br> |
---|
1134 | |
---|
1135 | <span class="codefrag">Hello World Bye World</span> |
---|
1136 | <br> |
---|
1137 | |
---|
1138 | <br> |
---|
1139 | |
---|
1140 | <span class="codefrag">$ bin/hadoop dfs -cat /usr/joe/wordcount/input/file02</span> |
---|
1141 | <br> |
---|
1142 | |
---|
1143 | <span class="codefrag">Hello Hadoop Goodbye Hadoop</span> |
---|
1144 | |
---|
1145 | </p> |
---|
1146 | <p>Run the application:</p> |
---|
1147 | <p> |
---|
1148 | |
---|
1149 | <span class="codefrag"> |
---|
1150 | $ bin/hadoop jar /usr/joe/wordcount.jar org.myorg.WordCount |
---|
1151 | /usr/joe/wordcount/input /usr/joe/wordcount/output |
---|
1152 | </span> |
---|
1153 | |
---|
1154 | </p> |
---|
1155 | <p>Output:</p> |
---|
1156 | <p> |
---|
1157 | |
---|
1158 | <span class="codefrag"> |
---|
1159 | $ bin/hadoop dfs -cat /usr/joe/wordcount/output/part-00000 |
---|
1160 | </span> |
---|
1161 | |
---|
1162 | <br> |
---|
1163 | |
---|
1164 | <span class="codefrag">Bye 1</span> |
---|
1165 | <br> |
---|
1166 | |
---|
1167 | <span class="codefrag">Goodbye 1</span> |
---|
1168 | <br> |
---|
1169 | |
---|
1170 | <span class="codefrag">Hadoop 2</span> |
---|
1171 | <br> |
---|
1172 | |
---|
1173 | <span class="codefrag">Hello 2</span> |
---|
1174 | <br> |
---|
1175 | |
---|
1176 | <span class="codefrag">World 2</span> |
---|
1177 | <br> |
---|
1178 | |
---|
1179 | </p> |
---|
1180 | <p> Applications can specify a comma separated list of paths which |
---|
1181 | would be present in the current working directory of the task |
---|
1182 | using the option <span class="codefrag">-files</span>. The <span class="codefrag">-libjars</span> |
---|
1183 | option allows applications to add jars to the classpaths of the maps |
---|
1184 | and reduces. The <span class="codefrag">-archives</span> allows them to pass archives |
---|
1185 | as arguments that are unzipped/unjarred and a link with name of the |
---|
1186 | jar/zip are created in the current working directory of tasks. More |
---|
1187 | details about the command line options are available at |
---|
1188 | <a href="commands_manual.html"> Hadoop Command Guide.</a> |
---|
1189 | </p> |
---|
1190 | <p>Running <span class="codefrag">wordcount</span> example with |
---|
1191 | <span class="codefrag">-libjars</span> and <span class="codefrag">-files</span>:<br> |
---|
1192 | |
---|
1193 | <span class="codefrag"> hadoop jar hadoop-examples.jar wordcount -files cachefile.txt |
---|
1194 | -libjars mylib.jar input output </span> |
---|
1195 | |
---|
1196 | </p> |
---|
1197 | <a name="N10504"></a><a name="Walk-through"></a> |
---|
1198 | <h3 class="h4">Walk-through</h3> |
---|
1199 | <p>The <span class="codefrag">WordCount</span> application is quite straight-forward.</p> |
---|
1200 | <p>The <span class="codefrag">Mapper</span> implementation (lines 14-26), via the |
---|
1201 | <span class="codefrag">map</span> method (lines 18-25), processes one line at a time, |
---|
1202 | as provided by the specified <span class="codefrag">TextInputFormat</span> (line 49). |
---|
1203 | It then splits the line into tokens separated by whitespaces, via the |
---|
1204 | <span class="codefrag">StringTokenizer</span>, and emits a key-value pair of |
---|
1205 | <span class="codefrag">< <word>, 1></span>.</p> |
---|
1206 | <p> |
---|
1207 | For the given sample input the first map emits:<br> |
---|
1208 | |
---|
1209 | <span class="codefrag">< Hello, 1></span> |
---|
1210 | <br> |
---|
1211 | |
---|
1212 | <span class="codefrag">< World, 1></span> |
---|
1213 | <br> |
---|
1214 | |
---|
1215 | <span class="codefrag">< Bye, 1></span> |
---|
1216 | <br> |
---|
1217 | |
---|
1218 | <span class="codefrag">< World, 1></span> |
---|
1219 | <br> |
---|
1220 | |
---|
1221 | </p> |
---|
1222 | <p> |
---|
1223 | The second map emits:<br> |
---|
1224 | |
---|
1225 | <span class="codefrag">< Hello, 1></span> |
---|
1226 | <br> |
---|
1227 | |
---|
1228 | <span class="codefrag">< Hadoop, 1></span> |
---|
1229 | <br> |
---|
1230 | |
---|
1231 | <span class="codefrag">< Goodbye, 1></span> |
---|
1232 | <br> |
---|
1233 | |
---|
1234 | <span class="codefrag">< Hadoop, 1></span> |
---|
1235 | <br> |
---|
1236 | |
---|
1237 | </p> |
---|
1238 | <p>We'll learn more about the number of maps spawned for a given job, and |
---|
1239 | how to control them in a fine-grained manner, a bit later in the |
---|
1240 | tutorial.</p> |
---|
1241 | <p> |
---|
1242 | <span class="codefrag">WordCount</span> also specifies a <span class="codefrag">combiner</span> (line |
---|
1243 | 46). Hence, the output of each map is passed through the local combiner |
---|
1244 | (which is same as the <span class="codefrag">Reducer</span> as per the job |
---|
1245 | configuration) for local aggregation, after being sorted on the |
---|
1246 | <em>key</em>s.</p> |
---|
1247 | <p> |
---|
1248 | The output of the first map:<br> |
---|
1249 | |
---|
1250 | <span class="codefrag">< Bye, 1></span> |
---|
1251 | <br> |
---|
1252 | |
---|
1253 | <span class="codefrag">< Hello, 1></span> |
---|
1254 | <br> |
---|
1255 | |
---|
1256 | <span class="codefrag">< World, 2></span> |
---|
1257 | <br> |
---|
1258 | |
---|
1259 | </p> |
---|
1260 | <p> |
---|
1261 | The output of the second map:<br> |
---|
1262 | |
---|
1263 | <span class="codefrag">< Goodbye, 1></span> |
---|
1264 | <br> |
---|
1265 | |
---|
1266 | <span class="codefrag">< Hadoop, 2></span> |
---|
1267 | <br> |
---|
1268 | |
---|
1269 | <span class="codefrag">< Hello, 1></span> |
---|
1270 | <br> |
---|
1271 | |
---|
1272 | </p> |
---|
1273 | <p>The <span class="codefrag">Reducer</span> implementation (lines 28-36), via the |
---|
1274 | <span class="codefrag">reduce</span> method (lines 29-35) just sums up the values, |
---|
1275 | which are the occurence counts for each key (i.e. words in this example). |
---|
1276 | </p> |
---|
1277 | <p> |
---|
1278 | Thus the output of the job is:<br> |
---|
1279 | |
---|
1280 | <span class="codefrag">< Bye, 1></span> |
---|
1281 | <br> |
---|
1282 | |
---|
1283 | <span class="codefrag">< Goodbye, 1></span> |
---|
1284 | <br> |
---|
1285 | |
---|
1286 | <span class="codefrag">< Hadoop, 2></span> |
---|
1287 | <br> |
---|
1288 | |
---|
1289 | <span class="codefrag">< Hello, 2></span> |
---|
1290 | <br> |
---|
1291 | |
---|
1292 | <span class="codefrag">< World, 2></span> |
---|
1293 | <br> |
---|
1294 | |
---|
1295 | </p> |
---|
1296 | <p>The <span class="codefrag">run</span> method specifies various facets of the job, such |
---|
1297 | as the input/output paths (passed via the command line), key/value |
---|
1298 | types, input/output formats etc., in the <span class="codefrag">JobConf</span>. |
---|
1299 | It then calls the <span class="codefrag">JobClient.runJob</span> (line 55) to submit the |
---|
1300 | and monitor its progress.</p> |
---|
1301 | <p>We'll learn more about <span class="codefrag">JobConf</span>, <span class="codefrag">JobClient</span>, |
---|
1302 | <span class="codefrag">Tool</span> and other interfaces and classes a bit later in the |
---|
1303 | tutorial.</p> |
---|
1304 | </div> |
---|
1305 | |
---|
1306 | |
---|
1307 | <a name="N105BB"></a><a name="Map%2FReduce+-+User+Interfaces"></a> |
---|
1308 | <h2 class="h3">Map/Reduce - User Interfaces</h2> |
---|
1309 | <div class="section"> |
---|
1310 | <p>This section provides a reasonable amount of detail on every user-facing |
---|
1311 | aspect of the Map/Reduce framwork. This should help users implement, |
---|
1312 | configure and tune their jobs in a fine-grained manner. However, please |
---|
1313 | note that the javadoc for each class/interface remains the most |
---|
1314 | comprehensive documentation available; this is only meant to be a tutorial. |
---|
1315 | </p> |
---|
1316 | <p>Let us first take the <span class="codefrag">Mapper</span> and <span class="codefrag">Reducer</span> |
---|
1317 | interfaces. Applications typically implement them to provide the |
---|
1318 | <span class="codefrag">map</span> and <span class="codefrag">reduce</span> methods.</p> |
---|
1319 | <p>We will then discuss other core interfaces including |
---|
1320 | <span class="codefrag">JobConf</span>, <span class="codefrag">JobClient</span>, <span class="codefrag">Partitioner</span>, |
---|
1321 | <span class="codefrag">OutputCollector</span>, <span class="codefrag">Reporter</span>, |
---|
1322 | <span class="codefrag">InputFormat</span>, <span class="codefrag">OutputFormat</span>, |
---|
1323 | <span class="codefrag">OutputCommitter</span> and others.</p> |
---|
1324 | <p>Finally, we will wrap up by discussing some useful features of the |
---|
1325 | framework such as the <span class="codefrag">DistributedCache</span>, |
---|
1326 | <span class="codefrag">IsolationRunner</span> etc.</p> |
---|
1327 | <a name="N105F7"></a><a name="Payload"></a> |
---|
1328 | <h3 class="h4">Payload</h3> |
---|
1329 | <p>Applications typically implement the <span class="codefrag">Mapper</span> and |
---|
1330 | <span class="codefrag">Reducer</span> interfaces to provide the <span class="codefrag">map</span> and |
---|
1331 | <span class="codefrag">reduce</span> methods. These form the core of the job.</p> |
---|
1332 | <a name="N1060C"></a><a name="Mapper"></a> |
---|
1333 | <h4>Mapper</h4> |
---|
1334 | <p> |
---|
1335 | <a href="api/org/apache/hadoop/mapred/Mapper.html"> |
---|
1336 | Mapper</a> maps input key/value pairs to a set of intermediate |
---|
1337 | key/value pairs.</p> |
---|
1338 | <p>Maps are the individual tasks that transform input records into |
---|
1339 | intermediate records. The transformed intermediate records do not need |
---|
1340 | to be of the same type as the input records. A given input pair may |
---|
1341 | map to zero or many output pairs.</p> |
---|
1342 | <p>The Hadoop Map/Reduce framework spawns one map task for each |
---|
1343 | <span class="codefrag">InputSplit</span> generated by the <span class="codefrag">InputFormat</span> for |
---|
1344 | the job.</p> |
---|
1345 | <p>Overall, <span class="codefrag">Mapper</span> implementations are passed the |
---|
1346 | <span class="codefrag">JobConf</span> for the job via the |
---|
1347 | <a href="api/org/apache/hadoop/mapred/JobConfigurable.html#configure(org.apache.hadoop.mapred.JobConf)"> |
---|
1348 | JobConfigurable.configure(JobConf)</a> method and override it to |
---|
1349 | initialize themselves. The framework then calls |
---|
1350 | <a href="api/org/apache/hadoop/mapred/Mapper.html#map(K1, V1, org.apache.hadoop.mapred.OutputCollector, org.apache.hadoop.mapred.Reporter)"> |
---|
1351 | map(WritableComparable, Writable, OutputCollector, Reporter)</a> for |
---|
1352 | each key/value pair in the <span class="codefrag">InputSplit</span> for that task. |
---|
1353 | Applications can then override the |
---|
1354 | <a href="api/org/apache/hadoop/io/Closeable.html#close()"> |
---|
1355 | Closeable.close()</a> method to perform any required cleanup.</p> |
---|
1356 | <p>Output pairs do not need to be of the same types as input pairs. A |
---|
1357 | given input pair may map to zero or many output pairs. Output pairs |
---|
1358 | are collected with calls to |
---|
1359 | <a href="api/org/apache/hadoop/mapred/OutputCollector.html#collect(K, V)"> |
---|
1360 | OutputCollector.collect(WritableComparable,Writable)</a>.</p> |
---|
1361 | <p>Applications can use the <span class="codefrag">Reporter</span> to report |
---|
1362 | progress, set application-level status messages and update |
---|
1363 | <span class="codefrag">Counters</span>, or just indicate that they are alive.</p> |
---|
1364 | <p>All intermediate values associated with a given output key are |
---|
1365 | subsequently grouped by the framework, and passed to the |
---|
1366 | <span class="codefrag">Reducer</span>(s) to determine the final output. Users can |
---|
1367 | control the grouping by specifying a <span class="codefrag">Comparator</span> via |
---|
1368 | <a href="api/org/apache/hadoop/mapred/JobConf.html#setOutputKeyComparatorClass(java.lang.Class)"> |
---|
1369 | JobConf.setOutputKeyComparatorClass(Class)</a>.</p> |
---|
1370 | <p>The <span class="codefrag">Mapper</span> outputs are sorted and then |
---|
1371 | partitioned per <span class="codefrag">Reducer</span>. The total number of partitions is |
---|
1372 | the same as the number of reduce tasks for the job. Users can control |
---|
1373 | which keys (and hence records) go to which <span class="codefrag">Reducer</span> by |
---|
1374 | implementing a custom <span class="codefrag">Partitioner</span>.</p> |
---|
1375 | <p>Users can optionally specify a <span class="codefrag">combiner</span>, via |
---|
1376 | <a href="api/org/apache/hadoop/mapred/JobConf.html#setCombinerClass(java.lang.Class)"> |
---|
1377 | JobConf.setCombinerClass(Class)</a>, to perform local aggregation of |
---|
1378 | the intermediate outputs, which helps to cut down the amount of data |
---|
1379 | transferred from the <span class="codefrag">Mapper</span> to the <span class="codefrag">Reducer</span>. |
---|
1380 | </p> |
---|
1381 | <p>The intermediate, sorted outputs are always stored in a simple |
---|
1382 | (key-len, key, value-len, value) format. |
---|
1383 | Applications can control if, and how, the |
---|
1384 | intermediate outputs are to be compressed and the |
---|
1385 | <a href="api/org/apache/hadoop/io/compress/CompressionCodec.html"> |
---|
1386 | CompressionCodec</a> to be used via the <span class="codefrag">JobConf</span>. |
---|
1387 | </p> |
---|
1388 | <a name="N10682"></a><a name="How+Many+Maps%3F"></a> |
---|
1389 | <h5>How Many Maps?</h5> |
---|
1390 | <p>The number of maps is usually driven by the total size of the |
---|
1391 | inputs, that is, the total number of blocks of the input files.</p> |
---|
1392 | <p>The right level of parallelism for maps seems to be around 10-100 |
---|
1393 | maps per-node, although it has been set up to 300 maps for very |
---|
1394 | cpu-light map tasks. Task setup takes awhile, so it is best if the |
---|
1395 | maps take at least a minute to execute.</p> |
---|
1396 | <p>Thus, if you expect 10TB of input data and have a blocksize of |
---|
1397 | <span class="codefrag">128MB</span>, you'll end up with 82,000 maps, unless |
---|
1398 | <a href="api/org/apache/hadoop/mapred/JobConf.html#setNumMapTasks(int)"> |
---|
1399 | setNumMapTasks(int)</a> (which only provides a hint to the framework) |
---|
1400 | is used to set it even higher.</p> |
---|
1401 | <a name="N1069A"></a><a name="Reducer"></a> |
---|
1402 | <h4>Reducer</h4> |
---|
1403 | <p> |
---|
1404 | <a href="api/org/apache/hadoop/mapred/Reducer.html"> |
---|
1405 | Reducer</a> reduces a set of intermediate values which share a key to |
---|
1406 | a smaller set of values.</p> |
---|
1407 | <p>The number of reduces for the job is set by the user |
---|
1408 | via <a href="api/org/apache/hadoop/mapred/JobConf.html#setNumReduceTasks(int)"> |
---|
1409 | JobConf.setNumReduceTasks(int)</a>.</p> |
---|
1410 | <p>Overall, <span class="codefrag">Reducer</span> implementations are passed the |
---|
1411 | <span class="codefrag">JobConf</span> for the job via the |
---|
1412 | <a href="api/org/apache/hadoop/mapred/JobConfigurable.html#configure(org.apache.hadoop.mapred.JobConf)"> |
---|
1413 | JobConfigurable.configure(JobConf)</a> method and can override it to |
---|
1414 | initialize themselves. The framework then calls |
---|
1415 | <a href="api/org/apache/hadoop/mapred/Reducer.html#reduce(K2, java.util.Iterator, org.apache.hadoop.mapred.OutputCollector, org.apache.hadoop.mapred.Reporter)"> |
---|
1416 | reduce(WritableComparable, Iterator, OutputCollector, Reporter)</a> |
---|
1417 | method for each <span class="codefrag"><key, (list of values)></span> |
---|
1418 | pair in the grouped inputs. Applications can then override the |
---|
1419 | <a href="api/org/apache/hadoop/io/Closeable.html#close()"> |
---|
1420 | Closeable.close()</a> method to perform any required cleanup.</p> |
---|
1421 | <p> |
---|
1422 | <span class="codefrag">Reducer</span> has 3 primary phases: shuffle, sort and reduce. |
---|
1423 | </p> |
---|
1424 | <a name="N106CA"></a><a name="Shuffle"></a> |
---|
1425 | <h5>Shuffle</h5> |
---|
1426 | <p>Input to the <span class="codefrag">Reducer</span> is the sorted output of the |
---|
1427 | mappers. In this phase the framework fetches the relevant partition |
---|
1428 | of the output of all the mappers, via HTTP.</p> |
---|
1429 | <a name="N106D7"></a><a name="Sort"></a> |
---|
1430 | <h5>Sort</h5> |
---|
1431 | <p>The framework groups <span class="codefrag">Reducer</span> inputs by keys (since |
---|
1432 | different mappers may have output the same key) in this stage.</p> |
---|
1433 | <p>The shuffle and sort phases occur simultaneously; while |
---|
1434 | map-outputs are being fetched they are merged.</p> |
---|
1435 | <a name="N106E6"></a><a name="Secondary+Sort"></a> |
---|
1436 | <h5>Secondary Sort</h5> |
---|
1437 | <p>If equivalence rules for grouping the intermediate keys are |
---|
1438 | required to be different from those for grouping keys before |
---|
1439 | reduction, then one may specify a <span class="codefrag">Comparator</span> via |
---|
1440 | <a href="api/org/apache/hadoop/mapred/JobConf.html#setOutputValueGroupingComparator(java.lang.Class)"> |
---|
1441 | JobConf.setOutputValueGroupingComparator(Class)</a>. Since |
---|
1442 | <a href="api/org/apache/hadoop/mapred/JobConf.html#setOutputKeyComparatorClass(java.lang.Class)"> |
---|
1443 | JobConf.setOutputKeyComparatorClass(Class)</a> can be used to |
---|
1444 | control how intermediate keys are grouped, these can be used in |
---|
1445 | conjunction to simulate <em>secondary sort on values</em>.</p> |
---|
1446 | <a name="N106FF"></a><a name="Reduce"></a> |
---|
1447 | <h5>Reduce</h5> |
---|
1448 | <p>In this phase the |
---|
1449 | <a href="api/org/apache/hadoop/mapred/Reducer.html#reduce(K2, java.util.Iterator, org.apache.hadoop.mapred.OutputCollector, org.apache.hadoop.mapred.Reporter)"> |
---|
1450 | reduce(WritableComparable, Iterator, OutputCollector, Reporter)</a> |
---|
1451 | method is called for each <span class="codefrag"><key, (list of values)></span> |
---|
1452 | pair in the grouped inputs.</p> |
---|
1453 | <p>The output of the reduce task is typically written to the |
---|
1454 | <a href="api/org/apache/hadoop/fs/FileSystem.html"> |
---|
1455 | FileSystem</a> via |
---|
1456 | <a href="api/org/apache/hadoop/mapred/OutputCollector.html#collect(K, V)"> |
---|
1457 | OutputCollector.collect(WritableComparable, Writable)</a>.</p> |
---|
1458 | <p>Applications can use the <span class="codefrag">Reporter</span> to report |
---|
1459 | progress, set application-level status messages and update |
---|
1460 | <span class="codefrag">Counters</span>, or just indicate that they are alive.</p> |
---|
1461 | <p>The output of the <span class="codefrag">Reducer</span> is <em>not sorted</em>.</p> |
---|
1462 | <a name="N1072D"></a><a name="How+Many+Reduces%3F"></a> |
---|
1463 | <h5>How Many Reduces?</h5> |
---|
1464 | <p>The right number of reduces seems to be <span class="codefrag">0.95</span> or |
---|
1465 | <span class="codefrag">1.75</span> multiplied by (<<em>no. of nodes</em>> * |
---|
1466 | <span class="codefrag">mapred.tasktracker.reduce.tasks.maximum</span>).</p> |
---|
1467 | <p>With <span class="codefrag">0.95</span> all of the reduces can launch immediately |
---|
1468 | and start transfering map outputs as the maps finish. With |
---|
1469 | <span class="codefrag">1.75</span> the faster nodes will finish their first round of |
---|
1470 | reduces and launch a second wave of reduces doing a much better job |
---|
1471 | of load balancing.</p> |
---|
1472 | <p>Increasing the number of reduces increases the framework overhead, |
---|
1473 | but increases load balancing and lowers the cost of failures.</p> |
---|
1474 | <p>The scaling factors above are slightly less than whole numbers to |
---|
1475 | reserve a few reduce slots in the framework for speculative-tasks and |
---|
1476 | failed tasks.</p> |
---|
1477 | <a name="N10752"></a><a name="Reducer+NONE"></a> |
---|
1478 | <h5>Reducer NONE</h5> |
---|
1479 | <p>It is legal to set the number of reduce-tasks to <em>zero</em> if |
---|
1480 | no reduction is desired.</p> |
---|
1481 | <p>In this case the outputs of the map-tasks go directly to the |
---|
1482 | <span class="codefrag">FileSystem</span>, into the output path set by |
---|
1483 | <a href="api/org/apache/hadoop/mapred/FileOutputFormat.html#setOutputPath(org.apache.hadoop.mapred.JobConf,%20org.apache.hadoop.fs.Path)"> |
---|
1484 | setOutputPath(Path)</a>. The framework does not sort the |
---|
1485 | map-outputs before writing them out to the <span class="codefrag">FileSystem</span>. |
---|
1486 | </p> |
---|
1487 | <a name="N1076D"></a><a name="Partitioner"></a> |
---|
1488 | <h4>Partitioner</h4> |
---|
1489 | <p> |
---|
1490 | <a href="api/org/apache/hadoop/mapred/Partitioner.html"> |
---|
1491 | Partitioner</a> partitions the key space.</p> |
---|
1492 | <p>Partitioner controls the partitioning of the keys of the |
---|
1493 | intermediate map-outputs. The key (or a subset of the key) is used to |
---|
1494 | derive the partition, typically by a <em>hash function</em>. The total |
---|
1495 | number of partitions is the same as the number of reduce tasks for the |
---|
1496 | job. Hence this controls which of the <span class="codefrag">m</span> reduce tasks the |
---|
1497 | intermediate key (and hence the record) is sent to for reduction.</p> |
---|
1498 | <p> |
---|
1499 | <a href="api/org/apache/hadoop/mapred/lib/HashPartitioner.html"> |
---|
1500 | HashPartitioner</a> is the default <span class="codefrag">Partitioner</span>.</p> |
---|
1501 | <a name="N1078C"></a><a name="Reporter"></a> |
---|
1502 | <h4>Reporter</h4> |
---|
1503 | <p> |
---|
1504 | <a href="api/org/apache/hadoop/mapred/Reporter.html"> |
---|
1505 | Reporter</a> is a facility for Map/Reduce applications to report |
---|
1506 | progress, set application-level status messages and update |
---|
1507 | <span class="codefrag">Counters</span>.</p> |
---|
1508 | <p> |
---|
1509 | <span class="codefrag">Mapper</span> and <span class="codefrag">Reducer</span> implementations can use |
---|
1510 | the <span class="codefrag">Reporter</span> to report progress or just indicate |
---|
1511 | that they are alive. In scenarios where the application takes a |
---|
1512 | significant amount of time to process individual key/value pairs, |
---|
1513 | this is crucial since the framework might assume that the task has |
---|
1514 | timed-out and kill that task. Another way to avoid this is to |
---|
1515 | set the configuration parameter <span class="codefrag">mapred.task.timeout</span> to a |
---|
1516 | high-enough value (or even set it to <em>zero</em> for no time-outs). |
---|
1517 | </p> |
---|
1518 | <p>Applications can also update <span class="codefrag">Counters</span> using the |
---|
1519 | <span class="codefrag">Reporter</span>.</p> |
---|
1520 | <a name="N107B6"></a><a name="OutputCollector"></a> |
---|
1521 | <h4>OutputCollector</h4> |
---|
1522 | <p> |
---|
1523 | <a href="api/org/apache/hadoop/mapred/OutputCollector.html"> |
---|
1524 | OutputCollector</a> is a generalization of the facility provided by |
---|
1525 | the Map/Reduce framework to collect data output by the |
---|
1526 | <span class="codefrag">Mapper</span> or the <span class="codefrag">Reducer</span> (either the |
---|
1527 | intermediate outputs or the output of the job).</p> |
---|
1528 | <p>Hadoop Map/Reduce comes bundled with a |
---|
1529 | <a href="api/org/apache/hadoop/mapred/lib/package-summary.html"> |
---|
1530 | library</a> of generally useful mappers, reducers, and partitioners.</p> |
---|
1531 | <a name="N107D1"></a><a name="Job+Configuration"></a> |
---|
1532 | <h3 class="h4">Job Configuration</h3> |
---|
1533 | <p> |
---|
1534 | <a href="api/org/apache/hadoop/mapred/JobConf.html"> |
---|
1535 | JobConf</a> represents a Map/Reduce job configuration.</p> |
---|
1536 | <p> |
---|
1537 | <span class="codefrag">JobConf</span> is the primary interface for a user to describe |
---|
1538 | a Map/Reduce job to the Hadoop framework for execution. The framework |
---|
1539 | tries to faithfully execute the job as described by <span class="codefrag">JobConf</span>, |
---|
1540 | however:</p> |
---|
1541 | <ul> |
---|
1542 | |
---|
1543 | <li>f |
---|
1544 | Some configuration parameters may have been marked as |
---|
1545 | <a href="api/org/apache/hadoop/conf/Configuration.html#FinalParams"> |
---|
1546 | final</a> by administrators and hence cannot be altered. |
---|
1547 | </li> |
---|
1548 | |
---|
1549 | <li> |
---|
1550 | While some job parameters are straight-forward to set (e.g. |
---|
1551 | <a href="api/org/apache/hadoop/mapred/JobConf.html#setNumReduceTasks(int)"> |
---|
1552 | setNumReduceTasks(int)</a>), other parameters interact subtly with |
---|
1553 | the rest of the framework and/or job configuration and are |
---|
1554 | more complex to set (e.g. |
---|
1555 | <a href="api/org/apache/hadoop/mapred/JobConf.html#setNumMapTasks(int)"> |
---|
1556 | setNumMapTasks(int)</a>). |
---|
1557 | </li> |
---|
1558 | |
---|
1559 | </ul> |
---|
1560 | <p> |
---|
1561 | <span class="codefrag">JobConf</span> is typically used to specify the |
---|
1562 | <span class="codefrag">Mapper</span>, combiner (if any), <span class="codefrag">Partitioner</span>, |
---|
1563 | <span class="codefrag">Reducer</span>, <span class="codefrag">InputFormat</span>, |
---|
1564 | <span class="codefrag">OutputFormat</span> and <span class="codefrag">OutputCommitter</span> |
---|
1565 | implementations. <span class="codefrag">JobConf</span> also |
---|
1566 | indicates the set of input files |
---|
1567 | (<a href="api/org/apache/hadoop/mapred/FileInputFormat.html#setInputPaths(org.apache.hadoop.mapred.JobConf,%20org.apache.hadoop.fs.Path[])">setInputPaths(JobConf, Path...)</a> |
---|
1568 | /<a href="api/org/apache/hadoop/mapred/FileInputFormat.html#addInputPath(org.apache.hadoop.mapred.JobConf,%20org.apache.hadoop.fs.Path)">addInputPath(JobConf, Path)</a>) |
---|
1569 | and (<a href="api/org/apache/hadoop/mapred/FileInputFormat.html#setInputPaths(org.apache.hadoop.mapred.JobConf,%20java.lang.String)">setInputPaths(JobConf, String)</a> |
---|
1570 | /<a href="api/org/apache/hadoop/mapred/FileInputFormat.html#addInputPath(org.apache.hadoop.mapred.JobConf,%20java.lang.String)">addInputPaths(JobConf, String)</a>) |
---|
1571 | and where the output files should be written |
---|
1572 | (<a href="api/org/apache/hadoop/mapred/FileOutputFormat.html#setOutputPath(org.apache.hadoop.mapred.JobConf,%20org.apache.hadoop.fs.Path)">setOutputPath(Path)</a>).</p> |
---|
1573 | <p>Optionally, <span class="codefrag">JobConf</span> is used to specify other advanced |
---|
1574 | facets of the job such as the <span class="codefrag">Comparator</span> to be used, files |
---|
1575 | to be put in the <span class="codefrag">DistributedCache</span>, whether intermediate |
---|
1576 | and/or job outputs are to be compressed (and how), debugging via |
---|
1577 | user-provided scripts |
---|
1578 | (<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>) |
---|
1579 | , whether job tasks can be executed in a <em>speculative</em> manner |
---|
1580 | (<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>) |
---|
1581 | , maximum number of attempts per task |
---|
1582 | (<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>) |
---|
1583 | , percentage of tasks failure which can be tolerated by the job |
---|
1584 | (<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>) |
---|
1585 | etc.</p> |
---|
1586 | <p>Of course, users can use |
---|
1587 | <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> |
---|
1588 | to set/get arbitrary parameters needed by applications. However, use the |
---|
1589 | <span class="codefrag">DistributedCache</span> for large amounts of (read-only) data.</p> |
---|
1590 | <a name="N10866"></a><a name="Task+Execution+%26+Environment"></a> |
---|
1591 | <h3 class="h4">Task Execution & Environment</h3> |
---|
1592 | <p>The <span class="codefrag">TaskTracker</span> executes the <span class="codefrag">Mapper</span>/ |
---|
1593 | <span class="codefrag">Reducer</span> <em>task</em> as a child process in a separate jvm. |
---|
1594 | </p> |
---|
1595 | <p>The child-task inherits the environment of the parent |
---|
1596 | <span class="codefrag">TaskTracker</span>. The user can specify additional options to the |
---|
1597 | child-jvm via the <span class="codefrag">mapred.child.java.opts</span> configuration |
---|
1598 | parameter in the <span class="codefrag">JobConf</span> such as non-standard paths for the |
---|
1599 | run-time linker to search shared libraries via |
---|
1600 | <span class="codefrag">-Djava.library.path=<></span> etc. If the |
---|
1601 | <span class="codefrag">mapred.child.java.opts</span> contains the symbol <em>@taskid@</em> |
---|
1602 | it is interpolated with value of <span class="codefrag">taskid</span> of the map/reduce |
---|
1603 | task.</p> |
---|
1604 | <p>Here is an example with multiple arguments and substitutions, |
---|
1605 | showing jvm GC logging, and start of a passwordless JVM JMX agent so that |
---|
1606 | it can connect with jconsole and the likes to watch child memory, |
---|
1607 | threads and get thread dumps. It also sets the maximum heap-size of the |
---|
1608 | child jvm to 512MB and adds an additional path to the |
---|
1609 | <span class="codefrag">java.library.path</span> of the child-jvm.</p> |
---|
1610 | <p> |
---|
1611 | |
---|
1612 | <span class="codefrag"><property></span> |
---|
1613 | <br> |
---|
1614 | <span class="codefrag"><name>mapred.child.java.opts</name></span> |
---|
1615 | <br> |
---|
1616 | <span class="codefrag"><value></span> |
---|
1617 | <br> |
---|
1618 | <span class="codefrag"> |
---|
1619 | -Xmx512M -Djava.library.path=/home/mycompany/lib |
---|
1620 | -verbose:gc -Xloggc:/tmp/@taskid@.gc</span> |
---|
1621 | <br> |
---|
1622 | <span class="codefrag"> |
---|
1623 | -Dcom.sun.management.jmxremote.authenticate=false |
---|
1624 | -Dcom.sun.management.jmxremote.ssl=false</span> |
---|
1625 | <br> |
---|
1626 | <span class="codefrag"></value></span> |
---|
1627 | <br> |
---|
1628 | |
---|
1629 | <span class="codefrag"></property></span> |
---|
1630 | |
---|
1631 | </p> |
---|
1632 | <a name="N108B7"></a><a name="Memory+management"></a> |
---|
1633 | <h4> Memory management</h4> |
---|
1634 | <p>Users/admins can also specify the maximum virtual memory |
---|
1635 | of the launched child-task, and any sub-process it launches |
---|
1636 | recursively, using <span class="codefrag">mapred.child.ulimit</span>. Note that |
---|
1637 | the value set here is a per process limit. |
---|
1638 | The value for <span class="codefrag">mapred.child.ulimit</span> should be specified |
---|
1639 | in kilo bytes (KB). And also the value must be greater than |
---|
1640 | or equal to the -Xmx passed to JavaVM, else the VM might not start. |
---|
1641 | </p> |
---|
1642 | <p>Note: <span class="codefrag">mapred.child.java.opts</span> are used only for |
---|
1643 | configuring the launched child tasks from task tracker. Configuring |
---|
1644 | the memory options for daemons is documented in |
---|
1645 | <a href="cluster_setup.html#Configuring+the+Environment+of+the+Hadoop+Daemons"> |
---|
1646 | cluster_setup.html </a> |
---|
1647 | </p> |
---|
1648 | <p>The memory available to some parts of the framework is also |
---|
1649 | configurable. In map and reduce tasks, performance may be influenced |
---|
1650 | by adjusting parameters influencing the concurrency of operations and |
---|
1651 | the frequency with which data will hit disk. Monitoring the filesystem |
---|
1652 | counters for a job- particularly relative to byte counts from the map |
---|
1653 | and into the reduce- is invaluable to the tuning of these |
---|
1654 | parameters.</p> |
---|
1655 | <p>Users can choose to override default limits of Virtual Memory and RAM |
---|
1656 | enforced by the task tracker, if memory management is enabled. |
---|
1657 | Users can set the following parameter per job:</p> |
---|
1658 | <table class="ForrestTable" cellspacing="1" cellpadding="4"> |
---|
1659 | |
---|
1660 | <tr> |
---|
1661 | <th colspan="1" rowspan="1">Name</th><th colspan="1" rowspan="1">Type</th><th colspan="1" rowspan="1">Description</th> |
---|
1662 | </tr> |
---|
1663 | |
---|
1664 | <tr> |
---|
1665 | <td colspan="1" rowspan="1"><span class="codefrag">mapred.task.maxvmem</span></td><td colspan="1" rowspan="1">int</td> |
---|
1666 | <td colspan="1" rowspan="1">A number, in bytes, that represents the maximum Virtual Memory |
---|
1667 | task-limit for each task of the job. A task will be killed if |
---|
1668 | it consumes more Virtual Memory than this number. |
---|
1669 | </td> |
---|
1670 | </tr> |
---|
1671 | |
---|
1672 | <tr> |
---|
1673 | <td colspan="1" rowspan="1">mapred.task.maxpmem</td><td colspan="1" rowspan="1">int</td> |
---|
1674 | <td colspan="1" rowspan="1">A number, in bytes, that represents the maximum RAM task-limit |
---|
1675 | for each task of the job. This number can be optionally used by |
---|
1676 | Schedulers to prevent over-scheduling of tasks on a node based |
---|
1677 | on RAM needs. |
---|
1678 | </td> |
---|
1679 | </tr> |
---|
1680 | |
---|
1681 | </table> |
---|
1682 | <a name="N10906"></a><a name="Map+Parameters"></a> |
---|
1683 | <h4>Map Parameters</h4> |
---|
1684 | <p>A record emitted from a map will be serialized into a buffer and |
---|
1685 | metadata will be stored into accounting buffers. As described in the |
---|
1686 | following options, when either the serialization buffer or the |
---|
1687 | metadata exceed a threshold, the contents of the buffers will be |
---|
1688 | sorted and written to disk in the background while the map continues |
---|
1689 | to output records. If either buffer fills completely while the spill |
---|
1690 | is in progress, the map thread will block. When the map is finished, |
---|
1691 | any remaining records are written to disk and all on-disk segments |
---|
1692 | are merged into a single file. Minimizing the number of spills to |
---|
1693 | disk can decrease map time, but a larger buffer also decreases the |
---|
1694 | memory available to the mapper.</p> |
---|
1695 | <table class="ForrestTable" cellspacing="1" cellpadding="4"> |
---|
1696 | |
---|
1697 | <tr> |
---|
1698 | <th colspan="1" rowspan="1">Name</th><th colspan="1" rowspan="1">Type</th><th colspan="1" rowspan="1">Description</th> |
---|
1699 | </tr> |
---|
1700 | |
---|
1701 | <tr> |
---|
1702 | <td colspan="1" rowspan="1">io.sort.mb</td><td colspan="1" rowspan="1">int</td> |
---|
1703 | <td colspan="1" rowspan="1">The cumulative size of the serialization and accounting |
---|
1704 | buffers storing records emitted from the map, in megabytes. |
---|
1705 | </td> |
---|
1706 | </tr> |
---|
1707 | |
---|
1708 | <tr> |
---|
1709 | <td colspan="1" rowspan="1">io.sort.record.percent</td><td colspan="1" rowspan="1">float</td> |
---|
1710 | <td colspan="1" rowspan="1">The ratio of serialization to accounting space can be |
---|
1711 | adjusted. Each serialized record requires 16 bytes of |
---|
1712 | accounting information in addition to its serialized size to |
---|
1713 | effect the sort. This percentage of space allocated from |
---|
1714 | <span class="codefrag">io.sort.mb</span> affects the probability of a spill to |
---|
1715 | disk being caused by either exhaustion of the serialization |
---|
1716 | buffer or the accounting space. Clearly, for a map outputting |
---|
1717 | small records, a higher value than the default will likely |
---|
1718 | decrease the number of spills to disk.</td> |
---|
1719 | </tr> |
---|
1720 | |
---|
1721 | <tr> |
---|
1722 | <td colspan="1" rowspan="1">io.sort.spill.percent</td><td colspan="1" rowspan="1">float</td> |
---|
1723 | <td colspan="1" rowspan="1">This is the threshold for the accounting and serialization |
---|
1724 | buffers. When this percentage of either buffer has filled, |
---|
1725 | their contents will be spilled to disk in the background. Let |
---|
1726 | <span class="codefrag">io.sort.record.percent</span> be <em>r</em>, |
---|
1727 | <span class="codefrag">io.sort.mb</span> be <em>x</em>, and this value be |
---|
1728 | <em>q</em>. The maximum number of records collected before the |
---|
1729 | collection thread will spill is <span class="codefrag">r * x * q * 2^16</span>. |
---|
1730 | Note that a higher value may decrease the number of- or even |
---|
1731 | eliminate- merges, but will also increase the probability of |
---|
1732 | the map task getting blocked. The lowest average map times are |
---|
1733 | usually obtained by accurately estimating the size of the map |
---|
1734 | output and preventing multiple spills.</td> |
---|
1735 | </tr> |
---|
1736 | |
---|
1737 | </table> |
---|
1738 | <p>Other notes</p> |
---|
1739 | <ul> |
---|
1740 | |
---|
1741 | <li>If either spill threshold is exceeded while a spill is in |
---|
1742 | progress, collection will continue until the spill is finished. |
---|
1743 | For example, if <span class="codefrag">io.sort.buffer.spill.percent</span> is set |
---|
1744 | to 0.33, and the remainder of the buffer is filled while the spill |
---|
1745 | runs, the next spill will include all the collected records, or |
---|
1746 | 0.66 of the buffer, and will not generate additional spills. In |
---|
1747 | other words, the thresholds are defining triggers, not |
---|
1748 | blocking.</li> |
---|
1749 | |
---|
1750 | <li>A record larger than the serialization buffer will first |
---|
1751 | trigger a spill, then be spilled to a separate file. It is |
---|
1752 | undefined whether or not this record will first pass through the |
---|
1753 | combiner.</li> |
---|
1754 | |
---|
1755 | </ul> |
---|
1756 | <a name="N10972"></a><a name="Shuffle%2FReduce+Parameters"></a> |
---|
1757 | <h4>Shuffle/Reduce Parameters</h4> |
---|
1758 | <p>As described previously, each reduce fetches the output assigned |
---|
1759 | to it by the Partitioner via HTTP into memory and periodically |
---|
1760 | merges these outputs to disk. If intermediate compression of map |
---|
1761 | outputs is turned on, each output is decompressed into memory. The |
---|
1762 | following options affect the frequency of these merges to disk prior |
---|
1763 | to the reduce and the memory allocated to map output during the |
---|
1764 | reduce.</p> |
---|
1765 | <table class="ForrestTable" cellspacing="1" cellpadding="4"> |
---|
1766 | |
---|
1767 | <tr> |
---|
1768 | <th colspan="1" rowspan="1">Name</th><th colspan="1" rowspan="1">Type</th><th colspan="1" rowspan="1">Description</th> |
---|
1769 | </tr> |
---|
1770 | |
---|
1771 | <tr> |
---|
1772 | <td colspan="1" rowspan="1">io.sort.factor</td><td colspan="1" rowspan="1">int</td> |
---|
1773 | <td colspan="1" rowspan="1">Specifies the number of segments on disk to be merged at |
---|
1774 | the same time. It limits the number of open files and |
---|
1775 | compression codecs during the merge. If the number of files |
---|
1776 | exceeds this limit, the merge will proceed in several passes. |
---|
1777 | Though this limit also applies to the map, most jobs should be |
---|
1778 | configured so that hitting this limit is unlikely |
---|
1779 | there.</td> |
---|
1780 | </tr> |
---|
1781 | |
---|
1782 | <tr> |
---|
1783 | <td colspan="1" rowspan="1">mapred.inmem.merge.threshold</td><td colspan="1" rowspan="1">int</td> |
---|
1784 | <td colspan="1" rowspan="1">The number of sorted map outputs fetched into memory |
---|
1785 | before being merged to disk. Like the spill thresholds in the |
---|
1786 | preceding note, this is not defining a unit of partition, but |
---|
1787 | a trigger. In practice, this is usually set very high (1000) |
---|
1788 | or disabled (0), since merging in-memory segments is often |
---|
1789 | less expensive than merging from disk (see notes following |
---|
1790 | this table). This threshold influences only the frequency of |
---|
1791 | in-memory merges during the shuffle.</td> |
---|
1792 | </tr> |
---|
1793 | |
---|
1794 | <tr> |
---|
1795 | <td colspan="1" rowspan="1">mapred.job.shuffle.merge.percent</td><td colspan="1" rowspan="1">float</td> |
---|
1796 | <td colspan="1" rowspan="1">The memory threshold for fetched map outputs before an |
---|
1797 | in-memory merge is started, expressed as a percentage of |
---|
1798 | memory allocated to storing map outputs in memory. Since map |
---|
1799 | outputs that can't fit in memory can be stalled, setting this |
---|
1800 | high may decrease parallelism between the fetch and merge. |
---|
1801 | Conversely, values as high as 1.0 have been effective for |
---|
1802 | reduces whose input can fit entirely in memory. This parameter |
---|
1803 | influences only the frequency of in-memory merges during the |
---|
1804 | shuffle.</td> |
---|
1805 | </tr> |
---|
1806 | |
---|
1807 | <tr> |
---|
1808 | <td colspan="1" rowspan="1">mapred.job.shuffle.input.buffer.percent</td><td colspan="1" rowspan="1">float</td> |
---|
1809 | <td colspan="1" rowspan="1">The percentage of memory- relative to the maximum heapsize |
---|
1810 | as typically specified in <span class="codefrag">mapred.child.java.opts</span>- |
---|
1811 | that can be allocated to storing map outputs during the |
---|
1812 | shuffle. Though some memory should be set aside for the |
---|
1813 | framework, in general it is advantageous to set this high |
---|
1814 | enough to store large and numerous map outputs.</td> |
---|
1815 | </tr> |
---|
1816 | |
---|
1817 | <tr> |
---|
1818 | <td colspan="1" rowspan="1">mapred.job.reduce.input.buffer.percent</td><td colspan="1" rowspan="1">float</td> |
---|
1819 | <td colspan="1" rowspan="1">The percentage of memory relative to the maximum heapsize |
---|
1820 | in which map outputs may be retained during the reduce. When |
---|
1821 | the reduce begins, map outputs will be merged to disk until |
---|
1822 | those that remain are under the resource limit this defines. |
---|
1823 | By default, all map outputs are merged to disk before the |
---|
1824 | reduce begins to maximize the memory available to the reduce. |
---|
1825 | For less memory-intensive reduces, this should be increased to |
---|
1826 | avoid trips to disk.</td> |
---|
1827 | </tr> |
---|
1828 | |
---|
1829 | </table> |
---|
1830 | <p>Other notes</p> |
---|
1831 | <ul> |
---|
1832 | |
---|
1833 | <li>If a map output is larger than 25 percent of the memory |
---|
1834 | allocated to copying map outputs, it will be written directly to |
---|
1835 | disk without first staging through memory.</li> |
---|
1836 | |
---|
1837 | <li>When running with a combiner, the reasoning about high merge |
---|
1838 | thresholds and large buffers may not hold. For merges started |
---|
1839 | before all map outputs have been fetched, the combiner is run |
---|
1840 | while spilling to disk. In some cases, one can obtain better |
---|
1841 | reduce times by spending resources combining map outputs- making |
---|
1842 | disk spills small and parallelizing spilling and fetching- rather |
---|
1843 | than aggressively increasing buffer sizes.</li> |
---|
1844 | |
---|
1845 | <li>When merging in-memory map outputs to disk to begin the |
---|
1846 | reduce, if an intermediate merge is necessary because there are |
---|
1847 | segments to spill and at least <span class="codefrag">io.sort.factor</span> |
---|
1848 | segments already on disk, the in-memory map outputs will be part |
---|
1849 | of the intermediate merge.</li> |
---|
1850 | |
---|
1851 | </ul> |
---|
1852 | <a name="N109ED"></a><a name="Directory+Structure"></a> |
---|
1853 | <h4> Directory Structure </h4> |
---|
1854 | <p>The task tracker has local directory, |
---|
1855 | <span class="codefrag"> ${mapred.local.dir}/taskTracker/</span> to create localized |
---|
1856 | cache and localized job. It can define multiple local directories |
---|
1857 | (spanning multiple disks) and then each filename is assigned to a |
---|
1858 | semi-random local directory. When the job starts, task tracker |
---|
1859 | creates a localized job directory relative to the local directory |
---|
1860 | specified in the configuration. Thus the task tracker directory |
---|
1861 | structure looks the following: </p> |
---|
1862 | <ul> |
---|
1863 | |
---|
1864 | <li> |
---|
1865 | <span class="codefrag">${mapred.local.dir}/taskTracker/archive/</span> : |
---|
1866 | The distributed cache. This directory holds the localized distributed |
---|
1867 | cache. Thus localized distributed cache is shared among all |
---|
1868 | the tasks and jobs </li> |
---|
1869 | |
---|
1870 | <li> |
---|
1871 | <span class="codefrag">${mapred.local.dir}/taskTracker/jobcache/$jobid/</span> : |
---|
1872 | The localized job directory |
---|
1873 | <ul> |
---|
1874 | |
---|
1875 | <li> |
---|
1876 | <span class="codefrag">${mapred.local.dir}/taskTracker/jobcache/$jobid/work/</span> |
---|
1877 | : The job-specific shared directory. The tasks can use this space as |
---|
1878 | scratch space and share files among them. This directory is exposed |
---|
1879 | to the users through the configuration property |
---|
1880 | <span class="codefrag">job.local.dir</span>. The directory can accessed through |
---|
1881 | api <a href="api/org/apache/hadoop/mapred/JobConf.html#getJobLocalDir()"> |
---|
1882 | JobConf.getJobLocalDir()</a>. It is available as System property also. |
---|
1883 | So, users (streaming etc.) can call |
---|
1884 | <span class="codefrag">System.getProperty("job.local.dir")</span> to access the |
---|
1885 | directory.</li> |
---|
1886 | |
---|
1887 | <li> |
---|
1888 | <span class="codefrag">${mapred.local.dir}/taskTracker/jobcache/$jobid/jars/</span> |
---|
1889 | : The jars directory, which has the job jar file and expanded jar. |
---|
1890 | The <span class="codefrag">job.jar</span> is the application's jar file that is |
---|
1891 | automatically distributed to each machine. It is expanded in jars |
---|
1892 | directory before the tasks for the job start. The job.jar location |
---|
1893 | is accessible to the application through the api |
---|
1894 | <a href="api/org/apache/hadoop/mapred/JobConf.html#getJar()"> |
---|
1895 | JobConf.getJar() </a>. To access the unjarred directory, |
---|
1896 | JobConf.getJar().getParent() can be called.</li> |
---|
1897 | |
---|
1898 | <li> |
---|
1899 | <span class="codefrag">${mapred.local.dir}/taskTracker/jobcache/$jobid/job.xml</span> |
---|
1900 | : The job.xml file, the generic job configuration, localized for |
---|
1901 | the job. </li> |
---|
1902 | |
---|
1903 | <li> |
---|
1904 | <span class="codefrag">${mapred.local.dir}/taskTracker/jobcache/$jobid/$taskid</span> |
---|
1905 | : The task directory for each task attempt. Each task directory |
---|
1906 | again has the following structure : |
---|
1907 | <ul> |
---|
1908 | |
---|
1909 | <li> |
---|
1910 | <span class="codefrag">${mapred.local.dir}/taskTracker/jobcache/$jobid/$taskid/job.xml</span> |
---|
1911 | : A job.xml file, task localized job configuration, Task localization |
---|
1912 | means that properties have been set that are specific to |
---|
1913 | this particular task within the job. The properties localized for |
---|
1914 | each task are described below.</li> |
---|
1915 | |
---|
1916 | <li> |
---|
1917 | <span class="codefrag">${mapred.local.dir}/taskTracker/jobcache/$jobid/$taskid/output</span> |
---|
1918 | : A directory for intermediate output files. This contains the |
---|
1919 | temporary map reduce data generated by the framework |
---|
1920 | such as map output files etc. </li> |
---|
1921 | |
---|
1922 | <li> |
---|
1923 | <span class="codefrag">${mapred.local.dir}/taskTracker/jobcache/$jobid/$taskid/work</span> |
---|
1924 | : The curernt working directory of the task. |
---|
1925 | With <a href="#Task+JVM+Reuse">jvm reuse</a> enabled for tasks, this |
---|
1926 | directory will be the directory on which the jvm has started</li> |
---|
1927 | |
---|
1928 | <li> |
---|
1929 | <span class="codefrag">${mapred.local.dir}/taskTracker/jobcache/$jobid/$taskid/work/tmp</span> |
---|
1930 | : The temporary directory for the task. |
---|
1931 | (User can specify the property <span class="codefrag">mapred.child.tmp</span> to set |
---|
1932 | the value of temporary directory for map and reduce tasks. This |
---|
1933 | defaults to <span class="codefrag">./tmp</span>. If the value is not an absolute path, |
---|
1934 | it is prepended with task's working directory. Otherwise, it is |
---|
1935 | directly assigned. The directory will be created if it doesn't exist. |
---|
1936 | Then, the child java tasks are executed with option |
---|
1937 | <span class="codefrag">-Djava.io.tmpdir='the absolute path of the tmp dir'</span>. |
---|
1938 | Anp pipes and streaming are set with environment variable, |
---|
1939 | <span class="codefrag">TMPDIR='the absolute path of the tmp dir'</span>). This |
---|
1940 | directory is created, if <span class="codefrag">mapred.child.tmp</span> has the value |
---|
1941 | <span class="codefrag">./tmp</span> |
---|
1942 | </li> |
---|
1943 | |
---|
1944 | </ul> |
---|
1945 | |
---|
1946 | </li> |
---|
1947 | |
---|
1948 | </ul> |
---|
1949 | |
---|
1950 | </li> |
---|
1951 | |
---|
1952 | </ul> |
---|
1953 | <a name="N10A5C"></a><a name="Task+JVM+Reuse"></a> |
---|
1954 | <h4>Task JVM Reuse</h4> |
---|
1955 | <p>Jobs can enable task JVMs to be reused by specifying the job |
---|
1956 | configuration <span class="codefrag">mapred.job.reuse.jvm.num.tasks</span>. If the |
---|
1957 | value is 1 (the default), then JVMs are not reused |
---|
1958 | (i.e. 1 task per JVM). If it is -1, there is no limit to the number |
---|
1959 | of tasks a JVM can run (of the same job). One can also specify some |
---|
1960 | value greater than 1 using the api |
---|
1961 | <a href="api/org/apache/hadoop/mapred/JobConf.html#setNumTasksToExecutePerJvm(int)"> |
---|
1962 | JobConf.setNumTasksToExecutePerJvm(int)</a> |
---|
1963 | </p> |
---|
1964 | <p>The following properties are localized in the job configuration |
---|
1965 | for each task's execution: </p> |
---|
1966 | <table class="ForrestTable" cellspacing="1" cellpadding="4"> |
---|
1967 | |
---|
1968 | <tr> |
---|
1969 | <th colspan="1" rowspan="1">Name</th><th colspan="1" rowspan="1">Type</th><th colspan="1" rowspan="1">Description</th> |
---|
1970 | </tr> |
---|
1971 | |
---|
1972 | <tr> |
---|
1973 | <td colspan="1" rowspan="1">mapred.job.id</td><td colspan="1" rowspan="1">String</td><td colspan="1" rowspan="1">The job id</td> |
---|
1974 | </tr> |
---|
1975 | |
---|
1976 | <tr> |
---|
1977 | <td colspan="1" rowspan="1">mapred.jar</td><td colspan="1" rowspan="1">String</td> |
---|
1978 | <td colspan="1" rowspan="1">job.jar location in job directory</td> |
---|
1979 | </tr> |
---|
1980 | |
---|
1981 | <tr> |
---|
1982 | <td colspan="1" rowspan="1">job.local.dir</td><td colspan="1" rowspan="1"> String</td> |
---|
1983 | <td colspan="1" rowspan="1"> The job specific shared scratch space</td> |
---|
1984 | </tr> |
---|
1985 | |
---|
1986 | <tr> |
---|
1987 | <td colspan="1" rowspan="1">mapred.tip.id</td><td colspan="1" rowspan="1"> String</td> |
---|
1988 | <td colspan="1" rowspan="1"> The task id</td> |
---|
1989 | </tr> |
---|
1990 | |
---|
1991 | <tr> |
---|
1992 | <td colspan="1" rowspan="1">mapred.task.id</td><td colspan="1" rowspan="1"> String</td> |
---|
1993 | <td colspan="1" rowspan="1"> The task attempt id</td> |
---|
1994 | </tr> |
---|
1995 | |
---|
1996 | <tr> |
---|
1997 | <td colspan="1" rowspan="1">mapred.task.is.map</td><td colspan="1" rowspan="1"> boolean </td> |
---|
1998 | <td colspan="1" rowspan="1">Is this a map task</td> |
---|
1999 | </tr> |
---|
2000 | |
---|
2001 | <tr> |
---|
2002 | <td colspan="1" rowspan="1">mapred.task.partition</td><td colspan="1" rowspan="1"> int </td> |
---|
2003 | <td colspan="1" rowspan="1">The id of the task within the job</td> |
---|
2004 | </tr> |
---|
2005 | |
---|
2006 | <tr> |
---|
2007 | <td colspan="1" rowspan="1">map.input.file</td><td colspan="1" rowspan="1"> String</td> |
---|
2008 | <td colspan="1" rowspan="1"> The filename that the map is reading from</td> |
---|
2009 | </tr> |
---|
2010 | |
---|
2011 | <tr> |
---|
2012 | <td colspan="1" rowspan="1">map.input.start</td><td colspan="1" rowspan="1"> long</td> |
---|
2013 | <td colspan="1" rowspan="1"> The offset of the start of the map input split</td> |
---|
2014 | </tr> |
---|
2015 | |
---|
2016 | <tr> |
---|
2017 | <td colspan="1" rowspan="1">map.input.length </td><td colspan="1" rowspan="1">long </td> |
---|
2018 | <td colspan="1" rowspan="1">The number of bytes in the map input split</td> |
---|
2019 | </tr> |
---|
2020 | |
---|
2021 | <tr> |
---|
2022 | <td colspan="1" rowspan="1">mapred.work.output.dir</td><td colspan="1" rowspan="1"> String </td> |
---|
2023 | <td colspan="1" rowspan="1">The task's temporary output directory</td> |
---|
2024 | </tr> |
---|
2025 | |
---|
2026 | </table> |
---|
2027 | <p>The standard output (stdout) and error (stderr) streams of the task |
---|
2028 | are read by the TaskTracker and logged to |
---|
2029 | <span class="codefrag">${HADOOP_LOG_DIR}/userlogs</span> |
---|
2030 | </p> |
---|
2031 | <p>The <a href="#DistributedCache">DistributedCache</a> can also be used |
---|
2032 | to distribute both jars and native libraries for use in the map |
---|
2033 | and/or reduce tasks. The child-jvm always has its |
---|
2034 | <em>current working directory</em> added to the |
---|
2035 | <span class="codefrag">java.library.path</span> and <span class="codefrag">LD_LIBRARY_PATH</span>. |
---|
2036 | And hence the cached libraries can be loaded via |
---|
2037 | <a href="http://java.sun.com/javase/6/docs/api/java/lang/System.html#loadLibrary(java.lang.String)"> |
---|
2038 | System.loadLibrary</a> or |
---|
2039 | <a href="http://java.sun.com/javase/6/docs/api/java/lang/System.html#load(java.lang.String)"> |
---|
2040 | System.load</a>. More details on how to load shared libraries through |
---|
2041 | distributed cache are documented at |
---|
2042 | <a href="native_libraries.html#Loading+native+libraries+through+DistributedCache"> |
---|
2043 | native_libraries.html</a> |
---|
2044 | </p> |
---|
2045 | <a name="N10B45"></a><a name="Job+Submission+and+Monitoring"></a> |
---|
2046 | <h3 class="h4">Job Submission and Monitoring</h3> |
---|
2047 | <p> |
---|
2048 | <a href="api/org/apache/hadoop/mapred/JobClient.html"> |
---|
2049 | JobClient</a> is the primary interface by which user-job interacts |
---|
2050 | with the <span class="codefrag">JobTracker</span>.</p> |
---|
2051 | <p> |
---|
2052 | <span class="codefrag">JobClient</span> provides facilities to submit jobs, track their |
---|
2053 | progress, access component-tasks' reports and logs, get the Map/Reduce |
---|
2054 | cluster's status information and so on.</p> |
---|
2055 | <p>The job submission process involves:</p> |
---|
2056 | <ol> |
---|
2057 | |
---|
2058 | <li>Checking the input and output specifications of the job.</li> |
---|
2059 | |
---|
2060 | <li>Computing the <span class="codefrag">InputSplit</span> values for the job.</li> |
---|
2061 | |
---|
2062 | <li> |
---|
2063 | Setting up the requisite accounting information for the |
---|
2064 | <span class="codefrag">DistributedCache</span> of the job, if necessary. |
---|
2065 | </li> |
---|
2066 | |
---|
2067 | <li> |
---|
2068 | Copying the job's jar and configuration to the Map/Reduce system |
---|
2069 | directory on the <span class="codefrag">FileSystem</span>. |
---|
2070 | </li> |
---|
2071 | |
---|
2072 | <li> |
---|
2073 | Submitting the job to the <span class="codefrag">JobTracker</span> and optionally |
---|
2074 | monitoring it's status. |
---|
2075 | </li> |
---|
2076 | |
---|
2077 | </ol> |
---|
2078 | <p> Job history files are also logged to user specified directory |
---|
2079 | <span class="codefrag">hadoop.job.history.user.location</span> |
---|
2080 | which defaults to job output directory. The files are stored in |
---|
2081 | "_logs/history/" in the specified directory. Hence, by default they |
---|
2082 | will be in mapred.output.dir/_logs/history. User can stop |
---|
2083 | logging by giving the value <span class="codefrag">none</span> for |
---|
2084 | <span class="codefrag">hadoop.job.history.user.location</span> |
---|
2085 | </p> |
---|
2086 | <p> User can view the history logs summary in specified directory |
---|
2087 | using the following command <br> |
---|
2088 | |
---|
2089 | <span class="codefrag">$ bin/hadoop job -history output-dir</span> |
---|
2090 | <br> |
---|
2091 | This command will print job details, failed and killed tip |
---|
2092 | details. <br> |
---|
2093 | More details about the job such as successful tasks and |
---|
2094 | task attempts made for each task can be viewed using the |
---|
2095 | following command <br> |
---|
2096 | |
---|
2097 | <span class="codefrag">$ bin/hadoop job -history all output-dir</span> |
---|
2098 | <br> |
---|
2099 | </p> |
---|
2100 | <p> User can use |
---|
2101 | <a href="api/org/apache/hadoop/mapred/OutputLogFilter.html">OutputLogFilter</a> |
---|
2102 | to filter log files from the output directory listing. </p> |
---|
2103 | <p>Normally the user creates the application, describes various facets |
---|
2104 | of the job via <span class="codefrag">JobConf</span>, and then uses the |
---|
2105 | <span class="codefrag">JobClient</span> to submit the job and monitor its progress.</p> |
---|
2106 | <a name="N10BA5"></a><a name="Job+Control"></a> |
---|
2107 | <h4>Job Control</h4> |
---|
2108 | <p>Users may need to chain Map/Reduce jobs to accomplish complex |
---|
2109 | tasks which cannot be done via a single Map/Reduce job. This is fairly |
---|
2110 | easy since the output of the job typically goes to distributed |
---|
2111 | file-system, and the output, in turn, can be used as the input for the |
---|
2112 | next job.</p> |
---|
2113 | <p>However, this also means that the onus on ensuring jobs are |
---|
2114 | complete (success/failure) lies squarely on the clients. In such |
---|
2115 | cases, the various job-control options are:</p> |
---|
2116 | <ul> |
---|
2117 | |
---|
2118 | <li> |
---|
2119 | |
---|
2120 | <a href="api/org/apache/hadoop/mapred/JobClient.html#runJob(org.apache.hadoop.mapred.JobConf)"> |
---|
2121 | runJob(JobConf)</a> : Submits the job and returns only after the |
---|
2122 | job has completed. |
---|
2123 | </li> |
---|
2124 | |
---|
2125 | <li> |
---|
2126 | |
---|
2127 | <a href="api/org/apache/hadoop/mapred/JobClient.html#submitJob(org.apache.hadoop.mapred.JobConf)"> |
---|
2128 | submitJob(JobConf)</a> : Only submits the job, then poll the |
---|
2129 | returned handle to the |
---|
2130 | <a href="api/org/apache/hadoop/mapred/RunningJob.html"> |
---|
2131 | RunningJob</a> to query status and make scheduling decisions. |
---|
2132 | </li> |
---|
2133 | |
---|
2134 | <li> |
---|
2135 | |
---|
2136 | <a href="api/org/apache/hadoop/mapred/JobConf.html#setJobEndNotificationURI(java.lang.String)"> |
---|
2137 | JobConf.setJobEndNotificationURI(String)</a> : Sets up a |
---|
2138 | notification upon job-completion, thus avoiding polling. |
---|
2139 | </li> |
---|
2140 | |
---|
2141 | </ul> |
---|
2142 | <a name="N10BCF"></a><a name="Job+Input"></a> |
---|
2143 | <h3 class="h4">Job Input</h3> |
---|
2144 | <p> |
---|
2145 | <a href="api/org/apache/hadoop/mapred/InputFormat.html"> |
---|
2146 | InputFormat</a> describes the input-specification for a Map/Reduce job. |
---|
2147 | </p> |
---|
2148 | <p>The Map/Reduce framework relies on the <span class="codefrag">InputFormat</span> of |
---|
2149 | the job to:</p> |
---|
2150 | <ol> |
---|
2151 | |
---|
2152 | <li>Validate the input-specification of the job.</li> |
---|
2153 | |
---|
2154 | <li> |
---|
2155 | Split-up the input file(s) into logical <span class="codefrag">InputSplit</span> |
---|
2156 | instances, each of which is then assigned to an individual |
---|
2157 | <span class="codefrag">Mapper</span>. |
---|
2158 | </li> |
---|
2159 | |
---|
2160 | <li> |
---|
2161 | Provide the <span class="codefrag">RecordReader</span> implementation used to |
---|
2162 | glean input records from the logical <span class="codefrag">InputSplit</span> for |
---|
2163 | processing by the <span class="codefrag">Mapper</span>. |
---|
2164 | </li> |
---|
2165 | |
---|
2166 | </ol> |
---|
2167 | <p>The default behavior of file-based <span class="codefrag">InputFormat</span> |
---|
2168 | implementations, typically sub-classes of |
---|
2169 | <a href="api/org/apache/hadoop/mapred/FileInputFormat.html"> |
---|
2170 | FileInputFormat</a>, is to split the input into <em>logical</em> |
---|
2171 | <span class="codefrag">InputSplit</span> instances based on the total size, in bytes, of |
---|
2172 | the input files. However, the <span class="codefrag">FileSystem</span> blocksize of the |
---|
2173 | input files is treated as an upper bound for input splits. A lower bound |
---|
2174 | on the split size can be set via <span class="codefrag">mapred.min.split.size</span>.</p> |
---|
2175 | <p>Clearly, logical splits based on input-size is insufficient for many |
---|
2176 | applications since record boundaries must be respected. In such cases, |
---|
2177 | the application should implement a <span class="codefrag">RecordReader</span>, who is |
---|
2178 | responsible for respecting record-boundaries and presents a |
---|
2179 | record-oriented view of the logical <span class="codefrag">InputSplit</span> to the |
---|
2180 | individual task.</p> |
---|
2181 | <p> |
---|
2182 | <a href="api/org/apache/hadoop/mapred/TextInputFormat.html"> |
---|
2183 | TextInputFormat</a> is the default <span class="codefrag">InputFormat</span>.</p> |
---|
2184 | <p>If <span class="codefrag">TextInputFormat</span> is the <span class="codefrag">InputFormat</span> for a |
---|
2185 | given job, the framework detects input-files with the <em>.gz</em> |
---|
2186 | extensions and automatically decompresses them using the |
---|
2187 | appropriate <span class="codefrag">CompressionCodec</span>. However, it must be noted that |
---|
2188 | compressed files with the above extensions cannot be <em>split</em> and |
---|
2189 | each compressed file is processed in its entirety by a single mapper.</p> |
---|
2190 | <a name="N10C36"></a><a name="InputSplit"></a> |
---|
2191 | <h4>InputSplit</h4> |
---|
2192 | <p> |
---|
2193 | <a href="api/org/apache/hadoop/mapred/InputSplit.html"> |
---|
2194 | InputSplit</a> represents the data to be processed by an individual |
---|
2195 | <span class="codefrag">Mapper</span>.</p> |
---|
2196 | <p>Typically <span class="codefrag">InputSplit</span> presents a byte-oriented view of |
---|
2197 | the input, and it is the responsibility of <span class="codefrag">RecordReader</span> |
---|
2198 | to process and present a record-oriented view.</p> |
---|
2199 | <p> |
---|
2200 | <a href="api/org/apache/hadoop/mapred/FileSplit.html"> |
---|
2201 | FileSplit</a> is the default <span class="codefrag">InputSplit</span>. It sets |
---|
2202 | <span class="codefrag">map.input.file</span> to the path of the input file for the |
---|
2203 | logical split.</p> |
---|
2204 | <a name="N10C5B"></a><a name="RecordReader"></a> |
---|
2205 | <h4>RecordReader</h4> |
---|
2206 | <p> |
---|
2207 | <a href="api/org/apache/hadoop/mapred/RecordReader.html"> |
---|
2208 | RecordReader</a> reads <span class="codefrag"><key, value></span> pairs from an |
---|
2209 | <span class="codefrag">InputSplit</span>.</p> |
---|
2210 | <p>Typically the <span class="codefrag">RecordReader</span> converts the byte-oriented |
---|
2211 | view of the input, provided by the <span class="codefrag">InputSplit</span>, and |
---|
2212 | presents a record-oriented to the <span class="codefrag">Mapper</span> implementations |
---|
2213 | for processing. <span class="codefrag">RecordReader</span> thus assumes the |
---|
2214 | responsibility of processing record boundaries and presents the tasks |
---|
2215 | with keys and values.</p> |
---|
2216 | <a name="N10C7E"></a><a name="Job+Output"></a> |
---|
2217 | <h3 class="h4">Job Output</h3> |
---|
2218 | <p> |
---|
2219 | <a href="api/org/apache/hadoop/mapred/OutputFormat.html"> |
---|
2220 | OutputFormat</a> describes the output-specification for a Map/Reduce |
---|
2221 | job.</p> |
---|
2222 | <p>The Map/Reduce framework relies on the <span class="codefrag">OutputFormat</span> of |
---|
2223 | the job to:</p> |
---|
2224 | <ol> |
---|
2225 | |
---|
2226 | <li> |
---|
2227 | Validate the output-specification of the job; for example, check that |
---|
2228 | the output directory doesn't already exist. |
---|
2229 | </li> |
---|
2230 | |
---|
2231 | <li> |
---|
2232 | Provide the <span class="codefrag">RecordWriter</span> implementation used to |
---|
2233 | write the output files of the job. Output files are stored in a |
---|
2234 | <span class="codefrag">FileSystem</span>. |
---|
2235 | </li> |
---|
2236 | |
---|
2237 | </ol> |
---|
2238 | <p> |
---|
2239 | <span class="codefrag">TextOutputFormat</span> is the default |
---|
2240 | <span class="codefrag">OutputFormat</span>.</p> |
---|
2241 | <a name="N10CA7"></a><a name="OutputCommitter"></a> |
---|
2242 | <h4>OutputCommitter</h4> |
---|
2243 | <p> |
---|
2244 | <a href="api/org/apache/hadoop/mapred/OutputCommitter.html"> |
---|
2245 | OutputCommitter</a> describes the commit of task output for a |
---|
2246 | Map/Reduce job.</p> |
---|
2247 | <p>The Map/Reduce framework relies on the <span class="codefrag">OutputCommitter</span> |
---|
2248 | of the job to:</p> |
---|
2249 | <ol> |
---|
2250 | |
---|
2251 | <li> |
---|
2252 | Setup the job during initialization. For example, create |
---|
2253 | the temporary output directory for the job during the |
---|
2254 | initialization of the job. |
---|
2255 | Job setup is done by a separate task when the job is |
---|
2256 | in PREP state and after initializing tasks. Once the setup task |
---|
2257 | completes, the job will be moved to RUNNING state. |
---|
2258 | </li> |
---|
2259 | |
---|
2260 | <li> |
---|
2261 | Cleanup the job after the job completion. For example, remove the |
---|
2262 | temporary output directory after the job completion. |
---|
2263 | Job cleanup is done by a separate task at the end of the job. |
---|
2264 | Job is declared SUCCEDED/FAILED/KILLED after the cleanup |
---|
2265 | task completes. |
---|
2266 | </li> |
---|
2267 | |
---|
2268 | <li> |
---|
2269 | Setup the task temporary output. |
---|
2270 | Task setup is done as part of the same task, during task initialization. |
---|
2271 | </li> |
---|
2272 | |
---|
2273 | <li> |
---|
2274 | Check whether a task needs a commit. This is to avoid the commit |
---|
2275 | procedure if a task does not need commit. |
---|
2276 | </li> |
---|
2277 | |
---|
2278 | <li> |
---|
2279 | Commit of the task output. |
---|
2280 | Once task is done, the task will commit it's output if required. |
---|
2281 | </li> |
---|
2282 | |
---|
2283 | <li> |
---|
2284 | Discard the task commit. |
---|
2285 | If the task has been failed/killed, the output will be cleaned-up. |
---|
2286 | If task could not cleanup (in exception block), a separate task |
---|
2287 | will be launched with same attempt-id to do the cleanup. |
---|
2288 | </li> |
---|
2289 | |
---|
2290 | </ol> |
---|
2291 | <p> |
---|
2292 | <span class="codefrag">FileOutputCommitter</span> is the default |
---|
2293 | <span class="codefrag">OutputCommitter</span>. Job setup/cleanup tasks occupy |
---|
2294 | map or reduce slots, whichever is free on the TaskTracker. And |
---|
2295 | JobCleanup task, TaskCleanup tasks and JobSetup task have the highest |
---|
2296 | priority, and in that order.</p> |
---|
2297 | <a name="N10CD7"></a><a name="Task+Side-Effect+Files"></a> |
---|
2298 | <h4>Task Side-Effect Files</h4> |
---|
2299 | <p>In some applications, component tasks need to create and/or write to |
---|
2300 | side-files, which differ from the actual job-output files.</p> |
---|
2301 | <p>In such cases there could be issues with two instances of the same |
---|
2302 | <span class="codefrag">Mapper</span> or <span class="codefrag">Reducer</span> running simultaneously (for |
---|
2303 | example, speculative tasks) trying to open and/or write to the same |
---|
2304 | file (path) on the <span class="codefrag">FileSystem</span>. Hence the |
---|
2305 | application-writer will have to pick unique names per task-attempt |
---|
2306 | (using the attemptid, say <span class="codefrag">attempt_200709221812_0001_m_000000_0</span>), |
---|
2307 | not just per task.</p> |
---|
2308 | <p>To avoid these issues the Map/Reduce framework, when the |
---|
2309 | <span class="codefrag">OutputCommitter</span> is <span class="codefrag">FileOutputCommitter</span>, |
---|
2310 | maintains a special |
---|
2311 | <span class="codefrag">${mapred.output.dir}/_temporary/_${taskid}</span> sub-directory |
---|
2312 | accessible via <span class="codefrag">${mapred.work.output.dir}</span> |
---|
2313 | for each task-attempt on the <span class="codefrag">FileSystem</span> where the output |
---|
2314 | of the task-attempt is stored. On successful completion of the |
---|
2315 | task-attempt, the files in the |
---|
2316 | <span class="codefrag">${mapred.output.dir}/_temporary/_${taskid}</span> (only) |
---|
2317 | are <em>promoted</em> to <span class="codefrag">${mapred.output.dir}</span>. Of course, |
---|
2318 | the framework discards the sub-directory of unsuccessful task-attempts. |
---|
2319 | This process is completely transparent to the application.</p> |
---|
2320 | <p>The application-writer can take advantage of this feature by |
---|
2321 | creating any side-files required in <span class="codefrag">${mapred.work.output.dir}</span> |
---|
2322 | during execution of a task via |
---|
2323 | <a href="api/org/apache/hadoop/mapred/FileOutputFormat.html#getWorkOutputPath(org.apache.hadoop.mapred.JobConf)"> |
---|
2324 | FileOutputFormat.getWorkOutputPath()</a>, and the framework will promote them |
---|
2325 | similarly for succesful task-attempts, thus eliminating the need to |
---|
2326 | pick unique paths per task-attempt.</p> |
---|
2327 | <p>Note: The value of <span class="codefrag">${mapred.work.output.dir}</span> during |
---|
2328 | execution of a particular task-attempt is actually |
---|
2329 | <span class="codefrag">${mapred.output.dir}/_temporary/_{$taskid}</span>, and this value is |
---|
2330 | set by the Map/Reduce framework. So, just create any side-files in the |
---|
2331 | path returned by |
---|
2332 | <a href="api/org/apache/hadoop/mapred/FileOutputFormat.html#getWorkOutputPath(org.apache.hadoop.mapred.JobConf)"> |
---|
2333 | FileOutputFormat.getWorkOutputPath() </a>from map/reduce |
---|
2334 | task to take advantage of this feature.</p> |
---|
2335 | <p>The entire discussion holds true for maps of jobs with |
---|
2336 | reducer=NONE (i.e. 0 reduces) since output of the map, in that case, |
---|
2337 | goes directly to HDFS.</p> |
---|
2338 | <a name="N10D25"></a><a name="RecordWriter"></a> |
---|
2339 | <h4>RecordWriter</h4> |
---|
2340 | <p> |
---|
2341 | <a href="api/org/apache/hadoop/mapred/RecordWriter.html"> |
---|
2342 | RecordWriter</a> writes the output <span class="codefrag"><key, value></span> |
---|
2343 | pairs to an output file.</p> |
---|
2344 | <p>RecordWriter implementations write the job outputs to the |
---|
2345 | <span class="codefrag">FileSystem</span>.</p> |
---|
2346 | <a name="N10D3C"></a><a name="Other+Useful+Features"></a> |
---|
2347 | <h3 class="h4">Other Useful Features</h3> |
---|
2348 | <a name="N10D42"></a><a name="Submitting+Jobs+to+Queues"></a> |
---|
2349 | <h4>Submitting Jobs to Queues</h4> |
---|
2350 | <p>Users submit jobs to Queues. Queues, as collection of jobs, |
---|
2351 | allow the system to provide specific functionality. For example, |
---|
2352 | queues use ACLs to control which users |
---|
2353 | who can submit jobs to them. Queues are expected to be primarily |
---|
2354 | used by Hadoop Schedulers. </p> |
---|
2355 | <p>Hadoop comes configured with a single mandatory queue, called |
---|
2356 | 'default'. Queue names are defined in the |
---|
2357 | <span class="codefrag">mapred.queue.names</span> property of the Hadoop site |
---|
2358 | configuration. Some job schedulers, such as the |
---|
2359 | <a href="capacity_scheduler.html">Capacity Scheduler</a>, |
---|
2360 | support multiple queues.</p> |
---|
2361 | <p>A job defines the queue it needs to be submitted to through the |
---|
2362 | <span class="codefrag">mapred.job.queue.name</span> property, or through the |
---|
2363 | <a href="api/org/apache/hadoop/mapred/JobConf.html#setQueueName(java.lang.String)">setQueueName(String)</a> |
---|
2364 | API. Setting the queue name is optional. If a job is submitted |
---|
2365 | without an associated queue name, it is submitted to the 'default' |
---|
2366 | queue.</p> |
---|
2367 | <a name="N10D60"></a><a name="Counters"></a> |
---|
2368 | <h4>Counters</h4> |
---|
2369 | <p> |
---|
2370 | <span class="codefrag">Counters</span> represent global counters, defined either by |
---|
2371 | the Map/Reduce framework or applications. Each <span class="codefrag">Counter</span> can |
---|
2372 | be of any <span class="codefrag">Enum</span> type. Counters of a particular |
---|
2373 | <span class="codefrag">Enum</span> are bunched into groups of type |
---|
2374 | <span class="codefrag">Counters.Group</span>.</p> |
---|
2375 | <p>Applications can define arbitrary <span class="codefrag">Counters</span> (of type |
---|
2376 | <span class="codefrag">Enum</span>) and update them via |
---|
2377 | <a href="api/org/apache/hadoop/mapred/Reporter.html#incrCounter(java.lang.Enum, long)"> |
---|
2378 | Reporter.incrCounter(Enum, long)</a> or |
---|
2379 | <a href="api/org/apache/hadoop/mapred/Reporter.html#incrCounter(java.lang.String, java.lang.String, long amount)"> |
---|
2380 | Reporter.incrCounter(String, String, long)</a> |
---|
2381 | in the <span class="codefrag">map</span> and/or |
---|
2382 | <span class="codefrag">reduce</span> methods. These counters are then globally |
---|
2383 | aggregated by the framework.</p> |
---|
2384 | <a name="N10D8F"></a><a name="DistributedCache"></a> |
---|
2385 | <h4>DistributedCache</h4> |
---|
2386 | <p> |
---|
2387 | <a href="api/org/apache/hadoop/filecache/DistributedCache.html"> |
---|
2388 | DistributedCache</a> distributes application-specific, large, read-only |
---|
2389 | files efficiently.</p> |
---|
2390 | <p> |
---|
2391 | <span class="codefrag">DistributedCache</span> is a facility provided by the |
---|
2392 | Map/Reduce framework to cache files (text, archives, jars and so on) |
---|
2393 | needed by applications.</p> |
---|
2394 | <p>Applications specify the files to be cached via urls (hdfs://) |
---|
2395 | in the <span class="codefrag">JobConf</span>. The <span class="codefrag">DistributedCache</span> |
---|
2396 | assumes that the files specified via hdfs:// urls are already present |
---|
2397 | on the <span class="codefrag">FileSystem</span>.</p> |
---|
2398 | <p>The framework will copy the necessary files to the slave node |
---|
2399 | before any tasks for the job are executed on that node. Its |
---|
2400 | efficiency stems from the fact that the files are only copied once |
---|
2401 | per job and the ability to cache archives which are un-archived on |
---|
2402 | the slaves.</p> |
---|
2403 | <p> |
---|
2404 | <span class="codefrag">DistributedCache</span> tracks the modification timestamps of |
---|
2405 | the cached files. Clearly the cache files should not be modified by |
---|
2406 | the application or externally while the job is executing.</p> |
---|
2407 | <p> |
---|
2408 | <span class="codefrag">DistributedCache</span> can be used to distribute simple, |
---|
2409 | read-only data/text files and more complex types such as archives and |
---|
2410 | jars. Archives (zip, tar, tgz and tar.gz files) are |
---|
2411 | <em>un-archived</em> at the slave nodes. Files |
---|
2412 | have <em>execution permissions</em> set. </p> |
---|
2413 | <p>The files/archives can be distributed by setting the property |
---|
2414 | <span class="codefrag">mapred.cache.{files|archives}</span>. If more than one |
---|
2415 | file/archive has to be distributed, they can be added as comma |
---|
2416 | separated paths. The properties can also be set by APIs |
---|
2417 | <a href="api/org/apache/hadoop/filecache/DistributedCache.html#addCacheFile(java.net.URI,%20org.apache.hadoop.conf.Configuration)"> |
---|
2418 | DistributedCache.addCacheFile(URI,conf)</a>/ |
---|
2419 | <a href="api/org/apache/hadoop/filecache/DistributedCache.html#addCacheArchive(java.net.URI,%20org.apache.hadoop.conf.Configuration)"> |
---|
2420 | DistributedCache.addCacheArchive(URI,conf)</a> and |
---|
2421 | <a href="api/org/apache/hadoop/filecache/DistributedCache.html#setCacheFiles(java.net.URI[],%20org.apache.hadoop.conf.Configuration)"> |
---|
2422 | DistributedCache.setCacheFiles(URIs,conf)</a>/ |
---|
2423 | <a href="api/org/apache/hadoop/filecache/DistributedCache.html#setCacheArchives(java.net.URI[],%20org.apache.hadoop.conf.Configuration)"> |
---|
2424 | DistributedCache.setCacheArchives(URIs,conf)</a> |
---|
2425 | where URI is of the form |
---|
2426 | <span class="codefrag">hdfs://host:port/absolute-path#link-name</span>. |
---|
2427 | In Streaming, the files can be distributed through command line |
---|
2428 | option <span class="codefrag">-cacheFile/-cacheArchive</span>.</p> |
---|
2429 | <p>Optionally users can also direct the <span class="codefrag">DistributedCache</span> |
---|
2430 | to <em>symlink</em> the cached file(s) into the <span class="codefrag">current working |
---|
2431 | directory</span> of the task via the |
---|
2432 | <a href="api/org/apache/hadoop/filecache/DistributedCache.html#createSymlink(org.apache.hadoop.conf.Configuration)"> |
---|
2433 | DistributedCache.createSymlink(Configuration)</a> api. Or by setting |
---|
2434 | the configuration property <span class="codefrag">mapred.create.symlink</span> |
---|
2435 | as <span class="codefrag">yes</span>. The DistributedCache will use the |
---|
2436 | <span class="codefrag">fragment</span> of the URI as the name of the symlink. |
---|
2437 | For example, the URI |
---|
2438 | <span class="codefrag">hdfs://namenode:port/lib.so.1#lib.so</span> |
---|
2439 | will have the symlink name as <span class="codefrag">lib.so</span> in task's cwd |
---|
2440 | for the file <span class="codefrag">lib.so.1</span> in distributed cache.</p> |
---|
2441 | <p>The <span class="codefrag">DistributedCache</span> can also be used as a |
---|
2442 | rudimentary software distribution mechanism for use in the |
---|
2443 | map and/or reduce tasks. It can be used to distribute both |
---|
2444 | jars and native libraries. The |
---|
2445 | <a href="api/org/apache/hadoop/filecache/DistributedCache.html#addArchiveToClassPath(org.apache.hadoop.fs.Path,%20org.apache.hadoop.conf.Configuration)"> |
---|
2446 | DistributedCache.addArchiveToClassPath(Path, Configuration)</a> or |
---|
2447 | <a href="api/org/apache/hadoop/filecache/DistributedCache.html#addFileToClassPath(org.apache.hadoop.fs.Path,%20org.apache.hadoop.conf.Configuration)"> |
---|
2448 | DistributedCache.addFileToClassPath(Path, Configuration)</a> api |
---|
2449 | can be used to cache files/jars and also add them to the |
---|
2450 | <em>classpath</em> of child-jvm. The same can be done by setting |
---|
2451 | the configuration properties |
---|
2452 | <span class="codefrag">mapred.job.classpath.{files|archives}</span>. Similarly the |
---|
2453 | cached files that are symlinked into the working directory of the |
---|
2454 | task can be used to distribute native libraries and load them.</p> |
---|
2455 | <a name="N10E12"></a><a name="Tool"></a> |
---|
2456 | <h4>Tool</h4> |
---|
2457 | <p>The <a href="api/org/apache/hadoop/util/Tool.html">Tool</a> |
---|
2458 | interface supports the handling of generic Hadoop command-line options. |
---|
2459 | </p> |
---|
2460 | <p> |
---|
2461 | <span class="codefrag">Tool</span> is the standard for any Map/Reduce tool or |
---|
2462 | application. The application should delegate the handling of |
---|
2463 | standard command-line options to |
---|
2464 | <a href="api/org/apache/hadoop/util/GenericOptionsParser.html"> |
---|
2465 | GenericOptionsParser</a> via |
---|
2466 | <a href="api/org/apache/hadoop/util/ToolRunner.html#run(org.apache.hadoop.util.Tool, java.lang.String[])"> |
---|
2467 | ToolRunner.run(Tool, String[])</a> and only handle its custom |
---|
2468 | arguments.</p> |
---|
2469 | <p> |
---|
2470 | The generic Hadoop command-line options are:<br> |
---|
2471 | |
---|
2472 | <span class="codefrag"> |
---|
2473 | -conf <configuration file> |
---|
2474 | </span> |
---|
2475 | |
---|
2476 | <br> |
---|
2477 | |
---|
2478 | <span class="codefrag"> |
---|
2479 | -D <property=value> |
---|
2480 | </span> |
---|
2481 | |
---|
2482 | <br> |
---|
2483 | |
---|
2484 | <span class="codefrag"> |
---|
2485 | -fs <local|namenode:port> |
---|
2486 | </span> |
---|
2487 | |
---|
2488 | <br> |
---|
2489 | |
---|
2490 | <span class="codefrag"> |
---|
2491 | -jt <local|jobtracker:port> |
---|
2492 | </span> |
---|
2493 | |
---|
2494 | </p> |
---|
2495 | <a name="N10E44"></a><a name="IsolationRunner"></a> |
---|
2496 | <h4>IsolationRunner</h4> |
---|
2497 | <p> |
---|
2498 | <a href="api/org/apache/hadoop/mapred/IsolationRunner.html"> |
---|
2499 | IsolationRunner</a> is a utility to help debug Map/Reduce programs.</p> |
---|
2500 | <p>To use the <span class="codefrag">IsolationRunner</span>, first set |
---|
2501 | <span class="codefrag">keep.failed.tasks.files</span> to <span class="codefrag">true</span> |
---|
2502 | (also see <span class="codefrag">keep.tasks.files.pattern</span>).</p> |
---|
2503 | <p> |
---|
2504 | Next, go to the node on which the failed task ran and go to the |
---|
2505 | <span class="codefrag">TaskTracker</span>'s local directory and run the |
---|
2506 | <span class="codefrag">IsolationRunner</span>:<br> |
---|
2507 | |
---|
2508 | <span class="codefrag">$ cd <local path>/taskTracker/${taskid}/work</span> |
---|
2509 | <br> |
---|
2510 | |
---|
2511 | <span class="codefrag"> |
---|
2512 | $ bin/hadoop org.apache.hadoop.mapred.IsolationRunner ../job.xml |
---|
2513 | </span> |
---|
2514 | |
---|
2515 | </p> |
---|
2516 | <p> |
---|
2517 | <span class="codefrag">IsolationRunner</span> will run the failed task in a single |
---|
2518 | jvm, which can be in the debugger, over precisely the same input.</p> |
---|
2519 | <a name="N10E77"></a><a name="Profiling"></a> |
---|
2520 | <h4>Profiling</h4> |
---|
2521 | <p>Profiling is a utility to get a representative (2 or 3) sample |
---|
2522 | of built-in java profiler for a sample of maps and reduces. </p> |
---|
2523 | <p>User can specify whether the system should collect profiler |
---|
2524 | information for some of the tasks in the job by setting the |
---|
2525 | configuration property <span class="codefrag">mapred.task.profile</span>. The |
---|
2526 | value can be set using the api |
---|
2527 | <a href="api/org/apache/hadoop/mapred/JobConf.html#setProfileEnabled(boolean)"> |
---|
2528 | JobConf.setProfileEnabled(boolean)</a>. If the value is set |
---|
2529 | <span class="codefrag">true</span>, the task profiling is enabled. The profiler |
---|
2530 | information is stored in the user log directory. By default, |
---|
2531 | profiling is not enabled for the job. </p> |
---|
2532 | <p>Once user configures that profiling is needed, she/he can use |
---|
2533 | the configuration property |
---|
2534 | <span class="codefrag">mapred.task.profile.{maps|reduces}</span> to set the ranges |
---|
2535 | of map/reduce tasks to profile. The value can be set using the api |
---|
2536 | <a href="api/org/apache/hadoop/mapred/JobConf.html#setProfileTaskRange(boolean,%20java.lang.String)"> |
---|
2537 | JobConf.setProfileTaskRange(boolean,String)</a>. |
---|
2538 | By default, the specified range is <span class="codefrag">0-2</span>.</p> |
---|
2539 | <p>User can also specify the profiler configuration arguments by |
---|
2540 | setting the configuration property |
---|
2541 | <span class="codefrag">mapred.task.profile.params</span>. The value can be specified |
---|
2542 | using the api |
---|
2543 | <a href="api/org/apache/hadoop/mapred/JobConf.html#setProfileParams(java.lang.String)"> |
---|
2544 | JobConf.setProfileParams(String)</a>. If the string contains a |
---|
2545 | <span class="codefrag">%s</span>, it will be replaced with the name of the profiling |
---|
2546 | output file when the task runs. These parameters are passed to the |
---|
2547 | task child JVM on the command line. The default value for |
---|
2548 | the profiling parameters is |
---|
2549 | <span class="codefrag">-agentlib:hprof=cpu=samples,heap=sites,force=n,thread=y,verbose=n,file=%s</span> |
---|
2550 | |
---|
2551 | </p> |
---|
2552 | <a name="N10EAB"></a><a name="Debugging"></a> |
---|
2553 | <h4>Debugging</h4> |
---|
2554 | <p>The Map/Reduce framework provides a facility to run user-provided |
---|
2555 | scripts for debugging. When a map/reduce task fails, a user can run |
---|
2556 | a debug script, to process task logs for example. The script is |
---|
2557 | given access to the task's stdout and stderr outputs, syslog and |
---|
2558 | jobconf. The output from the debug script's stdout and stderr is |
---|
2559 | displayed on the console diagnostics and also as part of the |
---|
2560 | job UI. </p> |
---|
2561 | <p> In the following sections we discuss how to submit a debug script |
---|
2562 | with a job. The script file needs to be distributed and submitted to |
---|
2563 | the framework.</p> |
---|
2564 | <a name="N10EB7"></a><a name="How+to+distribute+the+script+file%3A"></a> |
---|
2565 | <h5> How to distribute the script file: </h5> |
---|
2566 | <p> |
---|
2567 | The user needs to use |
---|
2568 | <a href="mapred_tutorial.html#DistributedCache">DistributedCache</a> |
---|
2569 | to <em>distribute</em> and <em>symlink</em> the script file.</p> |
---|
2570 | <a name="N10ECB"></a><a name="How+to+submit+the+script%3A"></a> |
---|
2571 | <h5> How to submit the script: </h5> |
---|
2572 | <p> A quick way to submit the debug script is to set values for the |
---|
2573 | properties <span class="codefrag">mapred.map.task.debug.script</span> and |
---|
2574 | <span class="codefrag">mapred.reduce.task.debug.script</span>, for debugging map and |
---|
2575 | reduce tasks respectively. These properties can also be set by using APIs |
---|
2576 | <a href="api/org/apache/hadoop/mapred/JobConf.html#setMapDebugScript(java.lang.String)"> |
---|
2577 | JobConf.setMapDebugScript(String) </a> and |
---|
2578 | <a href="api/org/apache/hadoop/mapred/JobConf.html#setReduceDebugScript(java.lang.String)"> |
---|
2579 | JobConf.setReduceDebugScript(String) </a>. In streaming mode, a debug |
---|
2580 | script can be submitted with the command-line options |
---|
2581 | <span class="codefrag">-mapdebug</span> and <span class="codefrag">-reducedebug</span>, for debugging |
---|
2582 | map and reduce tasks respectively.</p> |
---|
2583 | <p>The arguments to the script are the task's stdout, stderr, |
---|
2584 | syslog and jobconf files. The debug command, run on the node where |
---|
2585 | the map/reduce task failed, is: <br> |
---|
2586 | |
---|
2587 | <span class="codefrag"> $script $stdout $stderr $syslog $jobconf </span> |
---|
2588 | </p> |
---|
2589 | <p> Pipes programs have the c++ program name as a fifth argument |
---|
2590 | for the command. Thus for the pipes programs the command is <br> |
---|
2591 | |
---|
2592 | <span class="codefrag">$script $stdout $stderr $syslog $jobconf $program </span> |
---|
2593 | |
---|
2594 | </p> |
---|
2595 | <a name="N10EF9"></a><a name="Default+Behavior%3A"></a> |
---|
2596 | <h5> Default Behavior: </h5> |
---|
2597 | <p> For pipes, a default script is run to process core dumps under |
---|
2598 | gdb, prints stack trace and gives info about running threads. </p> |
---|
2599 | <a name="N10F04"></a><a name="JobControl"></a> |
---|
2600 | <h4>JobControl</h4> |
---|
2601 | <p> |
---|
2602 | <a href="api/org/apache/hadoop/mapred/jobcontrol/package-summary.html"> |
---|
2603 | JobControl</a> is a utility which encapsulates a set of Map/Reduce jobs |
---|
2604 | and their dependencies.</p> |
---|
2605 | <a name="N10F11"></a><a name="Data+Compression"></a> |
---|
2606 | <h4>Data Compression</h4> |
---|
2607 | <p>Hadoop Map/Reduce provides facilities for the application-writer to |
---|
2608 | specify compression for both intermediate map-outputs and the |
---|
2609 | job-outputs i.e. output of the reduces. It also comes bundled with |
---|
2610 | <a href="api/org/apache/hadoop/io/compress/CompressionCodec.html"> |
---|
2611 | CompressionCodec</a> implementation for the |
---|
2612 | <a href="http://www.zlib.net/">zlib</a> compression |
---|
2613 | algorithm. The <a href="http://www.gzip.org/">gzip</a> file format is also |
---|
2614 | supported.</p> |
---|
2615 | <p>Hadoop also provides native implementations of the above compression |
---|
2616 | codecs for reasons of both performance (zlib) and non-availability of |
---|
2617 | Java libraries. More details on their usage and availability are |
---|
2618 | available <a href="native_libraries.html">here</a>.</p> |
---|
2619 | <a name="N10F2D"></a><a name="Intermediate+Outputs"></a> |
---|
2620 | <h5>Intermediate Outputs</h5> |
---|
2621 | <p>Applications can control compression of intermediate map-outputs |
---|
2622 | via the |
---|
2623 | <a href="api/org/apache/hadoop/mapred/JobConf.html#setCompressMapOutput(boolean)"> |
---|
2624 | JobConf.setCompressMapOutput(boolean)</a> api and the |
---|
2625 | <span class="codefrag">CompressionCodec</span> to be used via the |
---|
2626 | <a href="api/org/apache/hadoop/mapred/JobConf.html#setMapOutputCompressorClass(java.lang.Class)"> |
---|
2627 | JobConf.setMapOutputCompressorClass(Class)</a> api.</p> |
---|
2628 | <a name="N10F42"></a><a name="Job+Outputs"></a> |
---|
2629 | <h5>Job Outputs</h5> |
---|
2630 | <p>Applications can control compression of job-outputs via the |
---|
2631 | <a href="api/org/apache/hadoop/mapred/FileOutputFormat.html#setCompressOutput(org.apache.hadoop.mapred.JobConf,%20boolean)"> |
---|
2632 | FileOutputFormat.setCompressOutput(JobConf, boolean)</a> api and the |
---|
2633 | <span class="codefrag">CompressionCodec</span> to be used can be specified via the |
---|
2634 | <a href="api/org/apache/hadoop/mapred/FileOutputFormat.html#setOutputCompressorClass(org.apache.hadoop.mapred.JobConf,%20java.lang.Class)"> |
---|
2635 | FileOutputFormat.setOutputCompressorClass(JobConf, Class)</a> api.</p> |
---|
2636 | <p>If the job outputs are to be stored in the |
---|
2637 | <a href="api/org/apache/hadoop/mapred/SequenceFileOutputFormat.html"> |
---|
2638 | SequenceFileOutputFormat</a>, the required |
---|
2639 | <span class="codefrag">SequenceFile.CompressionType</span> (i.e. <span class="codefrag">RECORD</span> / |
---|
2640 | <span class="codefrag">BLOCK</span> - defaults to <span class="codefrag">RECORD</span>) can be |
---|
2641 | specified via the |
---|
2642 | <a href="api/org/apache/hadoop/mapred/SequenceFileOutputFormat.html#setOutputCompressionType(org.apache.hadoop.mapred.JobConf,%20org.apache.hadoop.io.SequenceFile.CompressionType)"> |
---|
2643 | SequenceFileOutputFormat.setOutputCompressionType(JobConf, |
---|
2644 | SequenceFile.CompressionType)</a> api.</p> |
---|
2645 | <a name="N10F6F"></a><a name="Skipping+Bad+Records"></a> |
---|
2646 | <h4>Skipping Bad Records</h4> |
---|
2647 | <p>Hadoop provides an option where a certain set of bad input |
---|
2648 | records can be skipped when processing map inputs. Applications |
---|
2649 | can control this feature through the |
---|
2650 | <a href="api/org/apache/hadoop/mapred/SkipBadRecords.html"> |
---|
2651 | SkipBadRecords</a> class.</p> |
---|
2652 | <p>This feature can be used when map tasks crash deterministically |
---|
2653 | on certain input. This usually happens due to bugs in the |
---|
2654 | map function. Usually, the user would have to fix these bugs. |
---|
2655 | This is, however, not possible sometimes. The bug may be in third |
---|
2656 | party libraries, for example, for which the source code is not |
---|
2657 | available. In such cases, the task never completes successfully even |
---|
2658 | after multiple attempts, and the job fails. With this feature, only |
---|
2659 | a small portion of data surrounding the |
---|
2660 | bad records is lost, which may be acceptable for some applications |
---|
2661 | (those performing statistical analysis on very large data, for |
---|
2662 | example). </p> |
---|
2663 | <p>By default this feature is disabled. For enabling it, |
---|
2664 | refer to <a href="api/org/apache/hadoop/mapred/SkipBadRecords.html#setMapperMaxSkipRecords(org.apache.hadoop.conf.Configuration, long)"> |
---|
2665 | SkipBadRecords.setMapperMaxSkipRecords(Configuration, long)</a> and |
---|
2666 | <a href="api/org/apache/hadoop/mapred/SkipBadRecords.html#setReducerMaxSkipGroups(org.apache.hadoop.conf.Configuration, long)"> |
---|
2667 | SkipBadRecords.setReducerMaxSkipGroups(Configuration, long)</a>. |
---|
2668 | </p> |
---|
2669 | <p>With this feature enabled, the framework gets into 'skipping |
---|
2670 | mode' after a certain number of map failures. For more details, |
---|
2671 | see <a href="api/org/apache/hadoop/mapred/SkipBadRecords.html#setAttemptsToStartSkipping(org.apache.hadoop.conf.Configuration, int)"> |
---|
2672 | SkipBadRecords.setAttemptsToStartSkipping(Configuration, int)</a>. |
---|
2673 | In 'skipping mode', map tasks maintain the range of records being |
---|
2674 | processed. To do this, the framework relies on the processed record |
---|
2675 | counter. See <a href="api/org/apache/hadoop/mapred/SkipBadRecords.html#COUNTER_MAP_PROCESSED_RECORDS"> |
---|
2676 | SkipBadRecords.COUNTER_MAP_PROCESSED_RECORDS</a> and |
---|
2677 | <a href="api/org/apache/hadoop/mapred/SkipBadRecords.html#COUNTER_REDUCE_PROCESSED_GROUPS"> |
---|
2678 | SkipBadRecords.COUNTER_REDUCE_PROCESSED_GROUPS</a>. |
---|
2679 | This counter enables the framework to know how many records have |
---|
2680 | been processed successfully, and hence, what record range caused |
---|
2681 | a task to crash. On further attempts, this range of records is |
---|
2682 | skipped.</p> |
---|
2683 | <p>The number of records skipped depends on how frequently the |
---|
2684 | processed record counter is incremented by the application. |
---|
2685 | It is recommended that this counter be incremented after every |
---|
2686 | record is processed. This may not be possible in some applications |
---|
2687 | that typically batch their processing. In such cases, the framework |
---|
2688 | may skip additional records surrounding the bad record. Users can |
---|
2689 | control the number of skipped records through |
---|
2690 | <a href="api/org/apache/hadoop/mapred/SkipBadRecords.html#setMapperMaxSkipRecords(org.apache.hadoop.conf.Configuration, long)"> |
---|
2691 | SkipBadRecords.setMapperMaxSkipRecords(Configuration, long)</a> and |
---|
2692 | <a href="api/org/apache/hadoop/mapred/SkipBadRecords.html#setReducerMaxSkipGroups(org.apache.hadoop.conf.Configuration, long)"> |
---|
2693 | SkipBadRecords.setReducerMaxSkipGroups(Configuration, long)</a>. |
---|
2694 | The framework tries to narrow the range of skipped records using a |
---|
2695 | binary search-like approach. The skipped range is divided into two |
---|
2696 | halves and only one half gets executed. On subsequent |
---|
2697 | failures, the framework figures out which half contains |
---|
2698 | bad records. A task will be re-executed till the |
---|
2699 | acceptable skipped value is met or all task attempts are exhausted. |
---|
2700 | To increase the number of task attempts, use |
---|
2701 | <a href="api/org/apache/hadoop/mapred/JobConf.html#setMaxMapAttempts(int)"> |
---|
2702 | JobConf.setMaxMapAttempts(int)</a> and |
---|
2703 | <a href="api/org/apache/hadoop/mapred/JobConf.html#setMaxReduceAttempts(int)"> |
---|
2704 | JobConf.setMaxReduceAttempts(int)</a>. |
---|
2705 | </p> |
---|
2706 | <p>Skipped records are written to HDFS in the sequence file |
---|
2707 | format, for later analysis. The location can be changed through |
---|
2708 | <a href="api/org/apache/hadoop/mapred/SkipBadRecords.html#setSkipOutputPath(org.apache.hadoop.mapred.JobConf, org.apache.hadoop.fs.Path)"> |
---|
2709 | SkipBadRecords.setSkipOutputPath(JobConf, Path)</a>. |
---|
2710 | </p> |
---|
2711 | </div> |
---|
2712 | |
---|
2713 | |
---|
2714 | <a name="N10FB6"></a><a name="Example%3A+WordCount+v2.0"></a> |
---|
2715 | <h2 class="h3">Example: WordCount v2.0</h2> |
---|
2716 | <div class="section"> |
---|
2717 | <p>Here is a more complete <span class="codefrag">WordCount</span> which uses many of the |
---|
2718 | features provided by the Map/Reduce framework we discussed so far.</p> |
---|
2719 | <p>This needs the HDFS to be up and running, especially for the |
---|
2720 | <span class="codefrag">DistributedCache</span>-related features. Hence it only works with a |
---|
2721 | <a href="quickstart.html#SingleNodeSetup">pseudo-distributed</a> or |
---|
2722 | <a href="quickstart.html#Fully-Distributed+Operation">fully-distributed</a> |
---|
2723 | Hadoop installation.</p> |
---|
2724 | <a name="N10FD0"></a><a name="Source+Code-N10FD0"></a> |
---|
2725 | <h3 class="h4">Source Code</h3> |
---|
2726 | <table class="ForrestTable" cellspacing="1" cellpadding="4"> |
---|
2727 | |
---|
2728 | <tr> |
---|
2729 | |
---|
2730 | <th colspan="1" rowspan="1"></th> |
---|
2731 | <th colspan="1" rowspan="1">WordCount.java</th> |
---|
2732 | |
---|
2733 | </tr> |
---|
2734 | |
---|
2735 | <tr> |
---|
2736 | |
---|
2737 | <td colspan="1" rowspan="1">1.</td> |
---|
2738 | <td colspan="1" rowspan="1"> |
---|
2739 | <span class="codefrag">package org.myorg;</span> |
---|
2740 | </td> |
---|
2741 | |
---|
2742 | </tr> |
---|
2743 | |
---|
2744 | <tr> |
---|
2745 | |
---|
2746 | <td colspan="1" rowspan="1">2.</td> |
---|
2747 | <td colspan="1" rowspan="1"></td> |
---|
2748 | |
---|
2749 | </tr> |
---|
2750 | |
---|
2751 | <tr> |
---|
2752 | |
---|
2753 | <td colspan="1" rowspan="1">3.</td> |
---|
2754 | <td colspan="1" rowspan="1"> |
---|
2755 | <span class="codefrag">import java.io.*;</span> |
---|
2756 | </td> |
---|
2757 | |
---|
2758 | </tr> |
---|
2759 | |
---|
2760 | <tr> |
---|
2761 | |
---|
2762 | <td colspan="1" rowspan="1">4.</td> |
---|
2763 | <td colspan="1" rowspan="1"> |
---|
2764 | <span class="codefrag">import java.util.*;</span> |
---|
2765 | </td> |
---|
2766 | |
---|
2767 | </tr> |
---|
2768 | |
---|
2769 | <tr> |
---|
2770 | |
---|
2771 | <td colspan="1" rowspan="1">5.</td> |
---|
2772 | <td colspan="1" rowspan="1"></td> |
---|
2773 | |
---|
2774 | </tr> |
---|
2775 | |
---|
2776 | <tr> |
---|
2777 | |
---|
2778 | <td colspan="1" rowspan="1">6.</td> |
---|
2779 | <td colspan="1" rowspan="1"> |
---|
2780 | <span class="codefrag">import org.apache.hadoop.fs.Path;</span> |
---|
2781 | </td> |
---|
2782 | |
---|
2783 | </tr> |
---|
2784 | |
---|
2785 | <tr> |
---|
2786 | |
---|
2787 | <td colspan="1" rowspan="1">7.</td> |
---|
2788 | <td colspan="1" rowspan="1"> |
---|
2789 | <span class="codefrag">import org.apache.hadoop.filecache.DistributedCache;</span> |
---|
2790 | </td> |
---|
2791 | |
---|
2792 | </tr> |
---|
2793 | |
---|
2794 | <tr> |
---|
2795 | |
---|
2796 | <td colspan="1" rowspan="1">8.</td> |
---|
2797 | <td colspan="1" rowspan="1"> |
---|
2798 | <span class="codefrag">import org.apache.hadoop.conf.*;</span> |
---|
2799 | </td> |
---|
2800 | |
---|
2801 | </tr> |
---|
2802 | |
---|
2803 | <tr> |
---|
2804 | |
---|
2805 | <td colspan="1" rowspan="1">9.</td> |
---|
2806 | <td colspan="1" rowspan="1"> |
---|
2807 | <span class="codefrag">import org.apache.hadoop.io.*;</span> |
---|
2808 | </td> |
---|
2809 | |
---|
2810 | </tr> |
---|
2811 | |
---|
2812 | <tr> |
---|
2813 | |
---|
2814 | <td colspan="1" rowspan="1">10.</td> |
---|
2815 | <td colspan="1" rowspan="1"> |
---|
2816 | <span class="codefrag">import org.apache.hadoop.mapred.*;</span> |
---|
2817 | </td> |
---|
2818 | |
---|
2819 | </tr> |
---|
2820 | |
---|
2821 | <tr> |
---|
2822 | |
---|
2823 | <td colspan="1" rowspan="1">11.</td> |
---|
2824 | <td colspan="1" rowspan="1"> |
---|
2825 | <span class="codefrag">import org.apache.hadoop.util.*;</span> |
---|
2826 | </td> |
---|
2827 | |
---|
2828 | </tr> |
---|
2829 | |
---|
2830 | <tr> |
---|
2831 | |
---|
2832 | <td colspan="1" rowspan="1">12.</td> |
---|
2833 | <td colspan="1" rowspan="1"></td> |
---|
2834 | |
---|
2835 | </tr> |
---|
2836 | |
---|
2837 | <tr> |
---|
2838 | |
---|
2839 | <td colspan="1" rowspan="1">13.</td> |
---|
2840 | <td colspan="1" rowspan="1"> |
---|
2841 | <span class="codefrag">public class WordCount extends Configured implements Tool {</span> |
---|
2842 | </td> |
---|
2843 | |
---|
2844 | </tr> |
---|
2845 | |
---|
2846 | <tr> |
---|
2847 | |
---|
2848 | <td colspan="1" rowspan="1">14.</td> |
---|
2849 | <td colspan="1" rowspan="1"></td> |
---|
2850 | |
---|
2851 | </tr> |
---|
2852 | |
---|
2853 | <tr> |
---|
2854 | |
---|
2855 | <td colspan="1" rowspan="1">15.</td> |
---|
2856 | <td colspan="1" rowspan="1"> |
---|
2857 | |
---|
2858 | <span class="codefrag"> |
---|
2859 | public static class Map extends MapReduceBase |
---|
2860 | implements Mapper<LongWritable, Text, Text, IntWritable> { |
---|
2861 | </span> |
---|
2862 | </td> |
---|
2863 | |
---|
2864 | </tr> |
---|
2865 | |
---|
2866 | <tr> |
---|
2867 | |
---|
2868 | <td colspan="1" rowspan="1">16.</td> |
---|
2869 | <td colspan="1" rowspan="1"></td> |
---|
2870 | |
---|
2871 | </tr> |
---|
2872 | |
---|
2873 | <tr> |
---|
2874 | |
---|
2875 | <td colspan="1" rowspan="1">17.</td> |
---|
2876 | <td colspan="1" rowspan="1"> |
---|
2877 | |
---|
2878 | <span class="codefrag"> |
---|
2879 | static enum Counters { INPUT_WORDS } |
---|
2880 | </span> |
---|
2881 | </td> |
---|
2882 | |
---|
2883 | </tr> |
---|
2884 | |
---|
2885 | <tr> |
---|
2886 | |
---|
2887 | <td colspan="1" rowspan="1">18.</td> |
---|
2888 | <td colspan="1" rowspan="1"></td> |
---|
2889 | |
---|
2890 | </tr> |
---|
2891 | |
---|
2892 | <tr> |
---|
2893 | |
---|
2894 | <td colspan="1" rowspan="1">19.</td> |
---|
2895 | <td colspan="1" rowspan="1"> |
---|
2896 | |
---|
2897 | <span class="codefrag"> |
---|
2898 | private final static IntWritable one = new IntWritable(1); |
---|
2899 | </span> |
---|
2900 | </td> |
---|
2901 | |
---|
2902 | </tr> |
---|
2903 | |
---|
2904 | <tr> |
---|
2905 | |
---|
2906 | <td colspan="1" rowspan="1">20.</td> |
---|
2907 | <td colspan="1" rowspan="1"> |
---|
2908 | |
---|
2909 | <span class="codefrag">private Text word = new Text();</span> |
---|
2910 | </td> |
---|
2911 | |
---|
2912 | </tr> |
---|
2913 | |
---|
2914 | <tr> |
---|
2915 | |
---|
2916 | <td colspan="1" rowspan="1">21.</td> |
---|
2917 | <td colspan="1" rowspan="1"></td> |
---|
2918 | |
---|
2919 | </tr> |
---|
2920 | |
---|
2921 | <tr> |
---|
2922 | |
---|
2923 | <td colspan="1" rowspan="1">22.</td> |
---|
2924 | <td colspan="1" rowspan="1"> |
---|
2925 | |
---|
2926 | <span class="codefrag">private boolean caseSensitive = true;</span> |
---|
2927 | </td> |
---|
2928 | |
---|
2929 | </tr> |
---|
2930 | |
---|
2931 | <tr> |
---|
2932 | |
---|
2933 | <td colspan="1" rowspan="1">23.</td> |
---|
2934 | <td colspan="1" rowspan="1"> |
---|
2935 | |
---|
2936 | <span class="codefrag">private Set<String> patternsToSkip = new HashSet<String>();</span> |
---|
2937 | </td> |
---|
2938 | |
---|
2939 | </tr> |
---|
2940 | |
---|
2941 | <tr> |
---|
2942 | |
---|
2943 | <td colspan="1" rowspan="1">24.</td> |
---|
2944 | <td colspan="1" rowspan="1"></td> |
---|
2945 | |
---|
2946 | </tr> |
---|
2947 | |
---|
2948 | <tr> |
---|
2949 | |
---|
2950 | <td colspan="1" rowspan="1">25.</td> |
---|
2951 | <td colspan="1" rowspan="1"> |
---|
2952 | |
---|
2953 | <span class="codefrag">private long numRecords = 0;</span> |
---|
2954 | </td> |
---|
2955 | |
---|
2956 | </tr> |
---|
2957 | |
---|
2958 | <tr> |
---|
2959 | |
---|
2960 | <td colspan="1" rowspan="1">26.</td> |
---|
2961 | <td colspan="1" rowspan="1"> |
---|
2962 | |
---|
2963 | <span class="codefrag">private String inputFile;</span> |
---|
2964 | </td> |
---|
2965 | |
---|
2966 | </tr> |
---|
2967 | |
---|
2968 | <tr> |
---|
2969 | |
---|
2970 | <td colspan="1" rowspan="1">27.</td> |
---|
2971 | <td colspan="1" rowspan="1"></td> |
---|
2972 | |
---|
2973 | </tr> |
---|
2974 | |
---|
2975 | <tr> |
---|
2976 | |
---|
2977 | <td colspan="1" rowspan="1">28.</td> |
---|
2978 | <td colspan="1" rowspan="1"> |
---|
2979 | |
---|
2980 | <span class="codefrag">public void configure(JobConf job) {</span> |
---|
2981 | </td> |
---|
2982 | |
---|
2983 | </tr> |
---|
2984 | |
---|
2985 | <tr> |
---|
2986 | |
---|
2987 | <td colspan="1" rowspan="1">29.</td> |
---|
2988 | <td colspan="1" rowspan="1"> |
---|
2989 | |
---|
2990 | <span class="codefrag"> |
---|
2991 | caseSensitive = job.getBoolean("wordcount.case.sensitive", true); |
---|
2992 | </span> |
---|
2993 | </td> |
---|
2994 | |
---|
2995 | </tr> |
---|
2996 | |
---|
2997 | <tr> |
---|
2998 | |
---|
2999 | <td colspan="1" rowspan="1">30.</td> |
---|
3000 | <td colspan="1" rowspan="1"> |
---|
3001 | |
---|
3002 | <span class="codefrag">inputFile = job.get("map.input.file");</span> |
---|
3003 | </td> |
---|
3004 | |
---|
3005 | </tr> |
---|
3006 | |
---|
3007 | <tr> |
---|
3008 | |
---|
3009 | <td colspan="1" rowspan="1">31.</td> |
---|
3010 | <td colspan="1" rowspan="1"></td> |
---|
3011 | |
---|
3012 | </tr> |
---|
3013 | |
---|
3014 | <tr> |
---|
3015 | |
---|
3016 | <td colspan="1" rowspan="1">32.</td> |
---|
3017 | <td colspan="1" rowspan="1"> |
---|
3018 | |
---|
3019 | <span class="codefrag">if (job.getBoolean("wordcount.skip.patterns", false)) {</span> |
---|
3020 | </td> |
---|
3021 | |
---|
3022 | </tr> |
---|
3023 | |
---|
3024 | <tr> |
---|
3025 | |
---|
3026 | <td colspan="1" rowspan="1">33.</td> |
---|
3027 | <td colspan="1" rowspan="1"> |
---|
3028 | |
---|
3029 | <span class="codefrag">Path[] patternsFiles = new Path[0];</span> |
---|
3030 | </td> |
---|
3031 | |
---|
3032 | </tr> |
---|
3033 | |
---|
3034 | <tr> |
---|
3035 | |
---|
3036 | <td colspan="1" rowspan="1">34.</td> |
---|
3037 | <td colspan="1" rowspan="1"> |
---|
3038 | |
---|
3039 | <span class="codefrag">try {</span> |
---|
3040 | </td> |
---|
3041 | |
---|
3042 | </tr> |
---|
3043 | |
---|
3044 | <tr> |
---|
3045 | |
---|
3046 | <td colspan="1" rowspan="1">35.</td> |
---|
3047 | <td colspan="1" rowspan="1"> |
---|
3048 | |
---|
3049 | <span class="codefrag"> |
---|
3050 | patternsFiles = DistributedCache.getLocalCacheFiles(job); |
---|
3051 | </span> |
---|
3052 | </td> |
---|
3053 | |
---|
3054 | </tr> |
---|
3055 | |
---|
3056 | <tr> |
---|
3057 | |
---|
3058 | <td colspan="1" rowspan="1">36.</td> |
---|
3059 | <td colspan="1" rowspan="1"> |
---|
3060 | |
---|
3061 | <span class="codefrag">} catch (IOException ioe) {</span> |
---|
3062 | </td> |
---|
3063 | |
---|
3064 | </tr> |
---|
3065 | |
---|
3066 | <tr> |
---|
3067 | |
---|
3068 | <td colspan="1" rowspan="1">37.</td> |
---|
3069 | <td colspan="1" rowspan="1"> |
---|
3070 | |
---|
3071 | <span class="codefrag"> |
---|
3072 | System.err.println("Caught exception while getting cached files: " |
---|
3073 | + StringUtils.stringifyException(ioe)); |
---|
3074 | </span> |
---|
3075 | </td> |
---|
3076 | |
---|
3077 | </tr> |
---|
3078 | |
---|
3079 | <tr> |
---|
3080 | |
---|
3081 | <td colspan="1" rowspan="1">38.</td> |
---|
3082 | <td colspan="1" rowspan="1"> |
---|
3083 | |
---|
3084 | <span class="codefrag">}</span> |
---|
3085 | </td> |
---|
3086 | |
---|
3087 | </tr> |
---|
3088 | |
---|
3089 | <tr> |
---|
3090 | |
---|
3091 | <td colspan="1" rowspan="1">39.</td> |
---|
3092 | <td colspan="1" rowspan="1"> |
---|
3093 | |
---|
3094 | <span class="codefrag">for (Path patternsFile : patternsFiles) {</span> |
---|
3095 | </td> |
---|
3096 | |
---|
3097 | </tr> |
---|
3098 | |
---|
3099 | <tr> |
---|
3100 | |
---|
3101 | <td colspan="1" rowspan="1">40.</td> |
---|
3102 | <td colspan="1" rowspan="1"> |
---|
3103 | |
---|
3104 | <span class="codefrag">parseSkipFile(patternsFile);</span> |
---|
3105 | </td> |
---|
3106 | |
---|
3107 | </tr> |
---|
3108 | |
---|
3109 | <tr> |
---|
3110 | |
---|
3111 | <td colspan="1" rowspan="1">41.</td> |
---|
3112 | <td colspan="1" rowspan="1"> |
---|
3113 | |
---|
3114 | <span class="codefrag">}</span> |
---|
3115 | </td> |
---|
3116 | |
---|
3117 | </tr> |
---|
3118 | |
---|
3119 | <tr> |
---|
3120 | |
---|
3121 | <td colspan="1" rowspan="1">42.</td> |
---|
3122 | <td colspan="1" rowspan="1"> |
---|
3123 | |
---|
3124 | <span class="codefrag">}</span> |
---|
3125 | </td> |
---|
3126 | |
---|
3127 | </tr> |
---|
3128 | |
---|
3129 | <tr> |
---|
3130 | |
---|
3131 | <td colspan="1" rowspan="1">43.</td> |
---|
3132 | <td colspan="1" rowspan="1"> |
---|
3133 | |
---|
3134 | <span class="codefrag">}</span> |
---|
3135 | </td> |
---|
3136 | |
---|
3137 | </tr> |
---|
3138 | |
---|
3139 | <tr> |
---|
3140 | |
---|
3141 | <td colspan="1" rowspan="1">44.</td> |
---|
3142 | <td colspan="1" rowspan="1"></td> |
---|
3143 | |
---|
3144 | </tr> |
---|
3145 | |
---|
3146 | <tr> |
---|
3147 | |
---|
3148 | <td colspan="1" rowspan="1">45.</td> |
---|
3149 | <td colspan="1" rowspan="1"> |
---|
3150 | |
---|
3151 | <span class="codefrag">private void parseSkipFile(Path patternsFile) {</span> |
---|
3152 | </td> |
---|
3153 | |
---|
3154 | </tr> |
---|
3155 | |
---|
3156 | <tr> |
---|
3157 | |
---|
3158 | <td colspan="1" rowspan="1">46.</td> |
---|
3159 | <td colspan="1" rowspan="1"> |
---|
3160 | |
---|
3161 | <span class="codefrag">try {</span> |
---|
3162 | </td> |
---|
3163 | |
---|
3164 | </tr> |
---|
3165 | |
---|
3166 | <tr> |
---|
3167 | |
---|
3168 | <td colspan="1" rowspan="1">47.</td> |
---|
3169 | <td colspan="1" rowspan="1"> |
---|
3170 | |
---|
3171 | <span class="codefrag"> |
---|
3172 | BufferedReader fis = |
---|
3173 | new BufferedReader(new FileReader(patternsFile.toString())); |
---|
3174 | </span> |
---|
3175 | </td> |
---|
3176 | |
---|
3177 | </tr> |
---|
3178 | |
---|
3179 | <tr> |
---|
3180 | |
---|
3181 | <td colspan="1" rowspan="1">48.</td> |
---|
3182 | <td colspan="1" rowspan="1"> |
---|
3183 | |
---|
3184 | <span class="codefrag">String pattern = null;</span> |
---|
3185 | </td> |
---|
3186 | |
---|
3187 | </tr> |
---|
3188 | |
---|
3189 | <tr> |
---|
3190 | |
---|
3191 | <td colspan="1" rowspan="1">49.</td> |
---|
3192 | <td colspan="1" rowspan="1"> |
---|
3193 | |
---|
3194 | <span class="codefrag">while ((pattern = fis.readLine()) != null) {</span> |
---|
3195 | </td> |
---|
3196 | |
---|
3197 | </tr> |
---|
3198 | |
---|
3199 | <tr> |
---|
3200 | |
---|
3201 | <td colspan="1" rowspan="1">50.</td> |
---|
3202 | <td colspan="1" rowspan="1"> |
---|
3203 | |
---|
3204 | <span class="codefrag">patternsToSkip.add(pattern);</span> |
---|
3205 | </td> |
---|
3206 | |
---|
3207 | </tr> |
---|
3208 | |
---|
3209 | <tr> |
---|
3210 | |
---|
3211 | <td colspan="1" rowspan="1">51.</td> |
---|
3212 | <td colspan="1" rowspan="1"> |
---|
3213 | |
---|
3214 | <span class="codefrag">}</span> |
---|
3215 | </td> |
---|
3216 | |
---|
3217 | </tr> |
---|
3218 | |
---|
3219 | <tr> |
---|
3220 | |
---|
3221 | <td colspan="1" rowspan="1">52.</td> |
---|
3222 | <td colspan="1" rowspan="1"> |
---|
3223 | |
---|
3224 | <span class="codefrag">} catch (IOException ioe) {</span> |
---|
3225 | </td> |
---|
3226 | |
---|
3227 | </tr> |
---|
3228 | |
---|
3229 | <tr> |
---|
3230 | |
---|
3231 | <td colspan="1" rowspan="1">53.</td> |
---|
3232 | <td colspan="1" rowspan="1"> |
---|
3233 | |
---|
3234 | <span class="codefrag"> |
---|
3235 | System.err.println("Caught exception while parsing the cached file '" + |
---|
3236 | patternsFile + "' : " + |
---|
3237 | StringUtils.stringifyException(ioe)); |
---|
3238 | |
---|
3239 | </span> |
---|
3240 | </td> |
---|
3241 | |
---|
3242 | </tr> |
---|
3243 | |
---|
3244 | <tr> |
---|
3245 | |
---|
3246 | <td colspan="1" rowspan="1">54.</td> |
---|
3247 | <td colspan="1" rowspan="1"> |
---|
3248 | |
---|
3249 | <span class="codefrag">}</span> |
---|
3250 | </td> |
---|
3251 | |
---|
3252 | </tr> |
---|
3253 | |
---|
3254 | <tr> |
---|
3255 | |
---|
3256 | <td colspan="1" rowspan="1">55.</td> |
---|
3257 | <td colspan="1" rowspan="1"> |
---|
3258 | |
---|
3259 | <span class="codefrag">}</span> |
---|
3260 | </td> |
---|
3261 | |
---|
3262 | </tr> |
---|
3263 | |
---|
3264 | <tr> |
---|
3265 | |
---|
3266 | <td colspan="1" rowspan="1">56.</td> |
---|
3267 | <td colspan="1" rowspan="1"></td> |
---|
3268 | |
---|
3269 | </tr> |
---|
3270 | |
---|
3271 | <tr> |
---|
3272 | |
---|
3273 | <td colspan="1" rowspan="1">57.</td> |
---|
3274 | <td colspan="1" rowspan="1"> |
---|
3275 | |
---|
3276 | <span class="codefrag"> |
---|
3277 | public void map(LongWritable key, Text value, |
---|
3278 | OutputCollector<Text, IntWritable> output, |
---|
3279 | Reporter reporter) throws IOException { |
---|
3280 | </span> |
---|
3281 | </td> |
---|
3282 | |
---|
3283 | </tr> |
---|
3284 | |
---|
3285 | <tr> |
---|
3286 | |
---|
3287 | <td colspan="1" rowspan="1">58.</td> |
---|
3288 | <td colspan="1" rowspan="1"> |
---|
3289 | |
---|
3290 | <span class="codefrag"> |
---|
3291 | String line = |
---|
3292 | (caseSensitive) ? value.toString() : |
---|
3293 | value.toString().toLowerCase(); |
---|
3294 | </span> |
---|
3295 | </td> |
---|
3296 | |
---|
3297 | </tr> |
---|
3298 | |
---|
3299 | <tr> |
---|
3300 | |
---|
3301 | <td colspan="1" rowspan="1">59.</td> |
---|
3302 | <td colspan="1" rowspan="1"></td> |
---|
3303 | |
---|
3304 | </tr> |
---|
3305 | |
---|
3306 | <tr> |
---|
3307 | |
---|
3308 | <td colspan="1" rowspan="1">60.</td> |
---|
3309 | <td colspan="1" rowspan="1"> |
---|
3310 | |
---|
3311 | <span class="codefrag">for (String pattern : patternsToSkip) {</span> |
---|
3312 | </td> |
---|
3313 | |
---|
3314 | </tr> |
---|
3315 | |
---|
3316 | <tr> |
---|
3317 | |
---|
3318 | <td colspan="1" rowspan="1">61.</td> |
---|
3319 | <td colspan="1" rowspan="1"> |
---|
3320 | |
---|
3321 | <span class="codefrag">line = line.replaceAll(pattern, "");</span> |
---|
3322 | </td> |
---|
3323 | |
---|
3324 | </tr> |
---|
3325 | |
---|
3326 | <tr> |
---|
3327 | |
---|
3328 | <td colspan="1" rowspan="1">62.</td> |
---|
3329 | <td colspan="1" rowspan="1"> |
---|
3330 | |
---|
3331 | <span class="codefrag">}</span> |
---|
3332 | </td> |
---|
3333 | |
---|
3334 | </tr> |
---|
3335 | |
---|
3336 | <tr> |
---|
3337 | |
---|
3338 | <td colspan="1" rowspan="1">63.</td> |
---|
3339 | <td colspan="1" rowspan="1"></td> |
---|
3340 | |
---|
3341 | </tr> |
---|
3342 | |
---|
3343 | <tr> |
---|
3344 | |
---|
3345 | <td colspan="1" rowspan="1">64.</td> |
---|
3346 | <td colspan="1" rowspan="1"> |
---|
3347 | |
---|
3348 | <span class="codefrag">StringTokenizer tokenizer = new StringTokenizer(line);</span> |
---|
3349 | </td> |
---|
3350 | |
---|
3351 | </tr> |
---|
3352 | |
---|
3353 | <tr> |
---|
3354 | |
---|
3355 | <td colspan="1" rowspan="1">65.</td> |
---|
3356 | <td colspan="1" rowspan="1"> |
---|
3357 | |
---|
3358 | <span class="codefrag">while (tokenizer.hasMoreTokens()) {</span> |
---|
3359 | </td> |
---|
3360 | |
---|
3361 | </tr> |
---|
3362 | |
---|
3363 | <tr> |
---|
3364 | |
---|
3365 | <td colspan="1" rowspan="1">66.</td> |
---|
3366 | <td colspan="1" rowspan="1"> |
---|
3367 | |
---|
3368 | <span class="codefrag">word.set(tokenizer.nextToken());</span> |
---|
3369 | </td> |
---|
3370 | |
---|
3371 | </tr> |
---|
3372 | |
---|
3373 | <tr> |
---|
3374 | |
---|
3375 | <td colspan="1" rowspan="1">67.</td> |
---|
3376 | <td colspan="1" rowspan="1"> |
---|
3377 | |
---|
3378 | <span class="codefrag">output.collect(word, one);</span> |
---|
3379 | </td> |
---|
3380 | |
---|
3381 | </tr> |
---|
3382 | |
---|
3383 | <tr> |
---|
3384 | |
---|
3385 | <td colspan="1" rowspan="1">68.</td> |
---|
3386 | <td colspan="1" rowspan="1"> |
---|
3387 | |
---|
3388 | <span class="codefrag">reporter.incrCounter(Counters.INPUT_WORDS, 1);</span> |
---|
3389 | </td> |
---|
3390 | |
---|
3391 | </tr> |
---|
3392 | |
---|
3393 | <tr> |
---|
3394 | |
---|
3395 | <td colspan="1" rowspan="1">69.</td> |
---|
3396 | <td colspan="1" rowspan="1"> |
---|
3397 | |
---|
3398 | <span class="codefrag">}</span> |
---|
3399 | </td> |
---|
3400 | |
---|
3401 | </tr> |
---|
3402 | |
---|
3403 | <tr> |
---|
3404 | |
---|
3405 | <td colspan="1" rowspan="1">70.</td> |
---|
3406 | <td colspan="1" rowspan="1"></td> |
---|
3407 | |
---|
3408 | </tr> |
---|
3409 | |
---|
3410 | <tr> |
---|
3411 | |
---|
3412 | <td colspan="1" rowspan="1">71.</td> |
---|
3413 | <td colspan="1" rowspan="1"> |
---|
3414 | |
---|
3415 | <span class="codefrag">if ((++numRecords % 100) == 0) {</span> |
---|
3416 | </td> |
---|
3417 | |
---|
3418 | </tr> |
---|
3419 | |
---|
3420 | <tr> |
---|
3421 | |
---|
3422 | <td colspan="1" rowspan="1">72.</td> |
---|
3423 | <td colspan="1" rowspan="1"> |
---|
3424 | |
---|
3425 | <span class="codefrag"> |
---|
3426 | reporter.setStatus("Finished processing " + numRecords + |
---|
3427 | " records " + "from the input file: " + |
---|
3428 | inputFile); |
---|
3429 | </span> |
---|
3430 | </td> |
---|
3431 | |
---|
3432 | </tr> |
---|
3433 | |
---|
3434 | <tr> |
---|
3435 | |
---|
3436 | <td colspan="1" rowspan="1">73.</td> |
---|
3437 | <td colspan="1" rowspan="1"> |
---|
3438 | |
---|
3439 | <span class="codefrag">}</span> |
---|
3440 | </td> |
---|
3441 | |
---|
3442 | </tr> |
---|
3443 | |
---|
3444 | <tr> |
---|
3445 | |
---|
3446 | <td colspan="1" rowspan="1">74.</td> |
---|
3447 | <td colspan="1" rowspan="1"> |
---|
3448 | |
---|
3449 | <span class="codefrag">}</span> |
---|
3450 | </td> |
---|
3451 | |
---|
3452 | </tr> |
---|
3453 | |
---|
3454 | <tr> |
---|
3455 | |
---|
3456 | <td colspan="1" rowspan="1">75.</td> |
---|
3457 | <td colspan="1" rowspan="1"> |
---|
3458 | |
---|
3459 | <span class="codefrag">}</span> |
---|
3460 | </td> |
---|
3461 | |
---|
3462 | </tr> |
---|
3463 | |
---|
3464 | <tr> |
---|
3465 | |
---|
3466 | <td colspan="1" rowspan="1">76.</td> |
---|
3467 | <td colspan="1" rowspan="1"></td> |
---|
3468 | |
---|
3469 | </tr> |
---|
3470 | |
---|
3471 | <tr> |
---|
3472 | |
---|
3473 | <td colspan="1" rowspan="1">77.</td> |
---|
3474 | <td colspan="1" rowspan="1"> |
---|
3475 | |
---|
3476 | <span class="codefrag"> |
---|
3477 | public static class Reduce extends MapReduceBase implements |
---|
3478 | Reducer<Text, IntWritable, Text, IntWritable> { |
---|
3479 | </span> |
---|
3480 | </td> |
---|
3481 | |
---|
3482 | </tr> |
---|
3483 | |
---|
3484 | <tr> |
---|
3485 | |
---|
3486 | <td colspan="1" rowspan="1">78.</td> |
---|
3487 | <td colspan="1" rowspan="1"> |
---|
3488 | |
---|
3489 | <span class="codefrag"> |
---|
3490 | public void reduce(Text key, Iterator<IntWritable> values, |
---|
3491 | OutputCollector<Text, IntWritable> output, |
---|
3492 | Reporter reporter) throws IOException { |
---|
3493 | </span> |
---|
3494 | </td> |
---|
3495 | |
---|
3496 | </tr> |
---|
3497 | |
---|
3498 | <tr> |
---|
3499 | |
---|
3500 | <td colspan="1" rowspan="1">79.</td> |
---|
3501 | <td colspan="1" rowspan="1"> |
---|
3502 | |
---|
3503 | <span class="codefrag">int sum = 0;</span> |
---|
3504 | </td> |
---|
3505 | |
---|
3506 | </tr> |
---|
3507 | |
---|
3508 | <tr> |
---|
3509 | |
---|
3510 | <td colspan="1" rowspan="1">80.</td> |
---|
3511 | <td colspan="1" rowspan="1"> |
---|
3512 | |
---|
3513 | <span class="codefrag">while (values.hasNext()) {</span> |
---|
3514 | </td> |
---|
3515 | |
---|
3516 | </tr> |
---|
3517 | |
---|
3518 | <tr> |
---|
3519 | |
---|
3520 | <td colspan="1" rowspan="1">81.</td> |
---|
3521 | <td colspan="1" rowspan="1"> |
---|
3522 | |
---|
3523 | <span class="codefrag">sum += values.next().get();</span> |
---|
3524 | </td> |
---|
3525 | |
---|
3526 | </tr> |
---|
3527 | |
---|
3528 | <tr> |
---|
3529 | |
---|
3530 | <td colspan="1" rowspan="1">82.</td> |
---|
3531 | <td colspan="1" rowspan="1"> |
---|
3532 | |
---|
3533 | <span class="codefrag">}</span> |
---|
3534 | </td> |
---|
3535 | |
---|
3536 | </tr> |
---|
3537 | |
---|
3538 | <tr> |
---|
3539 | |
---|
3540 | <td colspan="1" rowspan="1">83.</td> |
---|
3541 | <td colspan="1" rowspan="1"> |
---|
3542 | |
---|
3543 | <span class="codefrag">output.collect(key, new IntWritable(sum));</span> |
---|
3544 | </td> |
---|
3545 | |
---|
3546 | </tr> |
---|
3547 | |
---|
3548 | <tr> |
---|
3549 | |
---|
3550 | <td colspan="1" rowspan="1">84.</td> |
---|
3551 | <td colspan="1" rowspan="1"> |
---|
3552 | |
---|
3553 | <span class="codefrag">}</span> |
---|
3554 | </td> |
---|
3555 | |
---|
3556 | </tr> |
---|
3557 | |
---|
3558 | <tr> |
---|
3559 | |
---|
3560 | <td colspan="1" rowspan="1">85.</td> |
---|
3561 | <td colspan="1" rowspan="1"> |
---|
3562 | |
---|
3563 | <span class="codefrag">}</span> |
---|
3564 | </td> |
---|
3565 | |
---|
3566 | </tr> |
---|
3567 | |
---|
3568 | <tr> |
---|
3569 | |
---|
3570 | <td colspan="1" rowspan="1">86.</td> |
---|
3571 | <td colspan="1" rowspan="1"></td> |
---|
3572 | |
---|
3573 | </tr> |
---|
3574 | |
---|
3575 | <tr> |
---|
3576 | |
---|
3577 | <td colspan="1" rowspan="1">87.</td> |
---|
3578 | <td colspan="1" rowspan="1"> |
---|
3579 | |
---|
3580 | <span class="codefrag">public int run(String[] args) throws Exception {</span> |
---|
3581 | </td> |
---|
3582 | |
---|
3583 | </tr> |
---|
3584 | |
---|
3585 | <tr> |
---|
3586 | |
---|
3587 | <td colspan="1" rowspan="1">88.</td> |
---|
3588 | <td colspan="1" rowspan="1"> |
---|
3589 | |
---|
3590 | <span class="codefrag"> |
---|
3591 | JobConf conf = new JobConf(getConf(), WordCount.class); |
---|
3592 | </span> |
---|
3593 | </td> |
---|
3594 | |
---|
3595 | </tr> |
---|
3596 | |
---|
3597 | <tr> |
---|
3598 | |
---|
3599 | <td colspan="1" rowspan="1">89.</td> |
---|
3600 | <td colspan="1" rowspan="1"> |
---|
3601 | |
---|
3602 | <span class="codefrag">conf.setJobName("wordcount");</span> |
---|
3603 | </td> |
---|
3604 | |
---|
3605 | </tr> |
---|
3606 | |
---|
3607 | <tr> |
---|
3608 | |
---|
3609 | <td colspan="1" rowspan="1">90.</td> |
---|
3610 | <td colspan="1" rowspan="1"></td> |
---|
3611 | |
---|
3612 | </tr> |
---|
3613 | |
---|
3614 | <tr> |
---|
3615 | |
---|
3616 | <td colspan="1" rowspan="1">91.</td> |
---|
3617 | <td colspan="1" rowspan="1"> |
---|
3618 | |
---|
3619 | <span class="codefrag">conf.setOutputKeyClass(Text.class);</span> |
---|
3620 | </td> |
---|
3621 | |
---|
3622 | </tr> |
---|
3623 | |
---|
3624 | <tr> |
---|
3625 | |
---|
3626 | <td colspan="1" rowspan="1">92.</td> |
---|
3627 | <td colspan="1" rowspan="1"> |
---|
3628 | |
---|
3629 | <span class="codefrag">conf.setOutputValueClass(IntWritable.class);</span> |
---|
3630 | </td> |
---|
3631 | |
---|
3632 | </tr> |
---|
3633 | |
---|
3634 | <tr> |
---|
3635 | |
---|
3636 | <td colspan="1" rowspan="1">93.</td> |
---|
3637 | <td colspan="1" rowspan="1"></td> |
---|
3638 | |
---|
3639 | </tr> |
---|
3640 | |
---|
3641 | <tr> |
---|
3642 | |
---|
3643 | <td colspan="1" rowspan="1">94.</td> |
---|
3644 | <td colspan="1" rowspan="1"> |
---|
3645 | |
---|
3646 | <span class="codefrag">conf.setMapperClass(Map.class);</span> |
---|
3647 | </td> |
---|
3648 | |
---|
3649 | </tr> |
---|
3650 | |
---|
3651 | <tr> |
---|
3652 | |
---|
3653 | <td colspan="1" rowspan="1">95.</td> |
---|
3654 | <td colspan="1" rowspan="1"> |
---|
3655 | |
---|
3656 | <span class="codefrag">conf.setCombinerClass(Reduce.class);</span> |
---|
3657 | </td> |
---|
3658 | |
---|
3659 | </tr> |
---|
3660 | |
---|
3661 | <tr> |
---|
3662 | |
---|
3663 | <td colspan="1" rowspan="1">96.</td> |
---|
3664 | <td colspan="1" rowspan="1"> |
---|
3665 | |
---|
3666 | <span class="codefrag">conf.setReducerClass(Reduce.class);</span> |
---|
3667 | </td> |
---|
3668 | |
---|
3669 | </tr> |
---|
3670 | |
---|
3671 | <tr> |
---|
3672 | |
---|
3673 | <td colspan="1" rowspan="1">97.</td> |
---|
3674 | <td colspan="1" rowspan="1"></td> |
---|
3675 | |
---|
3676 | </tr> |
---|
3677 | |
---|
3678 | <tr> |
---|
3679 | |
---|
3680 | <td colspan="1" rowspan="1">98.</td> |
---|
3681 | <td colspan="1" rowspan="1"> |
---|
3682 | |
---|
3683 | <span class="codefrag">conf.setInputFormat(TextInputFormat.class);</span> |
---|
3684 | </td> |
---|
3685 | |
---|
3686 | </tr> |
---|
3687 | |
---|
3688 | <tr> |
---|
3689 | |
---|
3690 | <td colspan="1" rowspan="1">99.</td> |
---|
3691 | <td colspan="1" rowspan="1"> |
---|
3692 | |
---|
3693 | <span class="codefrag">conf.setOutputFormat(TextOutputFormat.class);</span> |
---|
3694 | </td> |
---|
3695 | |
---|
3696 | </tr> |
---|
3697 | |
---|
3698 | <tr> |
---|
3699 | |
---|
3700 | <td colspan="1" rowspan="1">100.</td> |
---|
3701 | <td colspan="1" rowspan="1"></td> |
---|
3702 | |
---|
3703 | </tr> |
---|
3704 | |
---|
3705 | <tr> |
---|
3706 | |
---|
3707 | <td colspan="1" rowspan="1">101.</td> |
---|
3708 | <td colspan="1" rowspan="1"> |
---|
3709 | |
---|
3710 | <span class="codefrag"> |
---|
3711 | List<String> other_args = new ArrayList<String>(); |
---|
3712 | </span> |
---|
3713 | </td> |
---|
3714 | |
---|
3715 | </tr> |
---|
3716 | |
---|
3717 | <tr> |
---|
3718 | |
---|
3719 | <td colspan="1" rowspan="1">102.</td> |
---|
3720 | <td colspan="1" rowspan="1"> |
---|
3721 | |
---|
3722 | <span class="codefrag">for (int i=0; i < args.length; ++i) {</span> |
---|
3723 | </td> |
---|
3724 | |
---|
3725 | </tr> |
---|
3726 | |
---|
3727 | <tr> |
---|
3728 | |
---|
3729 | <td colspan="1" rowspan="1">103.</td> |
---|
3730 | <td colspan="1" rowspan="1"> |
---|
3731 | |
---|
3732 | <span class="codefrag">if ("-skip".equals(args[i])) {</span> |
---|
3733 | </td> |
---|
3734 | |
---|
3735 | </tr> |
---|
3736 | |
---|
3737 | <tr> |
---|
3738 | |
---|
3739 | <td colspan="1" rowspan="1">104.</td> |
---|
3740 | <td colspan="1" rowspan="1"> |
---|
3741 | |
---|
3742 | <span class="codefrag"> |
---|
3743 | DistributedCache.addCacheFile(new Path(args[++i]).toUri(), conf); |
---|
3744 | </span> |
---|
3745 | </td> |
---|
3746 | |
---|
3747 | </tr> |
---|
3748 | |
---|
3749 | <tr> |
---|
3750 | |
---|
3751 | <td colspan="1" rowspan="1">105.</td> |
---|
3752 | <td colspan="1" rowspan="1"> |
---|
3753 | |
---|
3754 | <span class="codefrag"> |
---|
3755 | conf.setBoolean("wordcount.skip.patterns", true); |
---|
3756 | </span> |
---|
3757 | </td> |
---|
3758 | |
---|
3759 | </tr> |
---|
3760 | |
---|
3761 | <tr> |
---|
3762 | |
---|
3763 | <td colspan="1" rowspan="1">106.</td> |
---|
3764 | <td colspan="1" rowspan="1"> |
---|
3765 | |
---|
3766 | <span class="codefrag">} else {</span> |
---|
3767 | </td> |
---|
3768 | |
---|
3769 | </tr> |
---|
3770 | |
---|
3771 | <tr> |
---|
3772 | |
---|
3773 | <td colspan="1" rowspan="1">107.</td> |
---|
3774 | <td colspan="1" rowspan="1"> |
---|
3775 | |
---|
3776 | <span class="codefrag">other_args.add(args[i]);</span> |
---|
3777 | </td> |
---|
3778 | |
---|
3779 | </tr> |
---|
3780 | |
---|
3781 | <tr> |
---|
3782 | |
---|
3783 | <td colspan="1" rowspan="1">108.</td> |
---|
3784 | <td colspan="1" rowspan="1"> |
---|
3785 | |
---|
3786 | <span class="codefrag">}</span> |
---|
3787 | </td> |
---|
3788 | |
---|
3789 | </tr> |
---|
3790 | |
---|
3791 | <tr> |
---|
3792 | |
---|
3793 | <td colspan="1" rowspan="1">109.</td> |
---|
3794 | <td colspan="1" rowspan="1"> |
---|
3795 | |
---|
3796 | <span class="codefrag">}</span> |
---|
3797 | </td> |
---|
3798 | |
---|
3799 | </tr> |
---|
3800 | |
---|
3801 | <tr> |
---|
3802 | |
---|
3803 | <td colspan="1" rowspan="1">110.</td> |
---|
3804 | <td colspan="1" rowspan="1"></td> |
---|
3805 | |
---|
3806 | </tr> |
---|
3807 | |
---|
3808 | <tr> |
---|
3809 | |
---|
3810 | <td colspan="1" rowspan="1">111.</td> |
---|
3811 | <td colspan="1" rowspan="1"> |
---|
3812 | |
---|
3813 | <span class="codefrag">FileInputFormat.setInputPaths(conf, new Path(other_args.get(0)));</span> |
---|
3814 | </td> |
---|
3815 | |
---|
3816 | </tr> |
---|
3817 | |
---|
3818 | <tr> |
---|
3819 | |
---|
3820 | <td colspan="1" rowspan="1">112.</td> |
---|
3821 | <td colspan="1" rowspan="1"> |
---|
3822 | |
---|
3823 | <span class="codefrag">FileOutputFormat.setOutputPath(conf, new Path(other_args.get(1)));</span> |
---|
3824 | </td> |
---|
3825 | |
---|
3826 | </tr> |
---|
3827 | |
---|
3828 | <tr> |
---|
3829 | |
---|
3830 | <td colspan="1" rowspan="1">113.</td> |
---|
3831 | <td colspan="1" rowspan="1"></td> |
---|
3832 | |
---|
3833 | </tr> |
---|
3834 | |
---|
3835 | <tr> |
---|
3836 | |
---|
3837 | <td colspan="1" rowspan="1">114.</td> |
---|
3838 | <td colspan="1" rowspan="1"> |
---|
3839 | |
---|
3840 | <span class="codefrag">JobClient.runJob(conf);</span> |
---|
3841 | </td> |
---|
3842 | |
---|
3843 | </tr> |
---|
3844 | |
---|
3845 | <tr> |
---|
3846 | |
---|
3847 | <td colspan="1" rowspan="1">115.</td> |
---|
3848 | <td colspan="1" rowspan="1"> |
---|
3849 | |
---|
3850 | <span class="codefrag">return 0;</span> |
---|
3851 | </td> |
---|
3852 | |
---|
3853 | </tr> |
---|
3854 | |
---|
3855 | <tr> |
---|
3856 | |
---|
3857 | <td colspan="1" rowspan="1">116.</td> |
---|
3858 | <td colspan="1" rowspan="1"> |
---|
3859 | |
---|
3860 | <span class="codefrag">}</span> |
---|
3861 | </td> |
---|
3862 | |
---|
3863 | </tr> |
---|
3864 | |
---|
3865 | <tr> |
---|
3866 | |
---|
3867 | <td colspan="1" rowspan="1">117.</td> |
---|
3868 | <td colspan="1" rowspan="1"></td> |
---|
3869 | |
---|
3870 | </tr> |
---|
3871 | |
---|
3872 | <tr> |
---|
3873 | |
---|
3874 | <td colspan="1" rowspan="1">118.</td> |
---|
3875 | <td colspan="1" rowspan="1"> |
---|
3876 | |
---|
3877 | <span class="codefrag"> |
---|
3878 | public static void main(String[] args) throws Exception { |
---|
3879 | </span> |
---|
3880 | </td> |
---|
3881 | |
---|
3882 | </tr> |
---|
3883 | |
---|
3884 | <tr> |
---|
3885 | |
---|
3886 | <td colspan="1" rowspan="1">119.</td> |
---|
3887 | <td colspan="1" rowspan="1"> |
---|
3888 | |
---|
3889 | <span class="codefrag"> |
---|
3890 | int res = ToolRunner.run(new Configuration(), new WordCount(), |
---|
3891 | args); |
---|
3892 | </span> |
---|
3893 | </td> |
---|
3894 | |
---|
3895 | </tr> |
---|
3896 | |
---|
3897 | <tr> |
---|
3898 | |
---|
3899 | <td colspan="1" rowspan="1">120.</td> |
---|
3900 | <td colspan="1" rowspan="1"> |
---|
3901 | |
---|
3902 | <span class="codefrag">System.exit(res);</span> |
---|
3903 | </td> |
---|
3904 | |
---|
3905 | </tr> |
---|
3906 | |
---|
3907 | <tr> |
---|
3908 | |
---|
3909 | <td colspan="1" rowspan="1">121.</td> |
---|
3910 | <td colspan="1" rowspan="1"> |
---|
3911 | |
---|
3912 | <span class="codefrag">}</span> |
---|
3913 | </td> |
---|
3914 | |
---|
3915 | </tr> |
---|
3916 | |
---|
3917 | <tr> |
---|
3918 | |
---|
3919 | <td colspan="1" rowspan="1">122.</td> |
---|
3920 | <td colspan="1" rowspan="1"> |
---|
3921 | <span class="codefrag">}</span> |
---|
3922 | </td> |
---|
3923 | |
---|
3924 | </tr> |
---|
3925 | |
---|
3926 | <tr> |
---|
3927 | |
---|
3928 | <td colspan="1" rowspan="1">123.</td> |
---|
3929 | <td colspan="1" rowspan="1"></td> |
---|
3930 | |
---|
3931 | </tr> |
---|
3932 | |
---|
3933 | </table> |
---|
3934 | <a name="N11732"></a><a name="Sample+Runs"></a> |
---|
3935 | <h3 class="h4">Sample Runs</h3> |
---|
3936 | <p>Sample text-files as input:</p> |
---|
3937 | <p> |
---|
3938 | |
---|
3939 | <span class="codefrag">$ bin/hadoop dfs -ls /usr/joe/wordcount/input/</span> |
---|
3940 | <br> |
---|
3941 | |
---|
3942 | <span class="codefrag">/usr/joe/wordcount/input/file01</span> |
---|
3943 | <br> |
---|
3944 | |
---|
3945 | <span class="codefrag">/usr/joe/wordcount/input/file02</span> |
---|
3946 | <br> |
---|
3947 | |
---|
3948 | <br> |
---|
3949 | |
---|
3950 | <span class="codefrag">$ bin/hadoop dfs -cat /usr/joe/wordcount/input/file01</span> |
---|
3951 | <br> |
---|
3952 | |
---|
3953 | <span class="codefrag">Hello World, Bye World!</span> |
---|
3954 | <br> |
---|
3955 | |
---|
3956 | <br> |
---|
3957 | |
---|
3958 | <span class="codefrag">$ bin/hadoop dfs -cat /usr/joe/wordcount/input/file02</span> |
---|
3959 | <br> |
---|
3960 | |
---|
3961 | <span class="codefrag">Hello Hadoop, Goodbye to hadoop.</span> |
---|
3962 | |
---|
3963 | </p> |
---|
3964 | <p>Run the application:</p> |
---|
3965 | <p> |
---|
3966 | |
---|
3967 | <span class="codefrag"> |
---|
3968 | $ bin/hadoop jar /usr/joe/wordcount.jar org.myorg.WordCount |
---|
3969 | /usr/joe/wordcount/input /usr/joe/wordcount/output |
---|
3970 | </span> |
---|
3971 | |
---|
3972 | </p> |
---|
3973 | <p>Output:</p> |
---|
3974 | <p> |
---|
3975 | |
---|
3976 | <span class="codefrag"> |
---|
3977 | $ bin/hadoop dfs -cat /usr/joe/wordcount/output/part-00000 |
---|
3978 | </span> |
---|
3979 | |
---|
3980 | <br> |
---|
3981 | |
---|
3982 | <span class="codefrag">Bye 1</span> |
---|
3983 | <br> |
---|
3984 | |
---|
3985 | <span class="codefrag">Goodbye 1</span> |
---|
3986 | <br> |
---|
3987 | |
---|
3988 | <span class="codefrag">Hadoop, 1</span> |
---|
3989 | <br> |
---|
3990 | |
---|
3991 | <span class="codefrag">Hello 2</span> |
---|
3992 | <br> |
---|
3993 | |
---|
3994 | <span class="codefrag">World! 1</span> |
---|
3995 | <br> |
---|
3996 | |
---|
3997 | <span class="codefrag">World, 1</span> |
---|
3998 | <br> |
---|
3999 | |
---|
4000 | <span class="codefrag">hadoop. 1</span> |
---|
4001 | <br> |
---|
4002 | |
---|
4003 | <span class="codefrag">to 1</span> |
---|
4004 | <br> |
---|
4005 | |
---|
4006 | </p> |
---|
4007 | <p>Notice that the inputs differ from the first version we looked at, |
---|
4008 | and how they affect the outputs.</p> |
---|
4009 | <p>Now, lets plug-in a pattern-file which lists the word-patterns to be |
---|
4010 | ignored, via the <span class="codefrag">DistributedCache</span>.</p> |
---|
4011 | <p> |
---|
4012 | |
---|
4013 | <span class="codefrag">$ hadoop dfs -cat /user/joe/wordcount/patterns.txt</span> |
---|
4014 | <br> |
---|
4015 | |
---|
4016 | <span class="codefrag">\.</span> |
---|
4017 | <br> |
---|
4018 | |
---|
4019 | <span class="codefrag">\,</span> |
---|
4020 | <br> |
---|
4021 | |
---|
4022 | <span class="codefrag">\!</span> |
---|
4023 | <br> |
---|
4024 | |
---|
4025 | <span class="codefrag">to</span> |
---|
4026 | <br> |
---|
4027 | |
---|
4028 | </p> |
---|
4029 | <p>Run it again, this time with more options:</p> |
---|
4030 | <p> |
---|
4031 | |
---|
4032 | <span class="codefrag"> |
---|
4033 | $ bin/hadoop jar /usr/joe/wordcount.jar org.myorg.WordCount |
---|
4034 | -Dwordcount.case.sensitive=true /usr/joe/wordcount/input |
---|
4035 | /usr/joe/wordcount/output -skip /user/joe/wordcount/patterns.txt |
---|
4036 | </span> |
---|
4037 | |
---|
4038 | </p> |
---|
4039 | <p>As expected, the output:</p> |
---|
4040 | <p> |
---|
4041 | |
---|
4042 | <span class="codefrag"> |
---|
4043 | $ bin/hadoop dfs -cat /usr/joe/wordcount/output/part-00000 |
---|
4044 | </span> |
---|
4045 | |
---|
4046 | <br> |
---|
4047 | |
---|
4048 | <span class="codefrag">Bye 1</span> |
---|
4049 | <br> |
---|
4050 | |
---|
4051 | <span class="codefrag">Goodbye 1</span> |
---|
4052 | <br> |
---|
4053 | |
---|
4054 | <span class="codefrag">Hadoop 1</span> |
---|
4055 | <br> |
---|
4056 | |
---|
4057 | <span class="codefrag">Hello 2</span> |
---|
4058 | <br> |
---|
4059 | |
---|
4060 | <span class="codefrag">World 2</span> |
---|
4061 | <br> |
---|
4062 | |
---|
4063 | <span class="codefrag">hadoop 1</span> |
---|
4064 | <br> |
---|
4065 | |
---|
4066 | </p> |
---|
4067 | <p>Run it once more, this time switch-off case-sensitivity:</p> |
---|
4068 | <p> |
---|
4069 | |
---|
4070 | <span class="codefrag"> |
---|
4071 | $ bin/hadoop jar /usr/joe/wordcount.jar org.myorg.WordCount |
---|
4072 | -Dwordcount.case.sensitive=false /usr/joe/wordcount/input |
---|
4073 | /usr/joe/wordcount/output -skip /user/joe/wordcount/patterns.txt |
---|
4074 | </span> |
---|
4075 | |
---|
4076 | </p> |
---|
4077 | <p>Sure enough, the output:</p> |
---|
4078 | <p> |
---|
4079 | |
---|
4080 | <span class="codefrag"> |
---|
4081 | $ bin/hadoop dfs -cat /usr/joe/wordcount/output/part-00000 |
---|
4082 | </span> |
---|
4083 | |
---|
4084 | <br> |
---|
4085 | |
---|
4086 | <span class="codefrag">bye 1</span> |
---|
4087 | <br> |
---|
4088 | |
---|
4089 | <span class="codefrag">goodbye 1</span> |
---|
4090 | <br> |
---|
4091 | |
---|
4092 | <span class="codefrag">hadoop 2</span> |
---|
4093 | <br> |
---|
4094 | |
---|
4095 | <span class="codefrag">hello 2</span> |
---|
4096 | <br> |
---|
4097 | |
---|
4098 | <span class="codefrag">world 2</span> |
---|
4099 | <br> |
---|
4100 | |
---|
4101 | </p> |
---|
4102 | <a name="N11806"></a><a name="Highlights"></a> |
---|
4103 | <h3 class="h4">Highlights</h3> |
---|
4104 | <p>The second version of <span class="codefrag">WordCount</span> improves upon the |
---|
4105 | previous one by using some features offered by the Map/Reduce framework: |
---|
4106 | </p> |
---|
4107 | <ul> |
---|
4108 | |
---|
4109 | <li> |
---|
4110 | Demonstrates how applications can access configuration parameters |
---|
4111 | in the <span class="codefrag">configure</span> method of the <span class="codefrag">Mapper</span> (and |
---|
4112 | <span class="codefrag">Reducer</span>) implementations (lines 28-43). |
---|
4113 | </li> |
---|
4114 | |
---|
4115 | <li> |
---|
4116 | Demonstrates how the <span class="codefrag">DistributedCache</span> can be used to |
---|
4117 | distribute read-only data needed by the jobs. Here it allows the user |
---|
4118 | to specify word-patterns to skip while counting (line 104). |
---|
4119 | </li> |
---|
4120 | |
---|
4121 | <li> |
---|
4122 | Demonstrates the utility of the <span class="codefrag">Tool</span> interface and the |
---|
4123 | <span class="codefrag">GenericOptionsParser</span> to handle generic Hadoop |
---|
4124 | command-line options (lines 87-116, 119). |
---|
4125 | </li> |
---|
4126 | |
---|
4127 | <li> |
---|
4128 | Demonstrates how applications can use <span class="codefrag">Counters</span> (line 68) |
---|
4129 | and how they can set application-specific status information via |
---|
4130 | the <span class="codefrag">Reporter</span> instance passed to the <span class="codefrag">map</span> (and |
---|
4131 | <span class="codefrag">reduce</span>) method (line 72). |
---|
4132 | </li> |
---|
4133 | |
---|
4134 | </ul> |
---|
4135 | </div> |
---|
4136 | |
---|
4137 | |
---|
4138 | <p> |
---|
4139 | |
---|
4140 | <em>Java and JNI are trademarks or registered trademarks of |
---|
4141 | Sun Microsystems, Inc. in the United States and other countries.</em> |
---|
4142 | |
---|
4143 | </p> |
---|
4144 | |
---|
4145 | |
---|
4146 | </div> |
---|
4147 | <!--+ |
---|
4148 | |end content |
---|
4149 | +--> |
---|
4150 | <div class="clearboth"> </div> |
---|
4151 | </div> |
---|
4152 | <div id="footer"> |
---|
4153 | <!--+ |
---|
4154 | |start bottomstrip |
---|
4155 | +--> |
---|
4156 | <div class="lastmodified"> |
---|
4157 | <script type="text/javascript"><!-- |
---|
4158 | document.write("Last Published: " + document.lastModified); |
---|
4159 | // --></script> |
---|
4160 | </div> |
---|
4161 | <div class="copyright"> |
---|
4162 | Copyright © |
---|
4163 | 2008 <a href="http://www.apache.org/licenses/">The Apache Software Foundation.</a> |
---|
4164 | </div> |
---|
4165 | <!--+ |
---|
4166 | |end bottomstrip |
---|
4167 | +--> |
---|
4168 | </div> |
---|
4169 | </body> |
---|
4170 | </html> |
---|