1567 lines
105 KiB
HTML
1567 lines
105 KiB
HTML
<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Transitional//EN" "http://www.w3.org/TR/xhtml1/DTD/xhtml1-transitional.dtd">
|
|
<!--
|
|
| Generated by Apache Maven Doxia at 2023-05-24
|
|
| Rendered using Apache Maven Stylus Skin 1.5
|
|
-->
|
|
<html xmlns="http://www.w3.org/1999/xhtml">
|
|
<head>
|
|
<title>Apache Hadoop 3.4.0-SNAPSHOT – MapReduce Tutorial</title>
|
|
<style type="text/css" media="all">
|
|
@import url("./css/maven-base.css");
|
|
@import url("./css/maven-theme.css");
|
|
@import url("./css/site.css");
|
|
</style>
|
|
<link rel="stylesheet" href="./css/print.css" type="text/css" media="print" />
|
|
<meta name="Date-Revision-yyyymmdd" content="20230524" />
|
|
<meta http-equiv="Content-Type" content="text/html; charset=UTF-8" />
|
|
</head>
|
|
<body class="composite">
|
|
<div id="banner">
|
|
<a href="http://hadoop.apache.org/" id="bannerLeft">
|
|
<img src="http://hadoop.apache.org/images/hadoop-logo.jpg" alt="" />
|
|
</a>
|
|
<a href="http://www.apache.org/" id="bannerRight">
|
|
<img src="http://www.apache.org/images/asf_logo_wide.png" alt="" />
|
|
</a>
|
|
<div class="clear">
|
|
<hr/>
|
|
</div>
|
|
</div>
|
|
<div id="breadcrumbs">
|
|
|
|
<div class="xright"> <a href="http://wiki.apache.org/hadoop" class="externalLink">Wiki</a>
|
|
|
|
|
<a href="https://gitbox.apache.org/repos/asf/hadoop.git" class="externalLink">git</a>
|
|
|
|
|
<a href="http://hadoop.apache.org/" class="externalLink">Apache Hadoop</a>
|
|
|
|
| Last Published: 2023-05-24
|
|
| Version: 3.4.0-SNAPSHOT
|
|
</div>
|
|
<div class="clear">
|
|
<hr/>
|
|
</div>
|
|
</div>
|
|
<div id="leftColumn">
|
|
<div id="navcolumn">
|
|
|
|
<h5>General</h5>
|
|
<ul>
|
|
<li class="none">
|
|
<a href="../../index.html">Overview</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/SingleCluster.html">Single Node Setup</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/ClusterSetup.html">Cluster Setup</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/CommandsManual.html">Commands Reference</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/FileSystemShell.html">FileSystem Shell</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/Compatibility.html">Compatibility Specification</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/DownstreamDev.html">Downstream Developer's Guide</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/AdminCompatibilityGuide.html">Admin Compatibility Guide</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/InterfaceClassification.html">Interface Classification</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/filesystem/index.html">FileSystem Specification</a>
|
|
</li>
|
|
</ul>
|
|
<h5>Common</h5>
|
|
<ul>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/CLIMiniCluster.html">CLI Mini Cluster</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/FairCallQueue.html">Fair Call Queue</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/NativeLibraries.html">Native Libraries</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/Superusers.html">Proxy User</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/RackAwareness.html">Rack Awareness</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/SecureMode.html">Secure Mode</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/ServiceLevelAuth.html">Service Level Authorization</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/HttpAuthentication.html">HTTP Authentication</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/CredentialProviderAPI.html">Credential Provider API</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-kms/index.html">Hadoop KMS</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/Tracing.html">Tracing</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/UnixShellGuide.html">Unix Shell Guide</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/registry/index.html">Registry</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/AsyncProfilerServlet.html">Async Profiler</a>
|
|
</li>
|
|
</ul>
|
|
<h5>HDFS</h5>
|
|
<ul>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/HdfsDesign.html">Architecture</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/HdfsUserGuide.html">User Guide</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/HDFSCommands.html">Commands Reference</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithQJM.html">NameNode HA With QJM</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html">NameNode HA With NFS</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/ObserverNameNode.html">Observer NameNode</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/Federation.html">Federation</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/ViewFs.html">ViewFs</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/ViewFsOverloadScheme.html">ViewFsOverloadScheme</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/HdfsSnapshots.html">Snapshots</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/HdfsEditsViewer.html">Edits Viewer</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/HdfsImageViewer.html">Image Viewer</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/HdfsPermissionsGuide.html">Permissions and HDFS</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/HdfsQuotaAdminGuide.html">Quotas and HDFS</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/LibHdfs.html">libhdfs (C API)</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/WebHDFS.html">WebHDFS (REST API)</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-hdfs-httpfs/index.html">HttpFS</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/ShortCircuitLocalReads.html">Short Circuit Local Reads</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/CentralizedCacheManagement.html">Centralized Cache Management</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/HdfsNfsGateway.html">NFS Gateway</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/HdfsRollingUpgrade.html">Rolling Upgrade</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/ExtendedAttributes.html">Extended Attributes</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/TransparentEncryption.html">Transparent Encryption</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/HdfsMultihoming.html">Multihoming</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/ArchivalStorage.html">Storage Policies</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/MemoryStorage.html">Memory Storage Support</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/SLGUserGuide.html">Synthetic Load Generator</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/HDFSErasureCoding.html">Erasure Coding</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/HDFSDiskbalancer.html">Disk Balancer</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/HdfsUpgradeDomain.html">Upgrade Domain</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/HdfsDataNodeAdminGuide.html">DataNode Admin</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs-rbf/HDFSRouterFederation.html">Router Federation</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/HdfsProvidedStorage.html">Provided Storage</a>
|
|
</li>
|
|
</ul>
|
|
<h5>MapReduce</h5>
|
|
<ul>
|
|
<li class="none">
|
|
<a href="../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html">Tutorial</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapredCommands.html">Commands Reference</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduce_Compatibility_Hadoop1_Hadoop2.html">Compatibility with 1.x</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/EncryptedShuffle.html">Encrypted Shuffle</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/PluggableShuffleAndPluggableSort.html">Pluggable Shuffle/Sort</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/DistributedCacheDeploy.html">Distributed Cache Deploy</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/SharedCacheSupport.html">Support for YARN Shared Cache</a>
|
|
</li>
|
|
</ul>
|
|
<h5>MapReduce REST APIs</h5>
|
|
<ul>
|
|
<li class="none">
|
|
<a href="../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapredAppMasterRest.html">MR Application Master</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-mapreduce-client/hadoop-mapreduce-client-hs/HistoryServerRest.html">MR History Server</a>
|
|
</li>
|
|
</ul>
|
|
<h5>YARN</h5>
|
|
<ul>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/YARN.html">Architecture</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/YarnCommands.html">Commands Reference</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/CapacityScheduler.html">Capacity Scheduler</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/FairScheduler.html">Fair Scheduler</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/ResourceManagerRestart.html">ResourceManager Restart</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/ResourceManagerHA.html">ResourceManager HA</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/ResourceModel.html">Resource Model</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/NodeLabel.html">Node Labels</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/NodeAttributes.html">Node Attributes</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/WebApplicationProxy.html">Web Application Proxy</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/TimelineServer.html">Timeline Server</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/TimelineServiceV2.html">Timeline Service V.2</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/WritingYarnApplications.html">Writing YARN Applications</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/YarnApplicationSecurity.html">YARN Application Security</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/NodeManager.html">NodeManager</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/DockerContainers.html">Running Applications in Docker Containers</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/RuncContainers.html">Running Applications in runC Containers</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/NodeManagerCgroups.html">Using CGroups</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/SecureContainer.html">Secure Containers</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/ReservationSystem.html">Reservation System</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/GracefulDecommission.html">Graceful Decommission</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/OpportunisticContainers.html">Opportunistic Containers</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/Federation.html">YARN Federation</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/SharedCache.html">Shared Cache</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/UsingGpus.html">Using GPU</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/UsingFPGA.html">Using FPGA</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/PlacementConstraints.html">Placement Constraints</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/YarnUI2.html">YARN UI2</a>
|
|
</li>
|
|
</ul>
|
|
<h5>YARN REST APIs</h5>
|
|
<ul>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/WebServicesIntro.html">Introduction</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/ResourceManagerRest.html">Resource Manager</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/NodeManagerRest.html">Node Manager</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/TimelineServer.html#Timeline_Server_REST_API_v1">Timeline Server</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/TimelineServiceV2.html#Timeline_Service_v.2_REST_API">Timeline Service V.2</a>
|
|
</li>
|
|
</ul>
|
|
<h5>YARN Service</h5>
|
|
<ul>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/yarn-service/Overview.html">Overview</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/yarn-service/QuickStart.html">QuickStart</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/yarn-service/Concepts.html">Concepts</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/yarn-service/YarnServiceAPI.html">Yarn Service API</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/yarn-service/ServiceDiscovery.html">Service Discovery</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-site/yarn-service/SystemServices.html">System Services</a>
|
|
</li>
|
|
</ul>
|
|
<h5>Hadoop Compatible File Systems</h5>
|
|
<ul>
|
|
<li class="none">
|
|
<a href="../../hadoop-aliyun/tools/hadoop-aliyun/index.html">Aliyun OSS</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-aws/tools/hadoop-aws/index.html">Amazon S3</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-azure/index.html">Azure Blob Storage</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-azure-datalake/index.html">Azure Data Lake Storage</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-cos/cloud-storage/index.html">Tencent COS</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-huaweicloud/cloud-storage/index.html">Huaweicloud OBS</a>
|
|
</li>
|
|
</ul>
|
|
<h5>Auth</h5>
|
|
<ul>
|
|
<li class="none">
|
|
<a href="../../hadoop-auth/index.html">Overview</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-auth/Examples.html">Examples</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-auth/Configuration.html">Configuration</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-auth/BuildingIt.html">Building</a>
|
|
</li>
|
|
</ul>
|
|
<h5>Tools</h5>
|
|
<ul>
|
|
<li class="none">
|
|
<a href="../../hadoop-streaming/HadoopStreaming.html">Hadoop Streaming</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-archives/HadoopArchives.html">Hadoop Archives</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-archive-logs/HadoopArchiveLogs.html">Hadoop Archive Logs</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-distcp/DistCp.html">DistCp</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-federation-balance/HDFSFederationBalance.html">HDFS Federation Balance</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-gridmix/GridMix.html">GridMix</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-rumen/Rumen.html">Rumen</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-resourceestimator/ResourceEstimator.html">Resource Estimator Service</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-sls/SchedulerLoadSimulator.html">Scheduler Load Simulator</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/Benchmarking.html">Hadoop Benchmarking</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-dynamometer/Dynamometer.html">Dynamometer</a>
|
|
</li>
|
|
</ul>
|
|
<h5>Reference</h5>
|
|
<ul>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/release/">Changelog and Release Notes</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../api/index.html">Java API docs</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/UnixShellAPI.html">Unix Shell API</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/Metrics.html">Metrics</a>
|
|
</li>
|
|
</ul>
|
|
<h5>Configuration</h5>
|
|
<ul>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/core-default.xml">core-default.xml</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs/hdfs-default.xml">hdfs-default.xml</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-hdfs-rbf/hdfs-rbf-default.xml">hdfs-rbf-default.xml</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-mapreduce-client/hadoop-mapreduce-client-core/mapred-default.xml">mapred-default.xml</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-yarn/hadoop-yarn-common/yarn-default.xml">yarn-default.xml</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-kms/kms-default.html">kms-default.xml</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-hdfs-httpfs/httpfs-default.html">httpfs-default.xml</a>
|
|
</li>
|
|
<li class="none">
|
|
<a href="../../hadoop-project-dist/hadoop-common/DeprecatedProperties.html">Deprecated Properties</a>
|
|
</li>
|
|
</ul>
|
|
<a href="http://maven.apache.org/" title="Built by Maven" class="poweredBy">
|
|
<img alt="Built by Maven" src="./images/logos/maven-feather.png"/>
|
|
</a>
|
|
|
|
</div>
|
|
</div>
|
|
<div id="bodyColumn">
|
|
<div id="contentBox">
|
|
<!---
|
|
Licensed under the Apache License, Version 2.0 (the "License");
|
|
you may not use this file except in compliance with the License.
|
|
You may obtain a copy of the License at
|
|
|
|
http://www.apache.org/licenses/LICENSE-2.0
|
|
|
|
Unless required by applicable law or agreed to in writing, software
|
|
distributed under the License is distributed on an "AS IS" BASIS,
|
|
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
See the License for the specific language governing permissions and
|
|
limitations under the License. See accompanying LICENSE file.
|
|
-->
|
|
<h1>MapReduce Tutorial</h1>
|
|
<ul>
|
|
<li><a href="#Purpose">Purpose</a></li>
|
|
<li><a href="#Prerequisites">Prerequisites</a></li>
|
|
<li><a href="#Overview">Overview</a></li>
|
|
<li><a href="#Inputs_and_Outputs">Inputs and Outputs</a></li>
|
|
<li><a href="#Example:_WordCount_v1.0">Example: WordCount v1.0</a>
|
|
<ul>
|
|
<li><a href="#Source_Code">Source Code</a></li>
|
|
<li><a href="#Usage">Usage</a></li>
|
|
<li><a href="#Walk-through">Walk-through</a></li></ul></li>
|
|
<li><a href="#MapReduce_-_User_Interfaces">MapReduce - User Interfaces</a>
|
|
<ul>
|
|
<li><a href="#Payload">Payload</a>
|
|
<ul>
|
|
<li><a href="#Mapper">Mapper</a></li>
|
|
<li><a href="#Reducer">Reducer</a></li>
|
|
<li><a href="#Partitioner">Partitioner</a></li>
|
|
<li><a href="#Counter">Counter</a></li></ul></li>
|
|
<li><a href="#Job_Configuration">Job Configuration</a></li>
|
|
<li><a href="#Task_Execution_.26_Environment">Task Execution & Environment</a>
|
|
<ul>
|
|
<li><a href="#Memory_Management">Memory Management</a></li>
|
|
<li><a href="#Map_Parameters">Map Parameters</a></li>
|
|
<li><a href="#Shuffle.2FReduce_Parameters">Shuffle/Reduce Parameters</a></li>
|
|
<li><a href="#Configured_Parameters">Configured Parameters</a></li>
|
|
<li><a href="#Task_Logs">Task Logs</a></li>
|
|
<li><a href="#Distributing_Libraries">Distributing Libraries</a></li></ul></li>
|
|
<li><a href="#Job_Submission_and_Monitoring">Job Submission and Monitoring</a>
|
|
<ul>
|
|
<li><a href="#Job_Control">Job Control</a></li></ul></li>
|
|
<li><a href="#Job_Input">Job Input</a>
|
|
<ul>
|
|
<li><a href="#InputSplit">InputSplit</a></li>
|
|
<li><a href="#RecordReader">RecordReader</a></li></ul></li>
|
|
<li><a href="#Job_Output">Job Output</a>
|
|
<ul>
|
|
<li><a href="#OutputCommitter">OutputCommitter</a></li>
|
|
<li><a href="#Task_Side-Effect_Files">Task Side-Effect Files</a></li>
|
|
<li><a href="#RecordWriter">RecordWriter</a></li></ul></li>
|
|
<li><a href="#Other_Useful_Features">Other Useful Features</a>
|
|
<ul>
|
|
<li><a href="#Submitting_Jobs_to_Queues">Submitting Jobs to Queues</a></li>
|
|
<li><a href="#Counters">Counters</a></li>
|
|
<li><a href="#DistributedCache">DistributedCache</a></li>
|
|
<li><a href="#Profiling">Profiling</a></li>
|
|
<li><a href="#Debugging">Debugging</a></li>
|
|
<li><a href="#Data_Compression">Data Compression</a></li>
|
|
<li><a href="#Skipping_Bad_Records">Skipping Bad Records</a></li></ul></li>
|
|
<li><a href="#Example:_WordCount_v2.0">Example: WordCount v2.0</a>
|
|
<ul>
|
|
<li><a href="#Source_Code">Source Code</a></li>
|
|
<li><a href="#Sample_Runs">Sample Runs</a></li>
|
|
<li><a href="#Highlights">Highlights</a></li></ul></li></ul></li></ul>
|
|
<section>
|
|
<h2><a name="Purpose"></a>Purpose</h2>
|
|
<p>This document comprehensively describes all user-facing facets of the Hadoop MapReduce framework and serves as a tutorial.</p></section><section>
|
|
<h2><a name="Prerequisites"></a>Prerequisites</h2>
|
|
<p>Ensure that Hadoop is installed, configured and is running. More details:</p>
|
|
<ul>
|
|
|
|
<li>
|
|
|
|
<p><a href="../../hadoop-project-dist/hadoop-common/SingleCluster.html">Single Node Setup</a> for first-time users.</p>
|
|
</li>
|
|
<li>
|
|
|
|
<p><a href="../../hadoop-project-dist/hadoop-common/ClusterSetup.html">Cluster Setup</a> for large, distributed clusters.</p>
|
|
</li>
|
|
</ul></section><section>
|
|
<h2><a name="Overview"></a>Overview</h2>
|
|
<p>Hadoop MapReduce is a software framework for easily writing applications which process vast amounts of data (multi-terabyte data-sets) in-parallel on large clusters (thousands of nodes) of commodity hardware in a reliable, fault-tolerant manner.</p>
|
|
<p>A MapReduce <i>job</i> usually splits the input data-set into independent chunks which are processed by the <i>map tasks</i> in a completely parallel manner. The framework sorts the outputs of the maps, which are then input to the <i>reduce tasks</i>. Typically both the input and the output of the job are stored in a file-system. The framework takes care of scheduling tasks, monitoring them and re-executes the failed tasks.</p>
|
|
<p>Typically the compute nodes and the storage nodes are the same, that is, the MapReduce framework and the Hadoop Distributed File System (see <a href="../../hadoop-project-dist/hadoop-hdfs/HdfsDesign.html">HDFS Architecture Guide</a>) are running on the same set of nodes. This configuration allows the framework to effectively schedule tasks on the nodes where data is already present, resulting in very high aggregate bandwidth across the cluster.</p>
|
|
<p>The MapReduce framework consists of a single master <code>ResourceManager</code>, one worker <code>NodeManager</code> per cluster-node, and <code>MRAppMaster</code> per application (see <a href="../../hadoop-yarn/hadoop-yarn-site/YARN.html">YARN Architecture Guide</a>).</p>
|
|
<p>Minimally, applications specify the input/output locations and supply <i>map</i> and <i>reduce</i> functions via implementations of appropriate interfaces and/or abstract-classes. These, and other job parameters, comprise the <i>job configuration</i>.</p>
|
|
<p>The Hadoop <i>job client</i> then submits the job (jar/executable etc.) and configuration to the <code>ResourceManager</code> which then assumes the responsibility of distributing the software/configuration to the workers, scheduling tasks and monitoring them, providing status and diagnostic information to the job-client.</p>
|
|
<p>Although the Hadoop framework is implemented in Java™, MapReduce applications need not be written in Java.</p>
|
|
<ul>
|
|
|
|
<li>
|
|
|
|
<p><a href="../../api/org/apache/hadoop/streaming/package-summary.html">Hadoop Streaming</a> is a utility which allows users to create and run jobs with any executables (e.g. shell utilities) as the mapper and/or the reducer.</p>
|
|
</li>
|
|
<li>
|
|
|
|
<p><a href="../../api/org/apache/hadoop/mapred/pipes/package-summary.html">Hadoop Pipes</a> is a <a class="externalLink" href="http://www.swig.org/">SWIG</a>-compatible C++ API to implement MapReduce applications (non JNI™ based).</p>
|
|
</li>
|
|
</ul></section><section>
|
|
<h2><a name="Inputs_and_Outputs"></a>Inputs and Outputs</h2>
|
|
<p>The MapReduce framework operates exclusively on <code><key, value></code> pairs, that is, the framework views the input to the job as a set of <code><key, value></code> pairs and produces a set of <code><key, value></code> pairs as the output of the job, conceivably of different types.</p>
|
|
<p>The <code>key</code> and <code>value</code> classes have to be serializable by the framework and hence need to implement the <a href="../../api/org/apache/hadoop/io/Writable.html">Writable</a> interface. Additionally, the key classes have to implement the <a href="../../api/org/apache/hadoop/io/WritableComparable.html">WritableComparable</a> interface to facilitate sorting by the framework.</p>
|
|
<p>Input and Output types of a MapReduce job:</p>
|
|
<p>(input) <code><k1, v1> -></code> <b>map</b> <code>-> <k2, v2> -></code> <b>combine</b> <code>-> <k2, v2> -></code> <b>reduce</b> <code>-> <k3, v3></code> (output)</p></section><section>
|
|
<h2><a name="Example:_WordCount_v1.0"></a>Example: WordCount v1.0</h2>
|
|
<p>Before we jump into the details, lets walk through an example MapReduce application to get a flavour for how they work.</p>
|
|
<p><code>WordCount</code> is a simple application that counts the number of occurrences of each word in a given input set.</p>
|
|
<p>This works with a local-standalone, pseudo-distributed or fully-distributed Hadoop installation (<a href="../../hadoop-project-dist/hadoop-common/SingleCluster.html">Single Node Setup</a>).</p><section>
|
|
<h3><a name="Source_Code"></a>Source Code</h3>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>import java.io.IOException;
|
|
import java.util.StringTokenizer;
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.io.IntWritable;
|
|
import org.apache.hadoop.io.Text;
|
|
import org.apache.hadoop.mapreduce.Job;
|
|
import org.apache.hadoop.mapreduce.Mapper;
|
|
import org.apache.hadoop.mapreduce.Reducer;
|
|
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
|
|
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
|
|
|
public class WordCount {
|
|
|
|
public static class TokenizerMapper
|
|
extends Mapper<Object, Text, Text, IntWritable>{
|
|
|
|
private final static IntWritable one = new IntWritable(1);
|
|
private Text word = new Text();
|
|
|
|
public void map(Object key, Text value, Context context
|
|
) throws IOException, InterruptedException {
|
|
StringTokenizer itr = new StringTokenizer(value.toString());
|
|
while (itr.hasMoreTokens()) {
|
|
word.set(itr.nextToken());
|
|
context.write(word, one);
|
|
}
|
|
}
|
|
}
|
|
|
|
public static class IntSumReducer
|
|
extends Reducer<Text,IntWritable,Text,IntWritable> {
|
|
private IntWritable result = new IntWritable();
|
|
|
|
public void reduce(Text key, Iterable<IntWritable> values,
|
|
Context context
|
|
) throws IOException, InterruptedException {
|
|
int sum = 0;
|
|
for (IntWritable val : values) {
|
|
sum += val.get();
|
|
}
|
|
result.set(sum);
|
|
context.write(key, result);
|
|
}
|
|
}
|
|
|
|
public static void main(String[] args) throws Exception {
|
|
Configuration conf = new Configuration();
|
|
Job job = Job.getInstance(conf, "word count");
|
|
job.setJarByClass(WordCount.class);
|
|
job.setMapperClass(TokenizerMapper.class);
|
|
job.setCombinerClass(IntSumReducer.class);
|
|
job.setReducerClass(IntSumReducer.class);
|
|
job.setOutputKeyClass(Text.class);
|
|
job.setOutputValueClass(IntWritable.class);
|
|
FileInputFormat.addInputPath(job, new Path(args[0]));
|
|
FileOutputFormat.setOutputPath(job, new Path(args[1]));
|
|
System.exit(job.waitForCompletion(true) ? 0 : 1);
|
|
}
|
|
}
|
|
</pre></div></div>
|
|
</section><section>
|
|
<h3><a name="Usage"></a>Usage</h3>
|
|
<p>Assuming environment variables are set as follows:</p>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>export JAVA_HOME=/usr/java/default
|
|
export PATH=${JAVA_HOME}/bin:${PATH}
|
|
export HADOOP_CLASSPATH=${JAVA_HOME}/lib/tools.jar
|
|
</pre></div></div>
|
|
|
|
<p>Compile <code>WordCount.java</code> and create a jar:</p>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>$ bin/hadoop com.sun.tools.javac.Main WordCount.java
|
|
$ jar cf wc.jar WordCount*.class
|
|
</pre></div></div>
|
|
|
|
<p>Assuming that:</p>
|
|
<ul>
|
|
|
|
<li><code>/user/joe/wordcount/input</code> - input directory in HDFS</li>
|
|
<li><code>/user/joe/wordcount/output</code> - output directory in HDFS</li>
|
|
</ul>
|
|
<p>Sample text-files as input:</p>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>$ bin/hadoop fs -ls /user/joe/wordcount/input/
|
|
/user/joe/wordcount/input/file01
|
|
/user/joe/wordcount/input/file02
|
|
|
|
$ bin/hadoop fs -cat /user/joe/wordcount/input/file01
|
|
Hello World Bye World
|
|
|
|
$ bin/hadoop fs -cat /user/joe/wordcount/input/file02
|
|
Hello Hadoop Goodbye Hadoop
|
|
</pre></div></div>
|
|
|
|
<p>Run the application:</p>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>$ bin/hadoop jar wc.jar WordCount /user/joe/wordcount/input /user/joe/wordcount/output
|
|
</pre></div></div>
|
|
|
|
<p>Output:</p>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>$ bin/hadoop fs -cat /user/joe/wordcount/output/part-r-00000
|
|
Bye 1
|
|
Goodbye 1
|
|
Hadoop 2
|
|
Hello 2
|
|
World 2
|
|
</pre></div></div>
|
|
|
|
<p>Applications can specify a comma separated list of paths which would be present in the current working directory of the task using the option <code>-files</code>. The <code>-libjars</code> option allows applications to add jars to the classpaths of the maps and reduces. The option <code>-archives</code> allows them to pass comma separated list of archives as arguments. These archives are unarchived and a link with name of the archive is created in the current working directory of tasks. More details about the command line options are available at <a href="../../hadoop-project-dist/hadoop-common/CommandsManual.html">Commands Guide</a>.</p>
|
|
<p>Running <code>wordcount</code> example with <code>-libjars</code>, <code>-files</code> and <code>-archives</code>:</p>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>bin/hadoop jar hadoop-mapreduce-examples-<ver>.jar wordcount -files cachefile.txt -libjars mylib.jar -archives myarchive.zip input output
|
|
</pre></div></div>
|
|
|
|
<p>Here, myarchive.zip will be placed and unzipped into a directory by the name “myarchive.zip”.</p>
|
|
<p>Users can specify a different symbolic name for files and archives passed through <code>-files</code> and <code>-archives</code> option, using #.</p>
|
|
<p>For example,</p>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>bin/hadoop jar hadoop-mapreduce-examples-<ver>.jar wordcount -files dir1/dict.txt#dict1,dir2/dict.txt#dict2 -archives mytar.tgz#tgzdir input output
|
|
</pre></div></div>
|
|
|
|
<p>Here, the files dir1/dict.txt and dir2/dict.txt can be accessed by tasks using the symbolic names dict1 and dict2 respectively. The archive mytar.tgz will be placed and unarchived into a directory by the name “tgzdir”.</p>
|
|
<p>Applications can specify environment variables for mapper, reducer, and application master tasks by specifying them on the command line using the options -Dmapreduce.map.env, -Dmapreduce.reduce.env, and -Dyarn.app.mapreduce.am.env, respectively.</p>
|
|
<p>For example the following sets environment variables FOO_VAR=bar and LIST_VAR=a,b,c for the mappers and reducers,</p>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>bin/hadoop jar hadoop-mapreduce-examples-<ver>.jar wordcount -Dmapreduce.map.env.FOO_VAR=bar -Dmapreduce.map.env.LIST_VAR=a,b,c -Dmapreduce.reduce.env.FOO_VAR=bar -Dmapreduce.reduce.env.LIST_VAR=a,b,c input output
|
|
</pre></div></div>
|
|
</section><section>
|
|
<h3><a name="Walk-through"></a>Walk-through</h3>
|
|
<p>The <code>WordCount</code> application is quite straight-forward.</p>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>public void map(Object key, Text value, Context context
|
|
) throws IOException, InterruptedException {
|
|
StringTokenizer itr = new StringTokenizer(value.toString());
|
|
while (itr.hasMoreTokens()) {
|
|
word.set(itr.nextToken());
|
|
context.write(word, one);
|
|
}
|
|
}
|
|
</pre></div></div>
|
|
|
|
<p>The <code>Mapper</code> implementation, via the <code>map</code> method, processes one line at a time, as provided by the specified <code>TextInputFormat</code>. It then splits the line into tokens separated by whitespaces, via the <code>StringTokenizer</code>, and emits a key-value pair of <code>< <word>, 1></code>.</p>
|
|
<p>For the given sample input the first map emits:</p>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>< Hello, 1>
|
|
< World, 1>
|
|
< Bye, 1>
|
|
< World, 1>
|
|
</pre></div></div>
|
|
|
|
<p>The second map emits:</p>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>< Hello, 1>
|
|
< Hadoop, 1>
|
|
< Goodbye, 1>
|
|
< Hadoop, 1>
|
|
</pre></div></div>
|
|
|
|
<p>We’ll learn more about the number of maps spawned for a given job, and how to control them in a fine-grained manner, a bit later in the tutorial.</p>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre> job.setCombinerClass(IntSumReducer.class);
|
|
</pre></div></div>
|
|
|
|
<p><code>WordCount</code> also specifies a <code>combiner</code>. Hence, the output of each map is passed through the local combiner (which is same as the <code>Reducer</code> as per the job configuration) for local aggregation, after being sorted on the <i>key</i>s.</p>
|
|
<p>The output of the first map:</p>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>< Bye, 1>
|
|
< Hello, 1>
|
|
< World, 2>
|
|
</pre></div></div>
|
|
|
|
<p>The output of the second map:</p>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>< Goodbye, 1>
|
|
< Hadoop, 2>
|
|
< Hello, 1>
|
|
</pre></div></div>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>public void reduce(Text key, Iterable<IntWritable> values,
|
|
Context context
|
|
) throws IOException, InterruptedException {
|
|
int sum = 0;
|
|
for (IntWritable val : values) {
|
|
sum += val.get();
|
|
}
|
|
result.set(sum);
|
|
context.write(key, result);
|
|
}
|
|
</pre></div></div>
|
|
|
|
<p>The <code>Reducer</code> implementation, via the <code>reduce</code> method just sums up the values, which are the occurrence counts for each key (i.e. words in this example).</p>
|
|
<p>Thus the output of the job is:</p>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>< Bye, 1>
|
|
< Goodbye, 1>
|
|
< Hadoop, 2>
|
|
< Hello, 2>
|
|
< World, 2>
|
|
</pre></div></div>
|
|
|
|
<p>The <code>main</code> method specifies various facets of the job, such as the input/output paths (passed via the command line), key/value types, input/output formats etc., in the <code>Job</code>. It then calls the <code>job.waitForCompletion</code> to submit the job and monitor its progress.</p>
|
|
<p>We’ll learn more about <code>Job</code>, <code>InputFormat</code>, <code>OutputFormat</code> and other interfaces and classes a bit later in the tutorial.</p></section></section><section>
|
|
<h2><a name="MapReduce_-_User_Interfaces"></a>MapReduce - User Interfaces</h2>
|
|
<p>This section provides a reasonable amount of detail on every user-facing aspect of the MapReduce framework. This should help users implement, configure and tune their jobs in a fine-grained manner. However, please note that the javadoc for each class/interface remains the most comprehensive documentation available; this is only meant to be a tutorial.</p>
|
|
<p>Let us first take the <code>Mapper</code> and <code>Reducer</code> interfaces. Applications typically implement them to provide the <code>map</code> and <code>reduce</code> methods.</p>
|
|
<p>We will then discuss other core interfaces including <code>Job</code>, <code>Partitioner</code>, <code>InputFormat</code>, <code>OutputFormat</code>, and others.</p>
|
|
<p>Finally, we will wrap up by discussing some useful features of the framework such as the <code>DistributedCache</code>, <code>IsolationRunner</code> etc.</p><section>
|
|
<h3><a name="Payload"></a>Payload</h3>
|
|
<p>Applications typically implement the <code>Mapper</code> and <code>Reducer</code> interfaces to provide the <code>map</code> and <code>reduce</code> methods. These form the core of the job.</p><section>
|
|
<h4><a name="Mapper"></a>Mapper</h4>
|
|
<p><a href="../../api/org/apache/hadoop/mapreduce/Mapper.html">Mapper</a> maps input key/value pairs to a set of intermediate key/value pairs.</p>
|
|
<p>Maps are the individual tasks that transform input records into intermediate records. The transformed intermediate records do not need to be of the same type as the input records. A given input pair may map to zero or many output pairs.</p>
|
|
<p>The Hadoop MapReduce framework spawns one map task for each <code>InputSplit</code> generated by the <code>InputFormat</code> for the job.</p>
|
|
<p>Overall, mapper implementations are passed to the job via <a href="../../api/org/apache/hadoop/mapreduce/Job.html">Job.setMapperClass(Class)</a> method. The framework then calls <a href="../../api/org/apache/hadoop/mapreduce/Mapper.html">map(WritableComparable, Writable, Context)</a> for each key/value pair in the <code>InputSplit</code> for that task. Applications can then override the <code>cleanup(Context)</code> method to perform any required cleanup.</p>
|
|
<p>Output pairs do not need to be of the same types as input pairs. A given input pair may map to zero or many output pairs. Output pairs are collected with calls to context.write(WritableComparable, Writable).</p>
|
|
<p>Applications can use the <code>Counter</code> to report its statistics.</p>
|
|
<p>All intermediate values associated with a given output key are subsequently grouped by the framework, and passed to the <code>Reducer</code>(s) to determine the final output. Users can control the grouping by specifying a <code>Comparator</code> via <a href="../../api/org/apache/hadoop/mapreduce/Job.html">Job.setGroupingComparatorClass(Class)</a>.</p>
|
|
<p>The <code>Mapper</code> outputs are sorted and then partitioned per <code>Reducer</code>. The total number of partitions is the same as the number of reduce tasks for the job. Users can control which keys (and hence records) go to which <code>Reducer</code> by implementing a custom <code>Partitioner</code>.</p>
|
|
<p>Users can optionally specify a <code>combiner</code>, via <a href="../../api/org/apache/hadoop/mapreduce/Job.html">Job.setCombinerClass(Class)</a>, to perform local aggregation of the intermediate outputs, which helps to cut down the amount of data transferred from the <code>Mapper</code> to the <code>Reducer</code>.</p>
|
|
<p>The intermediate, sorted outputs are always stored in a simple (key-len, key, value-len, value) format. Applications can control if, and how, the intermediate outputs are to be compressed and the <a href="../../api/org/apache/hadoop/io/compress/CompressionCodec.html">CompressionCodec</a> to be used via the <code>Configuration</code>.</p><section>
|
|
<h5><a name="How_Many_Maps.3F"></a>How Many Maps?</h5>
|
|
<p>The number of maps is usually driven by the total size of the inputs, that is, the total number of blocks of the input files.</p>
|
|
<p>The right level of parallelism for maps seems to be around 10-100 maps per-node, although it has been set up to 300 maps for very cpu-light map tasks. Task setup takes a while, so it is best if the maps take at least a minute to execute.</p>
|
|
<p>Thus, if you expect 10TB of input data and have a blocksize of <code>128MB</code>, you’ll end up with 82,000 maps, unless Configuration.set(<code>MRJobConfig.NUM_MAPS</code>, int) (which only provides a hint to the framework) is used to set it even higher.</p></section></section><section>
|
|
<h4><a name="Reducer"></a>Reducer</h4>
|
|
<p><a href="../../api/org/apache/hadoop/mapreduce/Reducer.html">Reducer</a> reduces a set of intermediate values which share a key to a smaller set of values.</p>
|
|
<p>The number of reduces for the job is set by the user via <a href="../../api/org/apache/hadoop/mapreduce/Job.html">Job.setNumReduceTasks(int)</a>.</p>
|
|
<p>Overall, <code>Reducer</code> implementations are passed the <code>Job</code> for the job via the <a href="../../api/org/apache/hadoop/mapreduce/Job.html">Job.setReducerClass(Class)</a> method and can override it to initialize themselves. The framework then calls <a href="../../api/org/apache/hadoop/mapreduce/Reducer.html">reduce(WritableComparable, Iterable<Writable>, Context)</a> method for each <code><key, (list of values)></code> pair in the grouped inputs. Applications can then override the <code>cleanup(Context)</code> method to perform any required cleanup.</p>
|
|
<p><code>Reducer</code> has 3 primary phases: shuffle, sort and reduce.</p><section>
|
|
<h5><a name="Shuffle"></a>Shuffle</h5>
|
|
<p>Input to the <code>Reducer</code> is the sorted output of the mappers. In this phase the framework fetches the relevant partition of the output of all the mappers, via HTTP.</p></section><section>
|
|
<h5><a name="Sort"></a>Sort</h5>
|
|
<p>The framework groups <code>Reducer</code> inputs by keys (since different mappers may have output the same key) in this stage.</p>
|
|
<p>The shuffle and sort phases occur simultaneously; while map-outputs are being fetched they are merged.</p></section><section>
|
|
<h5><a name="Secondary_Sort"></a>Secondary Sort</h5>
|
|
<p>If equivalence rules for grouping the intermediate keys are required to be different from those for grouping keys before reduction, then one may specify a <code>Comparator</code> via <a href="../../api/org/apache/hadoop/mapreduce/Job.html">Job.setSortComparatorClass(Class)</a>. Since <a href="../../api/org/apache/hadoop/mapreduce/Job.html">Job.setGroupingComparatorClass(Class)</a> can be used to control how intermediate keys are grouped, these can be used in conjunction to simulate <i>secondary sort on values</i>.</p></section><section>
|
|
<h5><a name="Reduce"></a>Reduce</h5>
|
|
<p>In this phase the reduce(WritableComparable, Iterable<Writable>, Context) method is called for each <code><key, (list of values)></code> pair in the grouped inputs.</p>
|
|
<p>The output of the reduce task is typically written to the <a href="../../api/org/apache/hadoop/fs/FileSystem.html">FileSystem</a> via Context.write(WritableComparable, Writable).</p>
|
|
<p>Applications can use the <code>Counter</code> to report its statistics.</p>
|
|
<p>The output of the <code>Reducer</code> is <i>not sorted</i>.</p></section><section>
|
|
<h5><a name="How_Many_Reduces.3F"></a>How Many Reduces?</h5>
|
|
<p>The right number of reduces seems to be <code>0.95</code> or <code>1.75</code> multiplied by (<<i>no. of nodes</i>> * <<i>no. of maximum containers per node</i>>).</p>
|
|
<p>With <code>0.95</code> all of the reduces can launch immediately and start transferring map outputs as the maps finish. With <code>1.75</code> the faster nodes will finish their first round of reduces and launch a second wave of reduces doing a much better job of load balancing.</p>
|
|
<p>Increasing the number of reduces increases the framework overhead, but increases load balancing and lowers the cost of failures.</p>
|
|
<p>The scaling factors above are slightly less than whole numbers to reserve a few reduce slots in the framework for speculative-tasks and failed tasks.</p></section><section>
|
|
<h5><a name="Reducer_NONE"></a>Reducer NONE</h5>
|
|
<p>It is legal to set the number of reduce-tasks to <i>zero</i> if no reduction is desired.</p>
|
|
<p>In this case the outputs of the map-tasks go directly to the <code>FileSystem</code>, into the output path set by <a href="../../api/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.html">FileOutputFormat.setOutputPath(Job, Path)</a>. The framework does not sort the map-outputs before writing them out to the <code>FileSystem</code>.</p></section></section><section>
|
|
<h4><a name="Partitioner"></a>Partitioner</h4>
|
|
<p><a href="../../api/org/apache/hadoop/mapreduce/Partitioner.html">Partitioner</a> partitions the key space.</p>
|
|
<p>Partitioner controls the partitioning of the keys of the intermediate map-outputs. The key (or a subset of the key) is used to derive the partition, typically by a <i>hash function</i>. The total number of partitions is the same as the number of reduce tasks for the job. Hence this controls which of the <code>m</code> reduce tasks the intermediate key (and hence the record) is sent to for reduction.</p>
|
|
<p><a href="../../api/org/apache/hadoop/mapreduce/lib/partition/HashPartitioner.html">HashPartitioner</a> is the default <code>Partitioner</code>.</p></section><section>
|
|
<h4><a name="Counter"></a>Counter</h4>
|
|
<p><a href="../../api/org/apache/hadoop/mapreduce/Counter.html">Counter</a> is a facility for MapReduce applications to report its statistics.</p>
|
|
<p><code>Mapper</code> and <code>Reducer</code> implementations can use the <code>Counter</code> to report statistics.</p>
|
|
<p>Hadoop MapReduce comes bundled with a <a href="../../api/org/apache/hadoop/mapreduce/package-summary.html">library</a> of generally useful mappers, reducers, and partitioners.</p></section></section><section>
|
|
<h3><a name="Job_Configuration"></a>Job Configuration</h3>
|
|
<p><a href="../../api/org/apache/hadoop/mapreduce/Job.html">Job</a> represents a MapReduce job configuration.</p>
|
|
<p><code>Job</code> is the primary interface for a user to describe a MapReduce job to the Hadoop framework for execution. The framework tries to faithfully execute the job as described by <code>Job</code>, however:</p>
|
|
<ul>
|
|
|
|
<li>
|
|
|
|
<p>Some configuration parameters may have been marked as final by administrators (see <a href="../../api/org/apache/hadoop/conf/Configuration.html#FinalParams">Final Parameters</a>) and hence cannot be altered.</p>
|
|
</li>
|
|
<li>
|
|
|
|
<p>While some job parameters are straight-forward to set (e.g. <a href="../../api/org/apache/hadoop/mapreduce/Job.html">Job.setNumReduceTasks(int)</a>) , other parameters interact subtly with the rest of the framework and/or job configuration and are more complex to set (e.g. <a href="../../api/org/apache/hadoop/conf/Configuration.html">Configuration.set(<code>JobContext.NUM_MAPS</code>, int)</a>).</p>
|
|
</li>
|
|
</ul>
|
|
<p><code>Job</code> is typically used to specify the <code>Mapper</code>, combiner (if any), <code>Partitioner</code>, <code>Reducer</code>, <code>InputFormat</code>, <code>OutputFormat</code> implementations. <a href="../../api/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.html">FileInputFormat</a> indicates the set of input files (<a href="../../api/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.html">FileInputFormat.setInputPaths(Job, Path…)</a>/ <a href="../../api/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.html">FileInputFormat.addInputPath(Job, Path)</a>) and (<a href="../../api/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.html">FileInputFormat.setInputPaths(Job, String…)</a>/ <a href="../../api/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.html">FileInputFormat.addInputPaths(Job, String))</a> and where the output files should be written (<a href="../../api/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.html">FileOutputFormat.setOutputPath(Path)</a>).</p>
|
|
<p>Optionally, <code>Job</code> is used to specify other advanced facets of the job such as the <code>Comparator</code> to be used, files to be put in the <code>DistributedCache</code>, whether intermediate and/or job outputs are to be compressed (and how), whether job tasks can be executed in a <i>speculative</i> manner (<a href="../../api/org/apache/hadoop/mapreduce/Job.html">setMapSpeculativeExecution(boolean)</a>)/ <a href="../../api/org/apache/hadoop/mapreduce/Job.html">setReduceSpeculativeExecution(boolean)</a>), maximum number of attempts per task (<a href="../../api/org/apache/hadoop/mapreduce/Job.html">setMaxMapAttempts(int)</a>/ <a href="../../api/org/apache/hadoop/mapreduce/Job.html">setMaxReduceAttempts(int)</a>) etc.</p>
|
|
<p>Of course, users can use <a href="../../api/org/apache/hadoop/conf/Configuration.html">Configuration.set(String, String)</a>/ <a href="../../api/org/apache/hadoop/conf/Configuration.html">Configuration.get(String)</a> to set/get arbitrary parameters needed by applications. However, use the <code>DistributedCache</code> for large amounts of (read-only) data.</p></section><section>
|
|
<h3><a name="Task_Execution_.26_Environment"></a>Task Execution & Environment</h3>
|
|
<p>The <code>MRAppMaster</code> executes the <code>Mapper</code>/<code>Reducer</code> <i>task</i> as a child process in a separate jvm.</p>
|
|
<p>The child-task inherits the environment of the parent <code>MRAppMaster</code>. The user can specify additional options to the child-jvm via the <code>mapreduce.{map|reduce}.java.opts</code> and configuration parameter in the <code>Job</code> such as non-standard paths for the run-time linker to search shared libraries via <code>-Djava.library.path=<></code> etc. If the <code>mapreduce.{map|reduce}.java.opts</code> parameters contains the symbol <i>@taskid@</i> it is interpolated with value of <code>taskid</code> of the MapReduce task.</p>
|
|
<p>Here is an example with multiple arguments and substitutions, showing jvm GC logging, and start of a passwordless JVM JMX agent so that it can connect with jconsole and the likes to watch child memory, threads and get thread dumps. It also sets the maximum heap-size of the map and reduce child jvm to 512MB & 1024MB respectively. It also adds an additional path to the <code>java.library.path</code> of the child-jvm.</p>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre><property>
|
|
<name>mapreduce.map.java.opts</name>
|
|
<value>
|
|
-Xmx512M -Djava.library.path=/home/mycompany/lib -verbose:gc -Xloggc:/tmp/@taskid@.gc
|
|
-Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false
|
|
</value>
|
|
</property>
|
|
|
|
<property>
|
|
<name>mapreduce.reduce.java.opts</name>
|
|
<value>
|
|
-Xmx1024M -Djava.library.path=/home/mycompany/lib -verbose:gc -Xloggc:/tmp/@taskid@.gc
|
|
-Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false
|
|
</value>
|
|
</property>
|
|
</pre></div></div>
|
|
<section>
|
|
<h4><a name="Memory_Management"></a>Memory Management</h4>
|
|
<p>Users/admins can also specify the maximum virtual memory of the launched child-task, and any sub-process it launches recursively, using <code>mapreduce.{map|reduce}.memory.mb</code>. Note that the value set here is a per process limit. The value for <code>mapreduce.{map|reduce}.memory.mb</code> should be specified in mega bytes (MB). And also the value must be greater than or equal to the -Xmx passed to JavaVM, else the VM might not start.</p>
|
|
<p>Note: <code>mapreduce.{map|reduce}.java.opts</code> are used only for configuring the launched child tasks from MRAppMaster. Configuring the memory options for daemons is documented in <a href="../../hadoop-project-dist/hadoop-common/ClusterSetup.html#Configuring_Environment_of_Hadoop_Daemons">Configuring the Environment of the Hadoop Daemons</a>.</p>
|
|
<p>The memory available to some parts of the framework is also configurable. In map and reduce tasks, performance may be influenced by adjusting parameters influencing the concurrency of operations and the frequency with which data will hit disk. Monitoring the filesystem counters for a job- particularly relative to byte counts from the map and into the reduce- is invaluable to the tuning of these parameters.</p></section><section>
|
|
<h4><a name="Map_Parameters"></a>Map Parameters</h4>
|
|
<p>A record emitted from a map will be serialized into a buffer and metadata will be stored into accounting buffers. As described in the following options, when either the serialization buffer or the metadata exceed a threshold, the contents of the buffers will be sorted and written to disk in the background while the map continues to output records. If either buffer fills completely while the spill is in progress, the map thread will block. When the map is finished, any remaining records are written to disk and all on-disk segments are merged into a single file. Minimizing the number of spills to disk can decrease map time, but a larger buffer also decreases the memory available to the mapper.</p>
|
|
<table border="0" class="bodyTable">
|
|
<thead>
|
|
|
|
<tr class="a">
|
|
<th align="left"> Name </th>
|
|
<th align="left"> Type </th>
|
|
<th align="left"> Description </th></tr>
|
|
</thead><tbody>
|
|
|
|
<tr class="b">
|
|
<td align="left"> mapreduce.task.io.sort.mb </td>
|
|
<td align="left"> int </td>
|
|
<td align="left"> The cumulative size of the serialization and accounting buffers storing records emitted from the map, in megabytes. </td></tr>
|
|
<tr class="a">
|
|
<td align="left"> mapreduce.map.sort.spill.percent </td>
|
|
<td align="left"> float </td>
|
|
<td align="left"> The soft limit in the serialization buffer. Once reached, a thread will begin to spill the contents to disk in the background. </td></tr>
|
|
</tbody>
|
|
</table>
|
|
<p>Other notes</p>
|
|
<ul>
|
|
|
|
<li>
|
|
|
|
<p>If either spill threshold is exceeded while a spill is in progress, collection will continue until the spill is finished. For example, if <code>mapreduce.map.sort.spill.percent</code> is set to 0.33, and the remainder of the buffer is filled while the spill runs, the next spill will include all the collected records, or 0.66 of the buffer, and will not generate additional spills. In other words, the thresholds are defining triggers, not blocking.</p>
|
|
</li>
|
|
<li>
|
|
|
|
<p>A record larger than the serialization buffer will first trigger a spill, then be spilled to a separate file. It is undefined whether or not this record will first pass through the combiner.</p>
|
|
</li>
|
|
</ul></section><section>
|
|
<h4><a name="Shuffle.2FReduce_Parameters"></a>Shuffle/Reduce Parameters</h4>
|
|
<p>As described previously, each reduce fetches the output assigned to it by the Partitioner via HTTP into memory and periodically merges these outputs to disk. If intermediate compression of map outputs is turned on, each output is decompressed into memory. The following options affect the frequency of these merges to disk prior to the reduce and the memory allocated to map output during the reduce.</p>
|
|
<table border="0" class="bodyTable">
|
|
<thead>
|
|
|
|
<tr class="a">
|
|
<th align="left"> Name </th>
|
|
<th align="left"> Type </th>
|
|
<th align="left"> Description </th></tr>
|
|
</thead><tbody>
|
|
|
|
<tr class="b">
|
|
<td align="left"> mapreduce.task.io.soft.factor </td>
|
|
<td align="left"> int </td>
|
|
<td align="left"> Specifies the number of segments on disk to be merged at the same time. It limits the number of open files and compression codecs during merge. If the number of files exceeds this limit, the merge will proceed in several passes. Though this limit also applies to the map, most jobs should be configured so that hitting this limit is unlikely there. </td></tr>
|
|
<tr class="a">
|
|
<td align="left"> mapreduce.reduce.merge.inmem.thresholds </td>
|
|
<td align="left"> int </td>
|
|
<td align="left"> The number of sorted map outputs fetched into memory before being merged to disk. Like the spill thresholds in the preceding note, this is not defining a unit of partition, but a trigger. In practice, this is usually set very high (1000) or disabled (0), since merging in-memory segments is often less expensive than merging from disk (see notes following this table). This threshold influences only the frequency of in-memory merges during the shuffle. </td></tr>
|
|
<tr class="b">
|
|
<td align="left"> mapreduce.reduce.shuffle.merge.percent </td>
|
|
<td align="left"> float </td>
|
|
<td align="left"> The memory threshold for fetched map outputs before an in-memory merge is started, expressed as a percentage of memory allocated to storing map outputs in memory. Since map outputs that can’t fit in memory can be stalled, setting this high may decrease parallelism between the fetch and merge. Conversely, values as high as 1.0 have been effective for reduces whose input can fit entirely in memory. This parameter influences only the frequency of in-memory merges during the shuffle. </td></tr>
|
|
<tr class="a">
|
|
<td align="left"> mapreduce.reduce.shuffle.input.buffer.percent </td>
|
|
<td align="left"> float </td>
|
|
<td align="left"> The percentage of memory- relative to the maximum heapsize as typically specified in <code>mapreduce.reduce.java.opts</code>- that can be allocated to storing map outputs during the shuffle. Though some memory should be set aside for the framework, in general it is advantageous to set this high enough to store large and numerous map outputs. </td></tr>
|
|
<tr class="b">
|
|
<td align="left"> mapreduce.reduce.input.buffer.percent </td>
|
|
<td align="left"> float </td>
|
|
<td align="left"> The percentage of memory relative to the maximum heapsize in which map outputs may be retained during the reduce. When the reduce begins, map outputs will be merged to disk until those that remain are under the resource limit this defines. By default, all map outputs are merged to disk before the reduce begins to maximize the memory available to the reduce. For less memory-intensive reduces, this should be increased to avoid trips to disk. </td></tr>
|
|
</tbody>
|
|
</table>
|
|
<p>Other notes</p>
|
|
<ul>
|
|
|
|
<li>
|
|
|
|
<p>If a map output is larger than 25 percent of the memory allocated to copying map outputs, it will be written directly to disk without first staging through memory.</p>
|
|
</li>
|
|
<li>
|
|
|
|
<p>When running with a combiner, the reasoning about high merge thresholds and large buffers may not hold. For merges started before all map outputs have been fetched, the combiner is run while spilling to disk. In some cases, one can obtain better reduce times by spending resources combining map outputs- making disk spills small and parallelizing spilling and fetching- rather than aggressively increasing buffer sizes.</p>
|
|
</li>
|
|
<li>
|
|
|
|
<p>When merging in-memory map outputs to disk to begin the reduce, if an intermediate merge is necessary because there are segments to spill and at least <code>mapreduce.task.io.sort.factor</code> segments already on disk, the in-memory map outputs will be part of the intermediate merge.</p>
|
|
</li>
|
|
</ul></section><section>
|
|
<h4><a name="Configured_Parameters"></a>Configured Parameters</h4>
|
|
<p>The following properties are localized in the job configuration for each task’s execution:</p>
|
|
<table border="0" class="bodyTable">
|
|
<thead>
|
|
|
|
<tr class="a">
|
|
<th align="left"> Name </th>
|
|
<th align="left"> Type </th>
|
|
<th align="left"> Description </th></tr>
|
|
</thead><tbody>
|
|
|
|
<tr class="b">
|
|
<td align="left"> mapreduce.job.id </td>
|
|
<td align="left"> String </td>
|
|
<td align="left"> The job id </td></tr>
|
|
<tr class="a">
|
|
<td align="left"> mapreduce.job.jar </td>
|
|
<td align="left"> String </td>
|
|
<td align="left"> job.jar location in job directory </td></tr>
|
|
<tr class="b">
|
|
<td align="left"> mapreduce.job.local.dir </td>
|
|
<td align="left"> String </td>
|
|
<td align="left"> The job specific shared scratch space </td></tr>
|
|
<tr class="a">
|
|
<td align="left"> mapreduce.task.id </td>
|
|
<td align="left"> String </td>
|
|
<td align="left"> The task id </td></tr>
|
|
<tr class="b">
|
|
<td align="left"> mapreduce.task.attempt.id </td>
|
|
<td align="left"> String </td>
|
|
<td align="left"> The task attempt id </td></tr>
|
|
<tr class="a">
|
|
<td align="left"> mapreduce.task.is.map </td>
|
|
<td align="left"> boolean </td>
|
|
<td align="left"> Is this a map task </td></tr>
|
|
<tr class="b">
|
|
<td align="left"> mapreduce.task.partition </td>
|
|
<td align="left"> int </td>
|
|
<td align="left"> The id of the task within the job </td></tr>
|
|
<tr class="a">
|
|
<td align="left"> mapreduce.map.input.file </td>
|
|
<td align="left"> String </td>
|
|
<td align="left"> The filename that the map is reading from </td></tr>
|
|
<tr class="b">
|
|
<td align="left"> mapreduce.map.input.start </td>
|
|
<td align="left"> long </td>
|
|
<td align="left"> The offset of the start of the map input split </td></tr>
|
|
<tr class="a">
|
|
<td align="left"> mapreduce.map.input.length </td>
|
|
<td align="left"> long </td>
|
|
<td align="left"> The number of bytes in the map input split </td></tr>
|
|
<tr class="b">
|
|
<td align="left"> mapreduce.task.output.dir </td>
|
|
<td align="left"> String </td>
|
|
<td align="left"> The task’s temporary output directory </td></tr>
|
|
</tbody>
|
|
</table>
|
|
<p><b>Note:</b> During the execution of a streaming job, the names of the “mapreduce” parameters are transformed. The dots ( . ) become underscores ( _ ). For example, mapreduce.job.id becomes mapreduce_job_id and mapreduce.job.jar becomes mapreduce_job_jar. To get the values in a streaming job’s mapper/reducer use the parameter names with the underscores.</p></section><section>
|
|
<h4><a name="Task_Logs"></a>Task Logs</h4>
|
|
<p>The standard output (stdout) and error (stderr) streams and the syslog of the task are read by the NodeManager and logged to <code>${HADOOP_LOG_DIR}/userlogs</code>.</p></section><section>
|
|
<h4><a name="Distributing_Libraries"></a>Distributing Libraries</h4>
|
|
<p>The <a href="#DistributedCache">DistributedCache</a> can also be used to distribute both jars and native libraries for use in the map and/or reduce tasks. The child-jvm always has its <i>current working directory</i> added to the <code>java.library.path</code> and <code>LD_LIBRARY_PATH</code>. And hence the cached libraries can be loaded via <a class="externalLink" href="http://docs.oracle.com/javase/7/docs/api/java/lang/System.html">System.loadLibrary</a> or <a class="externalLink" href="http://docs.oracle.com/javase/7/docs/api/java/lang/System.html">System.load</a>. More details on how to load shared libraries through distributed cache are documented at <a href="../../hadoop-project-dist/hadoop-common/NativeLibraries.html#Native_Shared_Libraries">Native Libraries</a>.</p></section></section><section>
|
|
<h3><a name="Job_Submission_and_Monitoring"></a>Job Submission and Monitoring</h3>
|
|
<p><a href="../../api/org/apache/hadoop/mapreduce/Job.html">Job</a> is the primary interface by which user-job interacts with the <code>ResourceManager</code>.</p>
|
|
<p><code>Job</code> provides facilities to submit jobs, track their progress, access component-tasks’ reports and logs, get the MapReduce cluster’s status information and so on.</p>
|
|
<p>The job submission process involves:</p>
|
|
<ol style="list-style-type: decimal">
|
|
|
|
<li>
|
|
|
|
<p>Checking the input and output specifications of the job.</p>
|
|
</li>
|
|
<li>
|
|
|
|
<p>Computing the <code>InputSplit</code> values for the job.</p>
|
|
</li>
|
|
<li>
|
|
|
|
<p>Setting up the requisite accounting information for the <code>DistributedCache</code> of the job, if necessary.</p>
|
|
</li>
|
|
<li>
|
|
|
|
<p>Copying the job’s jar and configuration to the MapReduce system directory on the <code>FileSystem</code>.</p>
|
|
</li>
|
|
<li>
|
|
|
|
<p>Submitting the job to the <code>ResourceManager</code> and optionally monitoring it’s status.</p>
|
|
</li>
|
|
</ol>
|
|
<p>Job history files are also logged to user specified directory <code>mapreduce.jobhistory.intermediate-done-dir</code> and <code>mapreduce.jobhistory.done-dir</code>, which defaults to job output directory.</p>
|
|
<p>User can view the history logs summary in specified directory using the following command <code>$ mapred job -history output.jhist</code> This command will print job details, failed and killed tip details. More details about the job such as successful tasks and task attempts made for each task can be viewed using the following command <code>$ mapred job -history all output.jhist</code></p>
|
|
<p>Normally the user uses <code>Job</code> to create the application, describe various facets of the job, submit the job, and monitor its progress.</p><section>
|
|
<h4><a name="Job_Control"></a>Job Control</h4>
|
|
<p>Users may need to chain MapReduce jobs to accomplish complex tasks which cannot be done via a single MapReduce job. This is fairly easy since the output of the job typically goes to distributed file-system, and the output, in turn, can be used as the input for the next job.</p>
|
|
<p>However, this also means that the onus on ensuring jobs are complete (success/failure) lies squarely on the clients. In such cases, the various job-control options are:</p>
|
|
<ul>
|
|
|
|
<li>
|
|
|
|
<p><a href="../../api/org/apache/hadoop/mapreduce/Job.html">Job.submit()</a> : Submit the job to the cluster and return immediately.</p>
|
|
</li>
|
|
<li>
|
|
|
|
<p><a href="../../api/org/apache/hadoop/mapreduce/Job.html">Job.waitForCompletion(boolean)</a> : Submit the job to the cluster and wait for it to finish.</p>
|
|
</li>
|
|
</ul></section></section><section>
|
|
<h3><a name="Job_Input"></a>Job Input</h3>
|
|
<p><a href="../../api/org/apache/hadoop/mapreduce/InputFormat.html">InputFormat</a> describes the input-specification for a MapReduce job.</p>
|
|
<p>The MapReduce framework relies on the <code>InputFormat</code> of the job to:</p>
|
|
<ol style="list-style-type: decimal">
|
|
|
|
<li>
|
|
|
|
<p>Validate the input-specification of the job.</p>
|
|
</li>
|
|
<li>
|
|
|
|
<p>Split-up the input file(s) into logical <code>InputSplit</code> instances, each of which is then assigned to an individual <code>Mapper</code>.</p>
|
|
</li>
|
|
<li>
|
|
|
|
<p>Provide the <code>RecordReader</code> implementation used to glean input records from the logical <code>InputSplit</code> for processing by the <code>Mapper</code>.</p>
|
|
</li>
|
|
</ol>
|
|
<p>The default behavior of file-based <code>InputFormat</code> implementations, typically sub-classes of <a href="../../api/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.html">FileInputFormat</a>, is to split the input into <i>logical</i> <code>InputSplit</code> instances based on the total size, in bytes, of the input files. However, the <code>FileSystem</code> blocksize of the input files is treated as an upper bound for input splits. A lower bound on the split size can be set via <code>mapreduce.input.fileinputformat.split.minsize</code>.</p>
|
|
<p>Clearly, logical splits based on input-size is insufficient for many applications since record boundaries must be respected. In such cases, the application should implement a <code>RecordReader</code>, who is responsible for respecting record-boundaries and presents a record-oriented view of the logical <code>InputSplit</code> to the individual task.</p>
|
|
<p><a href="../../api/org/apache/hadoop/mapreduce/lib/input/TextInputFormat.html">TextInputFormat</a> is the default <code>InputFormat</code>.</p>
|
|
<p>If <code>TextInputFormat</code> is the <code>InputFormat</code> for a given job, the framework detects input-files with the <i>.gz</i> extensions and automatically decompresses them using the appropriate <code>CompressionCodec</code>. However, it must be noted that compressed files with the above extensions cannot be <i>split</i> and each compressed file is processed in its entirety by a single mapper.</p><section>
|
|
<h4><a name="InputSplit"></a>InputSplit</h4>
|
|
<p><a href="../../api/org/apache/hadoop/mapreduce/InputSplit.html">InputSplit</a> represents the data to be processed by an individual <code>Mapper</code>.</p>
|
|
<p>Typically <code>InputSplit</code> presents a byte-oriented view of the input, and it is the responsibility of <code>RecordReader</code> to process and present a record-oriented view.</p>
|
|
<p><a href="../../api/org/apache/hadoop/mapreduce/lib/input/FileSplit.html">FileSplit</a> is the default <code>InputSplit</code>. It sets <code>mapreduce.map.input.file</code> to the path of the input file for the logical split.</p></section><section>
|
|
<h4><a name="RecordReader"></a>RecordReader</h4>
|
|
<p><a href="../../api/org/apache/hadoop/mapreduce/RecordReader.html">RecordReader</a> reads <code><key, value></code> pairs from an <code>InputSplit</code>.</p>
|
|
<p>Typically the <code>RecordReader</code> converts the byte-oriented view of the input, provided by the <code>InputSplit</code>, and presents a record-oriented to the <code>Mapper</code> implementations for processing. <code>RecordReader</code> thus assumes the responsibility of processing record boundaries and presents the tasks with keys and values.</p></section></section><section>
|
|
<h3><a name="Job_Output"></a>Job Output</h3>
|
|
<p><a href="../../api/org/apache/hadoop/mapreduce/OutputFormat.html">OutputFormat</a> describes the output-specification for a MapReduce job.</p>
|
|
<p>The MapReduce framework relies on the <code>OutputFormat</code> of the job to:</p>
|
|
<ol style="list-style-type: decimal">
|
|
|
|
<li>
|
|
|
|
<p>Validate the output-specification of the job; for example, check that the output directory doesn’t already exist.</p>
|
|
</li>
|
|
<li>
|
|
|
|
<p>Provide the <code>RecordWriter</code> implementation used to write the output files of the job. Output files are stored in a <code>FileSystem</code>.</p>
|
|
</li>
|
|
</ol>
|
|
<p><code>TextOutputFormat</code> is the default <code>OutputFormat</code>.</p><section>
|
|
<h4><a name="OutputCommitter"></a>OutputCommitter</h4>
|
|
<p><a href="../../api/org/apache/hadoop/mapreduce/OutputCommitter.html">OutputCommitter</a> describes the commit of task output for a MapReduce job.</p>
|
|
<p>The MapReduce framework relies on the <code>OutputCommitter</code> of the job to:</p>
|
|
<ol style="list-style-type: decimal">
|
|
|
|
<li>
|
|
|
|
<p>Setup the job during initialization. For example, create the temporary output directory for the job during the initialization of the job. Job setup is done by a separate task when the job is in PREP state and after initializing tasks. Once the setup task completes, the job will be moved to RUNNING state.</p>
|
|
</li>
|
|
<li>
|
|
|
|
<p>Cleanup the job after the job completion. For example, remove the temporary output directory after the job completion. Job cleanup is done by a separate task at the end of the job. Job is declared SUCCEDED/FAILED/KILLED after the cleanup task completes.</p>
|
|
</li>
|
|
<li>
|
|
|
|
<p>Setup the task temporary output. Task setup is done as part of the same task, during task initialization.</p>
|
|
</li>
|
|
<li>
|
|
|
|
<p>Check whether a task needs a commit. This is to avoid the commit procedure if a task does not need commit.</p>
|
|
</li>
|
|
<li>
|
|
|
|
<p>Commit of the task output. Once task is done, the task will commit it’s output if required.</p>
|
|
</li>
|
|
<li>
|
|
|
|
<p>Discard the task commit. If the task has been failed/killed, the output will be cleaned-up. If task could not cleanup (in exception block), a separate task will be launched with same attempt-id to do the cleanup.</p>
|
|
</li>
|
|
</ol>
|
|
<p><code>FileOutputCommitter</code> is the default <code>OutputCommitter</code>. Job setup/cleanup tasks occupy map or reduce containers, whichever is available on the NodeManager. And JobCleanup task, TaskCleanup tasks and JobSetup task have the highest priority, and in that order.</p></section><section>
|
|
<h4><a name="Task_Side-Effect_Files"></a>Task Side-Effect Files</h4>
|
|
<p>In some applications, component tasks need to create and/or write to side-files, which differ from the actual job-output files.</p>
|
|
<p>In such cases there could be issues with two instances of the same <code>Mapper</code> or <code>Reducer</code> running simultaneously (for example, speculative tasks) trying to open and/or write to the same file (path) on the <code>FileSystem</code>. Hence the application-writer will have to pick unique names per task-attempt (using the attemptid, say <code>attempt_200709221812_0001_m_000000_0</code>), not just per task.</p>
|
|
<p>To avoid these issues the MapReduce framework, when the <code>OutputCommitter</code> is <code>FileOutputCommitter</code>, maintains a special <code>${mapreduce.output.fileoutputformat.outputdir}/_temporary/_${taskid}</code> sub-directory accessible via <code>${mapreduce.task.output.dir}</code> for each task-attempt on the <code>FileSystem</code> where the output of the task-attempt is stored. On successful completion of the task-attempt, the files in the <code>${mapreduce.output.fileoutputformat.outputdir}/_temporary/_${taskid}</code> (only) are <i>promoted</i> to <code>${mapreduce.output.fileoutputformat.outputdir}</code>. Of course, the framework discards the sub-directory of unsuccessful task-attempts. This process is completely transparent to the application.</p>
|
|
<p>The application-writer can take advantage of this feature by creating any side-files required in <code>${mapreduce.task.output.dir}</code> during execution of a task via <a href="../../api/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.html">FileOutputFormat.getWorkOutputPath(Conext)</a>, and the framework will promote them similarly for succesful task-attempts, thus eliminating the need to pick unique paths per task-attempt.</p>
|
|
<p>Note: The value of <code>${mapreduce.task.output.dir}</code> during execution of a particular task-attempt is actually <code>${mapreduce.output.fileoutputformat.outputdir}/_temporary/_{$taskid}</code>, and this value is set by the MapReduce framework. So, just create any side-files in the path returned by <a href="../../api/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.html">FileOutputFormat.getWorkOutputPath(Conext)</a> from MapReduce task to take advantage of this feature.</p>
|
|
<p>The entire discussion holds true for maps of jobs with reducer=NONE (i.e. 0 reduces) since output of the map, in that case, goes directly to HDFS.</p></section><section>
|
|
<h4><a name="RecordWriter"></a>RecordWriter</h4>
|
|
<p><a href="../../api/org/apache/hadoop/mapreduce/RecordWriter.html">RecordWriter</a> writes the output <code><key, value></code> pairs to an output file.</p>
|
|
<p>RecordWriter implementations write the job outputs to the <code>FileSystem</code>.</p></section></section><section>
|
|
<h3><a name="Other_Useful_Features"></a>Other Useful Features</h3><section>
|
|
<h4><a name="Submitting_Jobs_to_Queues"></a>Submitting Jobs to Queues</h4>
|
|
<p>Users submit jobs to Queues. Queues, as collection of jobs, allow the system to provide specific functionality. For example, queues use ACLs to control which users who can submit jobs to them. Queues are expected to be primarily used by Hadoop Schedulers.</p>
|
|
<p>Hadoop comes configured with a single mandatory queue, called ‘default’. Queue names are defined in the <code>mapreduce.job.queuename</code> property of the Hadoop site configuration. Some job schedulers, such as the <a href="../../hadoop-yarn/hadoop-yarn-site/CapacityScheduler.html">Capacity Scheduler</a>, support multiple queues.</p>
|
|
<p>A job defines the queue it needs to be submitted to through the <code>mapreduce.job.queuename</code> property, or through the Configuration.set(<code>MRJobConfig.QUEUE_NAME</code>, String) API. Setting the queue name is optional. If a job is submitted without an associated queue name, it is submitted to the ‘default’ queue.</p></section><section>
|
|
<h4><a name="Counters"></a>Counters</h4>
|
|
<p><code>Counters</code> represent global counters, defined either by the MapReduce framework or applications. Each <code>Counter</code> can be of any <code>Enum</code> type. Counters of a particular <code>Enum</code> are bunched into groups of type <code>Counters.Group</code>.</p>
|
|
<p>Applications can define arbitrary <code>Counters</code> (of type <code>Enum</code>) and update them via <a href="../../api/org/apache/hadoop/mapred/Counters.html">Counters.incrCounter(Enum, long)</a> or Counters.incrCounter(String, String, long) in the <code>map</code> and/or <code>reduce</code> methods. These counters are then globally aggregated by the framework.</p></section><section>
|
|
<h4><a name="DistributedCache"></a>DistributedCache</h4>
|
|
<p><code>DistributedCache</code> distributes application-specific, large, read-only files efficiently.</p>
|
|
<p><code>DistributedCache</code> is a facility provided by the MapReduce framework to cache files (text, archives, jars and so on) needed by applications.</p>
|
|
<p>Applications specify the files to be cached via urls (<a class="externalLink" href="hdfs://)">hdfs://)</a> in the <code>Job</code>. The <code>DistributedCache</code> assumes that the files specified via <a class="externalLink" href="hdfs://">hdfs://</a> urls are already present on the <code>FileSystem</code>.</p>
|
|
<p>The framework will copy the necessary files to the worker node before any tasks for the job are executed on that node. Its efficiency stems from the fact that the files are only copied once per job and the ability to cache archives which are un-archived on the workers.</p>
|
|
<p><code>DistributedCache</code> tracks the modification timestamps of the cached files. Clearly the cache files should not be modified by the application or externally while the job is executing.</p>
|
|
<p><code>DistributedCache</code> can be used to distribute simple, read-only data/text files and more complex types such as archives and jars. Archives (zip, tar, tgz and tar.gz files) are <i>un-archived</i> at the worker nodes. Files have <i>execution permissions</i> set.</p>
|
|
<p>The files/archives can be distributed by setting the property <code>mapreduce.job.cache.{files |archives}</code>. If more than one file/archive has to be distributed, they can be added as comma separated paths. The properties can also be set by APIs <a href="../../api/org/apache/hadoop/mapreduce/Job.html">Job.addCacheFile(URI)</a>/ <a href="../../api/org/apache/hadoop/mapreduce/Job.html">Job.addCacheArchive(URI)</a> and <a href="../../api/org/apache/hadoop/mapreduce/Job.html">Job.setCacheFiles(URI[])</a>/ <a href="../../api/org/apache/hadoop/mapreduce/Job.html">Job.setCacheArchives(URI[])</a> where URI is of the form <code>hdfs://host:port/absolute-path#link-name</code>. In Streaming, the files can be distributed through command line option <code>-cacheFile/-cacheArchive</code>.</p>
|
|
<p>The <code>DistributedCache</code> can also be used as a rudimentary software distribution mechanism for use in the map and/or reduce tasks. It can be used to distribute both jars and native libraries. The <a href="../../api/org/apache/hadoop/mapreduce/Job.html">Job.addArchiveToClassPath(Path)</a> or <a href="../../api/org/apache/hadoop/mapreduce/Job.html">Job.addFileToClassPath(Path)</a> api can be used to cache files/jars and also add them to the <i>classpath</i> of child-jvm. The same can be done by setting the configuration properties <code>mapreduce.job.classpath.{files |archives}</code>. Similarly the cached files that are symlinked into the working directory of the task can be used to distribute native libraries and load them.</p><section>
|
|
<h5><a name="Private_and_Public_DistributedCache_Files"></a>Private and Public DistributedCache Files</h5>
|
|
<p>DistributedCache files can be private or public, that determines how they can be shared on the worker nodes.</p>
|
|
<ul>
|
|
|
|
<li>
|
|
|
|
<p>“Private” DistributedCache files are cached in a localdirectory private to the user whose jobs need these files. These files are shared by all tasks and jobs of the specific user only and cannot be accessed by jobs of other users on the workers. A DistributedCache file becomes private by virtue of its permissions on the file system where the files are uploaded, typically HDFS. If the file has no world readable access, or if the directory path leading to the file has no world executable access for lookup, then the file becomes private.</p>
|
|
</li>
|
|
<li>
|
|
|
|
<p>“Public” DistributedCache files are cached in a global directory and the file access is setup such that they are publicly visible to all users. These files can be shared by tasks and jobs of all users on the workers. A DistributedCache file becomes public by virtue of its permissions on the file system where the files are uploaded, typically HDFS. If the file has world readable access, AND if the directory path leading to the file has world executable access for lookup, then the file becomes public. In other words, if the user intends to make a file publicly available to all users, the file permissions must be set to be world readable, and the directory permissions on the path leading to the file must be world executable.</p>
|
|
</li>
|
|
</ul></section></section><section>
|
|
<h4><a name="Profiling"></a>Profiling</h4>
|
|
<p>Profiling is a utility to get a representative (2 or 3) sample of built-in java profiler for a sample of maps and reduces.</p>
|
|
<p>User can specify whether the system should collect profiler information for some of the tasks in the job by setting the configuration property <code>mapreduce.task.profile</code>. The value can be set using the api Configuration.set(<code>MRJobConfig.TASK_PROFILE</code>, boolean). If the value is set <code>true</code>, the task profiling is enabled. The profiler information is stored in the user log directory. By default, profiling is not enabled for the job.</p>
|
|
<p>Once user configures that profiling is needed, she/he can use the configuration property <code>mapreduce.task.profile.{maps|reduces}</code> to set the ranges of MapReduce tasks to profile. The value can be set using the api Configuration.set(<code>MRJobConfig.NUM_{MAP|REDUCE}_PROFILES</code>, String). By default, the specified range is <code>0-2</code>.</p>
|
|
<p>User can also specify the profiler configuration arguments by setting the configuration property <code>mapreduce.task.profile.params</code>. The value can be specified using the api Configuration.set(<code>MRJobConfig.TASK_PROFILE_PARAMS</code>, String). If the string contains a <code>%s</code>, it will be replaced with the name of the profiling output file when the task runs. These parameters are passed to the task child JVM on the command line. The default value for the profiling parameters is <code>-agentlib:hprof=cpu=samples,heap=sites,force=n,thread=y,verbose=n,file=%s</code>.</p></section><section>
|
|
<h4><a name="Debugging"></a>Debugging</h4>
|
|
<p>The MapReduce framework provides a facility to run user-provided scripts for debugging. When a MapReduce task fails, a user can run a debug script, to process task logs for example. The script is given access to the task’s stdout and stderr outputs, syslog and jobconf. The output from the debug script’s stdout and stderr is displayed on the console diagnostics and also as part of the job UI.</p>
|
|
<p>In the following sections we discuss how to submit a debug script with a job. The script file needs to be distributed and submitted to the framework.</p><section>
|
|
<h5><a name="How_to_distribute_the_script_file:"></a>How to distribute the script file:</h5>
|
|
<p>The user needs to use <a href="#DistributedCache">DistributedCache</a> to <i>distribute</i> and <i>symlink</i> to the script file.</p></section><section>
|
|
<h5><a name="How_to_submit_the_script:"></a>How to submit the script:</h5>
|
|
<p>A quick way to submit the debug script is to set values for the properties <code>mapreduce.map.debug.script</code> and <code>mapreduce.reduce.debug.script</code>, for debugging map and reduce tasks respectively. These properties can also be set by using APIs <a href="../../api/org/apache/hadoop/conf/Configuration.html">Configuration.set(<code>MRJobConfig.MAP_DEBUG_SCRIPT</code>, String)</a> and <a href="../../api/org/apache/hadoop/conf/Configuration.html">Configuration.set(<code>MRJobConfig.REDUCE_DEBUG_SCRIPT</code>, String)</a>. In streaming mode, a debug script can be submitted with the command-line options <code>-mapdebug</code> and <code>-reducedebug</code>, for debugging map and reduce tasks respectively.</p>
|
|
<p>The arguments to the script are the task’s stdout, stderr, syslog and jobconf files. The debug command, run on the node where the MapReduce task failed, is:<br /> <code>$script $stdout $stderr $syslog $jobconf</code></p>
|
|
<p>Pipes programs have the c++ program name as a fifth argument for the command. Thus for the pipes programs the command is<br /> <code>$script $stdout $stderr $syslog $jobconf $program</code></p></section><section>
|
|
<h5><a name="Default_Behavior:"></a>Default Behavior:</h5>
|
|
<p>For pipes, a default script is run to process core dumps under gdb, prints stack trace and gives info about running threads.</p></section></section><section>
|
|
<h4><a name="Data_Compression"></a>Data Compression</h4>
|
|
<p>Hadoop MapReduce provides facilities for the application-writer to specify compression for both intermediate map-outputs and the job-outputs i.e. output of the reduces. It also comes bundled with <a href="../../api/org/apache/hadoop/io/compress/CompressionCodec.html">CompressionCodec</a> implementation for the <a class="externalLink" href="http://www.zlib.net">zlib</a> compression algorithm. The <a class="externalLink" href="http://www.gzip.org">gzip</a>, <a class="externalLink" href="http://www.bzip.org">bzip2</a>, <a class="externalLink" href="http://code.google.com/p/snappy/">snappy</a>, and <a class="externalLink" href="http://code.google.com/p/lz4/">lz4</a> file format are also supported.</p>
|
|
<p>Hadoop also provides native implementations of the above compression codecs for reasons of both performance (zlib) and non-availability of Java libraries. More details on their usage and availability are available <a href="../../hadoop-project-dist/hadoop-common/NativeLibraries.html">here</a>.</p><section>
|
|
<h5><a name="Intermediate_Outputs"></a>Intermediate Outputs</h5>
|
|
<p>Applications can control compression of intermediate map-outputs via the Configuration.set(<code>MRJobConfig.MAP_OUTPUT_COMPRESS</code>, boolean) api and the <code>CompressionCodec</code> to be used via the Configuration.set(<code>MRJobConfig.MAP_OUTPUT_COMPRESS_CODEC</code>, Class) api.</p></section><section>
|
|
<h5><a name="Job_Outputs"></a>Job Outputs</h5>
|
|
<p>Applications can control compression of job-outputs via the <a href="../../api/org/apache/hadoop/mapreduce/lib/output/FileOutputFormat.html">FileOutputFormat.setCompressOutput(Job, boolean)</a> api and the <code>CompressionCodec</code> to be used can be specified via the FileOutputFormat.setOutputCompressorClass(Job, Class) api.</p>
|
|
<p>If the job outputs are to be stored in the <a href="../../api/org/apache/hadoop/mapreduce/lib/output/SequenceFileOutputFormat.html">SequenceFileOutputFormat</a>, the required <code>SequenceFile.CompressionType</code> (i.e. <code>RECORD</code> / <code>BLOCK</code> - defaults to <code>RECORD</code>) can be specified via the SequenceFileOutputFormat.setOutputCompressionType(Job, SequenceFile.CompressionType) api.</p></section></section><section>
|
|
<h4><a name="Skipping_Bad_Records"></a>Skipping Bad Records</h4>
|
|
<p>Hadoop provides an option where a certain set of bad input records can be skipped when processing map inputs. Applications can control this feature through the <a href="../../api/org/apache/hadoop/mapred/SkipBadRecords.html">SkipBadRecords</a> class.</p>
|
|
<p>This feature can be used when map tasks crash deterministically on certain input. This usually happens due to bugs in the map function. Usually, the user would have to fix these bugs. This is, however, not possible sometimes. The bug may be in third party libraries, for example, for which the source code is not available. In such cases, the task never completes successfully even after multiple attempts, and the job fails. With this feature, only a small portion of data surrounding the bad records is lost, which may be acceptable for some applications (those performing statistical analysis on very large data, for example).</p>
|
|
<p>By default this feature is disabled. For enabling it, refer to <a href="../../api/org/apache/hadoop/mapred/SkipBadRecords.html">SkipBadRecords.setMapperMaxSkipRecords(Configuration, long)</a> and <a href="../../api/org/apache/hadoop/mapred/SkipBadRecords.html">SkipBadRecords.setReducerMaxSkipGroups(Configuration, long)</a>.</p>
|
|
<p>With this feature enabled, the framework gets into ‘skipping mode’ after a certain number of map failures. For more details, see <a href="../../api/org/apache/hadoop/mapred/SkipBadRecords.html">SkipBadRecords.setAttemptsToStartSkipping(Configuration, int)</a>. In ‘skipping mode’, map tasks maintain the range of records being processed. To do this, the framework relies on the processed record counter. See <a href="../../api/org/apache/hadoop/mapred/SkipBadRecords.html">SkipBadRecords.COUNTER_MAP_PROCESSED_RECORDS</a> and <a href="../../api/org/apache/hadoop/mapred/SkipBadRecords.html">SkipBadRecords.COUNTER_REDUCE_PROCESSED_GROUPS</a>. This counter enables the framework to know how many records have been processed successfully, and hence, what record range caused a task to crash. On further attempts, this range of records is skipped.</p>
|
|
<p>The number of records skipped depends on how frequently the processed record counter is incremented by the application. It is recommended that this counter be incremented after every record is processed. This may not be possible in some applications that typically batch their processing. In such cases, the framework may skip additional records surrounding the bad record. Users can control the number of skipped records through <a href="../../api/org/apache/hadoop/mapred/SkipBadRecords.html">SkipBadRecords.setMapperMaxSkipRecords(Configuration, long)</a> and <a href="../../api/org/apache/hadoop/mapred/SkipBadRecords.html">SkipBadRecords.setReducerMaxSkipGroups(Configuration, long)</a>. The framework tries to narrow the range of skipped records using a binary search-like approach. The skipped range is divided into two halves and only one half gets executed. On subsequent failures, the framework figures out which half contains bad records. A task will be re-executed till the acceptable skipped value is met or all task attempts are exhausted. To increase the number of task attempts, use <a href="../../api/org/apache/hadoop/mapreduce/Job.html">Job.setMaxMapAttempts(int)</a> and <a href="../../api/org/apache/hadoop/mapreduce/Job.html">Job.setMaxReduceAttempts(int)</a></p>
|
|
<p>Skipped records are written to HDFS in the sequence file format, for later analysis. The location can be changed through <a href="../../api/org/apache/hadoop/mapred/SkipBadRecords.html">SkipBadRecords.setSkipOutputPath(JobConf, Path)</a>.</p></section></section><section>
|
|
<h3><a name="Example:_WordCount_v2.0"></a>Example: WordCount v2.0</h3>
|
|
<p>Here is a more complete <code>WordCount</code> which uses many of the features provided by the MapReduce framework we discussed so far.</p>
|
|
<p>This needs the HDFS to be up and running, especially for the <code>DistributedCache</code>-related features. Hence it only works with a <a href="../../hadoop-project-dist/hadoop-common/SingleCluster.html">pseudo-distributed</a> or <a href="../../hadoop-project-dist/hadoop-common/ClusterSetup.html">fully-distributed</a> Hadoop installation.</p><section>
|
|
<h4><a name="Source_Code"></a>Source Code</h4>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>import java.io.BufferedReader;
|
|
import java.io.FileReader;
|
|
import java.io.IOException;
|
|
import java.net.URI;
|
|
import java.util.ArrayList;
|
|
import java.util.HashSet;
|
|
import java.util.List;
|
|
import java.util.Set;
|
|
import java.util.StringTokenizer;
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.io.IntWritable;
|
|
import org.apache.hadoop.io.Text;
|
|
import org.apache.hadoop.mapreduce.Job;
|
|
import org.apache.hadoop.mapreduce.Mapper;
|
|
import org.apache.hadoop.mapreduce.Reducer;
|
|
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
|
|
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
|
|
import org.apache.hadoop.mapreduce.Counter;
|
|
import org.apache.hadoop.util.GenericOptionsParser;
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
|
public class WordCount2 {
|
|
|
|
public static class TokenizerMapper
|
|
extends Mapper<Object, Text, Text, IntWritable>{
|
|
|
|
static enum CountersEnum { INPUT_WORDS }
|
|
|
|
private final static IntWritable one = new IntWritable(1);
|
|
private Text word = new Text();
|
|
|
|
private boolean caseSensitive;
|
|
private Set<String> patternsToSkip = new HashSet<String>();
|
|
|
|
private Configuration conf;
|
|
private BufferedReader fis;
|
|
|
|
@Override
|
|
public void setup(Context context) throws IOException,
|
|
InterruptedException {
|
|
conf = context.getConfiguration();
|
|
caseSensitive = conf.getBoolean("wordcount.case.sensitive", true);
|
|
if (conf.getBoolean("wordcount.skip.patterns", false)) {
|
|
URI[] patternsURIs = Job.getInstance(conf).getCacheFiles();
|
|
for (URI patternsURI : patternsURIs) {
|
|
Path patternsPath = new Path(patternsURI.getPath());
|
|
String patternsFileName = patternsPath.getName().toString();
|
|
parseSkipFile(patternsFileName);
|
|
}
|
|
}
|
|
}
|
|
|
|
private void parseSkipFile(String fileName) {
|
|
try {
|
|
fis = new BufferedReader(new FileReader(fileName));
|
|
String pattern = null;
|
|
while ((pattern = fis.readLine()) != null) {
|
|
patternsToSkip.add(pattern);
|
|
}
|
|
} catch (IOException ioe) {
|
|
System.err.println("Caught exception while parsing the cached file '"
|
|
+ StringUtils.stringifyException(ioe));
|
|
}
|
|
}
|
|
|
|
@Override
|
|
public void map(Object key, Text value, Context context
|
|
) throws IOException, InterruptedException {
|
|
String line = (caseSensitive) ?
|
|
value.toString() : value.toString().toLowerCase();
|
|
for (String pattern : patternsToSkip) {
|
|
line = line.replaceAll(pattern, "");
|
|
}
|
|
StringTokenizer itr = new StringTokenizer(line);
|
|
while (itr.hasMoreTokens()) {
|
|
word.set(itr.nextToken());
|
|
context.write(word, one);
|
|
Counter counter = context.getCounter(CountersEnum.class.getName(),
|
|
CountersEnum.INPUT_WORDS.toString());
|
|
counter.increment(1);
|
|
}
|
|
}
|
|
}
|
|
|
|
public static class IntSumReducer
|
|
extends Reducer<Text,IntWritable,Text,IntWritable> {
|
|
private IntWritable result = new IntWritable();
|
|
|
|
public void reduce(Text key, Iterable<IntWritable> values,
|
|
Context context
|
|
) throws IOException, InterruptedException {
|
|
int sum = 0;
|
|
for (IntWritable val : values) {
|
|
sum += val.get();
|
|
}
|
|
result.set(sum);
|
|
context.write(key, result);
|
|
}
|
|
}
|
|
|
|
public static void main(String[] args) throws Exception {
|
|
Configuration conf = new Configuration();
|
|
GenericOptionsParser optionParser = new GenericOptionsParser(conf, args);
|
|
String[] remainingArgs = optionParser.getRemainingArgs();
|
|
if ((remainingArgs.length != 2) && (remainingArgs.length != 4)) {
|
|
System.err.println("Usage: wordcount <in> <out> [-skip skipPatternFile]");
|
|
System.exit(2);
|
|
}
|
|
Job job = Job.getInstance(conf, "word count");
|
|
job.setJarByClass(WordCount2.class);
|
|
job.setMapperClass(TokenizerMapper.class);
|
|
job.setCombinerClass(IntSumReducer.class);
|
|
job.setReducerClass(IntSumReducer.class);
|
|
job.setOutputKeyClass(Text.class);
|
|
job.setOutputValueClass(IntWritable.class);
|
|
|
|
List<String> otherArgs = new ArrayList<String>();
|
|
for (int i=0; i < remainingArgs.length; ++i) {
|
|
if ("-skip".equals(remainingArgs[i])) {
|
|
job.addCacheFile(new Path(remainingArgs[++i]).toUri());
|
|
job.getConfiguration().setBoolean("wordcount.skip.patterns", true);
|
|
} else {
|
|
otherArgs.add(remainingArgs[i]);
|
|
}
|
|
}
|
|
FileInputFormat.addInputPath(job, new Path(otherArgs.get(0)));
|
|
FileOutputFormat.setOutputPath(job, new Path(otherArgs.get(1)));
|
|
|
|
System.exit(job.waitForCompletion(true) ? 0 : 1);
|
|
}
|
|
}
|
|
</pre></div></div>
|
|
</section><section>
|
|
<h4><a name="Sample_Runs"></a>Sample Runs</h4>
|
|
<p>Sample text-files as input:</p>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>$ bin/hadoop fs -ls /user/joe/wordcount/input/
|
|
/user/joe/wordcount/input/file01
|
|
/user/joe/wordcount/input/file02
|
|
|
|
$ bin/hadoop fs -cat /user/joe/wordcount/input/file01
|
|
Hello World, Bye World!
|
|
|
|
$ bin/hadoop fs -cat /user/joe/wordcount/input/file02
|
|
Hello Hadoop, Goodbye to hadoop.
|
|
</pre></div></div>
|
|
|
|
<p>Run the application:</p>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>$ bin/hadoop jar wc.jar WordCount2 /user/joe/wordcount/input /user/joe/wordcount/output
|
|
</pre></div></div>
|
|
|
|
<p>Output:</p>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>$ bin/hadoop fs -cat /user/joe/wordcount/output/part-r-00000
|
|
Bye 1
|
|
Goodbye 1
|
|
Hadoop, 1
|
|
Hello 2
|
|
World! 1
|
|
World, 1
|
|
hadoop. 1
|
|
to 1
|
|
</pre></div></div>
|
|
|
|
<p>Notice that the inputs differ from the first version we looked at, and how they affect the outputs.</p>
|
|
<p>Now, lets plug-in a pattern-file which lists the word-patterns to be ignored, via the <code>DistributedCache</code>.</p>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>$ bin/hadoop fs -cat /user/joe/wordcount/patterns.txt
|
|
\.
|
|
\,
|
|
\!
|
|
to
|
|
</pre></div></div>
|
|
|
|
<p>Run it again, this time with more options:</p>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>$ bin/hadoop jar wc.jar WordCount2 -Dwordcount.case.sensitive=true /user/joe/wordcount/input /user/joe/wordcount/output -skip /user/joe/wordcount/patterns.txt
|
|
</pre></div></div>
|
|
|
|
<p>As expected, the output:</p>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>$ bin/hadoop fs -cat /user/joe/wordcount/output/part-r-00000
|
|
Bye 1
|
|
Goodbye 1
|
|
Hadoop 1
|
|
Hello 2
|
|
World 2
|
|
hadoop 1
|
|
</pre></div></div>
|
|
|
|
<p>Run it once more, this time switch-off case-sensitivity:</p>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>$ bin/hadoop jar wc.jar WordCount2 -Dwordcount.case.sensitive=false /user/joe/wordcount/input /user/joe/wordcount/output -skip /user/joe/wordcount/patterns.txt
|
|
</pre></div></div>
|
|
|
|
<p>Sure enough, the output:</p>
|
|
|
|
<div class="source">
|
|
<div class="source">
|
|
<pre>$ bin/hadoop fs -cat /user/joe/wordcount/output/part-r-00000
|
|
bye 1
|
|
goodbye 1
|
|
hadoop 2
|
|
hello 2
|
|
world 2
|
|
</pre></div></div>
|
|
</section><section>
|
|
<h4><a name="Highlights"></a>Highlights</h4>
|
|
<p>The second version of <code>WordCount</code> improves upon the previous one by using some features offered by the MapReduce framework:</p>
|
|
<ul>
|
|
|
|
<li>
|
|
|
|
<p>Demonstrates how applications can access configuration parameters in the <code>setup</code> method of the <code>Mapper</code> (and <code>Reducer</code>) implementations.</p>
|
|
</li>
|
|
<li>
|
|
|
|
<p>Demonstrates how the <code>DistributedCache</code> can be used to distribute read-only data needed by the jobs. Here it allows the user to specify word-patterns to skip while counting.</p>
|
|
</li>
|
|
<li>
|
|
|
|
<p>Demonstrates the utility of the <code>GenericOptionsParser</code> to handle generic Hadoop command-line options.</p>
|
|
</li>
|
|
<li>
|
|
|
|
<p>Demonstrates how applications can use <code>Counters</code> and how they can set application-specific status information passed to the <code>map</code> (and <code>reduce</code>) method.</p>
|
|
</li>
|
|
</ul>
|
|
<p><i>Java and JNI are trademarks or registered trademarks of Oracle America, Inc. in the United States and other countries.</i></p></section></section></section>
|
|
</div>
|
|
</div>
|
|
<div class="clear">
|
|
<hr/>
|
|
</div>
|
|
<div id="footer">
|
|
<div class="xright">
|
|
© 2008-2023
|
|
Apache Software Foundation
|
|
|
|
- <a href="http://maven.apache.org/privacy-policy.html">Privacy Policy</a>.
|
|
Apache Maven, Maven, Apache, the Apache feather logo, and the Apache Maven project logos are trademarks of The Apache Software Foundation.
|
|
</div>
|
|
<div class="clear">
|
|
<hr/>
|
|
</div>
|
|
</div>
|
|
</body>
|
|
</html>
|