DistributedCache is a facility provided by the Map-Reduce framework to cache files (text, archives, jars etc.) needed by applications.

Applications specify the files, via urls (hdfs:// or http://) to be cached via the {@link org.apache.hadoop.mapred.JobConf}. The DistributedCache assumes that the files specified via urls are already present on the {@link FileSystem} at the path specified by the url and are accessible by every machine in the cluster.

The framework will copy the necessary files on to the slave node before any tasks for the job are executed on that node. Its efficiency stems from the fact that the files are only copied once per job and the ability to cache archives which are un-archived on the slaves.

DistributedCache can be used to distribute simple, read-only data/text files and/or more complex types such as archives, jars etc. Archives (zip, tar and tgz/tar.gz files) are un-archived at the slave nodes. Jars may be optionally added to the classpath of the tasks, a rudimentary software distribution mechanism. Files have execution permissions. Optionally users can also direct it to symlink the distributed cache file(s) into the working directory of the task.

DistributedCache tracks modification timestamps of the cache files. Clearly the cache files should not be modified by the application or externally while the job is executing.

Here is an illustrative example on how to use the DistributedCache:

     // Setting up the cache for the application
     
     1. Copy the requisite files to the FileSystem:
     
     $ bin/hadoop fs -copyFromLocal lookup.dat /myapp/lookup.dat  
     $ bin/hadoop fs -copyFromLocal map.zip /myapp/map.zip  
     $ bin/hadoop fs -copyFromLocal mylib.jar /myapp/mylib.jar
     $ bin/hadoop fs -copyFromLocal mytar.tar /myapp/mytar.tar
     $ bin/hadoop fs -copyFromLocal mytgz.tgz /myapp/mytgz.tgz
     $ bin/hadoop fs -copyFromLocal mytargz.tar.gz /myapp/mytargz.tar.gz
     
     2. Setup the application's JobConf:
     
     JobConf job = new JobConf();
     DistributedCache.addCacheFile(new URI("/myapp/lookup.dat#lookup.dat"), 
                                   job);
     DistributedCache.addCacheArchive(new URI("/myapp/map.zip", job);
     DistributedCache.addFileToClassPath(new Path("/myapp/mylib.jar"), job);
     DistributedCache.addCacheArchive(new URI("/myapp/mytar.tar", job);
     DistributedCache.addCacheArchive(new URI("/myapp/mytgz.tgz", job);
     DistributedCache.addCacheArchive(new URI("/myapp/mytargz.tar.gz", job);
     
     3. Use the cached files in the {@link org.apache.hadoop.mapred.Mapper}
     or {@link org.apache.hadoop.mapred.Reducer}:
     
     public static class MapClass extends MapReduceBase  
     implements Mapper<K, V, K, V> {
     
       private Path[] localArchives;
       private Path[] localFiles;
       
       public void configure(JobConf job) {
         // Get the cached archives/files
         localArchives = DistributedCache.getLocalCacheArchives(job);
         localFiles = DistributedCache.getLocalCacheFiles(job);
       }
       
       public void map(K key, V value, 
                       OutputCollector<K, V> output, Reporter reporter) 
       throws IOException {
         // Use data from the cached archives/files here
         // ...
         // ...
         output.collect(k, v);
       }
     }
     
 

It is also very common to use the DistributedCache by using {@link org.apache.hadoop.util.GenericOptionsParser}. This class includes methods that should be used by users (specifically those mentioned in the example above, as well as {@link DistributedCache#addArchiveToClassPath(Path, Configuration)}), as well as methods intended for use by the MapReduce framework (e.g., {@link org.apache.hadoop.mapred.JobClient}). For implementation details, see {@link TrackerDistributedCacheManager} and {@link TaskDistributedCacheManager}. @see org.apache.hadoop.mapred.JobConf @see org.apache.hadoop.mapred.JobClient @see org.apache.hadoop.mapreduce.Job @deprecated Use methods on {@link Job}.]]>
JobTracker, as {@link JobTracker.State} @return the current state of the JobTracker. @deprecated Use {@link #getJobTrackerStatus()} instead.]]> ClusterStatus provides clients with information such as:
  1. Size of the cluster.
  2. Name of the trackers.
  3. Task capacity of the cluster.
  4. The number of currently running map & reduce tasks.
  5. State of the JobTracker.
  6. Details regarding black listed trackers.

Clients can query for the latest ClusterStatus, via {@link JobClient#getClusterStatus()}.

@see JobClient @deprecated Use {@link ClusterMetrics} or {@link TaskTrackerInfo} instead]]>
Counters represent global counters, defined either by the Map-Reduce framework or applications. Each Counter can be of any {@link Enum} type.

Counters are bunched into {@link Group}s, each comprising of counters from a particular Enum class. @deprecated Use {@link org.apache.hadoop.mapreduce.Counters} instead.]]> Group of counters, comprising of counters from a particular counter {@link Enum} class.

Grouphandles localization of the class name and the counter names.

]]>
FileInputFormat implementations can override this and return false to ensure that individual input files are never split-up so that {@link Mapper}s process entire files. @param fs the file system that the file is on @param filename the file name to check @return is this file splitable?]]> FileInputFormat is the base class for all file-based InputFormats. This provides a generic implementation of {@link #getSplits(JobConf, int)}. Subclasses of FileInputFormat can also override the {@link #isSplitable(FileSystem, Path)} method to ensure input-files are not split-up and are processed as a whole by {@link Mapper}s. @deprecated Use {@link org.apache.hadoop.mapreduce.lib.input.FileInputFormat} instead.]]> true if the job output should be compressed, false otherwise]]> Tasks' Side-Effect Files

Note: The following is valid only if the {@link OutputCommitter} is {@link FileOutputCommitter}. If OutputCommitter is not a FileOutputCommitter, the task's temporary output directory is same as {@link #getOutputPath(JobConf)} i.e. ${mapreduce.output.fileoutputformat.outputdir}$

Some applications need to create/write-to side-files, which differ from the actual job-outputs.

In such cases there could be issues with 2 instances of the same TIP (running simultaneously e.g. speculative tasks) trying to open/write-to the same file (path) on HDFS. Hence the application-writer will have to pick unique names per task-attempt (e.g. using the attemptid, say attempt_200709221812_0001_m_000000_0), not just per TIP.

To get around this the Map-Reduce framework helps the application-writer out by maintaining a special ${mapreduce.output.fileoutputformat.outputdir}/_temporary/_${taskid} sub-directory for each task-attempt on HDFS where the output of the task-attempt goes. On successful completion of the task-attempt the files in the ${mapreduce.output.fileoutputformat.outputdir}/_temporary/_${taskid} (only) are promoted to ${mapreduce.output.fileoutputformat.outputdir}. Of course, the framework discards the sub-directory of unsuccessful task-attempts. This is completely transparent to the application.

The application-writer can take advantage of this by creating any side-files required in ${mapreduce.task.output.dir} during execution of his reduce-task i.e. via {@link #getWorkOutputPath(JobConf)}, and the framework will move them out similarly - thus she doesn't have to pick unique paths per task-attempt.

Note: the value of ${mapreduce.task.output.dir} during execution of a particular task-attempt is actually ${mapreduce.output.fileoutputformat.outputdir}/_temporary/_{$taskid}, and this value is set by the map-reduce framework. So, just create any side-files in the path returned by {@link #getWorkOutputPath(JobConf)} from map/reduce task to take advantage of this feature.

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.

@return the {@link Path} to the task's temporary output directory for the map-reduce job.]]>
The generated name can be used to create custom files from within the different tasks for the job, the names for different tasks will not collide with each other.

The given name is postfixed with the task type, 'm' for maps, 'r' for reduces and the task partition number. For example, give a name 'test' running on the first map o the job the generated name will be 'test-m-00000'.

@param conf the configuration for the job. @param name the name to make unique. @return a unique name accross all tasks of the job.]]>
The path can be used to create custom files from within the map and reduce tasks. The path name will be unique for each task. The path parent will be the job output directory.

ls

This method uses the {@link #getUniqueName} method to make the file name unique for the task.

@param conf the configuration for the job. @param name the name for the file. @return a unique path accross all tasks of the job.]]>
Each {@link InputSplit} is then assigned to an individual {@link Mapper} for processing.

Note: The split is a logical split of the inputs and the input files are not physically split into chunks. For e.g. a split could be <input-file-path, start, offset> tuple. @param job job configuration. @param numSplits the desired number of splits, a hint. @return an array of {@link InputSplit}s for the job.]]> It is the responsibility of the RecordReader to respect record boundaries while processing the logical split to present a record-oriented view to the individual task.

@param split the {@link InputSplit} @param job the job that this split belongs to @return a {@link RecordReader}]]>
InputFormat describes the input-specification for a Map-Reduce job.

The Map-Reduce framework relies on the InputFormat of the job to:

  1. Validate the input-specification of the job.
  2. Split-up the input file(s) into logical {@link InputSplit}s, each of which is then assigned to an individual {@link Mapper}.
  3. Provide the {@link RecordReader} implementation to be used to glean input records from the logical InputSplit for processing by the {@link Mapper}.

The default behavior of file-based {@link InputFormat}s, typically sub-classes of {@link FileInputFormat}, is to split the input into logical {@link InputSplit}s based on the total size, in bytes, of the input files. However, the {@link FileSystem} 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 mapreduce.input.fileinputformat.split.minsize.

Clearly, logical splits based on input-size is insufficient for many applications since record boundaries are to respected. In such cases, the application has to also implement a {@link RecordReader} on whom lies the responsibilty to respect record-boundaries and present a record-oriented view of the logical InputSplit to the individual task. @see InputSplit @see RecordReader @see JobClient @see FileInputFormat @deprecated Use {@link org.apache.hadoop.mapreduce.InputFormat} instead.]]> InputSplit. @return the number of bytes in the input split. @throws IOException]]> InputSplit is located as an array of Strings. @throws IOException]]> InputSplit represents the data to be processed by an individual {@link Mapper}.

Typically, it presents a byte-oriented view on the input and is the responsibility of {@link RecordReader} of the job to process this and present a record-oriented view. @see InputFormat @see RecordReader @deprecated Use {@link org.apache.hadoop.mapreduce.InputSplit} instead.]]> JobClient.]]> jobid doesn't correspond to any known job. @throws IOException]]> JobClient is the primary interface for the user-job to interact with the {@link JobTracker}. JobClient provides facilities to submit jobs, track their progress, access component-tasks' reports/logs, get the Map-Reduce cluster status information etc.

The job submission process involves:

  1. Checking the input and output specifications of the job.
  2. Computing the {@link InputSplit}s for the job.
  3. Setup the requisite accounting information for the {@link DistributedCache} of the job, if necessary.
  4. Copying the job's jar and configuration to the map-reduce system directory on the distributed file-system.
  5. Submitting the job to the JobTracker and optionally monitoring it's status.

