HBASE-15397 Create bulk load replication znode(hfile-refs) in ZK replication queue by default (Ashish Singhi)

This commit is contained in:
tedyu 2016-03-04 13:43:25 -08:00
parent 7dabcf23e8
commit 664575598e
3 changed files with 15 additions and 22 deletions

View File

@ -124,16 +124,15 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
// If only bulk load hfile replication is enabled then add peerId node to hfile-refs node
if (replicationForBulkLoadEnabled) {
try {
String peerId = ZKUtil.joinZNode(this.hfileRefsZNode, id);
LOG.info("Adding peer " + peerId + " to hfile reference queue.");
ZKUtil.createWithParents(this.zookeeper, peerId);
} catch (KeeperException e) {
throw new ReplicationException("Failed to add peer with id=" + id
+ ", node under hfile references node.", e);
}
// Irrespective of bulk load hfile replication is enabled or not we add peerId node to
// hfile-refs node -- HBASE-15397
try {
String peerId = ZKUtil.joinZNode(this.hfileRefsZNode, id);
LOG.info("Adding peer " + peerId + " to hfile reference queue.");
ZKUtil.createWithParents(this.zookeeper, peerId);
} catch (KeeperException e) {
throw new ReplicationException("Failed to add peer with id=" + id
+ ", node under hfile references node.", e);
}
List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();

View File

@ -84,14 +84,12 @@ public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements R
} catch (KeeperException e) {
throw new ReplicationException("Could not initialize replication queues.", e);
}
// If only bulk load hfile replication is enabled then create the hfile-refs znode
if (replicationForBulkLoadEnabled) {
try {
ZKUtil.createWithParents(this.zookeeper, this.hfileRefsZNode);
} catch (KeeperException e) {
throw new ReplicationException("Could not initialize hfile references replication queue.",
e);
}
// Irrespective of bulk load hfile replication is enabled or not we add peerId node to
// hfile-refs node -- HBASE-15397
try {
ZKUtil.createWithParents(this.zookeeper, this.hfileRefsZNode);
} catch (KeeperException e) {
throw new ReplicationException("Could not initialize hfile references replication queue.", e);
}
}

View File

@ -22,7 +22,6 @@ import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
@ -56,7 +55,6 @@ public abstract class ReplicationStateZKBase {
protected final ZooKeeperWatcher zookeeper;
protected final Configuration conf;
protected final Abortable abortable;
protected final boolean replicationForBulkLoadEnabled;
// Public for testing
public static final byte[] ENABLED_ZNODE_BYTES =
@ -72,8 +70,6 @@ public abstract class ReplicationStateZKBase {
this.zookeeper = zookeeper;
this.conf = conf;
this.abortable = abortable;
this.replicationForBulkLoadEnabled = conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
String peersZNodeName = conf.get("zookeeper.znode.replication.peers", "peers");