瀏覽代碼

MAPREDUCE-7014. Fix java doc errors in jdk1.8. Contributed by Steve Loughran.

Rohith Sharma K S 7 年之前
父節點
當前提交
3cd75845da

+ 6 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitterFactory.java

@@ -39,12 +39,12 @@ import org.apache.hadoop.util.ReflectionUtils;
  *
  * Algorithm:
  * <ol>
- *   <ul>If an explicit committer factory is named, it is used.</ul>
- *   <ul>The output path is examined.
+ *   <li>If an explicit committer factory is named, it is used.</li>
+ *   <li>The output path is examined.
  *   If is non null and there is an explicit schema for that filesystem,
- *   its factory is instantiated.</ul>
- *   <ul>Otherwise, an instance of {@link FileOutputCommitter} is
- *   created.</ul>
+ *   its factory is instantiated.</li>
+ *   <li>Otherwise, an instance of {@link FileOutputCommitter} is
+ *   created.</li>
  * </ol>
  *
  * In {@link FileOutputFormat}, the created factory has its method
@@ -186,7 +186,7 @@ public class PathOutputCommitterFactory extends Configured {
   }
 
   /**
-   * Create the committer factory for a task attempt & destination, then
+   * Create the committer factory for a task attempt and destination, then
    * create the committer from it.
    * @param outputPath the task's output path, or or null if no output path
    * has been defined.

+ 3 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Invoker.java

@@ -55,7 +55,7 @@ import org.apache.hadoop.io.retry.RetryPolicy;
  *
  * The static {@link #quietly(String, String, VoidOperation)} and
  * {@link #quietlyEval(String, String, Operation)} calls exist to take any
- * operation and quietly catch & log at debug. The return value of
+ * operation and quietly catch and log at debug. The return value of
  * {@link #quietlyEval(String, String, Operation)} is a java 8 optional,
  * which can then be used in java8-expressions.
  */
@@ -390,9 +390,11 @@ public class Invoker {
    * Execute an operation; any exception raised is caught and
    * logged at debug.
    * The result is only non-empty if the operation succeeded
+   * @param <T> type to return
    * @param action action to execute
    * @param path path (for exception construction)
    * @param operation operation
+   * @return the result of a successful operation
    */
   public static <T> Optional<T> quietlyEval(String action,
       String path,

+ 1 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java

@@ -1247,7 +1247,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities {
    * @param request last list objects request to continue
    * @param prevResult last paged result to continue from
    * @return the next result object
-   * @throws IOException: none, just there for retryUntranslated.
+   * @throws IOException none, just there for retryUntranslated.
    */
   @Retries.RetryRaw
   protected S3ListResult continueListObjects(S3ListRequest request,

+ 4 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java

@@ -949,8 +949,10 @@ public final class S3AUtils {
   /**
    * Map an operation to every {@link LocatedFileStatus} in a remote
    * iterator, returning a list of the results.
+   * @param <T> return type of map
    * @param iterator iterator from a list
    * @param eval closure to evaluate
+   * @return the list of mapped results.
    * @throws IOException anything in the closure, or iteration logic.
    */
   public static <T> List<T> mapLocatedFiles(
@@ -965,8 +967,10 @@ public final class S3AUtils {
   /**
    * Map an operation to every {@link LocatedFileStatus} in a remote
    * iterator, returning a list of the all results which were not empty.
+   * @param <T> return type of map
    * @param iterator iterator from a list
    * @param eval closure to evaluate
+   * @return the flattened list of mapped results.
    * @throws IOException anything in the closure, or iteration logic.
    */
   public static <T> List<T> flatmapLocatedFiles(

+ 1 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java

@@ -179,6 +179,7 @@ public class WriteOperationHelper {
   /**
    * Start the multipart upload process.
    * Retry policy: retrying, translated.
+   * @param destKey destination of upload
    * @return the upload result containing the ID
    * @throws IOException IO problem
    */

+ 1 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java

@@ -523,6 +523,7 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter {
    * Abort all pending uploads to the destination directory during
    * job cleanup operations.
    * @param suppressExceptions should exceptions be suppressed
+   * @throws IOException IO problem
    */
   protected void abortPendingUploadsInCleanup(
       boolean suppressExceptions) throws IOException {

+ 1 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java

@@ -529,7 +529,7 @@ public class CommitOperations {
    * {@link #maybeRethrow()} could be done as a map(), but because Java doesn't
    * allow checked exceptions in a map, the following code is invalid
    * <pre>
-   *   exception.map((e) -> {throw e;}
+   *   exception.map((e) -&gt; {throw e;}
    * </pre>
    * As a result, the code to work with exceptions would be almost as convoluted
    * as the original.

+ 1 - 0
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java

@@ -413,6 +413,7 @@ public class StagingCommitter extends AbstractS3ACommitter {
    * @param relative the path of a file relative to the task attempt path
    * @param context the JobContext or TaskAttemptContext for this job
    * @return the S3 Path where the file will be uploaded
+   * @throws IOException IO problem
    */
   protected final Path getFinalPath(String relative, JobContext context)
       throws IOException {

+ 1 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java

@@ -745,7 +745,7 @@ public class DynamoDBMetadataStore implements MetadataStore {
    * the call to {@link #processBatchWriteRequest(PrimaryKey[], Item[])}
    * is only tried once.
    * @param meta Directory listing metadata.
-   * @throws IOException
+   * @throws IOException IO problem
    */
   @Override
   @Retries.OnceTranslated("retry(listFullPaths); once(batchWrite)")