Normally the user creates the application, describes various facets of the job via {@link JobConf} and then uses the JobClient to submit the job and monitor its progress.

Here is an example on how to use JobClient:

     // Create a new JobConf
     JobConf job = new JobConf(new Configuration(), MyJob.class);
     
     // Specify various job-specific parameters     
     job.setJobName("myjob");
     
     job.setInputPath(new Path("in"));
     job.setOutputPath(new Path("out"));
     
     job.setMapperClass(MyJob.MyMapper.class);
     job.setReducerClass(MyJob.MyReducer.class);

     // Submit the job, then poll for progress until the job is complete
     JobClient.runJob(job);
 

Job Control

At times clients would chain map-reduce jobs to accomplish complex tasks which cannot be done via a single map-reduce job. This is fairly easy since the output of the job, typically, goes to distributed file-system and that can be used as the input for the next job.

However, this also means that the onus on ensuring jobs are complete (success/failure) lies squarely on the clients. In such situations the various job-control options are:

  1. {@link #runJob(JobConf)} : submits the job and returns only after the job has completed.
  2. {@link #submitJob(JobConf)} : only submits the job, then poll the returned handle to the {@link RunningJob} to query status and make scheduling decisions.
  3. {@link JobConf#setJobEndNotificationURI(String)} : setup a notification on job-completion, thus avoiding polling.

@see JobConf @see ClusterStatus @see Tool @see DistributedCache @deprecated Use {@link Job} and {@link Cluster} instead]]>
If the parameter {@code loadDefaults} is false, the new instance will not load resources from the default files. @param loadDefaults specifies whether to load from the default files]]> true if framework should keep the intermediate files for failed tasks, false otherwise.]]> true if the outputs of the maps are to be compressed, false otherwise.]]> This comparator should be provided if the equivalence rules for keys for sorting the intermediates are different from those for grouping keys before each call to {@link Reducer#reduce(Object, java.util.Iterator, OutputCollector, Reporter)}.

For key-value pairs (K1,V1) and (K2,V2), the values (V1, V2) are passed in a single call to the reduce function if K1 and K2 compare as equal.

Since {@link #setOutputKeyComparatorClass(Class)} can be used to control how keys are sorted, this can be used in conjunction to simulate secondary sort on values.

Note: This is not a guarantee of the reduce sort being stable in any sense. (In any case, with the order of available map-outputs to the reduce being non-deterministic, it wouldn't make that much sense.)

@param theClass the comparator class to be used for grouping keys. It should implement RawComparator. @see #setOutputKeyComparatorClass(Class)]]>
combiner class used to combine map-outputs before being sent to the reducers. Typically the combiner is same as the the {@link Reducer} for the job i.e. {@link #getReducerClass()}. @return the user-defined combiner class used to combine map-outputs.]]> combiner class used to combine map-outputs before being sent to the reducers.

The combiner is an application-specified aggregation operation, which can help cut down the amount of data transferred between the {@link Mapper} and the {@link Reducer}, leading to better performance.

The framework may invoke the combiner 0, 1, or multiple times, in both the mapper and reducer tasks. In general, the combiner is called as the sort/merge result is written to disk. The combiner must:

  • be side-effect free
  • have the same input and output key types and the same input and output value types

Typically the combiner is same as the Reducer for the job i.e. {@link #setReducerClass(Class)}.

@param theClass the user-defined combiner class used to combine map-outputs.]]>
true. @return true if speculative execution be used for this job, false otherwise.]]> true if speculative execution should be turned on, else false.]]> true. @return true if speculative execution be used for this job for map tasks, false otherwise.]]> true if speculative execution should be turned on for map tasks, else false.]]> true. @return true if speculative execution be used for reduce tasks for this job, false otherwise.]]> true if speculative execution should be turned on for reduce tasks, else false.]]> 1. @return the number of reduce tasks for this job.]]> Note: This is only a hint to the framework. The actual number of spawned map tasks depends on the number of {@link InputSplit}s generated by the job's {@link InputFormat#getSplits(JobConf, int)}. A custom {@link InputFormat} is typically used to accurately control the number of map tasks for the job.

How many maps?

The number of maps is usually driven by the total size of the inputs i.e. total number of blocks of the input files.

The right level of parallelism for maps seems to be around 10-100 maps per-node, although it has been set up to 300 or so for very cpu-light map tasks. Task setup takes awhile, so it is best if the maps take at least a minute to execute.

The default behavior of file-based {@link InputFormat}s is to split the input into logical {@link InputSplit}s based on the total size, in bytes, of input files. However, the {@link FileSystem} 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 mapreduce.input.fileinputformat.split.minsize.

Thus, if you expect 10TB of input data and have a blocksize of 128MB, you'll end up with 82,000 maps, unless {@link #setNumMapTasks(int)} is used to set it even higher.

@param n the number of map tasks for this job. @see InputFormat#getSplits(JobConf, int) @see FileInputFormat @see FileSystem#getDefaultBlockSize() @see FileStatus#getBlockSize()]]>
1. @return the number of reduce tasks for this job.]]> How many reduces?

The right number of reduces seems to be 0.95 or 1.75 multiplied by (<no. of nodes> * mapreduce.tasktracker.reduce.tasks.maximum).

With 0.95 all of the reduces can launch immediately and start transfering map outputs as the maps finish. With 1.75 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.

Increasing the number of reduces increases the framework overhead, but increases load balancing and lowers the cost of failures.

The scaling factors above are slightly less than whole numbers to reserve a few reduce slots in the framework for speculative-tasks, failures etc.

Reducer NONE

It is legal to set the number of reduce-tasks to zero.

In this case the output of the map-tasks directly go to distributed file-system, to the path set by {@link FileOutputFormat#setOutputPath(JobConf, Path)}. Also, the framework doesn't sort the map-outputs before writing it out to HDFS.

@param n the number of reduce tasks for this job.]]>
mapreduce.map.maxattempts property. If this property is not already set, the default is 4 attempts. @return the max number of attempts per map task.]]> mapreduce.reduce.maxattempts property. If this property is not already set, the default is 4 attempts. @return the max number of attempts per reduce task.]]> noFailures, the tasktracker is blacklisted for this job. @param noFailures maximum no. of failures of a given job per tasktracker.]]> blacklisted for this job. @return the maximum no. of failures of a given job per tasktracker.]]> failed. Defaults to zero, i.e. any failed map-task results in the job being declared as {@link JobStatus#FAILED}. @return the maximum percentage of map tasks that can fail without the job being aborted.]]> failed. @param percent the maximum percentage of map tasks that can fail without the job being aborted.]]> failed. Defaults to zero, i.e. any failed reduce-task results in the job being declared as {@link JobStatus#FAILED}. @return the maximum percentage of reduce tasks that can fail without the job being aborted.]]> failed. @param percent the maximum percentage of reduce tasks that can fail without the job being aborted.]]> The debug script can aid debugging of failed map tasks. The script is given task's stdout, stderr, syslog, jobconf files as arguments.

The debug command, run on the node where the map failed, is:

$script $stdout $stderr $syslog $jobconf.

The script file is distributed through {@link DistributedCache} APIs. The script needs to be symlinked.

Here is an example on how to submit a script

 job.setMapDebugScript("./myscript");
 DistributedCache.createSymlink(job);
 DistributedCache.addCacheFile("/debug/scripts/myscript#myscript");
 

@param mDbgScript the script name]]>
The debug script can aid debugging of failed reduce tasks. The script is given task's stdout, stderr, syslog, jobconf files as arguments.

The debug command, run on the node where the map failed, is:

$script $stdout $stderr $syslog $jobconf.

The script file is distributed through {@link DistributedCache} APIs. The script file needs to be symlinked

Here is an example on how to submit a script

 job.setReduceDebugScript("./myscript");
 DistributedCache.createSymlink(job);
 DistributedCache.addCacheFile("/debug/scripts/myscript#myscript");
 

@param rDbgScript the script name]]>
null if it hasn't been set. @see #setJobEndNotificationURI(String)]]> The uri can contain 2 special parameters: $jobId and $jobStatus. Those, if present, are replaced by the job's identifier and completion-status respectively.

This is typically used by application-writers to implement chaining of Map-Reduce jobs in an asynchronous manner.

@param uri the job end notification uri @see JobStatus @see Job Completion and Chaining]]>
When a job starts, a shared directory is created at location ${mapreduce.cluster.local.dir}/taskTracker/$user/jobcache/$jobid/work/ . This directory is exposed to the users through mapreduce.job.local.dir . So, the tasks can use this space as scratch space and share files among them.

