diff --git a/hbase-replication/pom.xml b/hbase-replication/pom.xml index ab221999ad1..cd4f74d7cd5 100644 --- a/hbase-replication/pom.xml +++ b/hbase-replication/pom.xml @@ -69,6 +69,22 @@ org.apache.maven.plugins maven-source-plugin + + org.apache.maven.plugins + maven-checkstyle-plugin + + + checkstyle + validate + + check + + + true + + + + diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java index 9f4ad18ae0c..6c83d6a27cc 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java @@ -19,17 +19,19 @@ package org.apache.hadoop.hbase.replication; import org.apache.commons.lang3.reflect.ConstructorUtils; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; +import org.apache.yetus.audience.InterfaceAudience; /** * A factory class for instantiating replication objects that deal with replication state. */ @InterfaceAudience.Private -public class ReplicationFactory { +public final class ReplicationFactory { + private ReplicationFactory() { + } public static ReplicationQueues getReplicationQueues(ReplicationQueuesArguments args) throws Exception { @@ -49,7 +51,7 @@ public class ReplicationFactory { } public static ReplicationPeers getReplicationPeers(final ZKWatcher zk, Configuration conf, - final ReplicationQueuesClient queuesClient, Abortable abortable) { + final ReplicationQueuesClient queuesClient, Abortable abortable) { return new ReplicationPeersZKImpl(zk, conf, queuesClient, abortable); } diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java index 454d09ce352..15523e74cdc 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java @@ -31,17 +31,17 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.log.HBaseMarkers; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; import org.apache.hadoop.hbase.zookeeper.ZKNodeTracker; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.KeeperException.NodeExistsException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; + @InterfaceAudience.Private public class ReplicationPeerZKImpl extends ReplicationStateZKBase implements ReplicationPeer, Abortable, Closeable { @@ -77,10 +77,9 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase * start a state tracker to check whether this peer is enabled or not * * @param peerStateNode path to zk node which stores peer state - * @throws KeeperException + * @throws KeeperException if creating the znode fails */ - public void startStateTracker(String peerStateNode) - throws KeeperException { + public void startStateTracker(String peerStateNode) throws KeeperException { ensurePeerEnabled(peerStateNode); this.peerStateTracker = new PeerStateTracker(peerStateNode, zookeeper, this); this.peerStateTracker.start(); @@ -101,10 +100,8 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase /** * start a table-cfs tracker to listen the (table, cf-list) map change * @param peerConfigNode path to zk node which stores table-cfs - * @throws KeeperException */ - public void startPeerConfigTracker(String peerConfigNode) - throws KeeperException { + public void startPeerConfigTracker(String peerConfigNode) throws KeeperException { this.peerConfigTracker = new PeerConfigTracker(peerConfigNode, zookeeper, this); this.peerConfigTracker.start(); @@ -208,7 +205,7 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase * Parse the raw data from ZK to get a peer's state * @param bytes raw ZK data * @return True if the passed in bytes are those of a pb serialized ENABLED state. - * @throws DeserializationException + * @throws DeserializationException if parsing the state fails */ public static boolean isStateEnabled(final byte[] bytes) throws DeserializationException { ReplicationProtos.ReplicationState.State state = parseStateFrom(bytes); @@ -218,7 +215,7 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase /** * @param bytes Content of a state znode. * @return State parsed from the passed bytes. - * @throws DeserializationException + * @throws DeserializationException if a ProtoBuf operation fails */ private static ReplicationProtos.ReplicationState.State parseStateFrom(final byte[] bytes) throws DeserializationException { @@ -240,11 +237,9 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase * Utility method to ensure an ENABLED znode is in place; if not present, we create it. * @param path Path to znode to check * @return True if we created the znode. - * @throws NodeExistsException - * @throws KeeperException + * @throws KeeperException if creating the znode fails */ - private boolean ensurePeerEnabled(final String path) - throws NodeExistsException, KeeperException { + private boolean ensurePeerEnabled(final String path) throws KeeperException { if (ZKUtil.checkExists(zookeeper, path) == -1) { // There is a race b/w PeerWatcher and ReplicationZookeeper#add method to create the // peer-state znode. This happens while adding a peer. diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java index 10936bf2657..542d8920951 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java @@ -25,8 +25,8 @@ import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.TableName; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.util.Pair; +import org.apache.yetus.audience.InterfaceAudience; /** * This provides an interface for maintaining a set of peer clusters. These peers are remote slave @@ -76,7 +76,7 @@ public interface ReplicationPeers { * newly connected cluster. * @param peerId a short that identifies the cluster * @return whether a ReplicationPeer was successfully created - * @throws ReplicationException + * @throws ReplicationException if connecting to the peer fails */ boolean peerConnected(String peerId) throws ReplicationException; @@ -182,7 +182,7 @@ public interface ReplicationPeers { * Update the peerConfig for the a given peer cluster * @param id a short that identifies the cluster * @param peerConfig new config for the peer cluster - * @throws ReplicationException + * @throws ReplicationException if updating the peer configuration fails */ void updatePeerConfig(String id, ReplicationPeerConfig peerConfig) throws ReplicationException; } diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java index 289d2aa954e..358721da67d 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java @@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil; import org.apache.hadoop.hbase.exceptions.DeserializationException; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.zookeeper.ZKConfig; @@ -48,6 +47,8 @@ import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; + /** * This class provides an implementation of the ReplicationPeers interface using ZooKeeper. The * peers znode contains a list of all peer replication clusters and the current replication state of @@ -462,8 +463,8 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re /** * Update the state znode of a peer cluster. - * @param id - * @param state + * @param id the id of the peer + * @param state the state to update to */ private void changePeerState(String id, ReplicationProtos.ReplicationState.State state) throws ReplicationException { @@ -491,7 +492,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re * Helper method to connect to a peer * @param peerId peer's identifier * @return object representing the peer - * @throws ReplicationException + * @throws ReplicationException if creating the peer fails */ private ReplicationPeerZKImpl createPeer(String peerId) throws ReplicationException { Pair pair = getPeerConf(peerId); @@ -520,7 +521,10 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re } private void checkQueuesDeleted(String peerId) throws ReplicationException { - if (queuesClient == null) return; + if (queuesClient == null) { + return; + } + try { List replicators = queuesClient.getListOfReplicators(); if (replicators == null || replicators.isEmpty()) { diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueInfo.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueInfo.java index ecd888f51ea..8ef193949c9 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueInfo.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueInfo.java @@ -18,15 +18,14 @@ */ package org.apache.hadoop.hbase.replication; - import java.util.ArrayList; import java.util.Collections; import java.util.List; +import org.apache.hadoop.hbase.ServerName; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.hbase.ServerName; /** * This class is responsible for the parsing logic for a znode representing a queue. @@ -66,8 +65,9 @@ public class ReplicationQueueInfo { */ private static void extractDeadServersFromZNodeString(String deadServerListStr, List result) { - - if(deadServerListStr == null || result == null || deadServerListStr.isEmpty()) return; + if(deadServerListStr == null || result == null || deadServerListStr.isEmpty()) { + return; + } // valid server name delimiter "-" has to be after "," in a server name int seenCommaCnt = 0; @@ -76,25 +76,25 @@ public class ReplicationQueueInfo { for (int i = 0; i < len; i++) { switch (deadServerListStr.charAt(i)) { - case ',': - seenCommaCnt += 1; - break; - case '-': - if(seenCommaCnt>=2) { - if (i > startIndex) { - String serverName = deadServerListStr.substring(startIndex, i); - if(ServerName.isFullServerName(serverName)){ - result.add(ServerName.valueOf(serverName)); - } else { - LOG.error("Found invalid server name:" + serverName); + case ',': + seenCommaCnt += 1; + break; + case '-': + if(seenCommaCnt>=2) { + if (i > startIndex) { + String serverName = deadServerListStr.substring(startIndex, i); + if(ServerName.isFullServerName(serverName)){ + result.add(ServerName.valueOf(serverName)); + } else { + LOG.error("Found invalid server name:" + serverName); + } + startIndex = i + 1; } - startIndex = i + 1; + seenCommaCnt = 0; } - seenCommaCnt = 0; - } - break; - default: - break; + break; + default: + break; } } diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java index 7f440b102be..81e94c97700 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java @@ -22,8 +22,8 @@ import java.util.List; import java.util.SortedSet; import org.apache.hadoop.fs.Path; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.util.Pair; +import org.apache.yetus.audience.InterfaceAudience; /** * This provides an interface for maintaining a region server's replication queues. These queues @@ -90,7 +90,8 @@ public interface ReplicationQueues { /** * Get a list of all queues for this region server. - * @return a list of queueIds, an empty list if this region server is dead and has no outstanding queues + * @return a list of queueIds, an empty list if this region server is dead and has no outstanding + * queues */ List getAllQueues(); @@ -111,7 +112,7 @@ public interface ReplicationQueues { /** * Remove the znode of region server if the queue is empty. - * @param regionserver + * @param regionserver the id of the region server */ void removeReplicatorIfQueueIsEmpty(String regionserver); diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java index 2c513faa1c6..7ef4004cac4 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java @@ -67,7 +67,7 @@ public interface ReplicationQueuesClient { * is concurrent queue failover. However, some newly created WALs during the call may * not be included. */ - Set getAllWALs() throws KeeperException; + Set getAllWALs() throws KeeperException; /** * Get the change version number of replication hfile references node. This can be used as diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java index e00a7a27501..4dccf7ffdad 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java @@ -23,8 +23,6 @@ import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet; -import org.apache.hbase.thirdparty.com.google.common.collect.Sets; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.hadoop.hbase.zookeeper.ZNodePaths; @@ -34,6 +32,9 @@ import org.apache.zookeeper.data.Stat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableSet; +import org.apache.hbase.thirdparty.com.google.common.collect.Sets; + @InterfaceAudience.Private public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implements ReplicationQueuesClient { diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java index 7551cb76608..56abed2ea66 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java @@ -302,8 +302,9 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R // add delete op for peer listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode)); - if (LOG.isTraceEnabled()) + if (LOG.isTraceEnabled()) { LOG.trace(" The multi list size is: " + listOfOps.size()); + } } ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false); diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java index 05bbc8436fa..f28a0dee3e7 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java @@ -24,10 +24,6 @@ import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.hbase.thirdparty.com.google.protobuf.CodedOutputStream; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; import org.apache.hadoop.hbase.zookeeper.ZKConfig; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; @@ -35,6 +31,12 @@ import org.apache.hadoop.hbase.zookeeper.ZNodePaths; import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.KeeperException; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hbase.thirdparty.com.google.protobuf.CodedOutputStream; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; + /** * This is a base class for maintaining replication state in zookeeper. */ @@ -104,7 +106,7 @@ public abstract class ReplicationStateZKBase { } /** - * @param state + * @param state the state to convert into a byte array * @return Serialized protobuf of state with pb magic prefix prepended suitable for * use as content of a peer-state znode under a peer cluster id as in * /hbase/replication/peers/PEER_ID/peer-state. diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTableBase.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTableBase.java index 0d8427c0f06..6aae08dee62 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTableBase.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTableBase.java @@ -18,32 +18,6 @@ */ package org.apache.hadoop.hbase.replication; -import org.apache.hadoop.hbase.CompareOperator; -import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Abortable; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.NamespaceDescriptor; -import org.apache.hadoop.hbase.TableExistsException; -import org.apache.hadoop.hbase.TableName; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.ResultScanner; -import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.filter.CompareFilter; -import org.apache.hadoop.hbase.filter.SingleColumnValueFilter; -import org.apache.hadoop.hbase.regionserver.BloomType; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.RetryCounter; -import org.apache.hadoop.hbase.util.RetryCounterFactory; - import java.io.IOException; import java.io.InterruptedIOException; import java.util.ArrayList; @@ -58,6 +32,32 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.NamespaceDescriptor; +import org.apache.hadoop.hbase.TableExistsException; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.filter.SingleColumnValueFilter; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.RetryCounter; +import org.apache.hadoop.hbase.util.RetryCounterFactory; +import org.apache.yetus.audience.InterfaceAudience; + +import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; + /* * Abstract class that provides an interface to the Replication Table. Which is currently * being used for WAL offset tracking. @@ -321,12 +321,12 @@ abstract class ReplicationTableBase { * * @param server name of the server * @return a ResultScanner over the QueueIds belonging to the server - * @throws IOException + * @throws IOException if getting the table or the scanner fails */ protected ResultScanner getQueuesBelongingToServer(String server) throws IOException { Scan scan = new Scan(); SingleColumnValueFilter filterMyQueues = new SingleColumnValueFilter(CF_QUEUE, COL_QUEUE_OWNER, - CompareOperator.EQUAL, Bytes.toBytes(server)); + CompareOperator.EQUAL, Bytes.toBytes(server)); scan.setFilter(filterMyQueues); scan.addColumn(CF_QUEUE, COL_QUEUE_OWNER); scan.addColumn(CF_QUEUE, COL_QUEUE_OWNER_HISTORY); @@ -341,7 +341,7 @@ abstract class ReplicationTableBase { * the CreateReplicationWorker thread. It is up to the caller of this method to close the * returned Table * @return the Replication Table when it is created - * @throws IOException + * @throws IOException if getting the table or the scanner fails */ protected Table getOrBlockOnReplicationTable() throws IOException { // Sleep until the Replication Table becomes available @@ -359,7 +359,7 @@ abstract class ReplicationTableBase { * Creates a new copy of the Replication Table and sets up the proper Table time outs for it * * @return the Replication Table - * @throws IOException + * @throws IOException if getting the table fails */ private Table getAndSetUpReplicationTable() throws IOException { Table replicationTable = connection.getTable(REPLICATION_TABLE_NAME); @@ -413,7 +413,7 @@ abstract class ReplicationTableBase { * Create the replication table with the provided HColumnDescriptor REPLICATION_COL_DESCRIPTOR * in TableBasedReplicationQueuesImpl * - * @throws IOException + * @throws IOException if creating the table fails */ private void createReplicationTable() throws IOException { HTableDescriptor replicationTableDescriptor = new HTableDescriptor(REPLICATION_TABLE_NAME); @@ -429,7 +429,6 @@ abstract class ReplicationTableBase { * Checks whether the Replication Table exists yet * * @return whether the Replication Table exists - * @throws IOException */ private boolean replicationTableExists() { try { diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTracker.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTracker.java index 95bb5db547d..93a32630d55 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTracker.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTracker.java @@ -35,7 +35,7 @@ public interface ReplicationTracker { /** * Register a replication listener to receive replication events. - * @param listener + * @param listener the listener to register */ public void registerListener(ReplicationListener listener); diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java index 9a1d9aaefb9..848052c8c28 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java @@ -22,13 +22,13 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.CopyOnWriteArrayList; -import org.apache.hadoop.hbase.zookeeper.ZKListener; -import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Stoppable; +import org.apache.hadoop.hbase.zookeeper.ZKListener; import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZKWatcher; +import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory;