瀏覽代碼

HADOOP-13419. Fix javadoc warnings by JDK8 in hadoop-common package. Contributed by Kai Sasaki.

(cherry picked from commit a13a607e2099153a6b63afe06be24c1bc453158f)
Masatake Iwasaki 8 年之前
父節點
當前提交
74782e7e42

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java

@@ -315,7 +315,7 @@ public class FileContext {
    * 
    * @throws UnsupportedFileSystemException If the file system for
    *           <code>absOrFqPath</code> is not supported.
-   * @throws IOExcepton If the file system for <code>absOrFqPath</code> could
+   * @throws IOException If the file system for <code>absOrFqPath</code> could
    *         not be instantiated.
    */
   protected AbstractFileSystem getFSofPath(final Path absOrFqPath)
@@ -2725,7 +2725,7 @@ public class FileContext {
   /**
    * Query the effective storage policy ID for the given file or directory.
    *
-   * @param src file or directory path.
+   * @param path file or directory path.
    * @return storage policy for give file.
    * @throws IOException
    */

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/FenceMethod.java

@@ -52,7 +52,7 @@ public interface FenceMethod {
   
   /**
    * Attempt to fence the target node.
-   * @param serviceAddr the address (host:ipcport) of the service to fence
+   * @param target the address (host:ipcport) of the service to fence
    * @param args the configured arguments, which were checked at startup by
    *             {@link #checkArgs(String)}
    * @return true if fencing was successful, false if unsuccessful or

+ 3 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAServiceProtocol.java

@@ -151,12 +151,13 @@ public interface HAServiceProtocol {
   /**
    * Return the current status of the service. The status indicates
    * the current <em>state</em> (e.g ACTIVE/STANDBY) as well as
-   * some additional information. {@see HAServiceStatus}
-   * 
+   * some additional information.
+   *
    * @throws AccessControlException
    *           if access is denied.
    * @throws IOException
    *           if other errors happen
+   * @see HAServiceStatus
    */
   @Idempotent
   public HAServiceStatus getServiceStatus() throws AccessControlException,

+ 22 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/package-info.java

@@ -15,6 +15,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
+/**
+ * A mechanism for selectively retrying methods that throw exceptions under
+ * certain circumstances.
+ * Typical usage is
+ *  UnreliableImplementation unreliableImpl = new UnreliableImplementation();
+ *  UnreliableInterface unreliable = (UnreliableInterface)
+ *  RetryProxy.create(UnreliableInterface.class, unreliableImpl,
+ *  RetryPolicies.retryUpToMaximumCountWithFixedSleep(4, 10,
+ *      TimeUnit.SECONDS));
+ *  unreliable.call();
+ *
+ * This will retry any method called on <code>unreliable</code> four times -
+ * in this case the <code>call()</code> method - sleeping 10 seconds between
+ * each retry. There are a number of
+ * {@link org.apache.hadoop.io.retry.RetryPolicies retry policies}
+ * available, or you can implement a custom one by implementing
+ * {@link org.apache.hadoop.io.retry.RetryPolicy}.
+ * It is also possible to specify retry policies on a
+ * {@link org.apache.hadoop.io.retry.RetryProxy#create(Class, Object, Map)
+ * per-method basis}.
+ */
 @InterfaceAudience.LimitedPrivate({"HBase", "HDFS", "MapReduce"})
 @InterfaceStability.Evolving
 package org.apache.hadoop.io.retry;

+ 0 - 48
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/package.html

@@ -1,48 +0,0 @@
-<html>
-
-<!--
-   Licensed to the Apache Software Foundation (ASF) under one or more
-   contributor license agreements.  See the NOTICE file distributed with
-   this work for additional information regarding copyright ownership.
-   The ASF licenses this file to You under the Apache License, Version 2.0
-   (the "License"); you may not use this file except in compliance with
-   the License.  You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-   Unless required by applicable law or agreed to in writing, software
-   distributed under the License is distributed on an "AS IS" BASIS,
-   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-   See the License for the specific language governing permissions and
-   limitations under the License.
--->
-
-<body>
-
-<p>
-A mechanism for selectively retrying methods that throw exceptions under certain circumstances.
-</p>
-
-<p>
-Typical usage is
-</p>
-
-<pre>
-UnreliableImplementation unreliableImpl = new UnreliableImplementation();
-UnreliableInterface unreliable = (UnreliableInterface)
-  RetryProxy.create(UnreliableInterface.class, unreliableImpl,
-    RetryPolicies.retryUpToMaximumCountWithFixedSleep(4, 10, TimeUnit.SECONDS));
-unreliable.call();
-</pre>
-
-<p>
-This will retry any method called on <code>unreliable</code> four times - in this case the <code>call()</code>
-method - sleeping 10 seconds between
-each retry. There are a number of {@link org.apache.hadoop.io.retry.RetryPolicies retry policies}
-available, or you can implement a custom one by implementing {@link org.apache.hadoop.io.retry.RetryPolicy}.
-It is also possible to specify retry policies on a 
-{@link org.apache.hadoop.io.retry.RetryProxy#create(Class, Object, Map) per-method basis}.
-</p>
-
-</body>
-</html>

+ 11 - 11
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java

@@ -2035,7 +2035,7 @@ public abstract class Server {
     }
 
     /** Reads the connection context following the connection header
-     * @param dis - DataInputStream from which to read the header 
+     * @param buffer - DataInputStream from which to read the header
      * @throws WrappedRpcServerException - if the header cannot be
      *         deserialized, or the user is not authorized
      */ 
@@ -2087,7 +2087,7 @@ public abstract class Server {
     /**
      * Process a wrapped RPC Request - unwrap the SASL packet and process
      * each embedded RPC request 
-     * @param buf - SASL wrapped request of one or more RPCs
+     * @param inBuf - SASL wrapped request of one or more RPCs
      * @throws IOException - SASL packet cannot be unwrapped
      * @throws InterruptedException
      */    
@@ -2131,7 +2131,7 @@ public abstract class Server {
     /**
      * Process an RPC Request - handle connection setup and decoding of
      * request into a Call
-     * @param buf - contains the RPC request header and the rpc request
+     * @param bb - contains the RPC request header and the rpc request
      * @throws IOException - internal error that should not be returned to
      *         client, typically failure to respond to client
      * @throws WrappedRpcServerException - an exception to be sent back to
@@ -2207,7 +2207,7 @@ public abstract class Server {
      * Process an RPC Request - the connection headers and context must
      * have been already read
      * @param header - RPC request header
-     * @param dis - stream to request payload
+     * @param buffer - stream to request payload
      * @throws WrappedRpcServerException - due to fatal rpc layer issues such
      *   as invalid header or deserialization error. In this case a RPC fatal
      *   status response will later be sent back to client.
@@ -2283,7 +2283,7 @@ public abstract class Server {
      * Establish RPC connection setup by negotiating SASL if required, then
      * reading and authorizing the connection header
      * @param header - RPC header
-     * @param dis - stream to request payload
+     * @param buffer - stream to request payload
      * @throws WrappedRpcServerException - setup failed due to SASL
      *         negotiation failure, premature or invalid connection context,
      *         or other state errors 
@@ -2351,8 +2351,8 @@ public abstract class Server {
     
     /**
      * Decode the a protobuf from the given input stream 
-     * @param builder - Builder of the protobuf to decode
-     * @param dis - DataInputStream to read the protobuf
+     * @param message - Representation of the type of message
+     * @param buffer - a buffer to read the protobuf
      * @return Message - decoded protobuf
      * @throws WrappedRpcServerException - deserialization failed
      */
@@ -2522,8 +2522,8 @@ public abstract class Server {
    * from configuration. Otherwise the configuration will be picked up.
    * 
    * If rpcRequestClass is null then the rpcRequestClass must have been 
-   * registered via {@link #registerProtocolEngine(RpcPayloadHeader.RpcKind,
-   *  Class, RPC.RpcInvoker)}
+   * registered via {@link #registerProtocolEngine(RPC.RpcKind, Class,
+   * RPC.RpcInvoker)}.
    * This parameter has been retained for compatibility with existing tests
    * and usage.
    */
@@ -2881,8 +2881,8 @@ public abstract class Server {
   
   /** 
    * Called for each call. 
-   * @deprecated Use  {@link #call(RpcPayloadHeader.RpcKind, String,
-   *  Writable, long)} instead
+   * @deprecated Use  {@link Server#call(RPC.RpcKind, String, Writable, long)}
+   * instead.
    */
   @Deprecated
   public Writable call(Writable param, long receiveTime) throws Exception {

+ 4 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/package-info.java

@@ -15,6 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
+/**
+ * Tools to help define network clients and servers.
+ */
 @InterfaceAudience.LimitedPrivate({"HBase", "HDFS", "MapReduce"})
 @InterfaceStability.Evolving
 package org.apache.hadoop.ipc;

+ 0 - 23
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/package.html

@@ -1,23 +0,0 @@
-<html>
-
-<!--
-   Licensed to the Apache Software Foundation (ASF) under one or more
-   contributor license agreements.  See the NOTICE file distributed with
-   this work for additional information regarding copyright ownership.
-   The ASF licenses this file to You under the Apache License, Version 2.0
-   (the "License"); you may not use this file except in compliance with
-   the License.  You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-   Unless required by applicable law or agreed to in writing, software
-   distributed under the License is distributed on an "AS IS" BASIS,
-   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-   See the License for the specific language governing permissions and
-   limitations under the License.
--->
-
-<body>
-Tools to help define network clients and servers.
-</body>
-</html>