This value is available as System property also. @return The localized job specific shared directory]]>
For backward compatibility, if the job configuration sets the key {@link #MAPRED_TASK_MAXVMEM_PROPERTY} to a value different from {@link #DISABLED_MEMORY_LIMIT}, that value will be used after converting it from bytes to MB. @return memory required to run a map task of the job, in MB, or {@link #DISABLED_MEMORY_LIMIT} if unset.]]> For backward compatibility, if the job configuration sets the key {@link #MAPRED_TASK_MAXVMEM_PROPERTY} to a value different from {@link #DISABLED_MEMORY_LIMIT}, that value will be used after converting it from bytes to MB. @return memory required to run a reduce task of the job, in MB, or {@link #DISABLED_MEMORY_LIMIT} if unset.]]> This method is deprecated. Now, different memory limits can be set for map and reduce tasks of a job, in MB.

For backward compatibility, if the job configuration sets the key {@link #MAPRED_TASK_MAXVMEM_PROPERTY} to a value different from {@link #DISABLED_MEMORY_LIMIT}, that value is returned. Otherwise, this method will return the larger of the values returned by {@link #getMemoryForMapTask()} and {@link #getMemoryForReduceTask()} after converting them into bytes. @return Memory required to run a task of this job, in bytes, or {@link #DISABLED_MEMORY_LIMIT}, if unset. @see #setMaxVirtualMemoryForTask(long) @deprecated Use {@link #getMemoryForMapTask()} and {@link #getMemoryForReduceTask()}]]> mapred.task.maxvmem is split into mapreduce.map.memory.mb and mapreduce.map.memory.mb,mapred each of the new key are set as mapred.task.maxvmem / 1024 as new values are in MB @param vmem Maximum amount of virtual memory in bytes any task of this job can use. @see #getMaxVirtualMemoryForTask() @deprecated Use {@link #setMemoryForMapTask(long mem)} and Use {@link #setMemoryForReduceTask(long mem)}]]> k1=v1,k2=v2. Further it can reference existing environment variables via $key. Example:

  • A=foo - This will set the env variable A to foo.
  • B=$X:c This is inherit tasktracker's X env variable.
@deprecated Use {@link #MAPRED_MAP_TASK_ENV} or {@link #MAPRED_REDUCE_TASK_ENV}]]>
k1=v1,k2=v2. Further it can reference existing environment variables via $key. Example:
  • A=foo - This will set the env variable A to foo.
  • B=$X:c This is inherit tasktracker's X env variable.
]]>
k1=v1,k2=v2. Further it can reference existing environment variables via $key. Example:
  • A=foo - This will set the env variable A to foo.
  • B=$X:c This is inherit tasktracker's X env variable.
]]>
JobConf is the primary interface for a user to describe a map-reduce job to the Hadoop framework for execution. The framework tries to faithfully execute the job as-is described by JobConf, however:
  1. Some configuration parameters might have been marked as final by administrators and hence cannot be altered.
  2. While some job parameters are straight-forward to set (e.g. {@link #setNumReduceTasks(int)}), some parameters interact subtly rest of the framework and/or job-configuration and is relatively more complex for the user to control finely (e.g. {@link #setNumMapTasks(int)}).

JobConf typically specifies the {@link Mapper}, combiner (if any), {@link Partitioner}, {@link Reducer}, {@link InputFormat} and {@link OutputFormat} implementations to be used etc.

Optionally JobConf is used to specify other advanced facets of the job such as Comparators to be used, files to be put in the {@link DistributedCache}, whether or not intermediate and/or job outputs are to be compressed (and how), debugability via user-provided scripts ( {@link #setMapDebugScript(String)}/{@link #setReduceDebugScript(String)}), for doing post-processing on task logs, task's stdout, stderr, syslog. and etc.

Here is an example on how to configure a job via JobConf:

     // Create a new JobConf
     JobConf job = new JobConf(new Configuration(), MyJob.class);
     
     // Specify various job-specific parameters     
     job.setJobName("myjob");
     
     FileInputFormat.setInputPaths(job, new Path("in"));
     FileOutputFormat.setOutputPath(job, new Path("out"));
     
     job.setMapperClass(MyJob.MyMapper.class);
     job.setCombinerClass(MyJob.MyReducer.class);
     job.setReducerClass(MyJob.MyReducer.class);
     
     job.setInputFormat(SequenceFileInputFormat.class);
     job.setOutputFormat(SequenceFileOutputFormat.class);
 

@see JobClient @see ClusterStatus @see Tool @see DistributedCache @deprecated Use {@link Configuration} instead]]>
any job run on the jobtracker started at 200707121733, we would use :
 
 JobID.getTaskIDsPattern("200707121733", null);
 
which will return :
 "job_200707121733_[0-9]*" 
@param jtIdentifier jobTracker identifier, or null @param jobId job number, or null @return a regex pattern matching JobIDs]]>
An example JobID is : job_200707121733_0003 , which represents the third job running at the jobtracker started at 200707121733.

Applications should never construct or parse JobID strings, but rather use appropriate constructors or {@link #forName(String)} method. @see TaskID @see TaskAttemptID]]> Output pairs need not 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 {@link OutputCollector#collect(Object,Object)}.

Applications can use the {@link Reporter} provided to report progress or just indicate that they are alive. In scenarios where the application takes an insignificant amount of time to process individual key/value pairs, this is crucial since the framework might assume that the task has timed-out and kill that task. The other way of avoiding this is to set mapreduce.task.timeout to a high-enough value (or even zero for no time-outs).

@param key the input key. @param value the input value. @param output collects mapped keys and values. @param reporter facility to report progress.]]>
Maps are the individual tasks which transform input records into a intermediate records. The transformed intermediate records need not be of the same type as the input records. A given input pair may map to zero or many output pairs.

The Hadoop Map-Reduce framework spawns one map task for each {@link InputSplit} generated by the {@link InputFormat} for the job. Mapper implementations can access the {@link JobConf} for the job via the {@link JobConfigurable#configure(JobConf)} and initialize themselves. Similarly they can use the {@link Closeable#close()} method for de-initialization.

The framework then calls {@link #map(Object, Object, OutputCollector, Reporter)} for each key/value pair in the InputSplit for that task.

All intermediate values associated with a given output key are subsequently grouped by the framework, and passed to a {@link Reducer} to determine the final output. Users can control the grouping by specifying a Comparator via {@link JobConf#setOutputKeyComparatorClass(Class)}.

The grouped Mapper outputs are partitioned per Reducer. Users can control which keys (and hence records) go to which Reducer by implementing a custom {@link Partitioner}.

Users can optionally specify a combiner, via {@link JobConf#setCombinerClass(Class)}, to perform local aggregation of the intermediate outputs, which helps to cut down the amount of data transferred from the Mapper to the Reducer.

The intermediate, grouped outputs are always stored in {@link SequenceFile}s. Applications can specify if and how the intermediate outputs are to be compressed and which {@link CompressionCodec}s are to be used via the JobConf.

If the job has zero reduces then the output of the Mapper is directly written to the {@link FileSystem} without grouping by keys.

Example:

     public class MyMapper<K extends WritableComparable, V extends Writable> 
     extends MapReduceBase implements Mapper<K, V, K, V> {
     
       static enum MyCounters { NUM_RECORDS }
       
       private String mapTaskId;
       private String inputFile;
       private int noRecords = 0;
       
       public void configure(JobConf job) {
         mapTaskId = job.get(JobContext.TASK_ATTEMPT_ID);
         inputFile = job.get(JobContext.MAP_INPUT_FILE);
       }
       
       public void map(K key, V val,
                       OutputCollector<K, V> output, Reporter reporter)
       throws IOException {
         // Process the <key, value> pair (assume this takes a while)
         // ...
         // ...
         
         // Let the framework know that we are alive, and kicking!
         // reporter.progress();
         
         // Process some more
         // ...
         // ...
         
         // Increment the no. of <key, value> pairs processed
         ++noRecords;

         // Increment counters
         reporter.incrCounter(NUM_RECORDS, 1);
        
         // Every 100 records update application-level status
         if ((noRecords%100) == 0) {
           reporter.setStatus(mapTaskId + " processed " + noRecords + 
                              " from input-file: " + inputFile); 
         }
         
         // Output the result
         output.collect(key, val);
       }
     }
 

Applications may write a custom {@link MapRunnable} to exert greater control on map processing e.g. multi-threaded Mappers etc.

@see JobConf @see InputFormat @see Partitioner @see Reducer @see MapReduceBase @see MapRunnable @see SequenceFile @deprecated Use {@link org.apache.hadoop.mapreduce.Mapper} instead.]]>
Provides default no-op implementations for a few methods, most non-trivial applications need to override some of them.

]]>
<key, value> pairs.

Mapping of input records to output records is complete when this method returns.

@param input the {@link RecordReader} to read the input records. @param output the {@link OutputCollector} to collect the outputrecords. @param reporter {@link Reporter} to report progress, status-updates etc. @throws IOException]]>
Custom implementations of MapRunnable can exert greater control on map processing e.g. multi-threaded, asynchronous mappers etc.

@see Mapper @deprecated Use {@link org.apache.hadoop.mapreduce.Mapper} instead.]]>
nearly equal content length.
Subclasses implement {@link #getRecordReader(InputSplit, JobConf, Reporter)} to construct RecordReader's for MultiFileSplit's. @see MultiFileSplit @deprecated Use {@link org.apache.hadoop.mapred.lib.CombineFileInputFormat} instead]]>
MultiFileSplit can be used to implement {@link RecordReader}'s, with reading one record per file. @see FileSplit @see MultiFileInputFormat @deprecated Use {@link org.apache.hadoop.mapred.lib.CombineFileSplit} instead]]> <key, value> pairs output by {@link Mapper}s and {@link Reducer}s.

OutputCollector is the generalization of the facility provided by the Map-Reduce framework to collect data output by either the Mapper or the Reducer i.e. intermediate outputs or the output of the job.

]]>
OutputCommitter describes the commit of task output for a Map-Reduce job.

The Map-Reduce framework relies on the OutputCommitter of the job to:

  1. Setup the job during initialization. For example, create the temporary output directory for the job during the initialization of the job.
  2. Cleanup the job after the job completion. For example, remove the temporary output directory after the job completion.
  3. Setup the task temporary output.
  4. Check whether a task needs a commit. This is to avoid the commit procedure if a task does not need commit.
  5. Commit of the task output.
  6. Discard the task commit.
@see FileOutputCommitter @see JobContext @see TaskAttemptContext @deprecated Use {@link org.apache.hadoop.mapreduce.OutputCommitter} instead.]]>
This is to validate the output specification for the job when it is a job is submitted. Typically checks that it does not already exist, throwing an exception when it already exists, so that output is not overwritten.

@param ignored @param job job configuration. @throws IOException when output should not be attempted]]>
OutputFormat describes the output-specification for a Map-Reduce job.

The Map-Reduce framework relies on the OutputFormat of the job to:

  1. Validate the output-specification of the job. For e.g. check that the output directory doesn't already exist.
  2. Provide the {@link RecordWriter} implementation to be used to write out the output files of the job. Output files are stored in a {@link FileSystem}.
@see RecordWriter @see JobConf @deprecated Use {@link org.apache.hadoop.mapreduce.OutputFormat} instead.]]>
Typically a hash function on a all or a subset of the key.

@param key the key to be paritioned. @param value the entry value. @param numPartitions the total number of partitions. @return the partition number for the key.]]>
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 hash function. The total number of partitions is the same as the number of reduce tasks for the job. Hence this controls which of the m reduce tasks the intermediate key (and hence the record) is sent for reduction.

@see Reducer @deprecated Use {@link org.apache.hadoop.mapreduce.Partitioner} instead.]]>
0.0 to 1.0. @throws IOException]]> RecordReader reads <key, value> pairs from an {@link InputSplit}.

RecordReader, typically, converts the byte-oriented view of the input, provided by the InputSplit, and presents a record-oriented view for the {@link Mapper} & {@link Reducer} tasks for processing. It thus assumes the responsibility of processing record boundaries and presenting the tasks with keys and values.

@see InputSplit @see InputFormat]]>
RecordWriter to future operations. @param reporter facility to report progress. @throws IOException]]> RecordWriter writes the output <key, value> pairs to an output file.

RecordWriter implementations write the job outputs to the {@link FileSystem}. @see OutputFormat]]> Reduces values for a given key.

The framework calls this method for each <key, (list of values)> pair in the grouped inputs. Output values must be of the same type as input values. Input keys must not be altered. The framework will reuse the key and value objects that are passed into the reduce, therefore the application should clone the objects they want to keep a copy of. In many cases, all values are combined into zero or one value.

