瀏覽代碼

HDFS-17008. Fix RBF JDK 11 javadoc warnings (#5648)

Viraj Jasani 2 年之前
父節點
當前提交
5d0cc455f5
共有 25 個文件被更改,包括 151 次插入58 次删除
  1. 6 3
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/rbfbalance/MountTableProcedure.java
  2. 3 1
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/fairness/RouterRpcFairnessPolicyController.java
  3. 7 3
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java
  4. 1 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/PathLocation.java
  5. 1 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java
  6. 3 2
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPool.java
  7. 3 3
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
  8. 2 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/MountTableRefresherService.java
  9. 4 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Quota.java
  10. 8 3
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java
  11. 31 22
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java
  12. 17 7
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcMonitor.java
  13. 10 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/security/RouterSecurityManager.java
  14. 9 1
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/security/token/DistributedSQLCounter.java
  15. 2 1
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/RecordStore.java
  16. 3 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java
  17. 6 4
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java
  18. 2 1
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreRecordOperations.java
  19. 8 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreSerializer.java
  20. 2 1
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileBaseImpl.java
  21. 4 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreMySQLImpl.java
  22. 8 2
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializableImpl.java
  23. 5 3
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/FederationProtocolPBTranslator.java
  24. 1 0
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MountTable.java
  25. 5 1
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java

+ 6 - 3
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/rbfbalance/MountTableProcedure.java

@@ -59,13 +59,16 @@ public class MountTableProcedure extends BalanceProcedure {
   /**
    * Update mount entry to specified dst uri.
    *
+   * @param name the name of the procedure.
+   * @param nextProcedure the name of the next procedure.
+   * @param delayDuration the delay duration when this procedure is delayed.
    * @param mount the mount entry to be updated.
    * @param dstPath the sub-cluster uri of the dst path.
+   * @param dstNs the destination sub-cluster name service id.
    * @param conf the configuration.
    */
-  public MountTableProcedure(String name, String nextProcedure,
-      long delayDuration, String mount, String dstPath, String dstNs,
-      Configuration conf) throws IOException {
+  public MountTableProcedure(String name, String nextProcedure, long delayDuration,
+      String mount, String dstPath, String dstNs, Configuration conf) {
     super(name, nextProcedure, delayDuration);
     this.mount = mount;
     this.dstPath = dstPath;

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/fairness/RouterRpcFairnessPolicyController.java

@@ -64,7 +64,9 @@ public interface RouterRpcFairnessPolicyController {
   void shutdown();
 
   /**
-   * Returns the JSON string of the available handler for each Ns.
+   * Returns the JSON string of the available handler for each name service.
+   *
+   * @return the JSON string of the available handler for each name service.
    */
   String getAvailableHandlerOnPerNs();
 }

+ 7 - 3
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java

@@ -348,9 +348,11 @@ public class MountTableResolver
   }
 
   /**
-   * Check if PATH is the trail associated with the Trash.
+   * Check if path is the trail associated with the Trash.
    *
-   * @param path A path.
+   * @param path a path.
+   * @return true if the path matches the trash path pattern, false otherwise.
+   * @throws IOException if retrieving current user's trash directory fails.
    */
   @VisibleForTesting
   public static boolean isTrashPath(String path) throws IOException {
@@ -370,7 +372,9 @@ public class MountTableResolver
   /**
    * Subtract a TrashCurrent to get a new path.
    *
-   * @param path A path.
+   * @param path a path.
+   * @return new path with subtracted trash current path.
+   * @throws IOException if retrieving current user's trash directory fails.
    */
   @VisibleForTesting
   public static String subtractTrashCurrentPath(String path)

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/PathLocation.java

@@ -100,6 +100,7 @@ public class PathLocation {
    *
    * @param base The base path location we'd like to prioritize on.
    * @param firstNsId Identifier of the namespace to place first.
+   * @return path location with the prioritized destinations.
    */
   public static PathLocation prioritizeDestination(
       PathLocation base, String firstNsId) {

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java

@@ -97,6 +97,7 @@ public class ConnectionManager {
    * Creates a proxy client connection pool manager.
    *
    * @param config Configuration for the connections.
+   * @param routerStateIdContext Federated namespace context for router.
    */
   public ConnectionManager(Configuration config, RouterStateIdContext routerStateIdContext) {
     this.conf = config;

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionPool.java

@@ -420,16 +420,17 @@ public class ConnectionPool {
    * context for a single user/security context. To maximize throughput it is
    * recommended to use multiple connection per user+server, allowing multiple
    * writes and reads to be dispatched in parallel.
-   * @param <T> Input type T.
    *
    * @param conf Configuration for the connection.
    * @param nnAddress Address of server supporting the ClientProtocol.
    * @param ugi User context.
    * @param proto Interface of the protocol.
    * @param enableMultiSocket Enable multiple socket or not.
+   * @param socketIndex Index for FederationConnectionId.
    * @param alignmentContext Client alignment context.
+   * @param <T> Input type T.
    * @return proto for the target ClientProtocol that contains the user's
-   *         security context.
+   * security context.
    * @throws IOException If it cannot be created.
    */
   protected static <T> ConnectionContext newConnection(Configuration conf,

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java

@@ -274,9 +274,9 @@ public final class FederationUtil {
   /**
    * Collect all configured nameservices.
    *
-   * @param conf
-   * @return Set of name services in config
-   * @throws IllegalArgumentException
+   * @param conf the configuration object.
+   * @return Set of name services in config.
+   * @throws IllegalArgumentException if monitored namenodes are not correctly configured.
    */
   public static Set<String> getAllConfiguredNS(Configuration conf)
       throws IllegalArgumentException {

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/MountTableRefresherService.java

@@ -204,6 +204,8 @@ public class MountTableRefresherService extends AbstractService {
 
   /**
    * Refresh mount table cache of this router as well as all other routers.
+   *
+   * @throws StateStoreUnavailableException if the state store is not available.
    */
   public void refresh() throws StateStoreUnavailableException {
     RouterStore routerStore = router.getRouterStateManager();

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Quota.java

@@ -328,7 +328,9 @@ public class Quota {
 
   /**
    * Invoke predicate by each storage type and bitwise inclusive OR the results.
+   *
    * @param predicate the function test the storage type.
+   * @return true if bitwise OR by all storage type returns true, false otherwise.
    */
   public static boolean orByStorageType(Predicate<StorageType> predicate) {
     boolean res = false;
@@ -340,7 +342,9 @@ public class Quota {
 
   /**
    * Invoke predicate by each storage type and bitwise AND the results.
+   *
    * @param predicate the function test the storage type.
+   * @return true if bitwise AND by all storage type returns true, false otherwise.
    */
   public static boolean andByStorageType(Predicate<StorageType> predicate) {
     boolean res = false;

+ 8 - 3
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Router.java

@@ -670,6 +670,9 @@ public class Router extends CompositeService implements
 
   /**
    * Compare router state.
+   *
+   * @param routerState the router service state.
+   * @return true if the given router state is same as the state maintained by the router object.
    */
   public boolean isRouterState(RouterServiceState routerState) {
     return routerState.equals(this.state);
@@ -725,9 +728,10 @@ public class Router extends CompositeService implements
   }
 
   /**
-   * Get the Namenode metrics.
+   * Get the namenode metrics.
    *
-   * @return Namenode metrics.
+   * @return the namenode metrics.
+   * @throws IOException if the namenode metrics are not initialized.
    */
   public NamenodeBeanMetrics getNamenodeMetrics() throws IOException {
     if (this.metrics == null) {
@@ -864,7 +868,8 @@ public class Router extends CompositeService implements
 
   /**
    * Set router configuration.
-   * @param conf
+   *
+   * @param conf the configuration.
    */
   @VisibleForTesting
   public void setConf(Configuration conf) {

+ 31 - 22
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java

@@ -165,6 +165,8 @@ public class RouterRpcClient {
    * @param router A router using this RPC client.
    * @param resolver A NN resolver to determine the currently active NN in HA.
    * @param monitor Optional performance monitor.
+   * @param routerStateIdContext the router state context object to hold the state ids for all
+   * namespaces.
    */
   public RouterRpcClient(Configuration conf, Router router,
       ActiveNamenodeResolver resolver, RouterRpcMonitor monitor,
@@ -484,7 +486,7 @@ public class RouterRpcClient {
    * Invokes a method against the ClientProtocol proxy server. If a standby
    * exception is generated by the call to the client, retries using the
    * alternate server.
-   *
+   * <p>
    * Re-throws exceptions generated by the remote RPC call as either
    * RemoteException or IOException.
    *
@@ -492,6 +494,7 @@ public class RouterRpcClient {
    * @param namenodes A prioritized list of namenodes within the same
    *                  nameservice.
    * @param useObserver Whether to use observer namenodes.
+   * @param protocol the protocol of the connection.
    * @param method Remote ClientProtocol method to invoke.
    * @param params Variable list of parameters matching the method.
    * @return The result of invoking the method.
@@ -690,7 +693,7 @@ public class RouterRpcClient {
   /**
    * Invokes a method on the designated object. Catches exceptions specific to
    * the invocation.
-   *
+   * <p>
    * Re-throws exceptions generated by the remote RPC call as either
    * RemoteException or IOException.
    *
@@ -700,7 +703,7 @@ public class RouterRpcClient {
    * @param obj Target object for the method
    * @param params Variable parameters
    * @return Response from the remote server
-   * @throws IOException
+   * @throws IOException If error occurs.
    */
   private Object invoke(String nsId, int retryCount, final Method method,
       final Object obj, final Object... params) throws IOException {
@@ -764,9 +767,10 @@ public class RouterRpcClient {
 
   /**
    * Check if the cluster of given nameservice id is available.
+   *
    * @param nsId nameservice ID.
-   * @return
-   * @throws IOException
+   * @return true if the cluster with given nameservice id is available.
+   * @throws IOException if error occurs.
    */
   private boolean isClusterUnAvailable(String nsId) throws IOException {
     List<? extends FederationNamenodeContext> nnState = this.namenodeResolver
@@ -871,14 +875,15 @@ public class RouterRpcClient {
   /**
    * Invokes a ClientProtocol method. Determines the target nameservice via a
    * provided block.
-   *
+   * <p>
    * Re-throws exceptions generated by the remote RPC call as either
    * RemoteException or IOException.
    *
    * @param block Block used to determine appropriate nameservice.
    * @param method The remote method and parameters to invoke.
    * @param locations The remote locations will be used.
-   * @param clazz – Class for the return type.
+   * @param clazz Class for the return type.
+   * @param <T> The type of the remote method return.
    * @return The result of invoking the method.
    * @throws IOException If the invoke generated an error.
    */
@@ -891,7 +896,7 @@ public class RouterRpcClient {
   /**
    * Invokes a ClientProtocol method. Determines the target nameservice using
    * the block pool id.
-   *
+   * <p>
    * Re-throws exceptions generated by the remote RPC call as either
    * RemoteException or IOException.
    *
@@ -908,7 +913,7 @@ public class RouterRpcClient {
 
   /**
    * Invokes a ClientProtocol method against the specified namespace.
-   *
+   * <p>
    * Re-throws exceptions generated by the remote RPC call as either
    * RemoteException or IOException.
    *
@@ -937,7 +942,7 @@ public class RouterRpcClient {
 
   /**
    * Invokes a remote method against the specified namespace.
-   *
+   * <p>
    * Re-throws exceptions generated by the remote RPC call as either
    * RemoteException or IOException.
    *
@@ -957,7 +962,7 @@ public class RouterRpcClient {
 
   /**
    * Invokes a remote method against the specified extendedBlock.
-   *
+   * <p>
    * Re-throws exceptions generated by the remote RPC call as either
    * RemoteException or IOException.
    *
@@ -978,12 +983,14 @@ public class RouterRpcClient {
 
   /**
    * Invokes a single proxy call for a single location.
-   *
+   * <p>
    * Re-throws exceptions generated by the remote RPC call as either
    * RemoteException or IOException.
    *
    * @param location RemoteLocation to invoke.
    * @param remoteMethod The remote method and parameters to invoke.
+   * @param clazz Class for the return type.
+   * @param <T> The type of the remote method return.
    * @return The result of invoking the method if successful.
    * @throws IOException If the invoke generated an error.
    */
@@ -1001,10 +1008,11 @@ public class RouterRpcClient {
    *
    * @param locations List of locations/nameservices to call concurrently.
    * @param remoteMethod The remote method and parameters to invoke.
+   * @param <T> The type of the remote method return.
    * @return The result of the first successful call, or if no calls are
-   *         successful, the result of the last RPC call executed.
+   * successful, the result of the last RPC call executed.
    * @throws IOException if the success condition is not met and one of the RPC
-   *           calls generated a remote exception.
+   * calls generated a remote exception.
    */
   public <T> T invokeSequential(
       final List<? extends RemoteLocationContext> locations,
@@ -1270,7 +1278,7 @@ public class RouterRpcClient {
   /**
    * Invoke multiple concurrent proxy calls to different clients. Returns an
    * array of results.
-   *
+   * <p>
    * Re-throws exceptions generated by the remote RPC call as either
    * RemoteException or IOException.
    *
@@ -1289,14 +1297,15 @@ public class RouterRpcClient {
   /**
    * Invoke multiple concurrent proxy calls to different clients. Returns an
    * array of results.
-   *
+   * <p>
    * Re-throws exceptions generated by the remote RPC call as either
    * RemoteException or IOException.
    *
-   * @param <T> The type of the remote location.
-   * @param <R> The type of the remote method return.
    * @param locations List of remote locations to call concurrently.
    * @param method The remote method and parameters to invoke.
+   * @param clazz Type of the remote return type.
+   * @param <T> The type of the remote location.
+   * @param <R> The type of the remote method return.
    * @return Result of invoking the method per subcluster: nsId to result.
    * @throws IOException If all the calls throw an exception.
    */
@@ -1309,7 +1318,7 @@ public class RouterRpcClient {
   /**
    * Invoke multiple concurrent proxy calls to different clients. Returns an
    * array of results.
-   *
+   * <p>
    * Re-throws exceptions generated by the remote RPC call as either
    * RemoteException or IOException.
    *
@@ -1332,7 +1341,7 @@ public class RouterRpcClient {
   /**
    * Invokes multiple concurrent proxy calls to different clients. Returns an
    * array of results.
-   *
+   * <p>
    * Re-throws exceptions generated by the remote RPC call as either
    * RemoteException or IOException.
    *
@@ -1360,7 +1369,7 @@ public class RouterRpcClient {
   /**
    * Invokes multiple concurrent proxy calls to different clients. Returns an
    * array of results.
-   *
+   * <p>
    * Re-throws exceptions generated by the remote RPC call as either
    * RemoteException or IOException.
    *
@@ -1422,7 +1431,7 @@ public class RouterRpcClient {
   /**
    * Invokes multiple concurrent proxy calls to different clients. Returns an
    * array of results.
-   *
+   * <p>
    * Re-throws exceptions generated by the remote RPC call as either
    * RemoteException or IOException.
    *

+ 17 - 7
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcMonitor.java

@@ -61,29 +61,37 @@ public interface RouterRpcMonitor {
 
   /**
    * Mark a proxy operation as completed.
-   * @param success If the operation was successful.
-   * @param state proxy namenode state.
+   *
+   * @param success if the operation was successful.
+   * @param nsId nameservice id.
+   * @param state namenode state in the federation.
    */
   void proxyOpComplete(boolean success, String nsId, FederationNamenodeServiceState state);
 
   /**
-   * Failed to proxy an operation to a Namenode because it was in standby.
+   * Failed to proxy an operation to a namenode because it was in standby.
+   * @param nsId nameservice id.
    */
   void proxyOpFailureStandby(String nsId);
 
   /**
-   * Failed to proxy an operation to a Namenode because of an unexpected
-   * exception.
+   * Failed to proxy an operation to a namenode because of an unexpected exception.
+   *
+   * @param nsId nameservice id.
    */
   void proxyOpFailureCommunicate(String nsId);
 
   /**
-   * Rejected to proxy an operation to a Namenode.
+   * Rejected to proxy an operation to a namenode.
+   *
+   * @param nsId nameservice id.
    */
   void proxyOpPermitRejected(String nsId);
 
   /**
-   * Accepted to proxy an operation to a Namenode.
+   * Accepted to proxy an operation to a namenode.
+   *
+   * @param nsId nameservice id.
    */
   void proxyOpPermitAccepted(String nsId);
 
@@ -106,6 +114,8 @@ public interface RouterRpcMonitor {
 
   /**
    * Failed to proxy an operation because of no namenodes available.
+   *
+   * @param nsId nameservice id.
    */
   void proxyOpNoNamenodes(String nsId);
 

+ 10 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/security/RouterSecurityManager.java

@@ -220,6 +220,12 @@ public class RouterSecurityManager {
   /**
    * A utility method for creating credentials.
    * Used by web hdfs to return url encoded token.
+   *
+   * @param router the router object.
+   * @param ugi object with username and group information for the given user.
+   * @param renewer the renewer for the token.
+   * @return the credentials object for tokens.
+   * @throws IOException if error occurs while obtaining the credentials.
    */
   public static Credentials createCredentials(
       final Router router, final UserGroupInformation ugi,
@@ -239,6 +245,10 @@ public class RouterSecurityManager {
   /**
    * Delegation token verification.
    * Used by web hdfs to verify url encoded token.
+   *
+   * @param identifier the delegation token identifier.
+   * @param password the password in the token.
+   * @throws SecretManager.InvalidToken if password doesn't match.
    */
   public void verifyToken(DelegationTokenIdentifier identifier,
       byte[] password) throws SecretManager.InvalidToken {

+ 9 - 1
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/security/token/DistributedSQLCounter.java

@@ -50,7 +50,9 @@ public class DistributedSQLCounter {
 
   /**
    * Obtains the value of the counter.
+   *
    * @return counter value.
+   * @throws SQLException if querying the database fails.
    */
   public int selectCounterValue() throws SQLException {
     try (Connection connection = connectionFactory.getConnection()) {
@@ -74,7 +76,9 @@ public class DistributedSQLCounter {
 
   /**
    * Sets the counter to the given value.
+   *
    * @param value Value to assign to counter.
+   * @throws SQLException if querying the database fails.
    */
   public void updateCounterValue(int value) throws SQLException {
     try (Connection connection = connectionFactory.getConnection(true)) {
@@ -84,8 +88,10 @@ public class DistributedSQLCounter {
 
   /**
    * Sets the counter to the given value.
-   * @param connection Connection to database hosting the counter table.
+   *
    * @param value Value to assign to counter.
+   * @param connection Connection to database hosting the counter table.
+   * @throws SQLException if querying the database fails.
    */
   public void updateCounterValue(int value, Connection connection) throws SQLException {
     String queryText = String.format("UPDATE %s SET %s = ?", table, field);
@@ -99,8 +105,10 @@ public class DistributedSQLCounter {
   /**
    * Increments the counter by the given amount and
    * returns the previous counter value.
+   *
    * @param amount Amount to increase the counter.
    * @return Previous counter value.
+   * @throws SQLException if querying the database fails.
    */
   public int incrementCounterValue(int amount) throws SQLException {
     // Disabling auto-commit to ensure that all statements on this transaction

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/RecordStore.java

@@ -82,8 +82,9 @@ public abstract class RecordStore<R extends BaseRecord> {
    *
    * @param clazz The specific interface implementation to create
    * @param driver The {@link StateStoreDriver} implementation in use.
+   * @param <T> Instance of type RecordStore.
    * @return An initialized instance of the specified state store API
-   *         implementation.
+   * implementation.
    */
   public static <T extends RecordStore<?>> T newInstance(
       final Class<T> clazz, final StateStoreDriver driver) {

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java

@@ -257,6 +257,7 @@ public class StateStoreService extends CompositeService {
    * Get the record store in this State Store for a given interface.
    *
    * @param recordStoreClass Class of the record store.
+   * @param <T> The type of the record store.
    * @return Registered record store or null if not found.
    */
   public <T extends RecordStore<?>> T getRegisteredRecordStore(
@@ -274,6 +275,8 @@ public class StateStoreService extends CompositeService {
 
   /**
    * Get the list of all RecordStores.
+   *
+   * @param <T> The type of the record stores that are returned.
    * @return a list of each RecordStore.
    */
   @SuppressWarnings("unchecked")

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java

@@ -54,13 +54,13 @@ public abstract class StateStoreDriver implements StateStoreRecordOperations {
   /** State Store metrics. */
   private StateStoreMetrics metrics;
 
-
   /**
    * Initialize the state store connection.
    *
    * @param config Configuration for the driver.
    * @param id Identifier for the driver.
    * @param records Records that are supported.
+   * @param stateStoreMetrics State store metrics.
    * @return If initialized and ready, false if failed to initialize driver.
    */
   public boolean init(final Configuration config, final String id,
@@ -133,10 +133,10 @@ public abstract class StateStoreDriver implements StateStoreRecordOperations {
    * Initialize storage for a single record class.
    *
    * @param className String reference of the record class to initialize,
-   *                  used to construct paths and file names for the record.
-   *                  Determined by configuration settings for the specific
-   *                  driver.
+   * used to construct paths and file names for the record.
+   * Determined by configuration settings for the specific driver.
    * @param clazz Record type corresponding to the provided name.
+   * @param <T> Type of the state store record.
    * @return True if successful, false otherwise.
    */
   public abstract <T extends BaseRecord> boolean initRecordStorage(
@@ -166,6 +166,8 @@ public abstract class StateStoreDriver implements StateStoreRecordOperations {
 
   /**
    * Close the State Store driver connection.
+   *
+   * @throws Exception if something goes wrong while closing the state store driver connection.
    */
   public abstract void close() throws Exception;
 

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreRecordOperations.java

@@ -143,8 +143,9 @@ public interface StateStoreRecordOperations {
    * Remove multiple records of a specific class that match a query. Requires
    * the getAll implementation to fetch fresh records on each call.
    *
-   * @param <T> Record class of the records.
+   * @param clazz The class to match the records with.
    * @param query Query to filter what to remove.
+   * @param <T> Record class of the records.
    * @return The number of records removed.
    * @throws IOException Throws exception if unable to query the data store.
    */

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreSerializer.java

@@ -69,7 +69,9 @@ public abstract class StateStoreSerializer {
 
   /**
    * Create a new record.
+   *
    * @param clazz Class of the new record.
+   * @param <T> Type of the record.
    * @return New record.
    */
   public static <T> T newRecord(Class<T> clazz) {
@@ -78,7 +80,9 @@ public abstract class StateStoreSerializer {
 
   /**
    * Create a new record.
+   *
    * @param clazz Class of the new record.
+   * @param <T> Type of the record.
    * @return New record.
    */
   public abstract <T> T newRecordInstance(Class<T> clazz);
@@ -99,8 +103,10 @@ public abstract class StateStoreSerializer {
 
   /**
    * Deserialize a bytes array into a record.
+   *
    * @param byteArray Byte array to deserialize.
    * @param clazz Class of the record.
+   * @param <T> Type of the record.
    * @return New record.
    * @throws IOException If it cannot deserialize the record.
    */
@@ -109,8 +115,10 @@ public abstract class StateStoreSerializer {
 
   /**
    * Deserialize a string into a record.
+   *
    * @param data String with the data to deserialize.
    * @param clazz Class of the record.
+   * @param <T> Type of the record.
    * @return New record.
    * @throws IOException If it cannot deserialize the record.
    */

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreFileBaseImpl.java

@@ -81,11 +81,11 @@ public abstract class StateStoreFileBaseImpl
 
   private ExecutorService concurrentStoreAccessPool;
 
-
   /**
    * Get the reader of a record for the file system.
    *
    * @param path Path of the record to read.
+   * @param <T> Type of the state store record.
    * @return Reader for the record.
    */
   protected abstract <T extends BaseRecord> BufferedReader getReader(
@@ -95,6 +95,7 @@ public abstract class StateStoreFileBaseImpl
    * Get the writer of a record for the file system.
    *
    * @param path Path of the record to write.
+   * @param <T> Type of the state store record.
    * @return Writer for the record.
    */
   @VisibleForTesting

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreMySQLImpl.java

@@ -294,8 +294,10 @@ public class StateStoreMySQLImpl extends StateStoreSerializableImpl {
 
   /**
    * Insert a record with a given key into the specified table.
+   *
    * @param tableName Name of table to modify
    * @param key Primary key for the record.
+   * @param data The record value for the given record key.
    * @return True is operation is successful, false otherwise.
    */
   protected boolean insertRecord(String tableName, String key, String data) {
@@ -314,8 +316,10 @@ public class StateStoreMySQLImpl extends StateStoreSerializableImpl {
 
   /**
    * Updates the record with a given key from the specified table.
+   *
    * @param tableName Name of table to modify
    * @param key Primary key for the record.
+   * @param data The record value for the given record key.
    * @return True is operation is successful, false otherwise.
    */
   protected boolean updateRecord(String tableName, String key, String data) {

+ 8 - 2
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/impl/StateStoreSerializableImpl.java

@@ -57,7 +57,9 @@ public abstract class StateStoreSerializableImpl extends StateStoreBaseImpl {
 
   /**
    * Serialize a record using the serializer.
+   *
    * @param record Record to serialize.
+   * @param <T> Type of the state store record.
    * @return Byte array with the serialization of the record.
    */
   protected <T extends BaseRecord> byte[] serialize(T record) {
@@ -66,7 +68,9 @@ public abstract class StateStoreSerializableImpl extends StateStoreBaseImpl {
 
   /**
    * Serialize a record using the serializer.
+   *
    * @param record Record to serialize.
+   * @param <T> Type of the state store record.
    * @return String with the serialization of the record.
    */
   protected <T extends BaseRecord> String serializeString(T record) {
@@ -75,11 +79,13 @@ public abstract class StateStoreSerializableImpl extends StateStoreBaseImpl {
 
   /**
    * Creates a record from an input data string.
+   *
    * @param data Serialized text of the record.
    * @param clazz Record class.
    * @param includeDates If dateModified and dateCreated are serialized.
-   * @return The created record.
-   * @throws IOException
+   * @param <T> Type of the state store record.
+   * @return The created record by deserializing the input text.
+   * @throws IOException If the record deserialization fails.
    */
   protected <T extends BaseRecord> T newRecord(
       String data, Class<T> clazz, boolean includeDates) throws IOException {

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/FederationProtocolPBTranslator.java

@@ -50,7 +50,8 @@ public class FederationProtocolPBTranslator<P extends GeneratedMessageV3,
    * stream.
    *
    * @param p The existing proto object to use to initialize the translator.
-   * @throws IllegalArgumentException
+   * @throws IllegalArgumentException If the given proto message is not instance of the class of
+   * the proto handler this translator holds.
    */
   @SuppressWarnings("unchecked")
   public void setProto(Message p) {
@@ -133,8 +134,9 @@ public class FederationProtocolPBTranslator<P extends GeneratedMessageV3,
 
   /**
    * Read instance from base64 data.
-   * @param base64String
-   * @throws IOException
+   *
+   * @param base64String String containing Base64 data.
+   * @throws IOException If the protobuf message build fails.
    */
   @SuppressWarnings("unchecked")
   public void readInstance(String base64String) throws IOException {

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/records/MountTable.java

@@ -125,6 +125,7 @@ public abstract class MountTable extends BaseRecord {
    *
    * @param src Source path in the mount entry.
    * @param destinations Name service destinations of the mount point.
+   * @return The MountTable object.
    * @throws IOException If it cannot be created.
    */
   public static MountTable newInstance(final String src,

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java

@@ -1391,7 +1391,11 @@ public class RouterAdmin extends Configured implements Tool {
 
   /**
    * Dumps the contents of the StateStore to stdout.
-   * @return true if it was successful
+   *
+   * @param conf the configuration.
+   * @param output the print output stream.
+   * @return true if it was successful.
+   * @throws IOException if the State store is not available.
    */
   public static boolean dumpStateStore(Configuration conf,
                                 PrintStream output) throws IOException {