MAPREDUCE-5657. Fix Javadoc errors caused by incorrect or illegal tags in doc comments. Contributed by Akira AJISAKA.

This commit is contained in:
Tsuyoshi Ozawa 2015-03-03 18:06:26 +09:00
parent 742f9d90c0
commit 9ae7f9eb7b
78 changed files with 249 additions and 261 deletions

View File

@ -181,6 +181,9 @@ Trunk (Unreleased)
MAPREDUCE-6234. TestHighRamJob fails due to the change in MAPREDUCE-5785.
(Masatake Iwasaki via kasha)
MAPREDUCE-5657. Fix Javadoc errors caused by incorrect or illegal tags in doc
comments. (Akira AJISAKA via ozawa)
BREAKDOWN OF MAPREDUCE-2841 (NATIVE TASK) SUBTASKS
MAPREDUCE-5985. native-task: Fix build on macosx. Contributed by

View File

@ -174,7 +174,7 @@ public InetSocketAddress getAddress() {
/**
* Child checking whether it can commit.
*
* <br/>
* <br>
* Commit is a two-phased protocol. First the attempt informs the
* ApplicationMaster that it is
* {@link #commitPending(TaskAttemptID, TaskStatus)}. Then it repeatedly polls
@ -208,7 +208,7 @@ public boolean canCommit(TaskAttemptID taskAttemptID) throws IOException {
* TaskAttempt is reporting that it is in commit_pending and it is waiting for
* the commit Response
*
* <br/>
* <br>
* Commit it a two-phased protocol. First the attempt informs the
* ApplicationMaster that it is
* {@link #commitPending(TaskAttemptID, TaskStatus)}. Then it repeatedly polls

View File

@ -44,7 +44,6 @@
* proxy if needed</li><li>
* The URL may contain sentinels which will be replaced by jobId and jobStatus
* (eg. SUCCEEDED/KILLED/FAILED) </li> </ul>
* </p>
*/
public class JobEndNotifier implements Configurable {
private static final String JOB_ID = "$jobId";

View File

@ -345,7 +345,7 @@ public static void setJobClassLoader(Configuration conf)
* {@link MRJobConfig#MAPREDUCE_JOB_CLASSLOADER} is set to true, and
* the APP_CLASSPATH environment variable is set.
* @param conf
* @returns the created job classloader, or null if the job classloader is not
* @return the created job classloader, or null if the job classloader is not
* enabled or the APP_CLASSPATH environment variable is not set
* @throws IOException
*/

View File

@ -113,7 +113,7 @@
* }
* }
*
* </pre></blockquote></p>
* </pre></blockquote>
*
* It is also very common to use the DistributedCache by using
* {@link org.apache.hadoop.util.GenericOptionsParser}.

View File

@ -48,7 +48,7 @@
* Task capacity of the cluster.
* </li>
* <li>
* The number of currently running map & reduce tasks.
* The number of currently running map and reduce tasks.
* </li>
* <li>
* State of the <code>JobTracker</code>.
@ -56,7 +56,7 @@
* <li>
* Details regarding black listed trackers.
* </li>
* </ol></p>
* </ol>
*
* <p>Clients can query for the latest <code>ClusterStatus</code>, via
* {@link JobClient#getClusterStatus()}.</p>

View File

@ -179,7 +179,7 @@ public static Path getOutputPath(JobConf conf) {
* Get the {@link Path} to the task's temporary output directory
* for the map-reduce job
*
* <h4 id="SideEffectFiles">Tasks' Side-Effect Files</h4>
* <b id="SideEffectFiles">Tasks' Side-Effect Files</b>
*
* <p><i>Note:</i> The following is valid only if the {@link OutputCommitter}
* is {@link FileOutputCommitter}. If <code>OutputCommitter</code> is not

View File

@ -47,7 +47,7 @@
import org.apache.commons.logging.LogFactory;
/**
* <code>IFile</code> is the simple <key-len, value-len, key, value> format
* <code>IFile</code> is the simple &lt;key-len, value-len, key, value&gt; format
* for the intermediate map-outputs in Map-Reduce.
*
* There is a <code>Writer</code> to write out map-outputs in this format and

View File

@ -101,7 +101,6 @@ boolean isMRAdmin(UserGroupInformation callerUGI) {
* @param jobOperation
* @param jobOwner
* @param jobACL
* @throws AccessControlException
*/
public boolean checkAccess(UserGroupInformation callerUGI,
JobACL jobOperation, String jobOwner, AccessControlList jobACL) {

View File

@ -79,7 +79,7 @@
* Submitting the job to the cluster and optionally monitoring
* it's status.
* </li>
* </ol></p>
* </ol>
*
* Normally the user creates the application, describes various facets of the
* job via {@link JobConf} and then uses the <code>JobClient</code> to submit
@ -101,9 +101,9 @@
*
* // Submit the job, then poll for progress until the job is complete
* JobClient.runJob(job);
* </pre></blockquote></p>
* </pre></blockquote>
*
* <h4 id="JobControl">Job Control</h4>
* <b id="JobControl">Job Control</b>
*
* <p>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
@ -127,7 +127,7 @@
* {@link JobConf#setJobEndNotificationURI(String)} : setup a notification
* on job-completion, thus avoiding polling.
* </li>
* </ol></p>
* </ol>
*
* @see JobConf
* @see ClusterStatus

View File

@ -74,7 +74,7 @@
* more complex for the user to control finely
* (e.g. {@link #setNumMapTasks(int)}).
* </li>
* </ol></p>
* </ol>
*
* <p><code>JobConf</code> typically specifies the {@link Mapper}, combiner
* (if any), {@link Partitioner}, {@link Reducer}, {@link InputFormat} and
@ -105,7 +105,7 @@
*
* job.setInputFormat(SequenceFileInputFormat.class);
* job.setOutputFormat(SequenceFileOutputFormat.class);
* </pre></blockquote></p>
* </pre></blockquote>
*
* @see JobClient
* @see ClusterStatus
@ -486,7 +486,7 @@ public JobConf(Path config) {
/** A new map/reduce configuration where the behavior of reading from the
* default resources can be turned off.
* <p/>
* <p>
* If the parameter {@code loadDefaults} is false, the new instance
* will not load resources from the default files.
*
@ -993,19 +993,19 @@ public RawComparator getOutputValueGroupingComparator() {
/**
* Set the user defined {@link RawComparator} comparator for
* grouping keys in the input to the combiner.
* <p/>
*
* <p>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)}.</p>
* <p/>
*
* <p>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.</p>
* <p/>
*
* <p>Since {@link #setOutputKeyComparatorClass(Class)} can be used to control
* how keys are sorted, this can be used in conjunction to simulate
* <i>secondary sort on values</i>.</p>
* <p/>
*
* <p><i>Note</i>: This is not a guarantee of the combiner sort being
* <i>stable</i> in any sense. (In any case, with the order of available
* map-outputs to the combiner being non-deterministic, it wouldn't make
@ -1210,7 +1210,7 @@ public Class<? extends Reducer> getCombinerClass() {
* <li> be side-effect free</li>
* <li> have the same input and output key types and the same input and
* output value types</li>
* </ul></p>
* </ul>
*
* <p>Typically the combiner is same as the <code>Reducer</code> for the
* job i.e. {@link #setReducerClass(Class)}.</p>
@ -1309,7 +1309,7 @@ public void setReduceSpeculativeExecution(boolean speculativeExecution) {
* A custom {@link InputFormat} is typically used to accurately control
* the number of map tasks for the job.</p>
*
* <h4 id="NoOfMaps">How many maps?</h4>
* <b id="NoOfMaps">How many maps?</b>
*
* <p>The number of maps is usually driven by the total size of the inputs
* i.e. total number of blocks of the input files.</p>
@ -1350,7 +1350,7 @@ public void setReduceSpeculativeExecution(boolean speculativeExecution) {
/**
* Set the requisite number of reduce tasks for this job.
*
* <h4 id="NoOfReduces">How many reduces?</h4>
* <b id="NoOfReduces">How many reduces?</b>
*
* <p>The right number of reduces seems to be <code>0.95</code> or
* <code>1.75</code> multiplied by (&lt;<i>no. of nodes</i>&gt; *
@ -1370,7 +1370,7 @@ public void setReduceSpeculativeExecution(boolean speculativeExecution) {
* reserve a few reduce slots in the framework for speculative-tasks, failures
* etc.</p>
*
* <h4 id="ReducerNone">Reducer NONE</h4>
* <b id="ReducerNone">Reducer NONE</b>
*
* <p>It is legal to set the number of reduce-tasks to <code>zero</code>.</p>
*
@ -1693,9 +1693,9 @@ public void setProfileTaskRange(boolean isMap, String newValue) {
* given task's stdout, stderr, syslog, jobconf files as arguments.</p>
*
* <p>The debug command, run on the node where the map failed, is:</p>
* <p><pre><blockquote>
* <p><blockquote><pre>
* $script $stdout $stderr $syslog $jobconf.
* </blockquote></pre></p>
* </pre></blockquote>
*
* <p> The script file is distributed through {@link DistributedCache}
* APIs. The script needs to be symlinked. </p>
@ -1705,7 +1705,7 @@ public void setProfileTaskRange(boolean isMap, String newValue) {
* job.setMapDebugScript("./myscript");
* DistributedCache.createSymlink(job);
* DistributedCache.addCacheFile("/debug/scripts/myscript#myscript");
* </pre></blockquote></p>
* </pre></blockquote>
*
* @param mDbgScript the script name
*/
@ -1730,9 +1730,9 @@ public String getMapDebugScript() {
* is given task's stdout, stderr, syslog, jobconf files as arguments.</p>
*
* <p>The debug command, run on the node where the map failed, is:</p>
* <p><pre><blockquote>
* <p><blockquote><pre>
* $script $stdout $stderr $syslog $jobconf.
* </blockquote></pre></p>
* </pre></blockquote>
*
* <p> The script file is distributed through {@link DistributedCache}
* APIs. The script file needs to be symlinked </p>
@ -1742,7 +1742,7 @@ public String getMapDebugScript() {
* job.setReduceDebugScript("./myscript");
* DistributedCache.createSymlink(job);
* DistributedCache.addCacheFile("/debug/scripts/myscript#myscript");
* </pre></blockquote></p>
* </pre></blockquote>
*
* @param rDbgScript the script name
*/
@ -1785,8 +1785,6 @@ public String getJobEndNotificationURI() {
*
* @param uri the job end notification uri
* @see JobStatus
* @see <a href="{@docRoot}/org/apache/hadoop/mapred/JobClient.html#
* JobCompletionAndChaining">Job Completion and Chaining</a>
*/
public void setJobEndNotificationURI(String uri) {
set(JobContext.MR_JOB_END_NOTIFICATION_URL, uri);
@ -1816,7 +1814,7 @@ public String getJobLocalDir() {
*
* If a value is specified in the configuration, it is returned.
* Else, it returns {@link JobContext#DEFAULT_MAP_MEMORY_MB}.
* <p/>
* <p>
* 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
@ -1842,7 +1840,7 @@ public void setMemoryForMapTask(long mem) {
*
* If a value is specified in the configuration, it is returned.
* Else, it returns {@link JobContext#DEFAULT_REDUCE_MEMORY_MB}.
* <p/>
* <p>
* 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
@ -1915,7 +1913,6 @@ public static long normalizeMemoryConfigValue(long val) {
*
* @param my_class the class to find.
* @return a jar file that contains the class, or null.
* @throws IOException
*/
public static String findContainingJar(Class my_class) {
return ClassUtil.findContainingJar(my_class);
@ -1924,10 +1921,10 @@ public static String findContainingJar(Class my_class) {
/**
* Get the memory required to run a task of this job, in bytes. See
* {@link #MAPRED_TASK_MAXVMEM_PROPERTY}
* <p/>
* <p>
* This method is deprecated. Now, different memory limits can be
* set for map and reduce tasks of a job, in MB.
* <p/>
* <p>
* For backward compatibility, if the job configuration sets the
* key {@link #MAPRED_TASK_MAXVMEM_PROPERTY}, that value is returned.
* Otherwise, this method will return the larger of the values returned by
@ -1953,7 +1950,7 @@ public long getMaxVirtualMemoryForTask() {
/**
* Set the maximum amount of memory any task of this job can use. See
* {@link #MAPRED_TASK_MAXVMEM_PROPERTY}
* <p/>
* <p>
* mapred.task.maxvmem is split into
* mapreduce.map.memory.mb
* and mapreduce.map.memory.mb,mapred
@ -2073,7 +2070,7 @@ public String getTaskJavaOpts(TaskType taskType) {
/**
* Parse the Maximum heap size from the java opts as specified by the -Xmx option
* Format: -Xmx<size>[g|G|m|M|k|K]
* Format: -Xmx&lt;size&gt;[g|G|m|M|k|K]
* @param javaOpts String to parse to read maximum heap size
* @return Maximum heap size in MB or -1 if not specified
*/

View File

@ -117,7 +117,7 @@
* output.collect(key, val);
* }
* }
* </pre></blockquote></p>
* </pre></blockquote>
*
* <p>Applications may write a custom {@link MapRunnable} to exert greater
* control on map processing e.g. multi-threaded <code>Mapper</code>s etc.</p>

View File

@ -46,20 +46,20 @@
/**
* Class that exposes information about queues maintained by the Hadoop
* Map/Reduce framework.
* <p/>
* <p>
* The Map/Reduce framework can be configured with one or more queues,
* depending on the scheduler it is configured with. While some
* schedulers work only with one queue, some schedulers support multiple
* queues. Some schedulers also support the notion of queues within
* queues - a feature called hierarchical queues.
* <p/>
* <p>
* Queue names are unique, and used as a key to lookup queues. Hierarchical
* queues are named by a 'fully qualified name' such as q1:q2:q3, where
* q2 is a child queue of q1 and q3 is a child queue of q2.
* <p/>
* <p>
* Leaf level queues are queues that contain no queues within them. Jobs
* can be submitted only to leaf level queues.
* <p/>
* <p>
* Queues can be configured with various properties. Some of these
* properties are common to all schedulers, and those are handled by this
* class. Schedulers might also associate several custom properties with
@ -69,11 +69,11 @@
* provided by the framework, but define their own mechanisms. In such cases,
* it is likely that the name of the queue will be used to relate the
* common properties of a queue with scheduler specific properties.
* <p/>
* <p>
* Information related to a queue, such as its name, properties, scheduling
* information and children are exposed by this class via a serializable
* class called {@link JobQueueInfo}.
* <p/>
* <p>
* Queues are configured in the configuration file mapred-queues.xml.
* To support backwards compatibility, queues can also be configured
* in mapred-site.xml. However, when configured in the latter, there is
@ -102,7 +102,7 @@ public class QueueManager {
/**
* Factory method to create an appropriate instance of a queue
* configuration parser.
* <p/>
* <p>
* Returns a parser that can parse either the deprecated property
* style queue configuration in mapred-site.xml, or one that can
* parse hierarchical queues in mapred-queues.xml. First preference
@ -157,7 +157,7 @@ static QueueConfigurationParser getQueueConfigurationParser(
/**
* Construct a new QueueManager using configuration specified in the passed
* in {@link org.apache.hadoop.conf.Configuration} object.
* <p/>
* <p>
* This instance supports queue configuration specified in mapred-site.xml,
* but without support for hierarchical queues. If no queue configuration
* is found in mapred-site.xml, it will then look for site configuration
@ -173,7 +173,7 @@ public QueueManager(Configuration clusterConf) {
/**
* Create an instance that supports hierarchical queues, defined in
* the passed in configuration file.
* <p/>
* <p>
* This is mainly used for testing purposes and should not called from
* production code.
*
@ -208,7 +208,7 @@ private void initialize(QueueConfigurationParser cp) {
/**
* Return the set of leaf level queues configured in the system to
* which jobs are submitted.
* <p/>
* <p>
* The number of queues configured should be dependent on the Scheduler
* configured. Note that some schedulers work with only one queue, whereas
* others can support multiple queues.
@ -222,7 +222,7 @@ public synchronized Set<String> getLeafQueueNames() {
/**
* Return true if the given user is part of the ACL for the given
* {@link QueueACL} name for the given queue.
* <p/>
* <p>
* An operation is allowed if all users are provided access for this
* operation, or if either the user or any of the groups specified is
* provided access.
@ -283,7 +283,7 @@ synchronized boolean isRunning(String queueName) {
/**
* Set a generic Object that represents scheduling information relevant
* to a queue.
* <p/>
* <p>
* A string representation of this Object will be used by the framework
* to display in user facing applications like the JobTracker web UI and
* the hadoop CLI.
@ -323,7 +323,7 @@ public synchronized Object getSchedulerInfo(String queueName) {
/**
* Refresh acls, state and scheduler properties for the configured queues.
* <p/>
* <p>
* This method reloads configuration related to queues, but does not
* support changes to the list of queues or hierarchy. The expected usage
* is that an administrator can modify the queue configuration file and
@ -431,7 +431,7 @@ synchronized JobQueueInfo getJobQueueInfo(String queue) {
/**
* JobQueueInfo for all the queues.
* <p/>
* <p>
* Contribs can use this data structure to either create a hierarchy or for
* traversing.
* They can also use this to refresh properties in case of refreshQueues
@ -450,7 +450,7 @@ synchronized Map<String, JobQueueInfo> getJobQueueInfoMapping() {
/**
* Generates the array of QueueAclsInfo object.
* <p/>
* <p>
* The array consists of only those queues for which user has acls.
*
* @return QueueAclsInfo[]

View File

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

View File

@ -42,7 +42,7 @@
* <ol>
* <li>
*
* <h4 id="Shuffle">Shuffle</h4>
* <b id="Shuffle">Shuffle</b>
*
* <p><code>Reducer</code> is input the grouped output of a {@link Mapper}.
* In the phase the framework, for each <code>Reducer</code>, fetches the
@ -51,7 +51,7 @@
* </li>
*
* <li>
* <h4 id="Sort">Sort</h4>
* <b id="Sort">Sort</b>
*
* <p>The framework groups <code>Reducer</code> inputs by <code>key</code>s
* (since different <code>Mapper</code>s may have output the same key) in this
@ -60,7 +60,7 @@
* <p>The shuffle and sort phases occur simultaneously i.e. while outputs are
* being fetched they are merged.</p>
*
* <h5 id="SecondarySort">SecondarySort</h5>
* <b id="SecondarySort">SecondarySort</b>
*
* <p>If equivalence rules for keys while grouping the intermediates are
* different from those for grouping keys before reduction, then one may
@ -86,11 +86,11 @@
* </li>
*
* <li>
* <h4 id="Reduce">Reduce</h4>
* <b id="Reduce">Reduce</b>
*
* <p>In this phase the
* {@link #reduce(Object, Iterator, OutputCollector, Reporter)}
* method is called for each <code>&lt;key, (list of values)></code> pair in
* method is called for each <code>&lt;key, (list of values)&gt;</code> pair in
* the grouped inputs.</p>
* <p>The output of the reduce task is typically written to the
* {@link FileSystem} via
@ -156,7 +156,7 @@
* }
* }
* }
* </pre></blockquote></p>
* </pre></blockquote>
*
* @see Mapper
* @see Partitioner
@ -171,7 +171,7 @@ public interface Reducer<K2, V2, K3, V3> extends JobConfigurable, Closeable {
* <i>Reduces</i> values for a given key.
*
* <p>The framework calls this method for each
* <code>&lt;key, (list of values)></code> pair in the grouped inputs.
* <code>&lt;key, (list of values)&gt;</code> 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 <b>reuse</b> the key and value objects
* that are passed into the reduce, therefore the application should clone

View File

@ -178,7 +178,6 @@ void preempted(TaskAttemptID taskId, TaskStatus taskStatus)
*
* @param taskID task's id
* @return the most recent checkpoint (if any) for this task
* @throws IOException
*/
TaskCheckpointID getCheckpointID(TaskID taskID);

View File

@ -29,61 +29,61 @@
/**
* The ChainMapper class allows to use multiple Mapper classes within a single
* Map task.
* <p/>
* <p>
* 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.
* <p/>
* <p>
* 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.
* <p/>
* <p>
* 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.
* <p/>
* <p>
* Using the ChainMapper and the ChainReducer classes is possible to compose
* Map/Reduce jobs that look like <code>[MAP+ / REDUCE MAP*]</code>. And
* immediate benefit of this pattern is a dramatic reduction in disk IO.
* <p/>
* <p>
* 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.
* <p/>
* <p>
* ChainMapper usage pattern:
* <p/>
* <p>
* <pre>
* ...
* conf.setJobName("chain");
* conf.setInputFormat(TextInputFormat.class);
* conf.setOutputFormat(TextOutputFormat.class);
* <p/>
*
* JobConf mapAConf = new JobConf(false);
* ...
* ChainMapper.addMapper(conf, AMap.class, LongWritable.class, Text.class,
* Text.class, Text.class, true, mapAConf);
* <p/>
*
* JobConf mapBConf = new JobConf(false);
* ...
* ChainMapper.addMapper(conf, BMap.class, Text.class, Text.class,
* LongWritable.class, Text.class, false, mapBConf);
* <p/>
*
* JobConf reduceConf = new JobConf(false);
* ...
* ChainReducer.setReducer(conf, XReduce.class, LongWritable.class, Text.class,
* Text.class, Text.class, true, reduceConf);
* <p/>
*
* ChainReducer.addMapper(conf, CMap.class, Text.class, Text.class,
* LongWritable.class, Text.class, false, null);
* <p/>
*
* ChainReducer.addMapper(conf, DMap.class, LongWritable.class, Text.class,
* LongWritable.class, LongWritable.class, true, null);
* <p/>
*
* FileInputFormat.setInputPaths(conf, inDir);
* FileOutputFormat.setOutputPath(conf, outDir);
* ...
* <p/>
*
* JobClient jc = new JobClient(conf);
* RunningJob job = jc.submitJob(conf);
* ...
@ -95,21 +95,21 @@ public class ChainMapper implements Mapper {
/**
* Adds a Mapper class to the chain job's JobConf.
* <p/>
* <p>
* 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.
* <p/>
* <p>
* For the added Mapper the configuration given for it,
* <code>mapperConf</code>, have precedence over the job's JobConf. This
* precedence is in effect when the task is running.
* <p/>
* <p>
* 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
* <p/>
* <p>
*
* @param job job's JobConf to add the Mapper class.
* @param klass the Mapper class to add.
@ -148,7 +148,7 @@ public ChainMapper() {
/**
* Configures the ChainMapper and all the Mappers in the chain.
* <p/>
* <p>
* If this method is overriden <code>super.configure(...)</code> should be
* invoked at the beginning of the overwriter method.
*/
@ -171,7 +171,7 @@ public void map(Object key, Object value, OutputCollector output,
/**
* Closes the ChainMapper and all the Mappers in the chain.
* <p/>
* <p>
* If this method is overriden <code>super.close()</code> should be
* invoked at the end of the overwriter method.
*/

View File

@ -27,63 +27,63 @@
/**
* The ChainReducer class allows to chain multiple Mapper classes after a
* Reducer within the Reducer task.
* <p/>
* <p>
* 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.
* <p/>
* <p>
* 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.
* <p/>
* <p>
* 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.
* <p/>
* <p>
* Using the ChainMapper and the ChainReducer classes is possible to compose
* Map/Reduce jobs that look like <code>[MAP+ / REDUCE MAP*]</code>. And
* immediate benefit of this pattern is a dramatic reduction in disk IO.
* <p/>
* <p>
* 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.
* <p/>
* <p>
* ChainReducer usage pattern:
* <p/>
* <p>
* <pre>
* ...
* conf.setJobName("chain");
* conf.setInputFormat(TextInputFormat.class);
* conf.setOutputFormat(TextOutputFormat.class);
* <p/>
*
* JobConf mapAConf = new JobConf(false);
* ...
* ChainMapper.addMapper(conf, AMap.class, LongWritable.class, Text.class,
* Text.class, Text.class, true, mapAConf);
* <p/>
*
* JobConf mapBConf = new JobConf(false);
* ...
* ChainMapper.addMapper(conf, BMap.class, Text.class, Text.class,
* LongWritable.class, Text.class, false, mapBConf);
* <p/>
*
* JobConf reduceConf = new JobConf(false);
* ...
* ChainReducer.setReducer(conf, XReduce.class, LongWritable.class, Text.class,
* Text.class, Text.class, true, reduceConf);
* <p/>
*
* ChainReducer.addMapper(conf, CMap.class, Text.class, Text.class,
* LongWritable.class, Text.class, false, null);
* <p/>
*
* ChainReducer.addMapper(conf, DMap.class, LongWritable.class, Text.class,
* LongWritable.class, LongWritable.class, true, null);
* <p/>
*
* FileInputFormat.setInputPaths(conf, inDir);
* FileOutputFormat.setOutputPath(conf, outDir);
* ...
* <p/>
*
* JobClient jc = new JobClient(conf);
* RunningJob job = jc.submitJob(conf);
* ...
@ -95,18 +95,18 @@ public class ChainReducer implements Reducer {
/**
* Sets the Reducer class to the chain job's JobConf.
* <p/>
* <p>
* 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.
* <p/>
* <p>
* For the added Reducer the configuration given for it,
* <code>reducerConf</code>, have precedence over the job's JobConf. This
* precedence is in effect when the task is running.
* <p/>
* <p>
* 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.
@ -139,18 +139,18 @@ public static <K1, V1, K2, V2> void setReducer(JobConf job,
/**
* Adds a Mapper class to the chain job's JobConf.
* <p/>
* <p>
* 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.
* <p/>
* <p>
* For the added Mapper the configuration given for it,
* <code>mapperConf</code>, have precedence over the job's JobConf. This
* precedence is in effect when the task is running.
* <p/>
* <p>
* 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
* .
@ -191,7 +191,7 @@ public ChainReducer() {
/**
* Configures the ChainReducer, the Reducer and all the Mappers in the chain.
* <p/>
* <p>
* If this method is overriden <code>super.configure(...)</code> should be
* invoked at the beginning of the overwriter method.
*/
@ -215,7 +215,7 @@ public void reduce(Object key, Iterator values, OutputCollector output,
/**
* Closes the ChainReducer, the Reducer and all the Mappers in the chain.
* <p/>
* <p>
* If this method is overriden <code>super.close()</code> should be
* invoked at the end of the overwriter method.
*/

View File

@ -31,29 +31,29 @@
* than the job default output via the <code>OutputCollector</code> passed to
* the <code>map()</code> and <code>reduce()</code> methods of the
* <code>Mapper</code> and <code>Reducer</code> implementations.
* <p/>
* <p>
* Each additional output, or named output, may be configured with its own
* <code>OutputFormat</code>, with its own key class and with its own value
* class.
* <p/>
* <p>
* A named output can be a single file or a multi file. The later is refered as
* a multi named output.
* <p/>
* <p>
* A multi named output is an unbound set of files all sharing the same
* <code>OutputFormat</code>, key class and value class configuration.
* <p/>
* <p>
* When named outputs are used within a <code>Mapper</code> implementation,
* key/values written to a name output are not part of the reduce phase, only
* key/values written to the job <code>OutputCollector</code> are part of the
* reduce phase.
* <p/>
* <p>
* MultipleOutputs supports counters, by default the are disabled. The counters
* group is the {@link MultipleOutputs} class name.
* </p>
* 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.
* <p/>
* <p>
* Job configuration usage pattern is:
* <pre>
*
@ -82,7 +82,7 @@
*
* ...
* </pre>
* <p/>
* <p>
* Job configuration usage pattern is:
* <pre>
*
@ -271,7 +271,6 @@ public static Class<?> getNamedOutputValueClass(JobConf conf,
/**
* Adds a named output for the job.
* <p/>
*
* @param conf job conf to add the named output
* @param namedOutput named output name, it has to be a word, letters
@ -291,7 +290,6 @@ public static void addNamedOutput(JobConf conf, String namedOutput,
/**
* Adds a multi named output for the job.
* <p/>
*
* @param conf job conf to add the named output
* @param namedOutput named output name, it has to be a word, letters
@ -311,7 +309,6 @@ public static void addMultiNamedOutput(JobConf conf, String namedOutput,
/**
* Adds a named output for the job.
* <p/>
*
* @param conf job conf to add the named output
* @param namedOutput named output name, it has to be a word, letters
@ -339,9 +336,9 @@ private static void addNamedOutput(JobConf conf, String namedOutput,
/**
* Enables or disables counters for the named outputs.
* <p/>
* <p>
* By default these counters are disabled.
* <p/>
* <p>
* MultipleOutputs supports counters, by default the are disabled.
* The counters group is the {@link MultipleOutputs} class name.
* </p>
@ -358,9 +355,9 @@ public static void setCountersEnabled(JobConf conf, boolean enabled) {
/**
* Returns if the counters for the named outputs are enabled or not.
* <p/>
* <p>
* By default these counters are disabled.
* <p/>
* <p>
* MultipleOutputs supports counters, by default the are disabled.
* The counters group is the {@link MultipleOutputs} class name.
* </p>
@ -465,7 +462,6 @@ public void close(Reporter reporter) throws IOException {
/**
* Gets the output collector for a named output.
* <p/>
*
* @param namedOutput the named output name
* @param reporter the reporter
@ -480,7 +476,6 @@ public OutputCollector getCollector(String namedOutput, Reporter reporter)
/**
* Gets the output collector for a multi named output.
* <p/>
*
* @param namedOutput the named output name
* @param multiName the multi name part
@ -525,7 +520,7 @@ public void collect(Object key, Object value) throws IOException {
/**
* Closes all the opened named outputs.
* <p/>
* <p>
* If overriden subclasses must invoke <code>super.close()</code> at the
* end of their <code>close()</code>
*

View File

@ -32,7 +32,7 @@
/**
* A {@link Mapper} that maps text values into <token,freq> pairs. Uses
* A {@link Mapper} that maps text values into &lt;token,freq&gt; pairs. Uses
* {@link StringTokenizer} to break text into tokens.
*/
@InterfaceAudience.Public

View File

@ -60,7 +60,7 @@
* The developer using Aggregate will need only to provide a plugin class
* conforming to the following interface:
*
* public interface ValueAggregatorDescriptor { public ArrayList<Entry>
* public interface ValueAggregatorDescriptor { public ArrayList&lt;Entry&gt;
* generateKeyValPairs(Object key, Object value); public void
* configure(JobConfjob); }
*

View File

@ -45,7 +45,8 @@ public class ValueAggregatorReducer<K1 extends WritableComparable,
* driven computing is achieved. It is assumed that each aggregator's
* getReport method emits appropriate output for the aggregator. This
* may be further customiized.
* @value the values to be aggregated
* @param values
* the values to be aggregated
*/
public void reduce(Text key, Iterator<Text> values,
OutputCollector<Text, Text> output, Reporter reporter) throws IOException {

View File

@ -195,8 +195,8 @@ public InputSplit[] getSplits(JobConf job, int chunks) throws IOException {
* @param inputClass the class object implementing DBWritable, which is the
* Java object holding tuple fields.
* @param tableName The table to read data from
* @param conditions The condition which to select data with, eg. '(updated >
* 20070101 AND length > 0)'
* @param conditions The condition which to select data with, eg. '(updated &gt;
* 20070101 AND length &gt; 0)'
* @param orderBy the fieldNames in the orderBy clause.
* @param fieldNames The field names in the table
* @see #setInput(JobConf, Class, String, String)

View File

@ -134,6 +134,7 @@ Configuration getConf() {
/**
* Close the <code>Cluster</code>.
* @throws IOException
*/
public synchronized void close() throws IOException {
clientProtocolProvider.close(client);

View File

@ -40,15 +40,15 @@
* Slot capacity of the cluster.
* </li>
* <li>
* The number of currently occupied/reserved map & reduce slots.
* The number of currently occupied/reserved map and reduce slots.
* </li>
* <li>
* The number of currently running map & reduce tasks.
* The number of currently running map and reduce tasks.
* </li>
* <li>
* The number of job submissions.
* </li>
* </ol></p>
* </ol>
*
* <p>Clients can query for the latest <code>ClusterMetrics</code>, via
* {@link Cluster#getClusterStatus()}.</p>

View File

@ -123,11 +123,11 @@ public static FSDataOutputStream wrapIfNecessary(Configuration conf,
* "mapreduce.job.encrypted-intermediate-data.buffer.kb" Job configuration
* variable.
*
* If the value of 'length' is > -1, The InputStream is additionally wrapped
* in a LimitInputStream. CryptoStreams are late buffering in nature. This
* means they will always try to read ahead if they can. The LimitInputStream
* will ensure that the CryptoStream does not read past the provided length
* from the given Input Stream.
* If the value of 'length' is &gt; -1, The InputStream is additionally
* wrapped in a LimitInputStream. CryptoStreams are late buffering in nature.
* This means they will always try to read ahead if they can. The
* LimitInputStream will ensure that the CryptoStream does not read past the
* provided length from the given Input Stream.
*
* @param conf
* @param in

View File

@ -69,7 +69,7 @@
*
* // Submit the job, then poll for progress until the job is complete
* job.waitForCompletion(true);
* </pre></blockquote></p>
* </pre></blockquote>
*
*
*/

View File

@ -289,7 +289,6 @@ public Class<? extends Partitioner<?,?>> getPartitionerClass()
* Get the timestamps of the archives. Used by internal
* DistributedCache and MapReduce code.
* @return a string array of timestamps
* @throws IOException
*/
public String[] getArchiveTimestamps();
@ -297,7 +296,6 @@ public Class<? extends Partitioner<?,?>> getPartitionerClass()
* Get the timestamps of the files. Used by internal
* DistributedCache and MapReduce code.
* @return a string array of timestamps
* @throws IOException
*/
public String[] getFileTimestamps();

View File

@ -100,7 +100,7 @@ public static Path getJobDistCacheLibjars(Path jobSubmitDir) {
/**
* Initializes the staging directory and returns the path. It also
* keeps track of all necessary ownership & permissions
* keeps track of all necessary ownership and permissions
* @param cluster
* @param conf
*/

View File

@ -42,9 +42,9 @@
*
* <p>The framework first calls
* {@link #setup(org.apache.hadoop.mapreduce.Mapper.Context)}, followed by
* {@link #map(Object, Object, Context)}
* {@link #map(Object, Object, org.apache.hadoop.mapreduce.Mapper.Context)}
* for each key/value pair in the <code>InputSplit</code>. Finally
* {@link #cleanup(Context)} is called.</p>
* {@link #cleanup(org.apache.hadoop.mapreduce.Mapper.Context)} is called.</p>
*
* <p>All intermediate values associated with a given output key are
* subsequently grouped by the framework, and passed to a {@link Reducer} to
@ -84,9 +84,10 @@
* }
* }
* }
* </pre></blockquote></p>
* </pre></blockquote>
*
* <p>Applications may override the {@link #run(Context)} method to exert
* <p>Applications may override the
* {@link #run(org.apache.hadoop.mapreduce.Mapper.Context)} method to exert
* greater control on map processing e.g. multi-threaded <code>Mapper</code>s
* etc.</p>
*

View File

@ -39,14 +39,14 @@
* <ol>
* <li>
*
* <h4 id="Shuffle">Shuffle</h4>
* <b id="Shuffle">Shuffle</b>
*
* <p>The <code>Reducer</code> copies the sorted output from each
* {@link Mapper} using HTTP across the network.</p>
* </li>
*
* <li>
* <h4 id="Sort">Sort</h4>
* <b id="Sort">Sort</b>
*
* <p>The framework merge sorts <code>Reducer</code> inputs by
* <code>key</code>s
@ -55,7 +55,7 @@
* <p>The shuffle and sort phases occur simultaneously i.e. while outputs are
* being fetched they are merged.</p>
*
* <h5 id="SecondarySort">SecondarySort</h5>
* <b id="SecondarySort">SecondarySort</b>
*
* <p>To achieve a secondary sort on the values returned by the value
* iterator, the application should extend the key with the secondary
@ -83,10 +83,10 @@
* </li>
*
* <li>
* <h4 id="Reduce">Reduce</h4>
* <b id="Reduce">Reduce</b>
*
* <p>In this phase the
* {@link #reduce(Object, Iterable, Context)}
* {@link #reduce(Object, Iterable, org.apache.hadoop.mapreduce.Reducer.Context)}
* method is called for each <code>&lt;key, (collection of values)&gt;</code> in
* the sorted inputs.</p>
* <p>The output of the reduce task is typically written to a
@ -113,7 +113,7 @@
* context.write(key, result);
* }
* }
* </pre></blockquote></p>
* </pre></blockquote>
*
* @see Mapper
* @see Partitioner

View File

@ -115,7 +115,7 @@
* }
* }
*
* </pre></blockquote></p>
* </pre></blockquote>
*
* It is also very common to use the DistributedCache by using
* {@link org.apache.hadoop.util.GenericOptionsParser}.
@ -235,7 +235,6 @@ private static long[] parseTimestamps(String[] strs) {
* DistributedCache and MapReduce code.
* @param conf The configuration which stored the timestamps
* @return a long array of timestamps
* @throws IOException
* @deprecated Use {@link JobContext#getArchiveTimestamps()} instead
*/
@Deprecated
@ -250,7 +249,6 @@ public static long[] getArchiveTimestamps(Configuration conf) {
* DistributedCache and MapReduce code.
* @param conf The configuration which stored the timestamps
* @return a long array of timestamps
* @throws IOException
* @deprecated Use {@link JobContext#getFileTimestamps()} instead
*/
@Deprecated
@ -434,7 +432,6 @@ private static boolean[] parseBooleans(String[] strs) {
* internal DistributedCache and MapReduce code.
* @param conf The configuration which stored the timestamps
* @return a string array of booleans
* @throws IOException
*/
public static boolean[] getFileVisibilities(Configuration conf) {
return parseBooleans(conf.getStrings(MRJobConfig.CACHE_FILE_VISIBILITIES));

View File

@ -60,7 +60,7 @@
* The developer using Aggregate will need only to provide a plugin class
* conforming to the following interface:
*
* public interface ValueAggregatorDescriptor { public ArrayList<Entry>
* public interface ValueAggregatorDescriptor { public ArrayList&lt;Entry&gt;
* generateKeyValPairs(Object key, Object value); public void
* configure(Configuration conf); }
*

View File

@ -600,7 +600,7 @@ protected static Configuration getChainElementConf(Configuration jobConf,
/**
* Adds a Mapper class to the chain job.
*
* <p/>
* <p>
* The configuration properties of the chain job have precedence over the
* configuration properties of the Mapper.
*
@ -738,7 +738,7 @@ protected static void setMapperConf(boolean isMap, Configuration jobConf,
/**
* Sets the Reducer class to the chain job.
*
* <p/>
* <p>
* The configuration properties of the chain job have precedence over the
* configuration properties of the Reducer.
*

View File

@ -57,24 +57,24 @@
* ChainMapper, this is done by the addMapper for the last mapper in the chain.
* </p>
* ChainMapper usage pattern:
* <p/>
* <p>
*
* <pre>
* ...
* Job = new Job(conf);
* <p/>
*
* Configuration mapAConf = new Configuration(false);
* ...
* ChainMapper.addMapper(job, AMap.class, LongWritable.class, Text.class,
* Text.class, Text.class, true, mapAConf);
* <p/>
*
* Configuration mapBConf = new Configuration(false);
* ...
* ChainMapper.addMapper(job, BMap.class, Text.class, Text.class,
* LongWritable.class, Text.class, false, mapBConf);
* <p/>
*
* ...
* <p/>
*
* job.waitForComplettion(true);
* ...
* </pre>

View File

@ -50,7 +50,7 @@
* 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.
* </p>
* </p> Using the ChainMapper and the ChainReducer classes is possible to
* <p> Using the ChainMapper and the ChainReducer classes is possible to
* compose Map/Reduce jobs that look like <code>[MAP+ / REDUCE MAP*]</code>. And
* immediate benefit of this pattern is a dramatic reduction in disk IO. </p>
* <p>
@ -59,26 +59,26 @@
* element in the chain.
* </p>
* ChainReducer usage pattern:
* <p/>
* <p>
*
* <pre>
* ...
* Job = new Job(conf);
* ....
* <p/>
*
* Configuration reduceConf = new Configuration(false);
* ...
* ChainReducer.setReducer(job, XReduce.class, LongWritable.class, Text.class,
* Text.class, Text.class, true, reduceConf);
* <p/>
*
* ChainReducer.addMapper(job, CMap.class, Text.class, Text.class,
* LongWritable.class, Text.class, false, null);
* <p/>
*
* ChainReducer.addMapper(job, DMap.class, LongWritable.class, Text.class,
* LongWritable.class, LongWritable.class, true, null);
* <p/>
*
* ...
* <p/>
*
* job.waitForCompletion(true);
* ...
* </pre>

View File

@ -319,7 +319,7 @@ protected String getCountQuery() {
* Java object holding tuple fields.
* @param tableName The table to read data from
* @param conditions The condition which to select data with,
* eg. '(updated > 20070101 AND length > 0)'
* eg. '(updated &gt; 20070101 AND length &gt; 0)'
* @param orderBy the fieldNames in the orderBy clause.
* @param fieldNames The field names in the table
* @see #setInput(Job, Class, String, String)

View File

@ -73,7 +73,7 @@
* timestamp = resultSet.getLong(2);
* }
* }
* </pre></p>
* </pre>
*/
@InterfaceAudience.Public
@InterfaceStability.Stable

View File

@ -144,7 +144,7 @@ public void remove() {
/**
* Convert Tuple to String as in the following.
* <tt>[<child1>,<child2>,...,<childn>]</tt>
* <tt>[&lt;child1&gt;,&lt;child2&gt;,...,&lt;childn&gt;]</tt>
*/
public String toString() {
StringBuffer buf = new StringBuffer("[");

View File

@ -44,15 +44,15 @@
* Multithreaded implementation for @link org.apache.hadoop.mapreduce.Mapper.
* <p>
* It can be used instead of the default implementation,
* @link org.apache.hadoop.mapred.MapRunner, when the Map operation is not CPU
* {@link org.apache.hadoop.mapred.MapRunner}, when the Map operation is not CPU
* bound in order to improve throughput.
* <p>
* Mapper implementations using this MapRunnable must be thread-safe.
* <p>
* The Map-Reduce job has to be configured with the mapper to use via
* {@link #setMapperClass(Configuration, Class)} and
* {@link #setMapperClass(Job, Class)} and
* the number of thread the thread-pool can use with the
* {@link #getNumberOfThreads(Configuration) method. The default
* {@link #getNumberOfThreads(JobContext)} method. The default
* value is 10 threads.
* <p>
*/

View File

@ -181,7 +181,7 @@ public static Path getOutputPath(JobContext job) {
* Get the {@link Path} to the task's temporary output directory
* for the map-reduce job
*
* <h4 id="SideEffectFiles">Tasks' Side-Effect Files</h4>
* <b id="SideEffectFiles">Tasks' Side-Effect Files</b>
*
* <p>Some applications need to create/write-to side-files, which differ from
* the actual job-outputs.

View File

@ -81,7 +81,7 @@
* <p>
* Usage in Reducer:
* <pre>
* <K, V> String generateFileName(K k, V v) {
* &lt;K, V&gt; String generateFileName(K k, V v) {
* return k.toString() + "_" + v.toString();
* }
*
@ -124,16 +124,16 @@
* </p>
*
* <pre>
* private MultipleOutputs<Text, Text> out;
* private MultipleOutputs&lt;Text, Text&gt; out;
*
* public void setup(Context context) {
* out = new MultipleOutputs<Text, Text>(context);
* out = new MultipleOutputs&lt;Text, Text&gt;(context);
* ...
* }
*
* public void reduce(Text key, Iterable<Text> values, Context context) throws IOException, InterruptedException {
* public void reduce(Text key, Iterable&lt;Text&gt; values, Context context) throws IOException, InterruptedException {
* for (Text t : values) {
* out.write(key, t, generateFileName(<<i>parameter list...</i>>));
* out.write(key, t, generateFileName(&lt;<i>parameter list...</i>&gt;));
* }
* }
*
@ -294,7 +294,6 @@ private static Class<?> getNamedOutputValueClass(
/**
* Adds a named output for the job.
* <p/>
*
* @param job job to add the named output
* @param namedOutput named output name, it has to be a word, letters

View File

@ -64,7 +64,7 @@
* <li>{@link #setOffsets}</li>
* <li>{@link #setLeftOffset}</li>
* <li>{@link #setRightOffset}</li>
* </ul></p>
* </ul>
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving

View File

@ -374,7 +374,6 @@ private static String[] toTimestampStrs(long[] timestamps) {
* Get the timestamps of the archives. Used by internal
* DistributedCache and MapReduce code.
* @return a string array of timestamps
* @throws IOException
*/
public String[] getArchiveTimestamps() {
return toTimestampStrs(DistributedCache.getArchiveTimestamps(conf));
@ -384,7 +383,6 @@ public String[] getArchiveTimestamps() {
* Get the timestamps of the files. Used by internal
* DistributedCache and MapReduce code.
* @return a string array of timestamps
* @throws IOException
*/
public String[] getFileTimestamps() {
return toTimestampStrs(DistributedCache.getFileTimestamps(conf));

View File

@ -42,7 +42,7 @@
* random sequence of words.
* In order for this program to generate data for terasort with a 5-10 words
* per key and 20-100 words per value, have the following config:
* <xmp>
* <pre>{@code
* <?xml version="1.0"?>
* <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
* <configuration>
@ -66,7 +66,7 @@
* <name>mapreduce.randomtextwriter.totalbytes</name>
* <value>1099511627776</value>
* </property>
* </configuration></xmp>
* </configuration>}</pre>
*
* Equivalently, {@link RandomTextWriter} also supports all the above options
* and ones supported by {@link Tool} via the command-line.

View File

@ -47,7 +47,7 @@
* random binary sequence file of BytesWritable.
* In order for this program to generate data for terasort with 10-byte keys
* and 90-byte values, have the following config:
* <xmp>
* <pre>{@code
* <?xml version="1.0"?>
* <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
* <configuration>
@ -71,8 +71,7 @@
* <name>mapreduce.randomwriter.totalbytes</name>
* <value>1099511627776</value>
* </property>
* </configuration></xmp>
*
* </configuration>}</pre>
* Equivalently, {@link RandomWriter} also supports all the above options
* and ones supported by {@link GenericOptionsParser} via the command-line.
*/

View File

@ -199,7 +199,7 @@ public Text getCurrentValue() throws IOException, InterruptedException {
}
/**
* This Mapper is similar to the one in {@link WordCount.MapClass}.
* This Mapper is similar to the one in {@link WordCount.TokenizerMapper}.
*/
public static class MapClass extends
Mapper<WordOffset, Text, Text, IntWritable> {

View File

@ -50,7 +50,7 @@
* where $S=[0,1)^2$ is a unit square,
* $x=(x_1,x_2)$ is a 2-dimensional point,
* and $f$ is a function describing the inscribed circle of the square $S$,
* $f(x)=1$ if $(2x_1-1)^2+(2x_2-1)^2 <= 1$ and $f(x)=0$, otherwise.
* $f(x)=1$ if $(2x_1-1)^2+(2x_2-1)^2 &lt;= 1$ and $f(x)=0$, otherwise.
* It is easy to see that Pi is equal to $4I$.
* So an approximation of Pi is obtained once $I$ is evaluated numerically.
*
@ -155,7 +155,7 @@ public static class QmcMapper extends
/** Map method.
* @param offset samples starting from the (offset+1)th sample.
* @param size the number of samples for this map
* @param context output {ture->numInside, false->numOutside}
* @param context output {ture-&gt;numInside, false-&gt;numOutside}
*/
public void map(LongWritable offset,
LongWritable size,

View File

@ -42,7 +42,7 @@
* random sequence of words.
* In order for this program to generate data for terasort with a 5-10 words
* per key and 20-100 words per value, have the following config:
* <xmp>
* <pre>{@code
* <?xml version="1.0"?>
* <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
* <configuration>
@ -66,7 +66,7 @@
* <name>mapreduce.randomtextwriter.totalbytes</name>
* <value>1099511627776</value>
* </property>
* </configuration></xmp>
* </configuration>}</pre>
*
* Equivalently, {@link RandomTextWriter} also supports all the above options
* and ones supported by {@link Tool} via the command-line.

View File

@ -47,7 +47,7 @@
* random binary sequence file of BytesWritable.
* In order for this program to generate data for terasort with 10-byte keys
* and 90-byte values, have the following config:
* <xmp>
* <pre>{@code
* <?xml version="1.0"?>
* <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
* <configuration>
@ -71,8 +71,7 @@
* <name>mapreduce.randomwriter.totalbytes</name>
* <value>1099511627776</value>
* </property>
* </configuration></xmp>
*
* </configuration>}</pre>
* Equivalently, {@link RandomWriter} also supports all the above options
* and ones supported by {@link GenericOptionsParser} via the command-line.
*/

View File

@ -74,7 +74,7 @@ public int getSecond() {
}
/**
* Read the two integers.
* Encoded as: MIN_VALUE -> 0, 0 -> -MIN_VALUE, MAX_VALUE-> -1
* Encoded as: MIN_VALUE -&gt; 0, 0 -&gt; -MIN_VALUE, MAX_VALUE-&gt; -1
*/
@Override
public void readFields(DataInput in) throws IOException {

View File

@ -35,7 +35,7 @@
* A map/reduce program that uses a BBP-type method to compute exact
* binary digits of Pi.
* This program is designed for computing the n th bit of Pi,
* for large n, say n >= 10^8.
* for large n, say n &gt;= 10^8.
* For computing lower bits of Pi, consider using bbp.
*
* The actually computation is done by DistSum jobs.

View File

@ -78,7 +78,7 @@ public static double addMod(double x, final double a) {
return x >= 1? x - 1: x < 0? x + 1: x;
}
/** Given 0 < x < y,
/** Given 0 &lt; x &lt; y,
* return x^(-1) mod y.
*/
public static long modInverse(final long x, final long y) {

View File

@ -28,7 +28,7 @@
/**
* A single process data generator for the terasort data. Based on gensort.c
* version 1.1 (3 Mar 2009) from Chris Nyberg <chris.nyberg@ordinal.com>.
* version 1.1 (3 Mar 2009) from Chris Nyberg &lt;chris.nyberg@ordinal.com&gt;.
*/
public class GenSort {

View File

@ -38,10 +38,10 @@
/**
* The CopyListing abstraction is responsible for how the list of
* sources and targets is constructed, for DistCp's copy function.
* The copy-listing should be a SequenceFile<Text, CopyListingFileStatus>,
* located at the path specified to buildListing(),
* each entry being a pair of (Source relative path, source file status),
* all the paths being fully qualified.
* The copy-listing should be a
* SequenceFile&lt;Text, CopyListingFileStatus&gt;, located at the path
* specified to buildListing(), each entry being a pair of (Source relative
* path, source file status), all the paths being fully qualified.
*/
public abstract class CopyListing extends Configured {
@ -95,8 +95,8 @@ public final void buildListing(Path pathToListFile,
* Validate input and output paths
*
* @param options - Input options
* @throws InvalidInputException: If inputs are invalid
* @throws IOException: any Exception with FS
* @throws InvalidInputException If inputs are invalid
* @throws IOException any Exception with FS
*/
protected abstract void validatePaths(DistCpOptions options)
throws IOException, InvalidInputException;
@ -105,7 +105,7 @@ protected abstract void validatePaths(DistCpOptions options)
* The interface to be implemented by sub-classes, to create the source/target file listing.
* @param pathToListFile Path on HDFS where the listing file is written.
* @param options Input Options for DistCp (indicating source/target paths.)
* @throws IOException: Thrown on failure to create the listing file.
* @throws IOException Thrown on failure to create the listing file.
*/
protected abstract void doBuildListing(Path pathToListFile,
DistCpOptions options) throws IOException;

View File

@ -76,7 +76,7 @@ public class DistCp extends Configured implements Tool {
* (E.g. source-paths, target-location, etc.)
* @param inputOptions Options (indicating source-paths, target-location.)
* @param configuration The Hadoop configuration against which the Copy-mapper must run.
* @throws Exception, on failure.
* @throws Exception
*/
public DistCp(Configuration configuration, DistCpOptions inputOptions) throws Exception {
Configuration config = new Configuration(configuration);
@ -142,7 +142,7 @@ public int run(String[] argv) {
* Implements the core-execution. Creates the file-list for copy,
* and launches the Hadoop-job, to do the copy.
* @return Job handle
* @throws Exception, on failure.
* @throws Exception
*/
public Job execute() throws Exception {
assert inputOptions != null;

View File

@ -105,7 +105,7 @@ public enum DistCpOptionSwitch {
* Copy all the source files and commit them atomically to the target
* This is typically useful in cases where there is a process
* polling for availability of a file/dir. This option is incompatible
* with SYNC_FOLDERS & DELETE_MISSING
* with SYNC_FOLDERS and DELETE_MISSING
*/
ATOMIC_COMMIT(DistCpConstants.CONF_LABEL_ATOMIC_COPY,
new Option("atomic", false, "Commit all changes or none")),

View File

@ -63,7 +63,7 @@ protected String[] flatten(Options options, String[] arguments, boolean stopAtNo
* @param args Command-line arguments (excluding the options consumed
* by the GenericOptionsParser).
* @return The Options object, corresponding to the specified command-line.
* @throws IllegalArgumentException: Thrown if the parse fails.
* @throws IllegalArgumentException Thrown if the parse fails.
*/
public static DistCpOptions parse(String args[]) throws IllegalArgumentException {

View File

@ -70,7 +70,7 @@ public CopyCommitter(Path outputPath, TaskAttemptContext context) throws IOExcep
this.taskAttemptContext = context;
}
/** @inheritDoc */
/** {@inheritDoc} */
@Override
public void commitJob(JobContext jobContext) throws IOException {
Configuration conf = jobContext.getConfiguration();
@ -102,7 +102,7 @@ public void commitJob(JobContext jobContext) throws IOException {
}
}
/** @inheritDoc */
/** {@inheritDoc} */
@Override
public void abortJob(JobContext jobContext,
JobStatus.State state) throws IOException {

View File

@ -45,7 +45,7 @@
/**
* Mapper class that executes the DistCp copy operation.
* Implements the o.a.h.mapreduce.Mapper<> interface.
* Implements the o.a.h.mapreduce.Mapper interface.
*/
public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text> {
@ -182,10 +182,11 @@ private Path findCacheFile(Path[] cacheFiles, String fileName) {
}
/**
* Implementation of the Mapper<>::map(). Does the copy.
* Implementation of the Mapper::map(). Does the copy.
* @param relPath The target path.
* @param sourceFileStatus The source path.
* @throws IOException
* @throws InterruptedException
*/
@Override
public void map(Text relPath, CopyListingFileStatus sourceFileStatus,

View File

@ -97,13 +97,13 @@ private static Path getCommitDirectory(Configuration conf) {
}
}
/** @inheritDoc */
/** {@inheritDoc} */
@Override
public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException {
return new CopyCommitter(getOutputPath(context), context);
}
/** @inheritDoc */
/** {@inheritDoc} */
@Override
public void checkOutputSpecs(JobContext context) throws IOException {
Configuration conf = context.getConfiguration();

View File

@ -84,8 +84,7 @@ public RetriableFileCopyCommand(boolean skipCrc, String description,
* This is the actual copy-implementation.
* @param arguments Argument-list to the command.
* @return Number of bytes copied.
* @throws Exception: CopyReadException, if there are read-failures. All other
* failures are IOExceptions.
* @throws Exception
*/
@SuppressWarnings("unchecked")
@Override

View File

@ -38,7 +38,7 @@
import java.util.ArrayList;
/**
* UniformSizeInputFormat extends the InputFormat<> class, to produce
* UniformSizeInputFormat extends the InputFormat class, to produce
* input-splits for DistCp.
* It looks at the copy-listing and groups the contents into input-splits such
* that the total-number of bytes to be copied for each input split is
@ -55,7 +55,7 @@ public class UniformSizeInputFormat
* approximately equal.
* @param context JobContext for the job.
* @return The list of uniformly-distributed input-splits.
* @throws IOException: On failure.
* @throws IOException
* @throws InterruptedException
*/
@Override

View File

@ -64,7 +64,7 @@ public class DynamicInputFormat<K, V> extends InputFormat<K, V> {
* tasks.
* @param jobContext JobContext for the map job.
* @return The list of (empty) dynamic input-splits.
* @throws IOException, on failure.
* @throws IOException
* @throws InterruptedException
*/
@Override
@ -343,7 +343,7 @@ static int getNumEntriesPerChunk(Configuration configuration) {
* @param inputSplit The split for which the RecordReader is required.
* @param taskAttemptContext TaskAttemptContext for the current attempt.
* @return DynamicRecordReader instance.
* @throws IOException, on failure.
* @throws IOException
* @throws InterruptedException
*/
@Override

View File

@ -57,7 +57,7 @@ public class DynamicRecordReader<K, V> extends RecordReader<K, V> {
* RecordReader to read from chunks.
* @param inputSplit The InputSplit for the map. Ignored entirely.
* @param taskAttemptContext The AttemptContext.
* @throws IOException, on failure.
* @throws IOException
* @throws InterruptedException
*/
@Override
@ -88,7 +88,7 @@ private int getTotalNumRecords() {
* been completely exhausted, an new chunk is acquired and read,
* transparently.
* @return True, if the nextValue() could be traversed to. False, otherwise.
* @throws IOException, on failure.
* @throws IOException
* @throws InterruptedException
*/
@Override
@ -130,7 +130,7 @@ public boolean nextKeyValue()
/**
* Implementation of RecordReader::getCurrentKey().
* @return The key of the current record. (i.e. the source-path.)
* @throws IOException, on failure.
* @throws IOException
* @throws InterruptedException
*/
@Override
@ -142,7 +142,7 @@ public K getCurrentKey()
/**
* Implementation of RecordReader::getCurrentValue().
* @return The value of the current record. (i.e. the target-path.)
* @throws IOException, on failure.
* @throws IOException
* @throws InterruptedException
*/
@Override
@ -154,7 +154,7 @@ public V getCurrentValue()
/**
* Implementation of RecordReader::getProgress().
* @return A fraction [0.0,1.0] indicating the progress of a DistCp mapper.
* @throws IOException, on failure.
* @throws IOException
* @throws InterruptedException
*/
@Override
@ -192,7 +192,7 @@ private int getNumChunksLeft() throws IOException {
/**
* Implementation of RecordReader::close().
* Closes the RecordReader.
* @throws IOException, on failure.
* @throws IOException
*/
@Override
public void close()

View File

@ -65,7 +65,7 @@ public class DistCpUtils {
* @param path The path of the file whose size is sought.
* @param configuration Configuration, to retrieve the appropriate FileSystem.
* @return The file-size, in number of bytes.
* @throws IOException, on failure.
* @throws IOException
*/
public static long getFileSize(Path path, Configuration configuration)
throws IOException {

View File

@ -77,7 +77,7 @@ public RetriableCommand(String description, RetryPolicy retryPolicy) {
* 2. the command may no longer be retried (e.g. runs out of retry-attempts).
* @param arguments The list of arguments for the command.
* @return Generic "Object" from doExecute(), on success.
* @throws IOException, IOException, on complete failure.
* @throws Exception
*/
public Object execute(Object... arguments) throws Exception {
Exception latestException;

View File

@ -62,7 +62,7 @@ public void close() throws IOException {
rawStream.close();
}
/** @inheritDoc */
/** {@inheritDoc} */
@Override
public int read() throws IOException {
throttle();
@ -73,7 +73,7 @@ public int read() throws IOException {
return data;
}
/** @inheritDoc */
/** {@inheritDoc} */
@Override
public int read(byte[] b) throws IOException {
throttle();
@ -84,7 +84,7 @@ public int read(byte[] b) throws IOException {
return readLen;
}
/** @inheritDoc */
/** {@inheritDoc} */
@Override
public int read(byte[] b, int off, int len) throws IOException {
throttle();
@ -155,7 +155,7 @@ public long getTotalSleepTime() {
return totalSleepTime;
}
/** @inheritDoc */
/** {@inheritDoc} */
@Override
public String toString() {
return "ThrottledInputStream{" +

View File

@ -60,7 +60,9 @@
* 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 <directory to archive logs> -analysis <directory> -logs <log-list uri> -grep <pattern> -sort <col1, col2> -separator <separator>
* Logalyzer -archive -archiveDir &lt;directory to archive logs&gt; -analysis
* &lt;directory&gt; -logs &lt;log-list uri&gt; -grep &lt;pattern&gt; -sort
* &lt;col1, col2&gt; -separator &lt;separator&gt;
* <p>
*/

View File

@ -35,7 +35,7 @@
* {@link ResourceUsageEmulatorPlugin} is also configured with a feedback module
* i.e a {@link ResourceCalculatorPlugin}, to monitor the current resource
* usage. {@link ResourceUsageMetrics} decides the final resource usage value to
* emulate. {@link Progressive} keeps track of the task's progress.</p>
* emulate. {@link Progressive} keeps track of the task's progress.
*
* <br><br>
*

View File

@ -31,10 +31,10 @@
/**
* This class implements the binding logic between Hadoop configurations
* and the swift rest client.
* <p/>
* <p>
* The swift rest client takes a Properties instance containing
* the string values it uses to bind to a swift endpoint.
* <p/>
* <p>
* This class extracts the values for a specific filesystem endpoint
* and then builds an appropriate Properties file.
*/
@ -188,7 +188,7 @@ private static void set(Properties props, String key, String optVal) {
/**
* Copy a (trimmed) property from the configuration file to the properties file.
* <p/>
* <p>
* If marked as required and not found in the configuration, an
* exception is raised.
* If not required -and missing- then the property will not be set.

View File

@ -1061,10 +1061,9 @@ protected void setup(PutMethod method) throws
* Authenticate to Openstack Keystone
* As well as returning the access token, the member fields {@link #token},
* {@link #endpointURI} and {@link #objectLocationURI} are set up for re-use.
* <p/>
* <p>
* This method is re-entrant -if more than one thread attempts to authenticate
* neither will block -but the field values with have those of the last caller.
* <p/>
*
* @return authenticated access token
*/
@ -1575,6 +1574,7 @@ private static StringRequestEntity toJsonEntity(String data) throws
* @param path path to object
* @param endpointURI damain url e.g. http://domain.com
* @return valid URI for object
* @throws SwiftException
*/
public static URI pathToURI(SwiftObjectPath path,
URI endpointURI) throws SwiftException {
@ -1820,7 +1820,7 @@ public boolean isLocationAware() {
/**
* Get the blocksize of this filesystem
* @return a blocksize >0
* @return a blocksize &gt; 0
*/
public long getBlocksizeKB() {
return blocksizeKB;

View File

@ -225,10 +225,10 @@ public boolean isDirectory(Path f) throws IOException {
* Return an array containing hostnames, offset and size of
* portions of the given file. For a nonexistent
* file or regions, null will be returned.
* <p/>
* <p>
* This call is most helpful with DFS, where it returns
* hostnames of machines that contain the given file.
* <p/>
* <p>
* The FileSystem will simply return an elt containing 'localhost'.
*/
@Override
@ -645,7 +645,7 @@ public List<DurationStats> getOperationStatistics() {
/**
* Low level method to do a deep listing of all entries, not stopping
* at the next directory entry. This is to let tests be confident that
* recursive deletes &c really are working.
* recursive deletes really are working.
* @param path path to recurse down
* @param newest ask for the newest data, potentially slower than not.
* @return a potentially empty array of file status

View File

@ -518,7 +518,7 @@ public boolean objectExists(SwiftObjectPath path) throws IOException {
* Rename through copy-and-delete. this is a consequence of the
* Swift filesystem using the path as the hash
* into the Distributed Hash Table, "the ring" of filenames.
* <p/>
* <p>
* Because of the nature of the operation, it is not atomic.
*
* @param src source file/dir
@ -847,7 +847,7 @@ public static List<URI> extractUris(String json, Path path) throws
}
/**
* Insert a throttled wait if the throttle delay >0
* Insert a throttled wait if the throttle delay &gt; 0
* @throws InterruptedIOException if interrupted during sleep
*/
public void throttle() throws InterruptedIOException {
@ -878,7 +878,7 @@ public List<DurationStats> getOperationStatistics() {
* raised. This lets the caller distinguish a file not found with
* other reasons for failure, so handles race conditions in recursive
* directory deletes better.
* <p/>
* <p>
* The problem being addressed is: caller A requests a recursive directory
* of directory /dir ; caller B requests a delete of a file /dir/file,
* between caller A enumerating the files contents, and requesting a delete

View File

@ -236,7 +236,7 @@ public static void compareByteArrays(byte[] src,
/**
* Convert a byte to a character for printing. If the
* byte value is < 32 -and hence unprintable- the byte is
* byte value is &lt; 32 -and hence unprintable- the byte is
* returned as a two digit hex value
* @param b byte
* @return the printable character string

View File

@ -45,12 +45,12 @@ public interface InputDemuxer extends Closeable {
public void bindTo(Path path, Configuration conf) throws IOException;
/**
* Get the next <name, input> pair. The name should preserve the original job
* Get the next &lt;name, input&gt; 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 <name, input> pair.
* @return the next &lt;name, input&gt; pair.
*/
public Pair<String, InputStream> getNext() throws IOException;
}

View File

@ -67,8 +67,9 @@
* ignoring user-specific and hard-to-parse keys but also provides a consistent
* view for all possible inputs. So if users invoke the
* {@link #parseJobProperty(String, String)} API with either
* <"mapreduce.job.user.name", "bob"> or <"user.name", "bob">, then the result
* would be a {@link UserName} {@link DataType} wrapping the user-name "bob".
* &lt;"mapreduce.job.user.name", "bob"&gt; or &lt;"user.name", "bob"&gt;,
* then the result would be a {@link UserName} {@link DataType} wrapping
* the user-name "bob".
*/
@SuppressWarnings("deprecation")
public class MapReduceJobPropertiesParser implements JobPropertyParser {

View File

@ -41,7 +41,7 @@
* String conf_filename = .. // assume the job configuration filename here
*
* // construct a list of interesting properties
* List<String> interestedProperties = new ArrayList<String>();
* List&lt;String&gt; interestedProperties = new ArrayList&lt;String&gt;();
* interestedProperties.add("mapreduce.job.name");
*
* JobConfigurationParser jcp =
@ -154,7 +154,7 @@
* TopologyBuilder tb = new TopologyBuilder();
*
* // construct a list of interesting properties
* List<String> interestingProperties = new ArrayList<Strng>();
* List&lt;String&gt; interestingProperties = new ArrayList%lt;String&gt;();
* // add the interesting properties here
* interestingProperties.add("mapreduce.job.name");
*
@ -207,7 +207,7 @@
* JobBuilder jb = new JobBuilder(jobID);
*
* // construct a list of interesting properties
* List<String> interestingProperties = new ArrayList<Strng>();
* List&lt;String&gt; interestingProperties = new ArrayList%lt;String&gt;();
* // add the interesting properties here
* interestingProperties.add("mapreduce.job.name");
*
@ -269,7 +269,7 @@
* TopologyBuilder tb = new TopologyBuilder();
*
* // construct a list of interesting properties
* List<String> interestingProperties = new ArrayList<Strng>();
* List&lt;String&gt; interestingProperties = new ArrayList%lt;String&gt;();
* // add the interesting properties here
* interestingProperties.add("mapreduce.job.name");
*