Output pairs are collected with calls to {@link OutputCollector#collect(Object,Object)}.

Applications can use the {@link Reporter} provided to report progress or just indicate that they are alive. In scenarios where the application takes an insignificant amount of time to process individual key/value pairs, this is crucial since the framework might assume that the task has timed-out and kill that task. The other way of avoiding this is to set mapreduce.task.timeout to a high-enough value (or even zero for no time-outs).

@param key the key. @param values the list of values to reduce. @param output to collect keys and combined values. @param reporter facility to report progress.]]>
The number of Reducers for the job is set by the user via {@link JobConf#setNumReduceTasks(int)}. Reducer implementations can access the {@link JobConf} for the job via the {@link JobConfigurable#configure(JobConf)} method and initialize themselves. Similarly they can use the {@link Closeable#close()} method for de-initialization.

Reducer has 3 primary phases:

  1. Shuffle

    Reducer is input the grouped output of a {@link Mapper}. In the phase the framework, for each Reducer, fetches the relevant partition of the output of all the Mappers, via HTTP.

  2. Sort

    The framework groups Reducer inputs by keys (since different Mappers may have output the same key) in this stage.

    The shuffle and sort phases occur simultaneously i.e. while outputs are being fetched they are merged.

    SecondarySort

    If equivalence rules for keys while grouping the intermediates are different from those for grouping keys before reduction, then one may specify a Comparator via {@link JobConf#setOutputValueGroupingComparator(Class)}.Since {@link JobConf#setOutputKeyComparatorClass(Class)} can be used to control how intermediate keys are grouped, these can be used in conjunction to simulate secondary sort on values.

    For example, say that you want to find duplicate web pages and tag them all with the url of the "best" known example. You would set up the job like:
    • Map Input Key: url
    • Map Input Value: document
    • Map Output Key: document checksum, url pagerank
    • Map Output Value: url
    • Partitioner: by checksum
    • OutputKeyComparator: by checksum and then decreasing pagerank
    • OutputValueGroupingComparator: by checksum
  3. Reduce

    In this phase the {@link #reduce(Object, Iterator, OutputCollector, Reporter)} method is called for each <key, (list of values)> pair in the grouped inputs.

    The output of the reduce task is typically written to the {@link FileSystem} via {@link OutputCollector#collect(Object, Object)}.

The output of the Reducer is not re-sorted.

Example:

     public class MyReducer<K extends WritableComparable, V extends Writable> 
     extends MapReduceBase implements Reducer<K, V, K, V> {
     
       static enum MyCounters { NUM_RECORDS }
        
       private String reduceTaskId;
       private int noKeys = 0;
       
       public void configure(JobConf job) {
         reduceTaskId = job.get(JobContext.TASK_ATTEMPT_ID);
       }
       
       public void reduce(K key, Iterator<V> values,
                          OutputCollector<K, V> output, 
                          Reporter reporter)
       throws IOException {
       
         // Process
         int noValues = 0;
         while (values.hasNext()) {
           V value = values.next();
           
           // Increment the no. of values for this key
           ++noValues;
           
           // Process the <key, value> pair (assume this takes a while)
           // ...
           // ...
           
           // Let the framework know that we are alive, and kicking!
           if ((noValues%10) == 0) {
             reporter.progress();
           }
         
           // Process some more
           // ...
           // ...
           
           // Output the <key, value> 
           output.collect(key, value);
         }
         
         // Increment the no. of <key, list of values> pairs processed
         ++noKeys;
         
         // Increment counters
         reporter.incrCounter(NUM_RECORDS, 1);
         
         // Every 100 keys update application-level status
         if ((noKeys%100) == 0) {
           reporter.setStatus(reduceTaskId + " processed " + noKeys);
         }
       }
     }
 

@see Mapper @see Partitioner @see Reporter @see MapReduceBase @deprecated Use {@link org.apache.hadoop.mapreduce.Reducer} instead.]]>
Counter of the given group/name.]]> Counter of the given group/name.]]> Enum. @param amount A non-negative amount by which the counter is to be incremented.]]> InputSplit that the map is reading from. @throws UnsupportedOperationException if called outside a mapper]]> {@link Mapper} and {@link Reducer} can use the Reporter provided to report progress or just indicate that they are alive. In scenarios where the application takes an insignificant amount of time to process individual key/value pairs, this is crucial since the framework might assume that the task has timed-out and kill that task.

Applications can also update {@link Counters} via the provided Reporter .

@see Progressable @see Counters]]>
progress of the job's map-tasks, as a float between 0.0 and 1.0. When all map tasks have completed, the function returns 1.0. @return the progress of the job's map-tasks. @throws IOException]]> progress of the job's reduce-tasks, as a float between 0.0 and 1.0. When all reduce tasks have completed, the function returns 1.0. @return the progress of the job's reduce-tasks. @throws IOException]]> progress of the job's cleanup-tasks, as a float between 0.0 and 1.0. When all cleanup tasks have completed, the function returns 1.0. @return the progress of the job's cleanup-tasks. @throws IOException]]> progress of the job's setup-tasks, as a float between 0.0 and 1.0. When all setup tasks have completed, the function returns 1.0. @return the progress of the job's setup-tasks. @throws IOException]]> true if the job is complete, else false. @throws IOException]]> true if the job succeeded, else false. @throws IOException]]> true if the job retired, else false. @throws IOException]]> RunningJob is the user-interface to query for details on a running Map-Reduce job.

Clients can get hold of RunningJob via the {@link JobClient} and then query the running-job for details such as name, configuration, progress etc.

@see JobClient @deprecated Use {@link org.apache.hadoop.mapreduce.Job} instead]]>
This allows the user to specify the key class to be different from the actual class ({@link BytesWritable}) used for writing

@param conf the {@link JobConf} to modify @param theClass the SequenceFile output key class.]]>
This allows the user to specify the value class to be different from the actual class ({@link BytesWritable}) used for writing

@param conf the {@link JobConf} to modify @param theClass the SequenceFile output key class.]]>
f. The filtering criteria is MD5(key) % f == 0.]]> f using the criteria record# % f == 0. For example, if the frequency is 10, one out of 10 records is returned.]]> true if auto increment {@link SkipBadRecords#COUNTER_MAP_PROCESSED_RECORDS}. false otherwise.]]> true if auto increment {@link SkipBadRecords#COUNTER_REDUCE_PROCESSED_GROUPS}. false otherwise.]]> Hadoop provides an optional mode of execution in which the bad records are detected and skipped in further attempts.

This feature can be used when map/reduce tasks crashes deterministically on certain input. This happens due to bugs in the map/reduce function. The usual course would be to fix these bugs. But sometimes this is not possible; perhaps the bug is in third party libraries for which the source code is not available. Due to this, the task never reaches to completion even with multiple attempts and complete data for that task is lost.

With this feature, only a small portion of data is lost surrounding the bad record, which may be acceptable for some user applications. see {@link SkipBadRecords#setMapperMaxSkipRecords(Configuration, long)}

The skipping mode gets kicked off after certain no of failures see {@link SkipBadRecords#setAttemptsToStartSkipping(Configuration, int)}

In the skipping mode, the map/reduce task maintains the record range which is getting processed at all times. Before giving the input to the map/reduce function, it sends this record range to the Task tracker. If task crashes, the Task tracker knows which one was the last reported range. On further attempts that range get skipped.

]]>
all task attempt IDs of any jobtracker, in any job, of the first map task, we would use :
 
 TaskAttemptID.getTaskAttemptIDsPattern(null, null, true, 1, null);
 
which will return :
 "attempt_[^_]*_[0-9]*_m_000001_[0-9]*" 
@param jtIdentifier jobTracker identifier, or null @param jobId job number, or null @param isMap whether the tip is a map, or null @param taskId taskId number, or null @param attemptId the task attempt number, or null @return a regex pattern matching TaskAttemptIDs]]>
all task attempt IDs of any jobtracker, in any job, of the first map task, we would use :
 
 TaskAttemptID.getTaskAttemptIDsPattern(null, null, TaskType.MAP, 1, null);
 
which will return :
 "attempt_[^_]*_[0-9]*_m_000001_[0-9]*" 
@param jtIdentifier jobTracker identifier, or null @param jobId job number, or null @param type the {@link TaskType} @param taskId taskId number, or null @param attemptId the task attempt number, or null @return a regex pattern matching TaskAttemptIDs]]>
An example TaskAttemptID is : attempt_200707121733_0003_m_000005_0 , which represents the zeroth task attempt for the fifth map task in the third job running at the jobtracker started at 200707121733.

Applications should never construct or parse TaskAttemptID strings , but rather use appropriate constructors or {@link #forName(String)} method. @see JobID @see TaskID]]> the first map task of any jobtracker, of any job, we would use :

 
 TaskID.getTaskIDsPattern(null, null, true, 1);
 
which will return :
 "task_[^_]*_[0-9]*_m_000001*" 
@param jtIdentifier jobTracker identifier, or null @param jobId job number, or null @param isMap whether the tip is a map, or null @param taskId taskId number, or null @return a regex pattern matching TaskIDs @deprecated Use {@link TaskID#getTaskIDsPattern(String, Integer, TaskType, Integer)}]]>
the first map task of any jobtracker, of any job, we would use :
 
 TaskID.getTaskIDsPattern(null, null, true, 1);
 
which will return :
 "task_[^_]*_[0-9]*_m_000001*" 
@param jtIdentifier jobTracker identifier, or null @param jobId job number, or null @param type the {@link TaskType}, or null @param taskId taskId number, or null @return a regex pattern matching TaskIDs]]>
An example TaskID is : task_200707121733_0003_m_000005 , which represents the fifth map task in the third job running at the jobtracker started at 200707121733.

Applications should never construct or parse TaskID strings , but rather use appropriate constructors or {@link #forName(String)} method. @see JobID @see TaskAttemptID]]> ([,]*) func ::= tbl(,"") class ::= @see java.lang.Class#forName(java.lang.String) path ::= @see org.apache.hadoop.fs.Path#Path(java.lang.String) } Reads expression from the mapred.join.expr property and user-supplied join types from mapred.join.define.<ident> types. Paths supplied to tbl are given as input paths to the InputFormat class listed. @see #compose(java.lang.String, java.lang.Class, java.lang.String...)]]> ,

) }]]> (tbl(,),tbl(,),...,tbl(,)) }]]> (tbl(,),tbl(,),...,tbl(,)) }]]> mapred.join.define.<ident> to a classname. In the expression mapred.join.expr, the identifier will be assumed to be a ComposableRecordReader. mapred.join.keycomparator can be a classname used to compare keys in the join. @see JoinRecordReader @see MultiFilterRecordReader @deprecated Use {@link org.apache.hadoop.mapreduce.lib.join.CompositeInputFormat} instead]]> ...... }]]> capacity children to position id in the parent reader. The id of a root CompositeRecordReader is -1 by convention, but relying on this is not recommended.]]> override(S1,S2,S3) will prefer values from S3 over S2, and values from S2 over S1 for all keys emitted from all sources. @deprecated Use {@link org.apache.hadoop.mapreduce.lib.join.OverrideRecordReader} instead]]> It has to be specified how key and values are passed from one element of the chain to the next, by value or by reference. If a Mapper leverages the assumed semantics that the key and values are not modified by the collector 'by value' must be used. If the Mapper does not expect this semantics, as an optimization to avoid serialization and deserialization 'by reference' can be used.

