Repository: hbase Updated Branches: refs/heads/branch-1 d089ac1ec -> 42c3fcabc
HBASE-15397 Create bulk load replication znode(hfile-refs) in ZK replication queue by default (Ashish Singhi) Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/42c3fcab Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/42c3fcab Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/42c3fcab Branch: refs/heads/branch-1 Commit: 42c3fcabc09fe4b05f99f78ce9aab9bdeb30932c Parents: d089ac1 Author: tedyu <[email protected]> Authored: Fri Mar 4 13:49:00 2016 -0800 Committer: tedyu <[email protected]> Committed: Fri Mar 4 13:49:00 2016 -0800 ---------------------------------------------------------------------- .../replication/ReplicationPeersZKImpl.java | 19 +++++++++---------- .../replication/ReplicationQueuesZKImpl.java | 14 ++++++-------- .../replication/ReplicationStateZKBase.java | 4 ---- 3 files changed, 15 insertions(+), 22 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/42c3fcab/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java ---------------------------------------------------------------------- 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 479b1c7..ad634fa 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<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>(); http://git-wip-us.apache.org/repos/asf/hbase/blob/42c3fcab/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java ---------------------------------------------------------------------- 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 e9be50b..93f3e27 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); } } http://git-wip-us.apache.org/repos/asf/hbase/blob/42c3fcab/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java ---------------------------------------------------------------------- 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");
