From 3b7ac2a6d3380f0ff72a5a3b7c79db9c7814db5c Mon Sep 17 00:00:00 2001 From: Ashish Singhi Date: Fri, 4 Mar 2016 16:25:27 +0530 Subject: [PATCH] HBASE-15397 Create bulk load replication znode(hfile-refs) in ZK replication queue by default --- .../hbase/replication/ReplicationPeersZKImpl.java | 19 +++++++++---------- .../hbase/replication/ReplicationQueuesZKImpl.java | 14 ++++++-------- .../hbase/replication/ReplicationStateZKBase.java | 4 ---- 3 files changed, 15 insertions(+), 22 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java index fd10b66..e14f2c6 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java @@ -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 listOfOps = new ArrayList(); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java index 43dd412..764b5b0 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java @@ -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); } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java index 762167f..a1dc1c8 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java @@ -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"); -- 1.9.2.msysgit.0