For the added Mapper the configuration given for it, mapperConf, have precedence over the job's JobConf. This precedence is in effect when the task is running.

IMPORTANT: There is no need to specify the output key/value classes for the ChainMapper, this is done by the addMapper for the last mapper in the chain

@param job job's JobConf to add the Mapper class. @param klass the Mapper class to add. @param inputKeyClass mapper input key class. @param inputValueClass mapper input value class. @param outputKeyClass mapper output key class. @param outputValueClass mapper output value class. @param byValue indicates if key/values should be passed by value to the next Mapper in the chain, if any. @param mapperConf a JobConf with the configuration for the Mapper class. It is recommended to use a JobConf without default values using the JobConf(boolean loadDefaults) constructor with FALSE.]]> If this method is overriden super.configure(...) should be invoked at the beginning of the overwriter method.]]> map(...) methods of the Mappers in the chain.]]> If this method is overriden super.close() should be invoked at the end of the overwriter method.]]> The Mapper classes are invoked in a chained (or piped) fashion, the output of the first becomes the input of the second, and so on until the last Mapper, the output of the last Mapper will be written to the task's output.

The key functionality of this feature is that the Mappers in the chain do not need to be aware that they are executed in a chain. This enables having reusable specialized Mappers that can be combined to perform composite operations within a single task.

Special care has to be taken when creating chains that the key/values output by a Mapper are valid for the following Mapper in the chain. It is assumed all Mappers and the Reduce in the chain use maching output and input key and value classes as no conversion is done by the chaining code.

Using the ChainMapper and the ChainReducer classes is possible to compose Map/Reduce jobs that look like [MAP+ / REDUCE MAP*]. And immediate benefit of this pattern is a dramatic reduction in disk IO.

IMPORTANT: There is no need to specify the output key/value classes for the ChainMapper, this is done by the addMapper for the last mapper in the chain.

ChainMapper usage pattern:

 ...
 conf.setJobName("chain");
 conf.setInputFormat(TextInputFormat.class);
 conf.setOutputFormat(TextOutputFormat.class);
 

JobConf mapAConf = new JobConf(false); ... ChainMapper.addMapper(conf, AMap.class, LongWritable.class, Text.class, Text.class, Text.class, true, mapAConf);

JobConf mapBConf = new JobConf(false); ... ChainMapper.addMapper(conf, BMap.class, Text.class, Text.class, LongWritable.class, Text.class, false, mapBConf);

JobConf reduceConf = new JobConf(false); ... ChainReducer.setReducer(conf, XReduce.class, LongWritable.class, Text.class, Text.class, Text.class, true, reduceConf);

ChainReducer.addMapper(conf, CMap.class, Text.class, Text.class, LongWritable.class, Text.class, false, null);

ChainReducer.addMapper(conf, DMap.class, LongWritable.class, Text.class, LongWritable.class, LongWritable.class, true, null);

FileInputFormat.setInputPaths(conf, inDir); FileOutputFormat.setOutputPath(conf, outDir); ...

JobClient jc = new JobClient(conf); RunningJob job = jc.submitJob(conf); ...

@deprecated Use {@link org.apache.hadoop.mapreduce.lib.chain.ChainMapper} instead]]>
It has to be specified how key and values are passed from one element of the chain to the next, by value or by reference. If a Reducer leverages the assumed semantics that the key and values are not modified by the collector 'by value' must be used. If the Reducer does not expect this semantics, as an optimization to avoid serialization and deserialization 'by reference' can be used.

For the added Reducer the configuration given for it, reducerConf, have precedence over the job's JobConf. This precedence is in effect when the task is running.

IMPORTANT: There is no need to specify the output key/value classes for the ChainReducer, this is done by the setReducer or the addMapper for the last element in the chain. @param job job's JobConf to add the Reducer class. @param klass the Reducer class to add. @param inputKeyClass reducer input key class. @param inputValueClass reducer input value class. @param outputKeyClass reducer output key class. @param outputValueClass reducer output value class. @param byValue indicates if key/values should be passed by value to the next Mapper in the chain, if any. @param reducerConf a JobConf with the configuration for the Reducer class. It is recommended to use a JobConf without default values using the JobConf(boolean loadDefaults) constructor with FALSE.]]> It has to be specified how key and values are passed from one element of the chain to the next, by value or by reference. If a Mapper leverages the assumed semantics that the key and values are not modified by the collector 'by value' must be used. If the Mapper does not expect this semantics, as an optimization to avoid serialization and deserialization 'by reference' can be used.

For the added Mapper the configuration given for it, mapperConf, have precedence over the job's JobConf. This precedence is in effect when the task is running.

IMPORTANT: There is no need to specify the output key/value classes for the ChainMapper, this is done by the addMapper for the last mapper in the chain . @param job chain job's JobConf to add the Mapper class. @param klass the Mapper class to add. @param inputKeyClass mapper input key class. @param inputValueClass mapper input value class. @param outputKeyClass mapper output key class. @param outputValueClass mapper output value class. @param byValue indicates if key/values should be passed by value to the next Mapper in the chain, if any. @param mapperConf a JobConf with the configuration for the Mapper class. It is recommended to use a JobConf without default values using the JobConf(boolean loadDefaults) constructor with FALSE.]]> If this method is overriden super.configure(...) should be invoked at the beginning of the overwriter method.]]> reduce(...) method of the Reducer with the map(...) methods of the Mappers in the chain.]]> If this method is overriden super.close() should be invoked at the end of the overwriter method.]]> For each record output by the Reducer, the Mapper classes are invoked in a chained (or piped) fashion, the output of the first becomes the input of the second, and so on until the last Mapper, the output of the last Mapper will be written to the task's output.

The key functionality of this feature is that the Mappers in the chain do not need to be aware that they are executed after the Reducer or in a chain. This enables having reusable specialized Mappers that can be combined to perform composite operations within a single task.

Special care has to be taken when creating chains that the key/values output by a Mapper are valid for the following Mapper in the chain. It is assumed all Mappers and the Reduce in the chain use maching output and input key and value classes as no conversion is done by the chaining code.

Using the ChainMapper and the ChainReducer classes is possible to compose Map/Reduce jobs that look like [MAP+ / REDUCE MAP*]. And immediate benefit of this pattern is a dramatic reduction in disk IO.

IMPORTANT: There is no need to specify the output key/value classes for the ChainReducer, this is done by the setReducer or the addMapper for the last element in the chain.

ChainReducer usage pattern:

 ...
 conf.setJobName("chain");
 conf.setInputFormat(TextInputFormat.class);
 conf.setOutputFormat(TextOutputFormat.class);
 

JobConf mapAConf = new JobConf(false); ... ChainMapper.addMapper(conf, AMap.class, LongWritable.class, Text.class, Text.class, Text.class, true, mapAConf);

JobConf mapBConf = new JobConf(false); ... ChainMapper.addMapper(conf, BMap.class, Text.class, Text.class, LongWritable.class, Text.class, false, mapBConf);

JobConf reduceConf = new JobConf(false); ... ChainReducer.setReducer(conf, XReduce.class, LongWritable.class, Text.class, Text.class, Text.class, true, reduceConf);

ChainReducer.addMapper(conf, CMap.class, Text.class, Text.class, LongWritable.class, Text.class, false, null);

ChainReducer.addMapper(conf, DMap.class, LongWritable.class, Text.class, LongWritable.class, LongWritable.class, true, null);

FileInputFormat.setInputPaths(conf, inDir); FileOutputFormat.setOutputPath(conf, outDir); ...

JobClient jc = new JobClient(conf); RunningJob job = jc.submitJob(conf); ...

@deprecated Use {@link org.apache.hadoop.mapreduce.lib.chain.ChainReducer} instead]]>
RecordReader's for CombineFileSplit's. @see CombineFileSplit @deprecated Use {@link org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat}]]> FilterRecordWriter is a convenience wrapper class that implements {@link RecordWriter}.]]> true if the name output is multi, false if it is single. If the name output is not defined it returns false]]> @param conf job conf to add the named output @param namedOutput named output name, it has to be a word, letters and numbers only, cannot be the word 'part' as that is reserved for the default output. @param outputFormatClass OutputFormat class. @param keyClass key class @param valueClass value class]]> @param conf job conf to add the named output @param namedOutput named output name, it has to be a word, letters and numbers only, cannot be the word 'part' as that is reserved for the default output. @param outputFormatClass OutputFormat class. @param keyClass key class @param valueClass value class]]> By default these counters are disabled.

MultipleOutputs supports counters, by default the are disabled. The counters group is the {@link MultipleOutputs} class name.

The names of the counters are the same as the named outputs. For multi named outputs the name of the counter is the concatenation of the named output, and underscore '_' and the multiname. @param conf job conf to enableadd the named output. @param enabled indicates if the counters will be enabled or not.]]>
By default these counters are disabled.

MultipleOutputs supports counters, by default the are disabled. The counters group is the {@link MultipleOutputs} class name.

The names of the counters are the same as the named outputs. For multi named outputs the name of the counter is the concatenation of the named output, and underscore '_' and the multiname. @param conf job conf to enableadd the named output. @return TRUE if the counters are enabled, FALSE if they are disabled.]]>
@param namedOutput the named output name @param reporter the reporter @return the output collector for the given named output @throws IOException thrown if output collector could not be created]]> @param namedOutput the named output name @param multiName the multi name part @param reporter the reporter @return the output collector for the given named output @throws IOException thrown if output collector could not be created]]> If overriden subclasses must invoke super.close() at the end of their close() @throws java.io.IOException thrown if any of the MultipleOutput files could not be closed properly.]]> OutputCollector passed to the map() and reduce() methods of the Mapper and Reducer implementations.

Each additional output, or named output, may be configured with its own OutputFormat, with its own key class and with its own value class.

A named output can be a single file or a multi file. The later is refered as a multi named output.

A multi named output is an unbound set of files all sharing the same OutputFormat, key class and value class configuration.

When named outputs are used within a Mapper implementation, key/values written to a name output are not part of the reduce phase, only key/values written to the job OutputCollector are part of the reduce phase.

MultipleOutputs supports counters, by default the are disabled. The counters group is the {@link MultipleOutputs} class name.

The names of the counters are the same as the named outputs. For multi named outputs the name of the counter is the concatenation of the named output, and underscore '_' and the multiname.

Job configuration usage pattern is:


 JobConf conf = new JobConf();

 conf.setInputPath(inDir);
 FileOutputFormat.setOutputPath(conf, outDir);

 conf.setMapperClass(MOMap.class);
 conf.setReducerClass(MOReduce.class);
 ...

 // Defines additional single text based output 'text' for the job
 MultipleOutputs.addNamedOutput(conf, "text", TextOutputFormat.class,
 LongWritable.class, Text.class);

 // Defines additional multi sequencefile based output 'sequence' for the
 // job
 MultipleOutputs.addMultiNamedOutput(conf, "seq",
   SequenceFileOutputFormat.class,
   LongWritable.class, Text.class);
 ...

 JobClient jc = new JobClient();
 RunningJob job = jc.submitJob(conf);

 ...
 

