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;