Job configuration usage pattern is:


 public class MOReduce implements
   Reducer<WritableComparable, Writable> {
 private MultipleOutputs mos;

 public void configure(JobConf conf) {
 ...
 mos = new MultipleOutputs(conf);
 }

 public void reduce(WritableComparable key, Iterator<Writable> values,
 OutputCollector output, Reporter reporter)
 throws IOException {
 ...
 mos.getCollector("text", reporter).collect(key, new Text("Hello"));
 mos.getCollector("seq", "A", reporter).collect(key, new Text("Bye"));
 mos.getCollector("seq", "B", reporter).collect(key, new Text("Chau"));
 ...
 }

 public void close() throws IOException {
 mos.close();
 ...
 }

 }
 
@deprecated Use {@link org.apache.hadoop.mapreduce.lib.output.MultipleOutputs} instead]]>
It can be used instead of the default implementation, @link org.apache.hadoop.mapred.MapRunner, when the Map operation is not CPU bound in order to improve throughput.

Map implementations using this MapRunnable must be thread-safe.

The Map-Reduce job has to be configured to use this MapRunnable class (using the JobConf.setMapRunnerClass method) and the number of thread the thread-pool can use with the mapred.map.multithreadedrunner.threads property, its default value is 10 threads.

@deprecated Use {@link MultithreadedMapper} instead.]]> pairs. Uses {@link StringTokenizer} to break text into tokens. @deprecated Use {@link org.apache.hadoop.mapreduce.lib.map.TokenCounterMapper} instead.]]> generateKeyValPairs(Object key, Object value); public void configure(JobConfjob); } The package also provides a base class, ValueAggregatorBaseDescriptor, implementing the above interface. The user can extend the base class and implement generateKeyValPairs accordingly. The primary work of generateKeyValPairs is to emit one or more key/value pairs based on the input key/value pair. The key in an output key/value pair encode two pieces of information: aggregation type and aggregation id. The value will be aggregated onto the aggregation id according the aggregation type. This class offers a function to generate a map/reduce job using Aggregate framework. The function takes the following parameters: input directory spec input format (text or sequence file) output directory a file specifying the user plugin class @deprecated Use {@link org.apache.hadoop.mapreduce.lib.aggregate.ValueAggregatorJob} instead]]> 20070101 AND length > 0)' @param orderBy the fieldNames in the orderBy clause. @param fieldNames The field names in the table @see #setInput(JobConf, Class, String, String)]]> Cluster.]]> ClusterMetrics provides clients with information such as:

  1. Size of the cluster.
  2. Number of blacklisted and decommissioned trackers.
  3. Slot capacity of the cluster.
  4. The number of currently occupied/reserved map & reduce slots.
  5. The number of currently running map & reduce tasks.
  6. The number of job submissions.

Clients can query for the latest ClusterMetrics, via {@link Cluster#getClusterStatus()}.

@see Cluster]]>
input key type @param input value type @param output key type @param output value type @param context the context to clone @param conf a new configuration @param reader Reader to read from. Null means to clone from context. @param writer Writer to write to. Null means to clone from context. @return a new context. it will not be the same class as the original. @throws IOException @throws InterruptedException]]> Counters represent global counters, defined either by the Map-Reduce framework or applications. Each Counter is named by an {@link Enum} and has a long for the value.

Counters are bunched into Groups, each comprising of counters from a particular Enum class.]]> Each {@link InputSplit} is then assigned to an individual {@link Mapper} for processing.

Note: The split is a logical split of the inputs and the input files are not physically split into chunks. For e.g. a split could be <input-file-path, start, offset> tuple. The InputFormat also creates the {@link RecordReader} to read the {@link InputSplit}. @param context job configuration. @return an array of {@link InputSplit}s for the job.]]> InputFormat describes the input-specification for a Map-Reduce job.

The Map-Reduce framework relies on the InputFormat of the job to:

  1. Validate the input-specification of the job.
  2. Split-up the input file(s) into logical {@link InputSplit}s, each of which is then assigned to an individual {@link Mapper}.
  3. Provide the {@link RecordReader} implementation to be used to glean input records from the logical InputSplit for processing by the {@link Mapper}.

The default behavior of file-based {@link InputFormat}s, typically sub-classes of {@link FileInputFormat}, is to split the input into logical {@link InputSplit}s based on the total size, in bytes, of the input files. However, the {@link FileSystem} 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 mapreduce.input.fileinputformat.split.minsize.

Clearly, logical splits based on input-size is insufficient for many applications since record boundaries are to respected. In such cases, the application has to also implement a {@link RecordReader} on whom lies the responsibility to respect record-boundaries and present a record-oriented view of the logical InputSplit to the individual task. @see InputSplit @see RecordReader @see FileInputFormat]]> InputSplit represents the data to be processed by an individual {@link Mapper}.

Typically, it presents a byte-oriented view on the input and is the responsibility of {@link RecordReader} of the job to process this and present a record-oriented view. @see InputFormat @see RecordReader]]> progress of the job's map-tasks, as a float between 0.0 and 1.0. When all map tasks have completed, the function returns 1.0. @return the progress of the job's map-tasks. @throws IOException]]> progress of the job's reduce-tasks, as a float between 0.0 and 1.0. When all reduce tasks have completed, the function returns 1.0. @return the progress of the job's reduce-tasks. @throws IOException]]> progress of the job's cleanup-tasks, as a float between 0.0 and 1.0. When all cleanup tasks have completed, the function returns 1.0. @return the progress of the job's cleanup-tasks. @throws IOException]]> progress of the job's setup-tasks, as a float between 0.0 and 1.0. When all setup tasks have completed, the function returns 1.0. @return the progress of the job's setup-tasks. @throws IOException]]> true if the job is complete, else false. @throws IOException]]> true if the job succeeded, else false. @throws IOException]]> InputFormat to use @throws IllegalStateException if the job is submitted]]> OutputFormat to use @throws IllegalStateException if the job is submitted]]> Mapper to use @throws IllegalStateException if the job is submitted]]> Reducer to use @throws IllegalStateException if the job is submitted]]> Partitioner to use @throws IllegalStateException if the job is submitted]]> true if speculative execution should be turned on, else false.]]> true if speculative execution should be turned on for map tasks, else false.]]> true if speculative execution should be turned on for reduce tasks, else false.]]> true, job-setup and job-cleanup will be considered from {@link OutputCommitter} else ignored.]]> JobTracker is lost]]> It allows the user to configure the job, submit it, control its execution, and query the state. The set methods only work until the job is submitted, afterwards they will throw an IllegalStateException.

Normally the user creates the application, describes various facets of the job via {@link Job} and then submits the job and monitor its progress.

Here is an example on how to submit a job:

     // Create a new Job
     Job job = new Job(new Configuration());
     job.setJarByClass(MyJob.class);
     
     // Specify various job-specific parameters     
     job.setJobName("myjob");
     
     job.setInputPath(new Path("in"));
     job.setOutputPath(new Path("out"));
     
     job.setMapperClass(MyJob.MyMapper.class);
     job.setReducerClass(MyJob.MyReducer.class);

     // Submit the job, then poll for progress until the job is complete
     job.waitForCompletion(true);
 

]]>
1. @return the number of reduce tasks for this job.]]> mapred.map.max.attempts property. If this property is not already set, the default is 4 attempts. @return the max number of attempts per map task.]]> mapred.reduce.max.attempts property. If this property is not already set, the default is 4 attempts. @return the max number of attempts per reduce task.]]> An example JobID is : job_200707121733_0003 , which represents the third job running at the jobtracker started at 200707121733.

Applications should never construct or parse JobID strings, but rather use appropriate constructors or {@link #forName(String)} method. @see TaskID @see TaskAttemptID @see org.apache.hadoop.mapred.JobTracker#getNewJobId() @see org.apache.hadoop.mapred.JobTracker#getStartTime()]]> the key input type to the Mapper @param the value input type to the Mapper @param the key output type from the Mapper @param the value output type from the Mapper]]> Maps are the individual tasks which transform input records into a intermediate records. The transformed intermediate records need not be of the same type as the input records. A given input pair may map to zero or many output pairs.

The Hadoop Map-Reduce framework spawns one map task for each {@link InputSplit} generated by the {@link InputFormat} for the job. Mapper implementations can access the {@link Configuration} for the job via the {@link JobContext#getConfiguration()}.

The framework first calls {@link #setup(org.apache.hadoop.mapreduce.Mapper.Context)}, followed by {@link #map(Object, Object, Context)} for each key/value pair in the InputSplit. Finally {@link #cleanup(Context)} is called.

All intermediate values associated with a given output key are subsequently grouped by the framework, and passed to a {@link Reducer} to determine the final output. Users can control the sorting and grouping by specifying two key {@link RawComparator} classes.

The Mapper outputs are partitioned per Reducer. Users can control which keys (and hence records) go to which Reducer by implementing a custom {@link Partitioner}.

Users can optionally specify a combiner, via {@link Job#setCombinerClass(Class)}, to perform local aggregation of the intermediate outputs, which helps to cut down the amount of data transferred from the Mapper to the Reducer.

Applications can specify if and how the intermediate outputs are to be compressed and which {@link CompressionCodec}s are to be used via the Configuration.

If the job has zero reduces then the output of the Mapper is directly written to the {@link OutputFormat} without sorting by keys.

Example:

 public class TokenCounterMapper 
     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);
     }
   }
 }
 

Applications may override the {@link #run(Context)} method to exert greater control on map processing e.g. multi-threaded Mappers etc.

@see InputFormat @see JobContext @see Partitioner @see Reducer]]>
Context passed on to the {@link Mapper} implementations.]]> MarkableIterator is a wrapper iterator class that implements the {@link MarkableIteratorInterface}.]]> OutputCommitter describes the commit of task output for a Map-Reduce job.

The Map-Reduce framework relies on the OutputCommitter of the job to:

  1. Setup the job during initialization. For example, create the temporary output directory for the job during the initialization of the job.
  2. Cleanup the job after the job completion. For example, remove the temporary output directory after the job completion.
  3. Setup the task temporary output.
  4. Check whether a task needs a commit. This is to avoid the commit procedure if a task does not need commit.
  5. Commit of the task output.
  6. Discard the task commit.
@see org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter @see JobContext @see TaskAttemptContext]]>
This is to validate the output specification for the job when it is a job is submitted. Typically checks that it does not already exist, throwing an exception when it already exists, so that output is not overwritten.

@param context information about the job @throws IOException when output should not be attempted]]>
OutputFormat describes the output-specification for a Map-Reduce job.

The Map-Reduce framework relies on the OutputFormat of the job to:

  1. Validate the output-specification of the job. For e.g. check that the output directory doesn't already exist.
  2. Provide the {@link RecordWriter} implementation to be used to write out the output files of the job. Output files are stored in a {@link FileSystem}.
@see RecordWriter]]>
Typically a hash function on a all or a subset of the key.

@param key the key to be partioned. @param value the entry value. @param numPartitions the total number of partitions. @return the partition number for the key.]]>
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 hash function. The total number of partitions is the same as the number of reduce tasks for the job. Hence this controls which of the m reduce tasks the intermediate key (and hence the record) is sent for reduction.

@see Reducer]]>
"N/A" @return Scheduling information associated to particular Job Queue]]> @param ]]> RecordWriter to future operations. @param context the context of the task @throws IOException]]> RecordWriter writes the output <key, value> pairs to an output file.

RecordWriter implementations write the job outputs to the {@link FileSystem}. @see OutputFormat]]> the class of the input keys @param the class of the input values @param the class of the output keys @param the class of the output values]]> Reducer implementations can access the {@link Configuration} for the job via the {@link JobContext#getConfiguration()} method.

Reducer has 3 primary phases:

  1. Shuffle

    The Reducer copies the sorted output from each {@link Mapper} using HTTP across the network.

  2. Sort

    The framework merge sorts Reducer inputs by keys (since different Mappers may have output the same key).

    The shuffle and sort phases occur simultaneously i.e. while outputs are being fetched they are merged.

    SecondarySort

    To achieve a secondary sort on the values returned by the value iterator, the application should extend the key with the secondary key and define a grouping comparator. The keys will be sorted using the entire key, but will be grouped using the grouping comparator to decide which keys and values are sent in the same call to reduce.The grouping comparator is specified via {@link Job#setGroupingComparatorClass(Class)}. The sort order is controlled by {@link Job#setSortComparatorClass(Class)}.

    For example, say that you want to find duplicate web pages and tag them all with the url of the "best" known example. You would set up the job like:
    • Map Input Key: url
    • Map Input Value: document
    • Map Output Key: document checksum, url pagerank
    • Map Output Value: url
    • Partitioner: by checksum
    • OutputKeyComparator: by checksum and then decreasing pagerank
    • OutputValueGroupingComparator: by checksum
  3. Reduce

    In this phase the {@link #reduce(Object, Iterable, Context)} method is called for each <key, (collection of values)> in the sorted inputs.

    The output of the reduce task is typically written to a {@link RecordWriter} via {@link Context#write(Object, Object)}.

The output of the Reducer is not re-sorted.

Example:

 public class IntSumReducer<Key> extends Reducer<Key,IntWritable,
                                                 Key,IntWritable> {
   private IntWritable result = new IntWritable();
 
   public void reduce(Key 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);
   }
 }
 

@see Mapper @see Partitioner]]>
Context passed on to the {@link Reducer} implementations.]]> counterName. @param counterName counter name @return the Counter for the given counterName]]> groupName and counterName. @param counterName counter name @return the Counter for the given groupName and counterName]]> An example TaskAttemptID is : attempt_200707121733_0003_m_000005_0 , which represents the zeroth task attempt for the fifth map task in the third job running at the jobtracker started at 200707121733.

Applications should never construct or parse TaskAttemptID strings , but rather use appropriate constructors or {@link #forName(String)} method. @see JobID @see TaskID]]> An example TaskID is : task_200707121733_0003_m_000005 , which represents the fifth map task in the third job running at the jobtracker started at 200707121733.

Applications should never construct or parse TaskID strings , but rather use appropriate constructors or {@link #forName(String)} method. @see JobID @see TaskAttemptID]]> OutputCommitter for the task-attempt]]> the input key type for the task @param the input value type for the task @param the output key type for the task @param the output value type for the task]]> generateKeyValPairs(Object key, Object value); public void configure(Configuration conf); } The package also provides a base class, ValueAggregatorBaseDescriptor, implementing the above interface. The user can extend the base class and implement generateKeyValPairs accordingly. The primary work of generateKeyValPairs is to emit one or more key/value pairs based on the input key/value pair. The key in an output key/value pair encode two pieces of information: aggregation type and aggregation id. The value will be aggregated onto the aggregation id according the aggregation type. This class offers a function to generate a map/reduce job using Aggregate framework. The function takes the following parameters: input directory spec input format (text or sequence file) output directory a file specifying the user plugin class]]> The key and values are passed from one element of the chain to the next, by value. For the added Mapper the configuration given for it, mapperConf, have precedence over the job's Configuration. This precedence is in effect when the task is running.

IMPORTANT: There is no need to specify the output key/value classes for the ChainMapper, this is done by the addMapper for the last mapper in the chain

@param job The job. @param klass the Mapper class to add. @param inputKeyClass mapper input key class. @param inputValueClass mapper input value class. @param outputKeyClass mapper output key class. @param outputValueClass mapper output value class. @param mapperConf a configuration for the Mapper class. It is recommended to use a Configuration without default values using the Configuration(boolean loadDefaults) constructor with FALSE.]]>
The Mapper classes are invoked in a chained (or piped) fashion, the output of the first becomes the input of the second, and so on until the last Mapper, the output of the last Mapper will be written to the task's output.

The key functionality of this feature is that the Mappers in the chain do not need to be aware that they are executed in a chain. This enables having reusable specialized Mappers that can be combined to perform composite operations within a single task.

Special care has to be taken when creating chains that the key/values output by a Mapper are valid for the following Mapper in the chain. It is assumed all Mappers and the Reduce in the chain use matching output and input key and value classes as no conversion is done by the chaining code.

Using the ChainMapper and the ChainReducer classes is possible to compose Map/Reduce jobs that look like [MAP+ / REDUCE MAP*]. And immediate benefit of this pattern is a dramatic reduction in disk IO.

IMPORTANT: There is no need to specify the output key/value classes for the ChainMapper, this is done by the addMapper for the last mapper in the chain.

ChainMapper usage pattern:

 ...
 Job = new Job(conf);
 

Configuration mapAConf = new Configuration(false); ... ChainMapper.addMapper(job, AMap.class, LongWritable.class, Text.class, Text.class, Text.class, true, mapAConf);

Configuration mapBConf = new Configuration(false); ... ChainMapper.addMapper(job, BMap.class, Text.class, Text.class, LongWritable.class, Text.class, false, mapBConf);

...

job.waitForComplettion(true); ...

]]>
The key and values are passed from one element of the chain to the next, by value. For the added Reducer the configuration given for it, reducerConf, have precedence over the job's Configuration. This precedence is in effect when the task is running.

IMPORTANT: There is no need to specify the output key/value classes for the ChainReducer, this is done by the setReducer or the addMapper for the last element in the chain.

@param job the job @param klass the Reducer class to add. @param inputKeyClass reducer input key class. @param inputValueClass reducer input value class. @param outputKeyClass reducer output key class. @param outputValueClass reducer output value class. @param reducerConf a configuration for the Reducer class. It is recommended to use a Configuration without default values using the Configuration(boolean loadDefaults) constructor with FALSE.]]>
The key and values are passed from one element of the chain to the next, by value For the added Mapper the configuration given for it, mapperConf, have precedence over the job's Configuration. This precedence is in effect when the task is running.

IMPORTANT: There is no need to specify the output key/value classes for the ChainMapper, this is done by the addMapper for the last mapper in the chain.

@param job The job. @param klass the Mapper class to add. @param inputKeyClass mapper input key class. @param inputValueClass mapper input value class. @param outputKeyClass mapper output key class. @param outputValueClass mapper output value class. @param mapperConf a configuration for the Mapper class. It is recommended to use a Configuration without default values using the Configuration(boolean loadDefaults) constructor with FALSE.]]>
For each record output by the Reducer, the Mapper classes are invoked in a chained (or piped) fashion. The output of the reducer becomes the input of the first mapper and output of first becomes the input of the second, and so on until the last Mapper, the output of the last Mapper will be written to the task's output.

The key functionality of this feature is that the Mappers in the chain do not need to be aware that they are executed after the Reducer or in a chain. This enables having reusable specialized Mappers that can be combined to perform composite operations within a single task.

Special care has to be taken when creating chains that the key/values output by a Mapper are valid for the following Mapper in the chain. It is assumed all Mappers and the Reduce in the chain use matching output and input key and value classes as no conversion is done by the chaining code.

Using the ChainMapper and the ChainReducer classes is possible to compose Map/Reduce jobs that look like [MAP+ / REDUCE MAP*]. And immediate benefit of this pattern is a dramatic reduction in disk IO.

IMPORTANT: There is no need to specify the output key/value classes for the ChainReducer, this is done by the setReducer or the addMapper for the last element in the chain.

ChainReducer usage pattern:

 ...
 Job = new Job(conf);
 ....
 

Configuration reduceConf = new Configuration(false); ... ChainReducer.setReducer(job, XReduce.class, LongWritable.class, Text.class, Text.class, Text.class, true, reduceConf);

ChainReducer.addMapper(job, CMap.class, Text.class, Text.class, LongWritable.class, Text.class, false, null);

ChainReducer.addMapper(job, DMap.class, LongWritable.class, Text.class, LongWritable.class, LongWritable.class, true, null);

...

job.waitForCompletion(true); ...

]]>
20070101 AND length > 0)' @param orderBy the fieldNames in the orderBy clause. @param fieldNames The field names in the table @see #setInput(Job, Class, String, String)]]> DBInputFormat emits LongWritables containing the record number as key and DBWritables as value. The SQL query, and input class can be using one of the two setInput methods.]]> {@link DBOutputFormat} accepts <key,value> pairs, where key has a type extending DBWritable. Returned {@link RecordWriter} writes only the key to the database with a batch SQL query.]]> DBWritable. DBWritable, is similar to {@link Writable} except that the {@link #write(PreparedStatement)} method takes a {@link PreparedStatement}, and {@link #readFields(ResultSet)} takes a {@link ResultSet}.

Implementations are responsible for writing the fields of the object to PreparedStatement, and reading the fields of the object from the ResultSet.

Example:

If we have the following table in the database :
 CREATE TABLE MyTable (
   counter        INTEGER NOT NULL,
   timestamp      BIGINT  NOT NULL,
 );
 
then we can read/write the tuples from/to the table with :

 public class MyWritable implements Writable, DBWritable {
   // Some data     
   private int counter;
   private long timestamp;
       
   //Writable#write() implementation
   public void write(DataOutput out) throws IOException {
     out.writeInt(counter);
     out.writeLong(timestamp);
   }
       
   //Writable#readFields() implementation
   public void readFields(DataInput in) throws IOException {
     counter = in.readInt();
     timestamp = in.readLong();
   }
       
   public void write(PreparedStatement statement) throws SQLException {
     statement.setInt(1, counter);
     statement.setLong(2, timestamp);
   }
       
   public void readFields(ResultSet resultSet) throws SQLException {
     counter = resultSet.getInt(1);
     timestamp = resultSet.getLong(2);
   } 
 }
 

]]>
RecordReader's for CombineFileSplit's. @see CombineFileSplit]]> th Path]]> th Path]]> th Path]]> CombineFileSplit can be used to implement {@link RecordReader}'s, with reading one record per file. @see FileSplit @see CombineFileInputFormat]]> FileInputFormat implementations can override this and return false to ensure that individual input files are never split-up so that {@link Mapper}s process entire files. @param context the job context @param filename the file name to check @return is this file splitable?]]> FileInputFormat is the base class for all file-based InputFormats. This provides a generic implementation of {@link #getSplits(JobContext)}. Subclasses of FileInputFormat can also override the {@link #isSplitable(JobContext, Path)} method to ensure input-files are not split-up and are processed as a whole by {@link Mapper}s.]]> f. The filtering criteria is MD5(key) % f == 0.]]> f using the criteria record# % f == 0. For example, if the frequency is 10, one out of 10 records is returned.]]> true if the Job was added.]]> ([,]*) func ::= tbl(,"") class ::= @see java.lang.Class#forName(java.lang.String) path ::= @see org.apache.hadoop.fs.Path#Path(java.lang.String) } Reads expression from the mapreduce.join.expr property and user-supplied join types from mapreduce.join.define.<ident> types. Paths supplied to tbl are given as input paths to the InputFormat class listed. @see #compose(java.lang.String, java.lang.Class, java.lang.String...)]]> ,

) }]]> (tbl(,),tbl(,),...,tbl(,)) }]]> (tbl(,),tbl(,),...,tbl(,)) }]]> mapreduce.join.define.<ident> to a classname. In the expression mapreduce.join.expr, the identifier will be assumed to be a ComposableRecordReader. mapreduce.join.keycomparator can be a classname used to compare keys in the join. @see JoinRecordReader @see MultiFilterRecordReader]]> ...... }]]> capacity children to position id in the parent reader. The id of a root CompositeRecordReader is -1 by convention, but relying on this is not recommended.]]> override(S1,S2,S3) will prefer values from S3 over S2, and values from S2 over S1 for all keys emitted from all sources.]]> [,,...,]]]> out. TupleWritable format: {@code ...... }]]> the map's input key type @param the map's input value type @param the map's output key type @param the map's output value type @param job the job @return the mapper class to run]]> the map input key type @param the map input value type @param the map output key type @param the map output value type @param job the job to modify @param cls the class to use as the mapper]]> It can be used instead of the default implementation, @link org.apache.hadoop.mapred.MapRunner, when the Map operation is not CPU bound in order to improve throughput.

Mapper implementations using this MapRunnable must be thread-safe.

The Map-Reduce job has to be configured with the mapper to use via {@link #setMapperClass(Configuration, Class)} and the number of thread the thread-pool can use with the {@link #getNumberOfThreads(Configuration) method. The default value is 10 threads.

]]> MapContext to be wrapped @return a wrapped Mapper.Context for custom implementations]]> true if the job output should be compressed, false otherwise]]> Tasks' Side-Effect Files

Some applications need to create/write-to side-files, which differ from the actual job-outputs.

In such cases there could be issues with 2 instances of the same TIP (running simultaneously e.g. speculative tasks) trying to open/write-to the same file (path) on HDFS. Hence the application-writer will have to pick unique names per task-attempt (e.g. using the attemptid, say attempt_200709221812_0001_m_000000_0), not just per TIP.

To get around this the Map-Reduce framework helps the application-writer out by maintaining a special ${mapreduce.output.fileoutputformat.outputdir}/_temporary/_${taskid} sub-directory for each task-attempt on HDFS where the output of the task-attempt goes. On successful completion of the task-attempt the files in the ${mapreduce.output.fileoutputformat.outputdir}/_temporary/_${taskid} (only) are promoted to ${mapreduce.output.fileoutputformat.outputdir}. Of course, the framework discards the sub-directory of unsuccessful task-attempts. This is completely transparent to the application.

The application-writer can take advantage of this by creating any side-files required in a work directory during execution of his task i.e. via {@link #getWorkOutputPath(TaskInputOutputContext)}, and the framework will move them out similarly - thus she doesn't have to pick unique paths per task-attempt.

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.

@return the {@link Path} to the task's temporary output directory for the map-reduce job.]]>
The path can be used to create custom files from within the map and reduce tasks. The path name will be unique for each task. The path parent will be the job output directory.

ls

This method uses the {@link #getUniqueFile} method to make the file name unique for the task.

@param context the context for the task. @param name the name for the file. @param extension the extension for the file @return a unique path accross all tasks of the job.]]>
FilterRecordWriter is a convenience wrapper class that extends the {@link RecordWriter}.]]> @param job job to add the named output @param namedOutput named output name, it has to be a word, letters and numbers only, cannot be the word 'part' as that is reserved for the default output. @param outputFormatClass OutputFormat class. @param keyClass key class @param valueClass value class]]> super.close() at the end of their close()]]> Case one: writing to additional outputs other than the job default output. Each additional output, or named output, may be configured with its own OutputFormat, with its own key class and with its own value class.

Case two: to write data to different files provided by user

MultipleOutputs supports counters, by default they are disabled. The counters group is the {@link MultipleOutputs} class name. The names of the counters are the same as the output name. These count the number records written to each output name.

Usage pattern for job submission:

 Job job = new Job();

 FileInputFormat.setInputPath(job, inDir);
 FileOutputFormat.setOutputPath(job, outDir);

 job.setMapperClass(MOMap.class);
 job.setReducerClass(MOReduce.class);
 ...

 // Defines additional single text based output 'text' for the job
 MultipleOutputs.addNamedOutput(job, "text", TextOutputFormat.class,
 LongWritable.class, Text.class);

 // Defines additional sequence-file based output 'sequence' for the job
 MultipleOutputs.addNamedOutput(job, "seq",
   SequenceFileOutputFormat.class,
   LongWritable.class, Text.class);
 ...

 job.waitForCompletion(true);
 ...
 

Usage in Reducer:

  String generateFileName(K k, V v) {
   return k.toString() + "_" + v.toString();
 }
 
 public class MOReduce extends
   Reducer<WritableComparable, Writable,WritableComparable, Writable> {
 private MultipleOutputs mos;
 public void setup(Context context) {
 ...
 mos = new MultipleOutputs(context);
 }

 public void reduce(WritableComparable key, Iterator<Writable> values,
 Context context)
 throws IOException {
 ...
 mos.write("text", , key, new Text("Hello"));
 mos.write("seq", LongWritable(1), new Text("Bye"), "seq_a");
 mos.write("seq", LongWritable(2), key, new Text("Chau"), "seq_b");
 mos.write(key, new Text("value"), generateFileName(key, new Text("value")));
 ...
 }

 public void cleanup(Context) throws IOException {
 mos.close();
 ...
 }

 }
 
]]>
This allows the user to specify the key class to be different from the actual class ({@link BytesWritable}) used for writing

@param job the {@link Job} to modify @param theClass the SequenceFile output key class.]]>
This allows the user to specify the value class to be different from the actual class ({@link BytesWritable}) used for writing

@param job the {@link Job} to modify @param theClass the SequenceFile output key class.]]>
bytes[left:(right+1)] in Python syntax. @param conf configuration object @param left left Python-style offset @param right right Python-style offset]]> bytes[offset:] in Python syntax. @param conf configuration object @param offset left Python-style offset]]> bytes[:(offset+1)] in Python syntax. @param conf configuration object @param offset right Python-style offset]]> Partition {@link BinaryComparable} keys using a configurable part of the bytes array returned by {@link BinaryComparable#getBytes()}.

The subarray to be used for the partitioning can be defined by means of the following properties:

  • mapreduce.partition.binarypartitioner.left.offset: left offset in array (0 by default)
  • mapreduce.partition.binarypartitioner.right.offset: right offset in array (-1 by default)
Like in Python, both negative and positive offsets are allowed, but the meaning is slightly different. In case of an array of length 5, for instance, the possible offsets are:

  +---+---+---+---+---+
  | B | B | B | B | B |
  +---+---+---+---+---+
    0   1   2   3   4
   -5  -4  -3  -2  -1
 
The first row of numbers gives the position of the offsets 0...5 in the array; the second row gives the corresponding negative offsets. Contrary to Python, the specified subarray has byte i and j as first and last element, repectively, when i and j are the left and right offset.

For Hadoop programs written in Java, it is advisable to use one of the following static convenience methods for setting the offsets:

  • {@link #setOffsets}
  • {@link #setLeftOffset}
  • {@link #setRightOffset}

]]>
all splits. @param freq The frequency with which records will be emitted.]]> all splits. This will read every split at the client, which is very expensive. @param freq Probability with which a key will be chosen. @param numSamples Total number of samples to obtain from all selected splits.]]> all splits. Takes the first numSamples / numSplits records from each split. @param numSamples Total number of samples to obtain from all selected splits.]]> total.order.partitioner.natural.order is not false, a trie of the first total.order.partitioner.max.trie.depth(2) + 1 bytes will be built. Otherwise, keys will be located using a binary search of the partition keyset using the {@link org.apache.hadoop.io.RawComparator} defined for this job. The input file must be sorted with the same comparator and contain {@link Job#getNumReduceTasks()} - 1 keys.]]> R reduces, there are R-1 keys in the SequenceFile.]]>
ReduceContext to be wrapped @return a wrapped Reducer.Context for custom implementations]]> This tool supports archiving and anaylzing (sort/grep) of log-files. It takes as input a) Input uri which will serve uris of the logs to be archived. b) Output directory (not mandatory). b) Directory on dfs to archive the logs. c) The sort/grep patterns for analyzing the files and separator for boundaries. Usage: Logalyzer -archive -archiveDir -analysis -logs -grep -sort -separator

]]> pair. The name should preserve the original job history file or job conf file name. The input object should be closed before calling getNext() again. The old input object would be invalid after calling getNext() again. @return the next pair.]]> JobConf for the job]]> Job]]> Job]]> TaskInfo for the given task]]> TaskAttemptInfo for the given task-attempt]]> TaskAttemptInfo for the given task-attempt]]> The type of the objects to be written.]]> hosts grouped into racks, then in onecluster you might have a node {@code node1} on rack {@code rack1}. This would be represented with a ArrayList of two layers, with two {@link String} s being {@code "rack1"} and {@code "node1"}. The details of this class are set up to meet the requirements of the Jackson JSON parser/generator. All of the public methods are simply accessors for the instance variables we want to write out in the JSON files.]]> map phase of the map-task attempt. @return the runtime for the map phase of the map-task attempt]]> reduce phase of the reduce task-attempt. @return the runtime for the reduce phase of the reduce task-attempt]]> shuffle phase of the reduce task-attempt. @return the runtime for the shuffle phase of the reduce task-attempt]]> merge phase of the reduce task-attempt @return the runtime for the merge phase of the reduce task-attempt]]> State of the task-attempt]]> TaskInfo for the given task-attempt]]>