You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2017/12/28 00:54:05 UTC

[01/26] hbase git commit: HBASE-19628 ByteBufferCell should extend ExtendedCell [Forced Update!]

Repository: hbase
Updated Branches:
  refs/heads/HBASE-19397 258114e5c -> 264890a91 (forced update)


http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
index aec9566..a0246a0 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java
@@ -39,7 +39,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.util.MapReduceCell;
+import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -674,7 +674,7 @@ public class TestImportExport {
       @Override
       public Void answer(InvocationOnMock invocation) throws Throwable {
         ImmutableBytesWritable writer = (ImmutableBytesWritable) invocation.getArgument(0);
-        MapReduceCell key = (MapReduceCell) invocation.getArgument(1);
+        MapReduceExtendedCell key = (MapReduceExtendedCell) invocation.getArgument(1);
         assertEquals("Key", Bytes.toString(writer.get()));
         assertEquals("row", Bytes.toString(CellUtil.cloneRow(key)));
         return null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java
index 23b3c04..1af9b1d 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java
@@ -47,12 +47,12 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.mapreduce.WALPlayer.WALKeyValueMapper;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MapReduceTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.LauncherSecurityManager;
-import org.apache.hadoop.hbase.util.MapReduceCell;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.mapreduce.Mapper;
@@ -185,7 +185,7 @@ public class TestWALPlayer {
       @Override
       public Void answer(InvocationOnMock invocation) throws Throwable {
         ImmutableBytesWritable writer = (ImmutableBytesWritable) invocation.getArgument(0);
-        MapReduceCell key = (MapReduceCell) invocation.getArgument(1);
+        MapReduceExtendedCell key = (MapReduceExtendedCell) invocation.getArgument(1);
         assertEquals("row", Bytes.toString(writer.get()));
         assertEquals("row", Bytes.toString(CellUtil.cloneRow(key)));
         return null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
index 50d5ddc..cfc3dd9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hbase.ByteBufferCell;
+import org.apache.hadoop.hbase.ByteBufferExtendedCell;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
@@ -383,14 +383,15 @@ public class HFileWriterImpl implements HFile.Writer {
           + CellUtil.getCellKeyAsString(left) + ", right=" + CellUtil.getCellKeyAsString(right));
     }
     byte[] midRow;
-    boolean bufferBacked = left instanceof ByteBufferCell && right instanceof ByteBufferCell;
+    boolean bufferBacked = left instanceof ByteBufferExtendedCell
+        && right instanceof ByteBufferExtendedCell;
     if (diff < 0) {
       // Left row is < right row.
       if (bufferBacked) {
-        midRow = getMinimumMidpointArray(((ByteBufferCell) left).getRowByteBuffer(),
-            ((ByteBufferCell) left).getRowPosition(), left.getRowLength(),
-            ((ByteBufferCell) right).getRowByteBuffer(),
-            ((ByteBufferCell) right).getRowPosition(), right.getRowLength());
+        midRow = getMinimumMidpointArray(((ByteBufferExtendedCell) left).getRowByteBuffer(),
+            ((ByteBufferExtendedCell) left).getRowPosition(), left.getRowLength(),
+            ((ByteBufferExtendedCell) right).getRowByteBuffer(),
+            ((ByteBufferExtendedCell) right).getRowPosition(), right.getRowLength());
       } else {
         midRow = getMinimumMidpointArray(left.getRowArray(), left.getRowOffset(),
             left.getRowLength(), right.getRowArray(), right.getRowOffset(), right.getRowLength());
@@ -407,10 +408,10 @@ public class HFileWriterImpl implements HFile.Writer {
     }
     if (diff < 0) {
       if (bufferBacked) {
-        midRow = getMinimumMidpointArray(((ByteBufferCell) left).getFamilyByteBuffer(),
-            ((ByteBufferCell) left).getFamilyPosition(), left.getFamilyLength(),
-            ((ByteBufferCell) right).getFamilyByteBuffer(),
-            ((ByteBufferCell) right).getFamilyPosition(), right.getFamilyLength());
+        midRow = getMinimumMidpointArray(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
+            ((ByteBufferExtendedCell) left).getFamilyPosition(), left.getFamilyLength(),
+            ((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
+            ((ByteBufferExtendedCell) right).getFamilyPosition(), right.getFamilyLength());
       } else {
         midRow = getMinimumMidpointArray(left.getFamilyArray(), left.getFamilyOffset(),
             left.getFamilyLength(), right.getFamilyArray(), right.getFamilyOffset(),
@@ -429,10 +430,10 @@ public class HFileWriterImpl implements HFile.Writer {
     }
     if (diff < 0) {
       if (bufferBacked) {
-        midRow = getMinimumMidpointArray(((ByteBufferCell) left).getQualifierByteBuffer(),
-            ((ByteBufferCell) left).getQualifierPosition(), left.getQualifierLength(),
-            ((ByteBufferCell) right).getQualifierByteBuffer(),
-            ((ByteBufferCell) right).getQualifierPosition(), right.getQualifierLength());
+        midRow = getMinimumMidpointArray(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
+            ((ByteBufferExtendedCell) left).getQualifierPosition(), left.getQualifierLength(),
+            ((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
+            ((ByteBufferExtendedCell) right).getQualifierPosition(), right.getQualifierLength());
       } else {
         midRow = getMinimumMidpointArray(left.getQualifierArray(), left.getQualifierOffset(),
             left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkCell.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkCell.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkCell.java
deleted file mode 100644
index 4597b55..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkCell.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.regionserver;
-
-import java.nio.ByteBuffer;
-
-import org.apache.hadoop.hbase.ByteBufferKeyValue;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.util.ByteBufferUtils;
-
-/**
- * ByteBuffer based cell which has the chunkid at the 0th offset
- * @see MemStoreLAB
- */
-//TODO : When moving this cell to CellChunkMap we will have the following things
-// to be serialized
-// chunkId (Integer) + offset (Integer) + length (Integer) + seqId (Long) = 20 bytes
-@InterfaceAudience.Private
-public class ByteBufferChunkCell extends ByteBufferKeyValue {
-  public ByteBufferChunkCell(ByteBuffer buf, int offset, int length) {
-    super(buf, offset, length);
-  }
-
-  public ByteBufferChunkCell(ByteBuffer buf, int offset, int length, long seqId) {
-    super(buf, offset, length, seqId);
-  }
-
-  @Override
-  public int getChunkId() {
-    // The chunkId is embedded at the 0th offset of the bytebuffer
-    return ByteBufferUtils.toInt(buf, 0);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkKeyValue.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkKeyValue.java
new file mode 100644
index 0000000..8278c42
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ByteBufferChunkKeyValue.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.ByteBufferKeyValue;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
+
+/**
+ * ByteBuffer based cell which has the chunkid at the 0th offset
+ * @see MemStoreLAB
+ */
+//TODO : When moving this cell to CellChunkMap we will have the following things
+// to be serialized
+// chunkId (Integer) + offset (Integer) + length (Integer) + seqId (Long) = 20 bytes
+@InterfaceAudience.Private
+public class ByteBufferChunkKeyValue extends ByteBufferKeyValue {
+  public ByteBufferChunkKeyValue(ByteBuffer buf, int offset, int length) {
+    super(buf, offset, length);
+  }
+
+  public ByteBufferChunkKeyValue(ByteBuffer buf, int offset, int length, long seqId) {
+    super(buf, offset, length, seqId);
+  }
+
+  @Override
+  public int getChunkId() {
+    // The chunkId is embedded at the 0th offset of the bytebuffer
+    return ByteBufferUtils.toInt(buf, 0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkMap.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkMap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkMap.java
index 3fbd46d..a3b45c3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkMap.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkMap.java
@@ -122,6 +122,6 @@ public class CellChunkMap extends CellFlatMap {
           + chunk.isFromPool() + ". We were looking for a cell at index " + i);
     }
 
-    return new ByteBufferChunkCell(buf, offsetOfCell, lengthOfCell, cellSeqID);
+    return new ByteBufferChunkKeyValue(buf, offsetOfCell, lengthOfCell, cellSeqID);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
index 6a253fa..41a6cf8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreLABImpl.java
@@ -159,9 +159,9 @@ public class MemStoreLABImpl implements MemStoreLAB {
       // which directly return tagsLen as 0. So we avoid parsing many length components in
       // reading the tagLength stored in the backing buffer. The Memstore addition of every Cell
       // call getTagsLength().
-      return new NoTagByteBufferChunkCell(buf, offset, len, cell.getSequenceId());
+      return new NoTagByteBufferChunkKeyValue(buf, offset, len, cell.getSequenceId());
     } else {
-      return new ByteBufferChunkCell(buf, offset, len, cell.getSequenceId());
+      return new ByteBufferChunkKeyValue(buf, offset, len, cell.getSequenceId());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkCell.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkCell.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkCell.java
deleted file mode 100644
index dba8681..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkCell.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.regionserver;
-
-import java.nio.ByteBuffer;
-
-import org.apache.hadoop.hbase.NoTagsByteBufferKeyValue;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.util.ByteBufferUtils;
-
-
-/**
- * ByteBuffer based cell which has the chunkid at the 0th offset and with no tags
- * @see MemStoreLAB
- */
-@InterfaceAudience.Private
-public class NoTagByteBufferChunkCell extends NoTagsByteBufferKeyValue {
-
-  public NoTagByteBufferChunkCell(ByteBuffer buf, int offset, int length) {
-    super(buf, offset, length);
-  }
-
-  public NoTagByteBufferChunkCell(ByteBuffer buf, int offset, int length, long seqId) {
-    super(buf, offset, length, seqId);
-  }
-
-  @Override
-  public int getChunkId() {
-    // The chunkId is embedded at the 0th offset of the bytebuffer
-    return ByteBufferUtils.toInt(buf, 0);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkKeyValue.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkKeyValue.java
new file mode 100644
index 0000000..1fb533a
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/NoTagByteBufferChunkKeyValue.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.hbase.NoTagsByteBufferKeyValue;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
+
+
+/**
+ * ByteBuffer based cell which has the chunkid at the 0th offset and with no tags
+ * @see MemStoreLAB
+ */
+@InterfaceAudience.Private
+public class NoTagByteBufferChunkKeyValue extends NoTagsByteBufferKeyValue {
+
+  public NoTagByteBufferChunkKeyValue(ByteBuffer buf, int offset, int length) {
+    super(buf, offset, length);
+  }
+
+  public NoTagByteBufferChunkKeyValue(ByteBuffer buf, int offset, int length, long seqId) {
+    super(buf, offset, length, seqId);
+  }
+
+  @Override
+  public int getChunkId() {
+    // The chunkId is embedded at the 0th offset of the bytebuffer
+    return ByteBufferUtils.toInt(buf, 0);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 584d0a2..4091316 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -46,7 +46,7 @@ import java.util.concurrent.atomic.LongAdder;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.ByteBufferCell;
+import org.apache.hadoop.hbase.ByteBufferExtendedCell;
 import org.apache.hadoop.hbase.CacheEvictionStats;
 import org.apache.hadoop.hbase.CacheEvictionStatsBuilder;
 import org.apache.hadoop.hbase.Cell;
@@ -1295,8 +1295,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
         // Since byte buffers can point all kinds of crazy places it's harder to keep track
         // of which blocks are kept alive by what byte buffer.
         // So we make a guess.
-        if (c instanceof ByteBufferCell) {
-          ByteBufferCell bbCell = (ByteBufferCell) c;
+        if (c instanceof ByteBufferExtendedCell) {
+          ByteBufferExtendedCell bbCell = (ByteBufferExtendedCell) c;
           ByteBuffer bb = bbCell.getValueByteBuffer();
           if (bb != lastBlock) {
             context.incrementResponseBlockSize(bb.capacity());

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java
index 7e14228..f43f147 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestBufferedDataBlockEncoder.java
@@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags;
-import org.apache.hadoop.hbase.io.encoding.BufferedDataBlockEncoder.OffheapDecodedCell;
 import org.apache.hadoop.hbase.io.encoding.BufferedDataBlockEncoder.OnheapDecodedCell;
 import org.apache.hadoop.hbase.codec.Codec.Decoder;
 import org.apache.hadoop.hbase.codec.Codec.Encoder;
@@ -108,7 +107,8 @@ public class TestBufferedDataBlockEncoder {
         kv2.getTagsLength());
     KeyValue kv3 = new KeyValue(Bytes.toBytes("r3"), Bytes.toBytes("cf"), Bytes.toBytes("qual"),
         HConstants.LATEST_TIMESTAMP, Bytes.toBytes("3"));
-    OffheapDecodedCell c3 = new OffheapDecodedCell(ByteBuffer.wrap(kv2.getKey()),
+    BufferedDataBlockEncoder.OffheapDecodedExtendedCell
+        c3 = new BufferedDataBlockEncoder.OffheapDecodedExtendedCell(ByteBuffer.wrap(kv2.getKey()),
         kv2.getRowLength(), kv2.getFamilyOffset() - KeyValue.ROW_OFFSET, kv2.getFamilyLength(),
         kv2.getQualifierOffset() - KeyValue.ROW_OFFSET, kv2.getQualifierLength(),
         kv2.getTimestamp(), kv2.getTypeByte(), ByteBuffer.wrap(kv2.getValueArray()),


[14/26] hbase git commit: HBASE-19579 Add peer lock test for shell command list_locks

Posted by zh...@apache.org.
HBASE-19579 Add peer lock test for shell command list_locks

Signed-off-by: zhangduo <zh...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/08f51e86
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/08f51e86
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/08f51e86

Branch: refs/heads/HBASE-19397
Commit: 08f51e8664cd03b4198752176ecc1c1fcfc31917
Parents: 8af1ce3
Author: Guanghao Zhang <zg...@apache.org>
Authored: Sat Dec 23 21:04:27 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Dec 28 08:51:59 2017 +0800

----------------------------------------------------------------------
 .../src/main/protobuf/LockService.proto          |  1 +
 .../src/test/ruby/shell/list_locks_test.rb       | 19 +++++++++++++++++++
 2 files changed, 20 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/08f51e86/hbase-protocol-shaded/src/main/protobuf/LockService.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/LockService.proto b/hbase-protocol-shaded/src/main/protobuf/LockService.proto
index b8d180c..0675070 100644
--- a/hbase-protocol-shaded/src/main/protobuf/LockService.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/LockService.proto
@@ -77,6 +77,7 @@ enum LockedResourceType {
   NAMESPACE = 2;
   TABLE = 3;
   REGION = 4;
+  PEER = 5;
 }
 
 message LockedResource {

http://git-wip-us.apache.org/repos/asf/hbase/blob/08f51e86/hbase-shell/src/test/ruby/shell/list_locks_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/shell/list_locks_test.rb b/hbase-shell/src/test/ruby/shell/list_locks_test.rb
index f465a6b..ef1c0ce 100644
--- a/hbase-shell/src/test/ruby/shell/list_locks_test.rb
+++ b/hbase-shell/src/test/ruby/shell/list_locks_test.rb
@@ -67,6 +67,25 @@ module Hbase
         proc_id)
     end
 
+    define_test 'list peer locks' do
+      lock = create_exclusive_lock(0)
+      peer_id = '1'
+
+      @scheduler.waitPeerExclusiveLock(lock, peer_id)
+      output = capture_stdout { @list_locks.command }
+      @scheduler.wakePeerExclusiveLock(lock, peer_id)
+
+      assert_equal(
+        "PEER(1)\n" \
+        "Lock type: EXCLUSIVE, procedure: {" \
+          "\"className\"=>\"org.apache.hadoop.hbase.master.locking.LockProcedure\", " \
+          "\"procId\"=>\"0\", \"submittedTime\"=>\"0\", \"state\"=>\"RUNNABLE\", " \
+          "\"lastUpdate\"=>\"0\", " \
+          "\"stateMessage\"=>[{\"lockType\"=>\"EXCLUSIVE\", \"description\"=>\"description\"}]" \
+        "}\n\n",
+        output)
+    end
+
     define_test 'list server locks' do
       lock = create_exclusive_lock(0)
 


[12/26] hbase git commit: HBASE-19216 Implement a general framework to execute remote procedure on RS

Posted by zh...@apache.org.
HBASE-19216 Implement a general framework to execute remote procedure on RS


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/f9ea9617
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/f9ea9617
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/f9ea9617

Branch: refs/heads/HBASE-19397
Commit: f9ea9617d1d923f578371c95eb904f7034fbe8c0
Parents: 2c65f03
Author: zhangduo <zh...@apache.org>
Authored: Fri Dec 15 21:06:44 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Dec 28 08:51:59 2017 +0800

----------------------------------------------------------------------
 .../hbase/procedure2/LockedResourceType.java    |   4 +-
 .../procedure2/RemoteProcedureDispatcher.java   |  23 +-
 .../src/main/protobuf/Admin.proto               |   9 +-
 .../src/main/protobuf/MasterProcedure.proto     |  30 +++
 .../src/main/protobuf/RegionServerStatus.proto  |  15 ++
 .../apache/hadoop/hbase/executor/EventType.java |  26 ++-
 .../hadoop/hbase/executor/ExecutorType.java     |   3 +-
 .../org/apache/hadoop/hbase/master/HMaster.java |  33 ++-
 .../hadoop/hbase/master/MasterRpcServices.java  |  13 ++
 .../assignment/RegionTransitionProcedure.java   |  18 +-
 .../procedure/MasterProcedureScheduler.java     | 224 +++++++++++++------
 .../procedure/PeerProcedureInterface.java       |  34 +++
 .../master/procedure/RSProcedureDispatcher.java |  90 ++++----
 .../master/replication/ModifyPeerProcedure.java | 127 +++++++++++
 .../master/replication/RefreshPeerCallable.java |  67 ++++++
 .../replication/RefreshPeerProcedure.java       | 197 ++++++++++++++++
 .../hbase/procedure2/RSProcedureCallable.java   |  43 ++++
 .../hbase/regionserver/HRegionServer.java       |  69 +++++-
 .../hbase/regionserver/RSRpcServices.java       |  56 +++--
 .../handler/RSProcedureHandler.java             |  51 +++++
 .../assignment/TestAssignmentManager.java       |  20 +-
 .../replication/DummyModifyPeerProcedure.java   |  41 ++++
 .../TestDummyModifyPeerProcedure.java           |  80 +++++++
 .../security/access/TestAccessController.java   |   6 +-
 24 files changed, 1109 insertions(+), 170 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f9ea9617/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java
index c5fe62b..dc9b5d4 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java
@@ -1,4 +1,4 @@
-/*
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -22,5 +22,5 @@ import org.apache.yetus.audience.InterfaceAudience;
 
 @InterfaceAudience.Private
 public enum LockedResourceType {
-  SERVER, NAMESPACE, TABLE, REGION
+  SERVER, NAMESPACE, TABLE, REGION, PEER
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9ea9617/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
index 54f2b08..e9a6906 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
@@ -226,13 +226,30 @@ public abstract class RemoteProcedureDispatcher<TEnv, TRemote extends Comparable
 
   /**
    * Remote procedure reference.
-   * @param <TEnv>
-   * @param <TRemote>
    */
   public interface RemoteProcedure<TEnv, TRemote> {
+    /**
+     * For building the remote operation.
+     */
     RemoteOperation remoteCallBuild(TEnv env, TRemote remote);
-    void remoteCallCompleted(TEnv env, TRemote remote, RemoteOperation response);
+
+    /**
+     * Called when the executeProcedure call is failed.
+     */
     void remoteCallFailed(TEnv env, TRemote remote, IOException exception);
+
+    /**
+     * Called when RS tells the remote procedure is succeeded through the
+     * {@code reportProcedureDone} method.
+     */
+    void remoteOperationCompleted(TEnv env);
+
+    /**
+     * Called when RS tells the remote procedure is failed through the {@code reportProcedureDone}
+     * method.
+     * @param error the error message
+     */
+    void remoteOperationFailed(TEnv env, String error);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9ea9617/hbase-protocol-shaded/src/main/protobuf/Admin.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Admin.proto b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
index 118c79b..ddcc266 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Admin.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Admin.proto
@@ -256,14 +256,19 @@ message ClearRegionBlockCacheResponse {
   required CacheEvictionStats stats = 1;
 }
 
+message RemoteProcedureRequest {
+  required uint64 proc_id = 1;
+  required string proc_class = 2;
+  optional bytes proc_data = 3;
+}
+
 message ExecuteProceduresRequest {
   repeated OpenRegionRequest open_region = 1;
   repeated CloseRegionRequest close_region = 2;
+  repeated RemoteProcedureRequest proc = 3;
 }
 
 message ExecuteProceduresResponse {
-  repeated OpenRegionResponse open_region = 1;
-  repeated CloseRegionResponse close_region = 2;
 }
 
 service AdminService {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9ea9617/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
index f9b8807..0e2bdba 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -365,3 +365,33 @@ message GCMergedRegionsStateData {
   required RegionInfo parent_b = 2;
   required RegionInfo merged_child = 3;
 }
+
+enum PeerModificationState {
+  UPDATE_PEER_STORAGE = 1;
+  REFRESH_PEER_ON_RS = 2;
+  POST_PEER_MODIFICATION = 3;
+}
+
+message PeerModificationStateData {
+  required string peer_id = 1;
+}
+
+enum PeerModificationType {
+  ADD_PEER = 1;
+  REMOVE_PEER = 2;
+  ENABLE_PEER = 3;
+  DISABLE_PEER = 4;
+  UPDATE_PEER_CONFIG = 5;
+}
+
+message RefreshPeerStateData {
+  required string peer_id = 1;
+  required PeerModificationType type = 2;
+  required ServerName target_server = 3;
+}
+
+message RefreshPeerParameter {
+  required string peer_id = 1;
+  required PeerModificationType type = 2;
+  required ServerName target_server = 3;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9ea9617/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
index f83bb20..eb396ac 100644
--- a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
@@ -143,7 +143,19 @@ message RegionSpaceUseReportRequest {
 }
 
 message RegionSpaceUseReportResponse {
+}
 
+message ReportProcedureDoneRequest {
+  required uint64 proc_id = 1;
+  enum Status {
+    SUCCESS = 1;
+    ERROR = 2;
+  }
+  required Status status = 2;
+  optional string error = 3;
+}
+
+message ReportProcedureDoneResponse {
 }
 
 service RegionServerStatusService {
@@ -181,4 +193,7 @@ service RegionServerStatusService {
    */
   rpc ReportRegionSpaceUse(RegionSpaceUseReportRequest)
     returns(RegionSpaceUseReportResponse);
+
+  rpc ReportProcedureDone(ReportProcedureDoneRequest)
+    returns(ReportProcedureDoneResponse);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9ea9617/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java
index 26fb63a..922deb8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventType.java
@@ -20,15 +20,14 @@ package org.apache.hadoop.hbase.executor;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
- * List of all HBase event handler types.  Event types are named by a
- * convention: event type names specify the component from which the event
- * originated and then where its destined -- e.g. RS2ZK_ prefix means the
- * event came from a regionserver destined for zookeeper -- and then what
- * the even is; e.g. REGION_OPENING.
- *
- * <p>We give the enums indices so we can add types later and keep them
- * grouped together rather than have to add them always to the end as we
- * would have to if we used raw enum ordinals.
+ * List of all HBase event handler types.
+ * <p>
+ * Event types are named by a convention: event type names specify the component from which the
+ * event originated and then where its destined -- e.g. RS_ZK_ prefix means the event came from a
+ * regionserver destined for zookeeper -- and then what the even is; e.g. REGION_OPENING.
+ * <p>
+ * We give the enums indices so we can add types later and keep them grouped together rather than
+ * have to add them always to the end as we would have to if we used raw enum ordinals.
  */
 @InterfaceAudience.Private
 public enum EventType {
@@ -275,7 +274,14 @@ public enum EventType {
    *
    * RS_COMPACTED_FILES_DISCHARGER
    */
-  RS_COMPACTED_FILES_DISCHARGER (83, ExecutorType.RS_COMPACTED_FILES_DISCHARGER);
+  RS_COMPACTED_FILES_DISCHARGER (83, ExecutorType.RS_COMPACTED_FILES_DISCHARGER),
+
+  /**
+   * RS refresh peer.<br>
+   *
+   * RS_REFRESH_PEER
+   */
+  RS_REFRESH_PEER (84, ExecutorType.RS_REFRESH_PEER);
 
   private final int code;
   private final ExecutorType executor;

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9ea9617/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java
index c75a0a9..7f130d1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/ExecutorType.java
@@ -46,7 +46,8 @@ public enum ExecutorType {
   RS_LOG_REPLAY_OPS          (27),
   RS_REGION_REPLICA_FLUSH_OPS  (28),
   RS_COMPACTED_FILES_DISCHARGER (29),
-  RS_OPEN_PRIORITY_REGION    (30);
+  RS_OPEN_PRIORITY_REGION    (30),
+  RS_REFRESH_PEER               (31);
 
   ExecutorType(int value) {
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9ea9617/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index efa17a4..9d0555f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -140,6 +139,7 @@ import org.apache.hadoop.hbase.procedure2.LockedResource;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
 import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.quotas.MasterSpaceQuotaObserver;
@@ -328,8 +328,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   // flag set after we become the active master (used for testing)
   private volatile boolean activeMaster = false;
 
-  // flag set after we complete initialization once active,
-  // it is not private since it's used in unit tests
+  // flag set after we complete initialization once active
   private final ProcedureEvent initialized = new ProcedureEvent("master initialized");
 
   // flag set after master services are started,
@@ -3525,4 +3524,28 @@ public class HMaster extends HRegionServer implements MasterServices {
   public SpaceQuotaSnapshotNotifier getSpaceQuotaSnapshotNotifier() {
     return this.spaceQuotaSnapshotNotifier;
   }
-}
+
+  @SuppressWarnings("unchecked")
+  private RemoteProcedure<MasterProcedureEnv, ?> getRemoteProcedure(long procId) {
+    Procedure<?> procedure = procedureExecutor.getProcedure(procId);
+    if (procedure == null) {
+      return null;
+    }
+    assert procedure instanceof RemoteProcedure;
+    return (RemoteProcedure<MasterProcedureEnv, ?>) procedure;
+  }
+
+  public void remoteProcedureCompleted(long procId) {
+    RemoteProcedure<MasterProcedureEnv, ?> procedure = getRemoteProcedure(procId);
+    if (procedure != null) {
+      procedure.remoteOperationCompleted(procedureExecutor.getEnvironment());
+    }
+  }
+
+  public void remoteProcedureFailed(long procId, String error) {
+    RemoteProcedure<MasterProcedureEnv, ?> procedure = getRemoteProcedure(procId);
+    if (procedure != null) {
+      procedure.remoteOperationFailed(procedureExecutor.getEnvironment(), error);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9ea9617/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 2e4b5e1..1d6949c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -264,6 +264,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
@@ -2247,4 +2249,15 @@ public class MasterRpcServices extends RSRpcServices
     }
     return response.build();
   }
+
+  @Override
+  public ReportProcedureDoneResponse reportProcedureDone(RpcController controller,
+      ReportProcedureDoneRequest request) throws ServiceException {
+    if (request.getStatus() == ReportProcedureDoneRequest.Status.SUCCESS) {
+      master.remoteProcedureCompleted(request.getProcId());
+    } else {
+      master.remoteProcedureFailed(request.getProcId(), request.getError());
+    }
+    return ReportProcedureDoneResponse.getDefaultInstance();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9ea9617/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
index 6e671c8..43e75ed 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
@@ -173,12 +173,6 @@ public abstract class RegionTransitionProcedure
       RegionStateNode regionNode, IOException exception);
 
   @Override
-  public void remoteCallCompleted(final MasterProcedureEnv env,
-      final ServerName serverName, final RemoteOperation response) {
-    // Ignore the response? reportTransition() is the one that count?
-  }
-
-  @Override
   public void remoteCallFailed(final MasterProcedureEnv env,
       final ServerName serverName, final IOException exception) {
     final RegionStateNode regionNode = getRegionState(env);
@@ -413,4 +407,16 @@ public abstract class RegionTransitionProcedure
    * @return ServerName the Assign or Unassign is going against.
    */
   public abstract ServerName getServer(final MasterProcedureEnv env);
+
+  @Override
+  public void remoteOperationCompleted(MasterProcedureEnv env) {
+    // should not be called for region operation until we modified the open/close region procedure
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void remoteOperationFailed(MasterProcedureEnv env, String error) {
+    // should not be called for region operation until we modified the open/close region procedure
+    throw new UnsupportedOperationException();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9ea9617/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
index c60de5c..dd85f5c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
@@ -24,7 +24,7 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
+import java.util.function.Function;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ServerName;
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.master.locking.LockProcedure;
+import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface.PeerOperationType;
 import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface.TableOperationType;
 import org.apache.hadoop.hbase.procedure2.AbstractProcedureScheduler;
 import org.apache.hadoop.hbase.procedure2.LockAndQueue;
@@ -109,12 +110,17 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       new ServerQueueKeyComparator();
   private final static TableQueueKeyComparator TABLE_QUEUE_KEY_COMPARATOR =
       new TableQueueKeyComparator();
+  private final static PeerQueueKeyComparator PEER_QUEUE_KEY_COMPARATOR =
+      new PeerQueueKeyComparator();
 
   private final FairQueue<ServerName> serverRunQueue = new FairQueue<>();
   private final FairQueue<TableName> tableRunQueue = new FairQueue<>();
+  private final FairQueue<String> peerRunQueue = new FairQueue<>();
 
   private final ServerQueue[] serverBuckets = new ServerQueue[128];
   private TableQueue tableMap = null;
+  private PeerQueue peerMap = null;
+
   private final SchemaLocking locking = new SchemaLocking();
 
   /**
@@ -161,6 +167,8 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       doAdd(tableRunQueue, getTableQueue(getTableName(proc)), proc, addFront);
     } else if (isServerProcedure(proc)) {
       doAdd(serverRunQueue, getServerQueue(getServerName(proc)), proc, addFront);
+    } else if (isPeerProcedure(proc)) {
+      doAdd(peerRunQueue, getPeerQueue(getPeerId(proc)), proc, addFront);
     } else {
       // TODO: at the moment we only have Table and Server procedures
       // if you are implementing a non-table/non-server procedure, you have two options: create
@@ -172,7 +180,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
   }
 
   private <T extends Comparable<T>> void doAdd(final FairQueue<T> fairq,
-      final Queue<T> queue, final Procedure proc, final boolean addFront) {
+      final Queue<T> queue, final Procedure<?> proc, final boolean addFront) {
     queue.add(proc, addFront);
     if (!queue.getLockStatus().hasExclusiveLock() || queue.getLockStatus().isLockOwner(proc.getProcId())) {
       // if the queue was not remove for an xlock execution
@@ -189,7 +197,8 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
 
   @Override
   protected boolean queueHasRunnables() {
-    return tableRunQueue.hasRunnables() || serverRunQueue.hasRunnables();
+    return tableRunQueue.hasRunnables() || serverRunQueue.hasRunnables() ||
+        peerRunQueue.hasRunnables();
   }
 
   @Override
@@ -197,7 +206,10 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     // For now, let server handling have precedence over table handling; presumption is that it
     // is more important handling crashed servers than it is running the
     // enabling/disabling tables, etc.
-    Procedure pollResult = doPoll(serverRunQueue);
+    Procedure<?> pollResult = doPoll(serverRunQueue);
+    if (pollResult == null) {
+      pollResult = doPoll(peerRunQueue);
+    }
     if (pollResult == null) {
       pollResult = doPoll(tableRunQueue);
     }
@@ -267,60 +279,30 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
         exclusiveLockOwnerProcedure, sharedLockCount, waitingProcedures);
   }
 
+  private <T> void addToLockedResources(List<LockedResource> lockedResources,
+      Map<T, LockAndQueue> locks, Function<T, String> keyTransformer,
+      LockedResourceType resourcesType) {
+    locks.entrySet().stream().filter(e -> e.getValue().isLocked())
+        .map(
+          e -> createLockedResource(resourcesType, keyTransformer.apply(e.getKey()), e.getValue()))
+        .forEachOrdered(lockedResources::add);
+  }
+
   @Override
   public List<LockedResource> getLocks() {
     schedLock();
-
     try {
       List<LockedResource> lockedResources = new ArrayList<>();
-
-      for (Entry<ServerName, LockAndQueue> entry : locking.serverLocks
-          .entrySet()) {
-        String serverName = entry.getKey().getServerName();
-        LockAndQueue queue = entry.getValue();
-
-        if (queue.isLocked()) {
-          LockedResource lockedResource =
-            createLockedResource(LockedResourceType.SERVER, serverName, queue);
-          lockedResources.add(lockedResource);
-        }
-      }
-
-      for (Entry<String, LockAndQueue> entry : locking.namespaceLocks
-          .entrySet()) {
-        String namespaceName = entry.getKey();
-        LockAndQueue queue = entry.getValue();
-
-        if (queue.isLocked()) {
-          LockedResource lockedResource =
-            createLockedResource(LockedResourceType.NAMESPACE, namespaceName, queue);
-          lockedResources.add(lockedResource);
-        }
-      }
-
-      for (Entry<TableName, LockAndQueue> entry : locking.tableLocks
-          .entrySet()) {
-        String tableName = entry.getKey().getNameAsString();
-        LockAndQueue queue = entry.getValue();
-
-        if (queue.isLocked()) {
-          LockedResource lockedResource =
-            createLockedResource(LockedResourceType.TABLE, tableName, queue);
-          lockedResources.add(lockedResource);
-        }
-      }
-
-      for (Entry<String, LockAndQueue> entry : locking.regionLocks.entrySet()) {
-        String regionName = entry.getKey();
-        LockAndQueue queue = entry.getValue();
-
-        if (queue.isLocked()) {
-          LockedResource lockedResource =
-            createLockedResource(LockedResourceType.REGION, regionName, queue);
-          lockedResources.add(lockedResource);
-        }
-      }
-
+      addToLockedResources(lockedResources, locking.serverLocks, sn -> sn.getServerName(),
+        LockedResourceType.SERVER);
+      addToLockedResources(lockedResources, locking.namespaceLocks, Function.identity(),
+        LockedResourceType.NAMESPACE);
+      addToLockedResources(lockedResources, locking.tableLocks, tn -> tn.getNameAsString(),
+        LockedResourceType.TABLE);
+      addToLockedResources(lockedResources, locking.regionLocks, Function.identity(),
+        LockedResourceType.REGION);
+      addToLockedResources(lockedResources, locking.peerLocks, Function.identity(),
+        LockedResourceType.PEER);
       return lockedResources;
     } finally {
       schedUnlock();
@@ -328,8 +310,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
   }
 
   @Override
-  public LockedResource getLockResource(LockedResourceType resourceType,
-      String resourceName) {
+  public LockedResource getLockResource(LockedResourceType resourceType, String resourceName) {
     LockAndQueue queue = null;
     schedLock();
     try {
@@ -346,8 +327,10 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
         case REGION:
           queue = locking.regionLocks.get(resourceName);
           break;
+        case PEER:
+          queue = locking.peerLocks.get(resourceName);
+          break;
       }
-
       return queue != null ? createLockedResource(resourceType, resourceName, queue) : null;
     } finally {
       schedUnlock();
@@ -431,6 +414,11 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
         markTableAsDeleted(iProcTable.getTableName(), proc);
         return;
       }
+    } else if (proc instanceof PeerProcedureInterface) {
+      PeerProcedureInterface iProcPeer = (PeerProcedureInterface) proc;
+      if (iProcPeer.getPeerOperationType() == PeerOperationType.REMOVE) {
+        removePeerQueue(iProcPeer.getPeerId());
+      }
     } else {
       // No cleanup for ServerProcedureInterface types, yet.
       return;
@@ -468,12 +456,11 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     locking.removeTableLock(tableName);
   }
 
-
-  private static boolean isTableProcedure(Procedure proc) {
+  private static boolean isTableProcedure(Procedure<?> proc) {
     return proc instanceof TableProcedureInterface;
   }
 
-  private static TableName getTableName(Procedure proc) {
+  private static TableName getTableName(Procedure<?> proc) {
     return ((TableProcedureInterface)proc).getTableName();
   }
 
@@ -494,15 +481,42 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     return Math.abs(hashCode) % buckets.length;
   }
 
-  private static boolean isServerProcedure(Procedure proc) {
+  private static boolean isServerProcedure(Procedure<?> proc) {
     return proc instanceof ServerProcedureInterface;
   }
 
-  private static ServerName getServerName(Procedure proc) {
+  private static ServerName getServerName(Procedure<?> proc) {
     return ((ServerProcedureInterface)proc).getServerName();
   }
 
   // ============================================================================
+  //  Peer Queue Lookup Helpers
+  // ============================================================================
+  private PeerQueue getPeerQueue(String peerId) {
+    PeerQueue node = AvlTree.get(peerMap, peerId, PEER_QUEUE_KEY_COMPARATOR);
+    if (node != null) {
+      return node;
+    }
+    node = new PeerQueue(peerId, locking.getPeerLock(peerId));
+    peerMap = AvlTree.insert(peerMap, node);
+    return node;
+  }
+
+  private void removePeerQueue(String peerId) {
+    peerMap = AvlTree.remove(peerMap, peerId, PEER_QUEUE_KEY_COMPARATOR);
+    locking.removePeerLock(peerId);
+  }
+
+
+  private static boolean isPeerProcedure(Procedure<?> proc) {
+    return proc instanceof PeerProcedureInterface;
+  }
+
+  private static String getPeerId(Procedure<?> proc) {
+    return ((PeerProcedureInterface) proc).getPeerId();
+  }
+
+  // ============================================================================
   //  Table and Server Queue Implementation
   // ============================================================================
   private static class ServerQueueKeyComparator implements AvlKeyComparator<ServerQueue> {
@@ -571,6 +585,26 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     }
   }
 
+  private static class PeerQueueKeyComparator implements AvlKeyComparator<PeerQueue> {
+
+    @Override
+    public int compareKey(PeerQueue node, Object key) {
+      return node.compareKey((String) key);
+    }
+  }
+
+  public static class PeerQueue extends Queue<String> {
+
+    public PeerQueue(String peerId, LockStatus lockStatus) {
+      super(peerId, lockStatus);
+    }
+
+    @Override
+    public boolean requireExclusiveLock(Procedure proc) {
+      return requirePeerExclusiveLock((PeerProcedureInterface) proc);
+    }
+  }
+
   // ============================================================================
   //  Table Locking Helpers
   // ============================================================================
@@ -958,7 +992,8 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
    * @param serverName Server to lock
    * @return true if the procedure has to wait for the server to be available
    */
-  public boolean waitServerExclusiveLock(final Procedure procedure, final ServerName serverName) {
+  public boolean waitServerExclusiveLock(final Procedure<?> procedure,
+      final ServerName serverName) {
     schedLock();
     try {
       final LockAndQueue lock = locking.getServerLock(serverName);
@@ -980,7 +1015,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
    * @param procedure the procedure releasing the lock
    * @param serverName the server that has the exclusive lock
    */
-  public void wakeServerExclusiveLock(final Procedure procedure, final ServerName serverName) {
+  public void wakeServerExclusiveLock(final Procedure<?> procedure, final ServerName serverName) {
     schedLock();
     try {
       final LockAndQueue lock = locking.getServerLock(serverName);
@@ -994,6 +1029,56 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
   }
 
   // ============================================================================
+  //  Peer Locking Helpers
+  // ============================================================================
+
+  private static boolean requirePeerExclusiveLock(PeerProcedureInterface proc) {
+    return proc.getPeerOperationType() != PeerOperationType.REFRESH;
+  }
+
+  /**
+   * Try to acquire the exclusive lock on the specified peer.
+   * @see #wakePeerExclusiveLock(Procedure, String)
+   * @param procedure the procedure trying to acquire the lock
+   * @param peerId peer to lock
+   * @return true if the procedure has to wait for the per to be available
+   */
+  public boolean waitPeerExclusiveLock(Procedure<?> procedure, String peerId) {
+    schedLock();
+    try {
+      final LockAndQueue lock = locking.getPeerLock(peerId);
+      if (lock.tryExclusiveLock(procedure)) {
+        removeFromRunQueue(peerRunQueue, getPeerQueue(peerId));
+        return false;
+      }
+      waitProcedure(lock, procedure);
+      logLockedResource(LockedResourceType.PEER, peerId);
+      return true;
+    } finally {
+      schedUnlock();
+    }
+  }
+
+  /**
+   * Wake the procedures waiting for the specified peer
+   * @see #waitPeerExclusiveLock(Procedure, String)
+   * @param procedure the procedure releasing the lock
+   * @param peerId the peer that has the exclusive lock
+   */
+  public void wakePeerExclusiveLock(Procedure<?> procedure, String peerId) {
+    schedLock();
+    try {
+      final LockAndQueue lock = locking.getPeerLock(peerId);
+      lock.releaseExclusiveLock(procedure);
+      addToRunQueue(peerRunQueue, getPeerQueue(peerId));
+      int waitingCount = wakeWaitingProcedures(lock);
+      wakePollIfNeeded(waitingCount);
+    } finally {
+      schedUnlock();
+    }
+  }
+
+  // ============================================================================
   //  Generic Helpers
   // ============================================================================
   private static abstract class Queue<TKey extends Comparable<TKey>>
@@ -1098,6 +1183,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     final Map<TableName, LockAndQueue> tableLocks = new HashMap<>();
     // Single map for all regions irrespective of tables. Key is encoded region name.
     final Map<String, LockAndQueue> regionLocks = new HashMap<>();
+    final Map<String, LockAndQueue> peerLocks = new HashMap<>();
 
     private <T> LockAndQueue getLock(Map<T, LockAndQueue> map, T key) {
       LockAndQueue lock = map.get(key);
@@ -1132,6 +1218,14 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       return getLock(serverLocks, serverName);
     }
 
+    LockAndQueue getPeerLock(String peerId) {
+      return getLock(peerLocks, peerId);
+    }
+
+    LockAndQueue removePeerLock(String peerId) {
+      return peerLocks.remove(peerId);
+    }
+
     /**
      * Removes all locks by clearing the maps.
      * Used when procedure executor is stopped for failure and recovery testing.
@@ -1142,6 +1236,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       namespaceLocks.clear();
       tableLocks.clear();
       regionLocks.clear();
+      peerLocks.clear();
     }
 
     @Override
@@ -1149,7 +1244,8 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
       return "serverLocks=" + filterUnlocked(this.serverLocks) +
         ", namespaceLocks=" + filterUnlocked(this.namespaceLocks) +
         ", tableLocks=" + filterUnlocked(this.tableLocks) +
-        ", regionLocks=" + filterUnlocked(this.regionLocks);
+        ", regionLocks=" + filterUnlocked(this.regionLocks) +
+        ", peerLocks=" + filterUnlocked(this.peerLocks);
     }
 
     private String filterUnlocked(Map<?, LockAndQueue> locks) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9ea9617/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerProcedureInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerProcedureInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerProcedureInterface.java
new file mode 100644
index 0000000..4abc9ad
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/PeerProcedureInterface.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.procedure;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public interface PeerProcedureInterface {
+
+  enum PeerOperationType {
+    ADD, REMOVE, ENABLE, DISABLE, UPDATE_CONFIG, REFRESH
+  }
+
+  String getPeerId();
+
+  PeerOperationType getPeerOperationType();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9ea9617/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
index 72e0846..22b70f1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.ipc.RemoteException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ArrayListMultimap;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
@@ -49,6 +50,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProc
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ExecuteProceduresResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RemoteProcedureRequest;
 
 /**
  * A remote procecdure dispatcher for regionservers.
@@ -222,7 +224,10 @@ public class RSProcedureDispatcher
 
   private interface RemoteProcedureResolver {
     void dispatchOpenRequests(MasterProcedureEnv env, List<RegionOpenOperation> operations);
+
     void dispatchCloseRequests(MasterProcedureEnv env, List<RegionCloseOperation> operations);
+
+    void dispatchServerOperations(MasterProcedureEnv env, List<ServerOperation> operations);
   }
 
   /**
@@ -231,22 +236,28 @@ public class RSProcedureDispatcher
    * Then {@code resolver} is used to dispatch {@link RegionOpenOperation}s and
    * {@link RegionCloseOperation}s.
    * @param serverName RegionServer to which the remote operations are sent
-   * @param remoteProcedures Remote procedures which are dispatched to the given server
+   * @param operations Remote procedures which are dispatched to the given server
    * @param resolver Used to dispatch remote procedures to given server.
    */
-  public void splitAndResolveOperation(final ServerName serverName,
-      final Set<RemoteProcedure> remoteProcedures, final RemoteProcedureResolver resolver) {
-    final ArrayListMultimap<Class<?>, RemoteOperation> reqsByType =
-      buildAndGroupRequestByType(procedureEnv, serverName, remoteProcedures);
+  public void splitAndResolveOperation(ServerName serverName, Set<RemoteProcedure> operations,
+      RemoteProcedureResolver resolver) {
+    MasterProcedureEnv env = master.getMasterProcedureExecutor().getEnvironment();
+    ArrayListMultimap<Class<?>, RemoteOperation> reqsByType =
+      buildAndGroupRequestByType(env, serverName, operations);
 
-    final List<RegionOpenOperation> openOps = fetchType(reqsByType, RegionOpenOperation.class);
+    List<RegionOpenOperation> openOps = fetchType(reqsByType, RegionOpenOperation.class);
     if (!openOps.isEmpty()) {
-      resolver.dispatchOpenRequests(procedureEnv, openOps);
+      resolver.dispatchOpenRequests(env, openOps);
     }
 
-    final List<RegionCloseOperation> closeOps = fetchType(reqsByType, RegionCloseOperation.class);
+    List<RegionCloseOperation> closeOps = fetchType(reqsByType, RegionCloseOperation.class);
     if (!closeOps.isEmpty()) {
-      resolver.dispatchCloseRequests(procedureEnv, closeOps);
+      resolver.dispatchCloseRequests(env, closeOps);
+    }
+
+    List<ServerOperation> refreshOps = fetchType(reqsByType, ServerOperation.class);
+    if (!refreshOps.isEmpty()) {
+      resolver.dispatchServerOperations(env, refreshOps);
     }
 
     if (!reqsByType.isEmpty()) {
@@ -277,8 +288,7 @@ public class RSProcedureDispatcher
       splitAndResolveOperation(getServerName(), remoteProcedures, this);
 
       try {
-        final ExecuteProceduresResponse response = sendRequest(getServerName(), request.build());
-        remoteCallCompleted(procedureEnv, response);
+        sendRequest(getServerName(), request.build());
       } catch (IOException e) {
         e = unwrapException(e);
         // TODO: In the future some operation may want to bail out early.
@@ -302,6 +312,11 @@ public class RSProcedureDispatcher
       }
     }
 
+    @Override
+    public void dispatchServerOperations(MasterProcedureEnv env, List<ServerOperation> operations) {
+      operations.stream().map(o -> o.buildRequest()).forEachOrdered(request::addProc);
+    }
+
     protected ExecuteProceduresResponse sendRequest(final ServerName serverName,
         final ExecuteProceduresRequest request) throws IOException {
       try {
@@ -311,17 +326,8 @@ public class RSProcedureDispatcher
       }
     }
 
-
-    private void remoteCallCompleted(final MasterProcedureEnv env,
-        final ExecuteProceduresResponse response) {
-      /*
-      for (RemoteProcedure proc: operations) {
-        proc.remoteCallCompleted(env, getServerName(), response);
-      }*/
-    }
-
     private void remoteCallFailed(final MasterProcedureEnv env, final IOException e) {
-      for (RemoteProcedure proc: remoteProcedures) {
+      for (RemoteProcedure proc : remoteProcedures) {
         proc.remoteCallFailed(env, getServerName(), e);
       }
     }
@@ -362,8 +368,7 @@ public class RSProcedureDispatcher
           buildOpenRegionRequest(procedureEnv, getServerName(), operations);
 
       try {
-        OpenRegionResponse response = sendRequest(getServerName(), request);
-        remoteCallCompleted(procedureEnv, response);
+        sendRequest(getServerName(), request);
       } catch (IOException e) {
         e = unwrapException(e);
         // TODO: In the future some operation may want to bail out early.
@@ -384,16 +389,6 @@ public class RSProcedureDispatcher
       }
     }
 
-    private void remoteCallCompleted(final MasterProcedureEnv env,
-        final OpenRegionResponse response) {
-      int index = 0;
-      for (RegionOpenOperation op: operations) {
-        OpenRegionResponse.RegionOpeningState state = response.getOpeningState(index++);
-        op.setFailedOpen(state == OpenRegionResponse.RegionOpeningState.FAILED_OPENING);
-        op.getRemoteProcedure().remoteCallCompleted(env, getServerName(), op);
-      }
-    }
-
     private void remoteCallFailed(final MasterProcedureEnv env, final IOException e) {
       for (RegionOpenOperation op: operations) {
         op.getRemoteProcedure().remoteCallFailed(env, getServerName(), e);
@@ -443,7 +438,6 @@ public class RSProcedureDispatcher
     private void remoteCallCompleted(final MasterProcedureEnv env,
         final CloseRegionResponse response) {
       operation.setClosed(response.getClosed());
-      operation.getRemoteProcedure().remoteCallCompleted(env, getServerName(), operation);
     }
 
     private void remoteCallFailed(final MasterProcedureEnv env, final IOException e) {
@@ -482,6 +476,11 @@ public class RSProcedureDispatcher
         submitTask(new CloseRegionRemoteCall(serverName, op));
       }
     }
+
+    @Override
+    public void dispatchServerOperations(MasterProcedureEnv env, List<ServerOperation> operations) {
+      throw new UnsupportedOperationException();
+    }
   }
 
   // ==========================================================================
@@ -489,13 +488,28 @@ public class RSProcedureDispatcher
   //  - ServerOperation: refreshConfig, grant, revoke, ... (TODO)
   //  - RegionOperation: open, close, flush, snapshot, ...
   // ==========================================================================
-  /* Currently unused
-  public static abstract class ServerOperation extends RemoteOperation {
-    protected ServerOperation(final RemoteProcedure remoteProcedure) {
+
+  public static final class ServerOperation extends RemoteOperation {
+
+    private final long procId;
+
+    private final Class<?> rsProcClass;
+
+    private final byte[] rsProcData;
+
+    public ServerOperation(RemoteProcedure remoteProcedure, long procId, Class<?> rsProcClass,
+        byte[] rsProcData) {
       super(remoteProcedure);
+      this.procId = procId;
+      this.rsProcClass = rsProcClass;
+      this.rsProcData = rsProcData;
+    }
+
+    public RemoteProcedureRequest buildRequest() {
+      return RemoteProcedureRequest.newBuilder().setProcId(procId)
+          .setProcClass(rsProcClass.getName()).setProcData(ByteString.copyFrom(rsProcData)).build();
     }
   }
-  */
 
   public static abstract class RegionOperation extends RemoteOperation {
     private final RegionInfo regionInfo;

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9ea9617/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
new file mode 100644
index 0000000..fca05a7
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
@@ -0,0 +1,127 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.replication;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationState;
+
+@InterfaceAudience.Private
+public abstract class ModifyPeerProcedure
+    extends StateMachineProcedure<MasterProcedureEnv, PeerModificationState>
+    implements PeerProcedureInterface {
+
+  private static final Log LOG = LogFactory.getLog(ModifyPeerProcedure.class);
+
+  protected String peerId;
+
+  protected ModifyPeerProcedure() {
+  }
+
+  protected ModifyPeerProcedure(String peerId) {
+    this.peerId = peerId;
+  }
+
+  @Override
+  public String getPeerId() {
+    return peerId;
+  }
+
+  /**
+   * Return {@code false} means that the operation is invalid and we should give up, otherwise
+   * {@code true}.
+   * <p>
+   * You need to call {@link #setFailure(String, Throwable)} to give the detail failure information.
+   */
+  protected abstract boolean updatePeerStorage() throws IOException;
+
+  protected void postPeerModification() {
+  }
+
+  @Override
+  protected Flow executeFromState(MasterProcedureEnv env, PeerModificationState state)
+      throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+    switch (state) {
+      case UPDATE_PEER_STORAGE:
+        try {
+          if (!updatePeerStorage()) {
+            assert isFailed() : "setFailure is not called";
+            return Flow.NO_MORE_STATE;
+          }
+        } catch (IOException e) {
+          LOG.warn("update peer storage failed, retry", e);
+          throw new ProcedureYieldException();
+        }
+        setNextState(PeerModificationState.REFRESH_PEER_ON_RS);
+        return Flow.HAS_MORE_STATE;
+      case REFRESH_PEER_ON_RS:
+        addChildProcedure(env.getMasterServices().getServerManager().getOnlineServersList().stream()
+            .map(sn -> new RefreshPeerProcedure(peerId, getPeerOperationType(), sn))
+            .toArray(RefreshPeerProcedure[]::new));
+        setNextState(PeerModificationState.POST_PEER_MODIFICATION);
+        return Flow.HAS_MORE_STATE;
+      case POST_PEER_MODIFICATION:
+        postPeerModification();
+        return Flow.NO_MORE_STATE;
+      default:
+        throw new UnsupportedOperationException("unhandled state=" + state);
+    }
+  }
+
+  @Override
+  protected LockState acquireLock(MasterProcedureEnv env) {
+    return env.getProcedureScheduler().waitPeerExclusiveLock(this, peerId)
+      ? LockState.LOCK_EVENT_WAIT
+      : LockState.LOCK_ACQUIRED;
+  }
+
+  @Override
+  protected void releaseLock(MasterProcedureEnv env) {
+    env.getProcedureScheduler().wakePeerExclusiveLock(this, peerId);
+  }
+
+  @Override
+  protected void rollbackState(MasterProcedureEnv env, PeerModificationState state)
+      throws IOException, InterruptedException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  protected PeerModificationState getState(int stateId) {
+    return PeerModificationState.forNumber(stateId);
+  }
+
+  @Override
+  protected int getStateId(PeerModificationState state) {
+    return state.getNumber();
+  }
+
+  @Override
+  protected PeerModificationState getInitialState() {
+    return PeerModificationState.UPDATE_PEER_STORAGE;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9ea9617/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerCallable.java
new file mode 100644
index 0000000..4e09107
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerCallable.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.replication;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshPeerParameter;
+
+/**
+ * The callable executed at RS side to refresh the peer config/state.
+ * <p>
+ * TODO: only a dummy implementation for verifying the framework, will add implementation later.
+ */
+@InterfaceAudience.Private
+public class RefreshPeerCallable implements RSProcedureCallable {
+
+  private HRegionServer rs;
+
+  private String peerId;
+
+  private Exception initError;
+
+  @Override
+  public Void call() throws Exception {
+    if (initError != null) {
+      throw initError;
+    }
+    rs.getFileSystem().create(new Path("/" + peerId + "/" + rs.getServerName().toString())).close();
+    return null;
+  }
+
+  @Override
+  public void init(byte[] parameter, HRegionServer rs) {
+    this.rs = rs;
+    try {
+      this.peerId = RefreshPeerParameter.parseFrom(parameter).getPeerId();
+    } catch (InvalidProtocolBufferException e) {
+      initError = e;
+      return;
+    }
+  }
+
+  @Override
+  public EventType getEventType() {
+    return EventType.RS_REFRESH_PEER;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9ea9617/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
new file mode 100644
index 0000000..18da487
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
@@ -0,0 +1,197 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.replication;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
+import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher.ServerOperation;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationType;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshPeerParameter;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshPeerStateData;
+
+@InterfaceAudience.Private
+public class RefreshPeerProcedure extends Procedure<MasterProcedureEnv>
+    implements PeerProcedureInterface, RemoteProcedure<MasterProcedureEnv, ServerName> {
+
+  private static final Log LOG = LogFactory.getLog(RefreshPeerProcedure.class);
+
+  private String peerId;
+
+  private PeerOperationType type;
+
+  private ServerName targetServer;
+
+  private boolean dispatched;
+
+  private ProcedureEvent<?> event;
+
+  private boolean succ;
+
+  public RefreshPeerProcedure() {
+  }
+
+  public RefreshPeerProcedure(String peerId, PeerOperationType type, ServerName targetServer) {
+    this.peerId = peerId;
+    this.type = type;
+    this.targetServer = targetServer;
+  }
+
+  @Override
+  public String getPeerId() {
+    return peerId;
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.REFRESH;
+  }
+
+  private static PeerModificationType toPeerModificationType(PeerOperationType type) {
+    switch (type) {
+      case ADD:
+        return PeerModificationType.ADD_PEER;
+      case REMOVE:
+        return PeerModificationType.REMOVE_PEER;
+      case ENABLE:
+        return PeerModificationType.ENABLE_PEER;
+      case DISABLE:
+        return PeerModificationType.DISABLE_PEER;
+      case UPDATE_CONFIG:
+        return PeerModificationType.UPDATE_PEER_CONFIG;
+      default:
+        throw new IllegalArgumentException("Unknown type: " + type);
+    }
+  }
+
+  private static PeerOperationType toPeerOperationType(PeerModificationType type) {
+    switch (type) {
+      case ADD_PEER:
+        return PeerOperationType.ADD;
+      case REMOVE_PEER:
+        return PeerOperationType.REMOVE;
+      case ENABLE_PEER:
+        return PeerOperationType.ENABLE;
+      case DISABLE_PEER:
+        return PeerOperationType.DISABLE;
+      case UPDATE_PEER_CONFIG:
+        return PeerOperationType.UPDATE_CONFIG;
+      default:
+        throw new IllegalArgumentException("Unknown type: " + type);
+    }
+  }
+
+  @Override
+  public RemoteOperation remoteCallBuild(MasterProcedureEnv env, ServerName remote) {
+    assert targetServer.equals(remote);
+    return new ServerOperation(this, getProcId(), RefreshPeerCallable.class,
+        RefreshPeerParameter.newBuilder().setPeerId(peerId).setType(toPeerModificationType(type))
+            .setTargetServer(ProtobufUtil.toServerName(remote)).build().toByteArray());
+  }
+
+  private void complete(MasterProcedureEnv env, boolean succ) {
+    if (event == null) {
+      LOG.warn("procedure event for " + getProcId() +
+          " is null, maybe the procedure is created when recovery", new Exception());
+      return;
+    }
+    LOG.info("Refresh peer " + peerId + " for " + type + " on " + targetServer +
+        (succ ? " suceeded" : " failed"));
+    this.succ = succ;
+    event.wake(env.getProcedureScheduler());
+    event = null;
+  }
+
+  @Override
+  public synchronized void remoteCallFailed(MasterProcedureEnv env, ServerName remote,
+      IOException exception) {
+    complete(env, false);
+  }
+
+  @Override
+  public synchronized void remoteOperationCompleted(MasterProcedureEnv env) {
+    complete(env, true);
+  }
+
+  @Override
+  public synchronized void remoteOperationFailed(MasterProcedureEnv env, String error) {
+    complete(env, false);
+  }
+
+  @Override
+  protected synchronized Procedure<MasterProcedureEnv>[] execute(MasterProcedureEnv env)
+      throws ProcedureYieldException, ProcedureSuspendedException, InterruptedException {
+    if (dispatched) {
+      if (succ) {
+        return null;
+      }
+      // retry
+      dispatched = false;
+    }
+    if (!env.getRemoteDispatcher().addOperationToNode(targetServer, this)) {
+      LOG.info("Can not add remote operation for refreshing peer " + peerId + " for " + type +
+          " to " + targetServer + ", this usually because the server is already dead," +
+          " give up and mark the procedure as complete");
+      return null;
+    }
+    dispatched = true;
+    event = new ProcedureEvent<>(this);
+    event.suspendIfNotReady(this);
+    throw new ProcedureSuspendedException();
+  }
+
+  @Override
+  protected void rollback(MasterProcedureEnv env) throws IOException, InterruptedException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  protected boolean abort(MasterProcedureEnv env) {
+    // TODO: no correctness problem if we just ignore this, implement later.
+    return false;
+  }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    serializer.serialize(
+      RefreshPeerStateData.newBuilder().setPeerId(peerId).setType(toPeerModificationType(type))
+          .setTargetServer(ProtobufUtil.toServerName(targetServer)).build());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    RefreshPeerStateData data = serializer.deserialize(RefreshPeerStateData.class);
+    peerId = data.getPeerId();
+    type = toPeerOperationType(data.getType());
+    targetServer = ProtobufUtil.toServerName(data.getTargetServer());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9ea9617/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/RSProcedureCallable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/RSProcedureCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/RSProcedureCallable.java
new file mode 100644
index 0000000..62c2e36
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure2/RSProcedureCallable.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.procedure2;
+
+import java.util.concurrent.Callable;
+
+import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A general interface for a sub procedure runs at RS side.
+ */
+@InterfaceAudience.Private
+public interface RSProcedureCallable extends Callable<Void> {
+
+  /**
+   * Initialize the callable
+   * @param parameter the parameter passed from master.
+   * @param rs the regionserver instance
+   */
+  void init(byte[] parameter, HRegionServer rs);
+
+  /**
+   * Event type used to select thread pool.
+   */
+  EventType getEventType();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9ea9617/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 8ead08c..0799ca6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -17,9 +17,6 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-import javax.servlet.http.HttpServlet;
 import java.io.IOException;
 import java.lang.Thread.UncaughtExceptionHandler;
 import java.lang.management.MemoryType;
@@ -51,6 +48,10 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.function.Function;
 
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import javax.servlet.http.HttpServlet;
+
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.commons.lang3.SystemUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -117,6 +118,7 @@ import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.mob.MobCacheConfig;
 import org.apache.hadoop.hbase.procedure.RegionServerProcedureManagerHost;
+import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
 import org.apache.hadoop.hbase.quotas.FileSystemUtilizationChore;
 import org.apache.hadoop.hbase.quotas.QuotaUtil;
 import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
@@ -127,6 +129,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequester;
 import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
 import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
+import org.apache.hadoop.hbase.regionserver.handler.RSProcedureHandler;
 import org.apache.hadoop.hbase.regionserver.handler.RegionReplicaFlushHandler;
 import org.apache.hadoop.hbase.regionserver.throttle.FlushThroughputControllerFactory;
 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
@@ -175,6 +178,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
@@ -206,6 +210,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
@@ -1932,6 +1937,8 @@ public class HRegionServer extends HasThread implements
           conf.getInt("hbase.regionserver.region.replica.flusher.threads",
               conf.getInt("hbase.regionserver.executor.openregion.threads", 3)));
     }
+    this.executorService.startExecutorService(ExecutorType.RS_REFRESH_PEER,
+      conf.getInt("hbase.regionserver.executor.refresh.peer.threads", 2));
 
     Threads.setDaemonThreadRunning(this.walRoller.getThread(), getName() + ".logRoller",
     uncaughtExceptionHandler);
@@ -3725,4 +3732,60 @@ public class HRegionServer extends HasThread implements
     return ConnectionUtils.createShortCircuitConnection(conf, null, user, this.serverName,
         this.rpcServices, this.rpcServices);
   }
+
+  public void executeProcedure(long procId, RSProcedureCallable callable) {
+    executorService.submit(new RSProcedureHandler(this, procId, callable));
+  }
+
+  public void reportProcedureDone(long procId, Throwable error) {
+    ReportProcedureDoneRequest.Builder builder =
+      ReportProcedureDoneRequest.newBuilder().setProcId(procId);
+    if (error != null) {
+      builder.setStatus(ReportProcedureDoneRequest.Status.ERROR)
+          .setError(Throwables.getStackTraceAsString(error));
+    } else {
+      builder.setStatus(ReportProcedureDoneRequest.Status.SUCCESS);
+    }
+    ReportProcedureDoneRequest request = builder.build();
+    int tries = 0;
+    long pauseTime = INIT_PAUSE_TIME_MS;
+    while (keepLooping()) {
+      RegionServerStatusService.BlockingInterface rss = rssStub;
+      try {
+        if (rss == null) {
+          createRegionServerStatusStub();
+          continue;
+        }
+        rss.reportProcedureDone(null, request);
+        // Log if we had to retry else don't log unless TRACE. We want to
+        // know if were successful after an attempt showed in logs as failed.
+        if (tries > 0 || LOG.isTraceEnabled()) {
+          LOG.info("PROCEDURE REPORTED " + request);
+        }
+        return;
+      } catch (ServiceException se) {
+        IOException ioe = ProtobufUtil.getRemoteException(se);
+        boolean pause =
+          ioe instanceof ServerNotRunningYetException || ioe instanceof PleaseHoldException;
+        if (pause) {
+          // Do backoff else we flood the Master with requests.
+          pauseTime = ConnectionUtils.getPauseTime(INIT_PAUSE_TIME_MS, tries);
+        } else {
+          pauseTime = INIT_PAUSE_TIME_MS; // Reset.
+        }
+        LOG.info(
+          "Failed to report transition " + TextFormat.shortDebugString(request) + "; retry (#" +
+            tries + ")" + (pause ? " after " + pauseTime + "ms delay (Master is coming online...)."
+              : " immediately."),
+          ioe);
+        if (pause) {
+          Threads.sleep(pauseTime);
+        }
+        tries++;
+        if (rssStub == rss) {
+          rssStub = null;
+        }
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9ea9617/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 4091316..92b0cda 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -99,6 +98,7 @@ import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.log.HBaseMarkers;
 import org.apache.hadoop.hbase.master.MasterRpcServices;
+import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
 import org.apache.hadoop.hbase.quotas.ActivePolicyEnforcement;
 import org.apache.hadoop.hbase.quotas.OperationQuota;
 import org.apache.hadoop.hbase.quotas.QuotaUtil;
@@ -171,6 +171,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionR
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionRequest.RegionOpenInfo;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionResponse.RegionOpeningState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RemoteProcedureRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ReplicateWALEntryResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.RollWALWriterRequest;
@@ -3434,23 +3435,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
   }
 
   @Override
-  public ExecuteProceduresResponse executeProcedures(RpcController controller,
-       ExecuteProceduresRequest request) throws ServiceException {
-    ExecuteProceduresResponse.Builder builder = ExecuteProceduresResponse.newBuilder();
-    if (request.getOpenRegionCount() > 0) {
-      for (OpenRegionRequest req : request.getOpenRegionList()) {
-        builder.addOpenRegion(openRegion(controller, req));
-      }
-    }
-    if (request.getCloseRegionCount() > 0) {
-      for (CloseRegionRequest req : request.getCloseRegionList()) {
-        builder.addCloseRegion(closeRegion(controller, req));
-      }
-    }
-    return builder.build();
-  }
-
-  @Override
   public ClearRegionBlockCacheResponse clearRegionBlockCache(RpcController controller,
       ClearRegionBlockCacheRequest request) {
     ClearRegionBlockCacheResponse.Builder builder =
@@ -3467,4 +3451,38 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     stats.withMaxCacheSize(regionServer.getCacheConfig().getBlockCache().getMaxSize());
     return builder.setStats(ProtobufUtil.toCacheEvictionStats(stats.build())).build();
   }
+
+  @Override
+  public ExecuteProceduresResponse executeProcedures(RpcController controller,
+      ExecuteProceduresRequest request) throws ServiceException {
+    if (request.getOpenRegionCount() > 0) {
+      for (OpenRegionRequest req : request.getOpenRegionList()) {
+        openRegion(controller, req);
+      }
+    }
+    if (request.getCloseRegionCount() > 0) {
+      for (CloseRegionRequest req : request.getCloseRegionList()) {
+        closeRegion(controller, req);
+      }
+    }
+    if (request.getProcCount() > 0) {
+      for (RemoteProcedureRequest req : request.getProcList()) {
+        RSProcedureCallable callable;
+        try {
+          callable =
+            Class.forName(req.getProcClass()).asSubclass(RSProcedureCallable.class).newInstance();
+        } catch (Exception e) {
+          // here we just ignore the error as this should not happen and we do not provide a general
+          // way to report errors for all types of remote procedure. The procedure will hang at
+          // master side but after you solve the problem and restart master it will be executed
+          // again and pass.
+          LOG.warn("create procedure of type " + req.getProcClass() + " failed, give up", e);
+          continue;
+        }
+        callable.init(req.getProcData().toByteArray(), regionServer);
+        regionServer.executeProcedure(req.getProcId(), callable);
+      }
+    }
+    return ExecuteProceduresResponse.getDefaultInstance();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9ea9617/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
new file mode 100644
index 0000000..94bcfec
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver.handler;
+
+import org.apache.hadoop.hbase.executor.EventHandler;
+import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A event handler for running procedure.
+ */
+@InterfaceAudience.Private
+public class RSProcedureHandler extends EventHandler {
+
+  private final long procId;
+
+  private final RSProcedureCallable callable;
+
+  public RSProcedureHandler(HRegionServer rs, long procId, RSProcedureCallable callable) {
+    super(rs, callable.getEventType());
+    this.procId = procId;
+    this.callable = callable;
+  }
+
+  @Override
+  public void process() {
+    Exception error = null;
+    try {
+      callable.call();
+    } catch (Exception e) {
+      error = e;
+    }
+    ((HRegionServer) server).reportProcedureDone(procId, error);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9ea9617/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
index 3c453bc..3ab915b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
@@ -531,26 +531,16 @@ public class TestAssignmentManager {
     @Override
     public ExecuteProceduresResponse sendRequest(ServerName server,
         ExecuteProceduresRequest request) throws IOException {
-      ExecuteProceduresResponse.Builder builder = ExecuteProceduresResponse.newBuilder();
       if (request.getOpenRegionCount() > 0) {
-        for (OpenRegionRequest req: request.getOpenRegionList()) {
-          OpenRegionResponse.Builder resp = OpenRegionResponse.newBuilder();
-          for (RegionOpenInfo openReq: req.getOpenInfoList()) {
-            RegionOpeningState state = execOpenRegion(server, openReq);
-            if (state != null) {
-              resp.addOpeningState(state);
-            }
+        for (OpenRegionRequest req : request.getOpenRegionList()) {
+          for (RegionOpenInfo openReq : req.getOpenInfoList()) {
+            execOpenRegion(server, openReq);
           }
-          builder.addOpenRegion(resp.build());
         }
       }
       if (request.getCloseRegionCount() > 0) {
-        for (CloseRegionRequest req: request.getCloseRegionList()) {
-          CloseRegionResponse resp = execCloseRegion(server,
-              req.getRegion().getValue().toByteArray());
-          if (resp != null) {
-            builder.addCloseRegion(resp);
-          }
+        for (CloseRegionRequest req : request.getCloseRegionList()) {
+          execCloseRegion(server, req.getRegion().getValue().toByteArray());
         }
       }
       return ExecuteProceduresResponse.newBuilder().build();

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9ea9617/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
new file mode 100644
index 0000000..44343d7
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.replication;
+
+import java.io.IOException;
+
+public class DummyModifyPeerProcedure extends ModifyPeerProcedure {
+
+  public DummyModifyPeerProcedure() {
+  }
+
+  public DummyModifyPeerProcedure(String peerId) {
+    super(peerId);
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.ADD;
+  }
+
+  @Override
+  protected boolean updatePeerStorage() throws IOException {
+    return true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9ea9617/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestDummyModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestDummyModifyPeerProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestDummyModifyPeerProcedure.java
new file mode 100644
index 0000000..ec06306
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestDummyModifyPeerProcedure.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.replication;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MasterTests.class, LargeTests.class })
+public class TestDummyModifyPeerProcedure {
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static String PEER_ID;
+
+  private static Path DIR;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.startMiniCluster(3);
+    PEER_ID = "testPeer";
+    DIR = new Path("/" + PEER_ID);
+    UTIL.getTestFileSystem().mkdirs(DIR);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void test() throws Exception {
+    ProcedureExecutor<?> executor =
+        UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor();
+    long procId = executor.submitProcedure(new DummyModifyPeerProcedure(PEER_ID));
+    UTIL.waitFor(30000, new Waiter.Predicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        return executor.isFinished(procId);
+      }
+    });
+    Set<String> serverNames = UTIL.getHBaseCluster().getRegionServerThreads().stream()
+        .map(t -> t.getRegionServer().getServerName().toString())
+        .collect(Collectors.toCollection(HashSet::new));
+    for (FileStatus s : UTIL.getTestFileSystem().listStatus(DIR)) {
+      assertTrue(serverNames.remove(s.getPath().getName()));
+    }
+    assertTrue(serverNames.isEmpty());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/f9ea9617/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index 138a40e..cade419 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -30,6 +30,7 @@ import com.google.protobuf.RpcCallback;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.Service;
 import com.google.protobuf.ServiceException;
+
 import java.io.IOException;
 import java.security.PrivilegedAction;
 import java.util.ArrayList;
@@ -119,8 +120,6 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.Permission.Action;
-import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
@@ -136,6 +135,9 @@ import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
+
 /**
  * Performs authorization checks for common operations, according to different
  * levels of authorized users.


[06/26] hbase git commit: HBASE-19647 Logging cleanups; emit regionname when RegionTooBusyException inside RetriesExhausted... make netty connect/disconnect TRACE-level

Posted by zh...@apache.org.
HBASE-19647 Logging cleanups; emit regionname when RegionTooBusyException inside RetriesExhausted... make netty connect/disconnect TRACE-level


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/16cd9169
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/16cd9169
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/16cd9169

Branch: refs/heads/HBASE-19397
Commit: 16cd9169768351fcdb6876d499888bd1834cca97
Parents: 6d9081b
Author: Michael Stack <st...@apache.org>
Authored: Wed Dec 27 11:55:35 2017 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Wed Dec 27 11:55:42 2017 -0800

----------------------------------------------------------------------
 .../RetriesExhaustedWithDetailsException.java   |  7 ++-
 ...estRetriesExhaustedWithDetailsException.java | 65 ++++++++++++++++++++
 .../hbase/ipc/NettyRpcServerRequestDecoder.java | 19 ++----
 .../hbase/regionserver/CompactingMemStore.java  |  8 +--
 .../hbase/regionserver/CompactionPipeline.java  | 21 +++----
 .../regionserver/FlushLargeStoresPolicy.java    | 39 ++++++------
 .../hadoop/hbase/regionserver/HRegion.java      |  2 +-
 .../MemStoreCompactionStrategy.java             |  7 +--
 .../hbase/regionserver/MemStoreCompactor.java   |  7 +--
 .../hadoop/hbase/regionserver/Segment.java      | 14 ++---
 10 files changed, 119 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/16cd9169/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java
index cb00675..e7eda2a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java
@@ -1,5 +1,4 @@
 /*
- *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -30,6 +29,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.RegionTooBusyException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.util.Bytes;
 
@@ -154,7 +154,9 @@ extends RetriesExhaustedException {
     for (Throwable t : ths) {
       if (t == null) continue;
       String name = "";
-      if (t instanceof DoNotRetryIOException) {
+      if (t instanceof DoNotRetryIOException ||
+          t instanceof RegionTooBusyException) {
+        // If RegionTooBusyException, print message since it has Region name in it.
         name = t.getMessage();
       } else {
         name = t.getClass().getSimpleName();
@@ -181,5 +183,4 @@ extends RetriesExhaustedException {
     }
     return classificatons.toString();
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/16cd9169/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRetriesExhaustedWithDetailsException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRetriesExhaustedWithDetailsException.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRetriesExhaustedWithDetailsException.java
new file mode 100644
index 0000000..46f44fa
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRetriesExhaustedWithDetailsException.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import org.apache.avro.generic.GenericData;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
+import org.apache.hadoop.hbase.RegionTooBusyException;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertTrue;
+
+@Category({SmallTests.class})
+public class TestRetriesExhaustedWithDetailsException {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestRetriesExhaustedWithDetailsException.class);
+  @Rule public TestName name = new TestName();
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder().
+      withTimeout(this.getClass()).
+      withLookingForStuckThread(true).
+      build();
+
+  /**
+   * Assert that a RetriesExhaustedException that has RegionTooBusyException outputs region name.
+   */
+  @Test
+  public void testRegionTooBusyException() {
+    List<Throwable> ts = new ArrayList<>(1);
+    final String regionName = this.name.getMethodName();
+    ts.add(new RegionTooBusyException(regionName));
+    List<Row> rows = new ArrayList<>(1);
+    rows.add(Mockito.mock(Row.class));
+    List<String> hostAndPorts = new ArrayList<>(1);
+    hostAndPorts.add("example.com:1234");
+    RetriesExhaustedException ree =
+        new RetriesExhaustedWithDetailsException(ts, rows, hostAndPorts);
+    assertTrue(ree.toString().contains(regionName));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/16cd9169/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerRequestDecoder.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerRequestDecoder.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerRequestDecoder.java
index e15048d..2eed79e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerRequestDecoder.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServerRequestDecoder.java
@@ -49,10 +49,8 @@ class NettyRpcServerRequestDecoder extends ChannelInboundHandlerAdapter {
   @Override
   public void channelActive(ChannelHandlerContext ctx) throws Exception {
     allChannels.add(ctx.channel());
-    if (NettyRpcServer.LOG.isDebugEnabled()) {
-      NettyRpcServer.LOG.debug("Connection from " + ctx.channel().remoteAddress() +
-          "; # active connections: " + (allChannels.size() - 1));
-    }
+    NettyRpcServer.LOG.trace("Connection {}; # active connections={}",
+        ctx.channel().remoteAddress(), (allChannels.size() - 1));
     super.channelActive(ctx);
   }
 
@@ -67,21 +65,16 @@ class NettyRpcServerRequestDecoder extends ChannelInboundHandlerAdapter {
   @Override
   public void channelInactive(ChannelHandlerContext ctx) throws Exception {
     allChannels.remove(ctx.channel());
-    if (NettyRpcServer.LOG.isDebugEnabled()) {
-      NettyRpcServer.LOG.debug("Disconnecting client: " + ctx.channel().remoteAddress() +
-          ". Number of active connections: " + (allChannels.size() - 1));
-    }
+    NettyRpcServer.LOG.trace("Disconnection {}; # active connections={}",
+        ctx.channel().remoteAddress(), (allChannels.size() - 1));
     super.channelInactive(ctx);
   }
 
   @Override
   public void exceptionCaught(ChannelHandlerContext ctx, Throwable e) {
     allChannels.remove(ctx.channel());
-    if (NettyRpcServer.LOG.isDebugEnabled()) {
-      NettyRpcServer.LOG.debug("Connection from " + ctx.channel().remoteAddress() +
-          " catch unexpected exception from downstream.",
-        e.getCause());
-    }
+    NettyRpcServer.LOG.trace("Connection {}; caught unexpected downstream exception.",
+        ctx.channel().remoteAddress(), e.getCause());
     ctx.channel().close();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/16cd9169/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
index 7b885ff..bb03117 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java
@@ -405,9 +405,7 @@ public class CompactingMemStore extends AbstractMemStore {
       // Phase I: Update the pipeline
       getRegionServices().blockUpdates();
       try {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("IN-MEMORY FLUSH: Pushing active segment into compaction pipeline");
-        }
+        LOG.trace("IN-MEMORY FLUSH: Pushing active segment into compaction pipeline");
         pushActiveToPipeline(this.active);
       } finally {
         getRegionServices().unblockUpdates();
@@ -429,9 +427,7 @@ public class CompactingMemStore extends AbstractMemStore {
       }
     } finally {
       inMemoryFlushInProgress.set(false);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("IN-MEMORY FLUSH: end");
-      }
+      LOG.trace("IN-MEMORY FLUSH: end");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/16cd9169/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionPipeline.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionPipeline.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionPipeline.java
index 49abe72..a07842c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionPipeline.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionPipeline.java
@@ -125,12 +125,8 @@ public class CompactionPipeline {
         return false;
       }
       suffix = versionedList.getStoreSegments();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Swapping pipeline suffix. "
-            + "Just before the swap the number of segments in pipeline is:"
-            + versionedList.getStoreSegments().size()
-            + ", and the new segment is:" + segment);
-      }
+      LOG.debug("Swapping pipeline suffix; before={}, new segement={}",
+          versionedList.getStoreSegments().size(), segment);
       swapSuffix(suffix, segment, closeSuffix);
       readOnlyCopy = new LinkedList<>(pipeline);
       version++;
@@ -146,11 +142,12 @@ public class CompactionPipeline {
       if(segment != null) newHeapSize = segment.heapSize();
       long heapSizeDelta = suffixHeapSize - newHeapSize;
       region.addMemStoreSize(new MemStoreSizing(-dataSizeDelta, -heapSizeDelta));
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Suffix data size: " + suffixDataSize + " new segment data size: "
-            + newDataSize + ". Suffix heap size: " + suffixHeapSize
-            + " new segment heap size: " + newHeapSize);
-      }
+      LOG.debug("Suffix data size={}, new segment data size={}, suffix heap size={}," +
+              "new segment heap size={}",
+          suffixDataSize,
+          newDataSize,
+          suffixHeapSize,
+          newHeapSize);
     }
     return true;
   }
@@ -206,7 +203,7 @@ public class CompactionPipeline {
             // upon flattening there is no change in the data size
             region.addMemStoreSize(new MemStoreSize(0, newMemstoreAccounting.getHeapSize()));
           }
-          LOG.debug("Compaction pipeline segment " + s + " was flattened");
+          LOG.debug("Compaction pipeline segment {} flattened", s);
           return true;
         }
         i++;

http://git-wip-us.apache.org/repos/asf/hbase/blob/16cd9169/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java
index 74bde60..2d2de24 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.procedure2.util.StringUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -42,8 +43,7 @@ public abstract class FlushLargeStoresPolicy extends FlushPolicy {
 
   protected long flushSizeLowerBound = -1;
 
-  protected long getFlushSizeLowerBound(HRegion region) {
-    int familyNumber = region.getTableDescriptor().getColumnFamilyCount();
+  protected long getFlushSizeLowerBound(HRegion region) { int familyNumber = region.getTableDescriptor().getColumnFamilyCount();
     // For multiple families, lower bound is the "average flush size" by default
     // unless setting in configuration is larger.
     long flushSizeLowerBound = region.getMemStoreFlushSize() / familyNumber;
@@ -57,20 +57,23 @@ public abstract class FlushLargeStoresPolicy extends FlushPolicy {
     String flushedSizeLowerBoundString =
         region.getTableDescriptor().getValue(HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND);
     if (flushedSizeLowerBoundString == null) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("No " + HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND
-            + " set in description of table " + region.getTableDescriptor().getTableName()
-            + ", use config (" + flushSizeLowerBound + ") instead");
-      }
+      LOG.debug("No {} set in table {} descriptor;" +
+          "using region.getMemStoreFlushSize/# of families ({}) instead.",
+          HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND,
+          region.getTableDescriptor().getTableName(),
+          StringUtils.humanSize(flushSizeLowerBound) + ")");
     } else {
       try {
         flushSizeLowerBound = Long.parseLong(flushedSizeLowerBoundString);
       } catch (NumberFormatException nfe) {
         // fall back for fault setting
-        LOG.warn("Number format exception when parsing "
-            + HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND + " for table "
-            + region.getTableDescriptor().getTableName() + ":" + flushedSizeLowerBoundString + ". " + nfe
-            + ", use config (" + flushSizeLowerBound + ") instead");
+        LOG.warn("Number format exception parsing {} for table {}: {}, {}; " +
+            "using region.getMemStoreFlushSize/# of families ({}) instead.",
+            HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND,
+            region.getTableDescriptor().getTableName(),
+            flushedSizeLowerBoundString,
+            nfe,
+            flushSizeLowerBound);
 
       }
     }
@@ -79,15 +82,13 @@ public abstract class FlushLargeStoresPolicy extends FlushPolicy {
 
   protected boolean shouldFlush(HStore store) {
     if (store.getMemStoreSize().getDataSize() > this.flushSizeLowerBound) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Flush Column Family " + store.getColumnFamilyName() + " of " +
-            region.getRegionInfo().getEncodedName() + " because memstoreSize=" +
-            store.getMemStoreSize().getDataSize() + " > lower bound="
-            + this.flushSizeLowerBound);
-      }
+      LOG.debug("Flush {} of {}; memstoreSize={} > lowerBound={}",
+          store.getColumnFamilyName(),
+          region.getRegionInfo().getEncodedName(),
+          store.getMemStoreSize().getDataSize(),
+          this.flushSizeLowerBound);
       return true;
     }
     return false;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/16cd9169/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 6401a8b..cde6874 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -4170,7 +4170,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     if (this.memstoreDataSize.get() > this.blockingMemStoreSize) {
       blockedRequestsCount.increment();
       requestFlush();
-      throw new RegionTooBusyException("Above memstore limit, " +
+      throw new RegionTooBusyException("Over memstore limit, " +
           "regionName=" + (this.getRegionInfo() == null ? "unknown" :
           this.getRegionInfo().getRegionNameAsString()) +
           ", server=" + (this.getRegionServerServices() == null ? "unknown" :

http://git-wip-us.apache.org/repos/asf/hbase/blob/16cd9169/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactionStrategy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactionStrategy.java
index 42302b2..d4aafed 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactionStrategy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactionStrategy.java
@@ -85,14 +85,13 @@ public abstract class MemStoreCompactionStrategy {
     int numOfSegments = versionedList.getNumOfSegments();
     if (numOfSegments > pipelineThreshold) {
       // to avoid too many segments, merge now
-      LOG.debug(strategy+" memory compaction for store " + cfName
-          + " merging " + numOfSegments + " segments");
+      LOG.debug("{} in-memory compaction of {}; merging {} segments",
+          strategy, cfName, numOfSegments);
       return getMergingAction();
     }
 
     // just flatten a segment
-    LOG.debug(strategy+" memory compaction for store " + cfName
-        + " flattening a segment in the pipeline");
+    LOG.debug("{} in-memory compaction of {}; flattening a segment", strategy, cfName);
     return getFlattenAction();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/16cd9169/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java
index 300218e..a5348de 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java
@@ -92,11 +92,8 @@ public class MemStoreCompactor {
     // get a snapshot of the list of the segments from the pipeline,
     // this local copy of the list is marked with specific version
     versionedList = compactingMemStore.getImmutableSegments();
-
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Starting the In-Memory Compaction for store "
-          + compactingMemStore.getStore().getColumnFamilyName());
-    }
+    LOG.debug("Starting In-Memory Compaction of {}",
+        compactingMemStore.getStore().getColumnFamilyName());
     HStore store = compactingMemStore.getStore();
     RegionCoprocessorHost cpHost = store.getCoprocessorHost();
     if (cpHost != null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/16cd9169/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java
index 121cbca..e09f587 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java
@@ -351,13 +351,13 @@ public abstract class Segment {
 
   @Override
   public String toString() {
-    String res = "Store segment of type "+this.getClass().getName()+"; ";
-    res += "isEmpty "+(isEmpty()?"yes":"no")+"; ";
-    res += "cellsCount "+getCellsCount()+"; ";
-    res += "cellsSize "+keySize()+"; ";
-    res += "totalHeapSize "+heapSize()+"; ";
-    res += "Min ts " + timeRangeTracker.getMin() + "; ";
-    res += "Max ts " + timeRangeTracker.getMax() + "; ";
+    String res = "Type=" + this.getClass().getSimpleName() + ", ";
+    res += "empty=" + (isEmpty()? "yes": "no") + ", ";
+    res += "cellCount=" + getCellsCount() + ", ";
+    res += "cellSize=" + keySize() + ", ";
+    res += "totalHeapSize=" + heapSize() + ", ";
+    res += "min timestamp=" + timeRangeTracker.getMin() + ", ";
+    res += "max timestamp=" + timeRangeTracker.getMax();
     return res;
   }
 }


[23/26] hbase git commit: HBASE-19525 RS side changes for moving peer modification from zk watcher to procedure

Posted by zh...@apache.org.
HBASE-19525 RS side changes for moving peer modification from zk watcher to procedure


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/5d91fb6e
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/5d91fb6e
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/5d91fb6e

Branch: refs/heads/HBASE-19397
Commit: 5d91fb6e74d1e0079f052e01b81e98a093e61036
Parents: 729b6a7
Author: huzheng <op...@gmail.com>
Authored: Wed Dec 20 10:47:18 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Dec 28 08:51:59 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |  11 +-
 .../hbase/shaded/protobuf/ProtobufUtil.java     |  13 +-
 .../hbase/replication/ReplicationListener.java  |  14 --
 .../hbase/replication/ReplicationPeer.java      |  28 ++-
 .../replication/ReplicationPeerZKImpl.java      | 180 ++++-----------
 .../replication/ReplicationPeersZKImpl.java     |  19 +-
 .../replication/ReplicationTrackerZKImpl.java   |  73 +-----
 .../regionserver/ReplicationSourceService.java  |   9 +-
 .../handler/RSProcedureHandler.java             |   3 +
 .../replication/BaseReplicationEndpoint.java    |   2 +-
 .../regionserver/PeerProcedureHandler.java      |  38 ++++
 .../regionserver/PeerProcedureHandlerImpl.java  |  81 +++++++
 .../regionserver/RefreshPeerCallable.java       |  39 +++-
 .../replication/regionserver/Replication.java   |  10 +
 .../regionserver/ReplicationSource.java         |   9 +-
 .../regionserver/ReplicationSourceManager.java  |  37 ++-
 .../replication/TestReplicationAdmin.java       |   2 +-
 .../TestReplicationAdminUsingProcedure.java     | 226 +++++++++++++++++++
 .../replication/DummyModifyPeerProcedure.java   |  48 ----
 .../TestDummyModifyPeerProcedure.java           |  80 -------
 .../TestReplicationTrackerZKImpl.java           |  51 -----
 .../TestReplicationSourceManager.java           |  32 ++-
 22 files changed, 533 insertions(+), 472 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5d91fb6e/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index 267dc7a..d5285dc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hbase.protobuf;
 
+import static org.apache.hadoop.hbase.protobuf.ProtobufMagic.PB_MAGIC;
+
 import com.google.protobuf.ByteString;
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.InvalidProtocolBufferException;
@@ -199,7 +201,7 @@ public final class ProtobufUtil {
    * byte array that is <code>bytes.length</code> plus {@link ProtobufMagic#PB_MAGIC}.length.
    */
   public static byte [] prependPBMagic(final byte [] bytes) {
-    return Bytes.add(ProtobufMagic.PB_MAGIC, bytes);
+    return Bytes.add(PB_MAGIC, bytes);
   }
 
   /**
@@ -224,10 +226,11 @@ public final class ProtobufUtil {
    * @param bytes bytes to check
    * @throws DeserializationException if we are missing the pb magic prefix
    */
-  public static void expectPBMagicPrefix(final byte [] bytes) throws DeserializationException {
+  public static void expectPBMagicPrefix(final byte[] bytes) throws DeserializationException {
     if (!isPBMagicPrefix(bytes)) {
-      throw new DeserializationException("Missing pb magic " +
-          Bytes.toString(ProtobufMagic.PB_MAGIC) + " prefix");
+      String bytesPrefix = bytes == null ? "null" : Bytes.toStringBinary(bytes, 0, PB_MAGIC.length);
+      throw new DeserializationException(
+          "Missing pb magic " + Bytes.toString(PB_MAGIC) + " prefix, bytes: " + bytesPrefix);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/5d91fb6e/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index 8f44135..fd61b8d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hbase.shaded.protobuf;
 
+import static org.apache.hadoop.hbase.protobuf.ProtobufMagic.PB_MAGIC;
+
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -282,7 +284,7 @@ public final class ProtobufUtil {
    * byte array that is <code>bytes.length</code> plus {@link ProtobufMagic#PB_MAGIC}.length.
    */
   public static byte [] prependPBMagic(final byte [] bytes) {
-    return Bytes.add(ProtobufMagic.PB_MAGIC, bytes);
+    return Bytes.add(PB_MAGIC, bytes);
   }
 
   /**
@@ -307,10 +309,11 @@ public final class ProtobufUtil {
    * @param bytes bytes to check
    * @throws DeserializationException if we are missing the pb magic prefix
    */
-  public static void expectPBMagicPrefix(final byte [] bytes) throws DeserializationException {
+  public static void expectPBMagicPrefix(final byte[] bytes) throws DeserializationException {
     if (!isPBMagicPrefix(bytes)) {
-      throw new DeserializationException("Missing pb magic " +
-          Bytes.toString(ProtobufMagic.PB_MAGIC) + " prefix");
+      String bytesPrefix = bytes == null ? "null" : Bytes.toStringBinary(bytes, 0, PB_MAGIC.length);
+      throw new DeserializationException(
+          "Missing pb magic " + Bytes.toString(PB_MAGIC) + " prefix" + ", bytes: " + bytesPrefix);
     }
   }
 
@@ -1962,7 +1965,7 @@ public final class ProtobufUtil {
   public static byte [] toDelimitedByteArray(final Message m) throws IOException {
     // Allocate arbitrary big size so we avoid resizing.
     ByteArrayOutputStream baos = new ByteArrayOutputStream(4096);
-    baos.write(ProtobufMagic.PB_MAGIC);
+    baos.write(PB_MAGIC);
     m.writeDelimitedTo(baos);
     return baos.toByteArray();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5d91fb6e/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationListener.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationListener.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationListener.java
index 3edfcf9..f040bf9 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationListener.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationListener.java
@@ -18,8 +18,6 @@
  */
 package org.apache.hadoop.hbase.replication;
 
-import java.util.List;
-
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -36,16 +34,4 @@ public interface ReplicationListener {
    * @param regionServer the removed region server
    */
   public void regionServerRemoved(String regionServer);
-
-  /**
-   * A peer cluster has been removed (i.e. unregistered) from replication.
-   * @param peerId The peer id of the cluster that has been removed
-   */
-  public void peerRemoved(String peerId);
-
-  /**
-   * The list of registered peer clusters has changed.
-   * @param peerIds A list of all currently registered peer clusters
-   */
-  public void peerListChanged(List<String> peerIds);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5d91fb6e/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
index 97e2ddb..b66d76d 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
@@ -55,12 +55,27 @@ public interface ReplicationPeer {
   public ReplicationPeerConfig getPeerConfig();
 
   /**
-   * Returns the state of the peer
+   * Get the peer config object. if loadFromBackingStore is true, it will load from backing store
+   * directly and update its load peer config. otherwise, just return the local cached peer config.
+   * @return the ReplicationPeerConfig for this peer
+   */
+  public ReplicationPeerConfig getPeerConfig(boolean loadFromBackingStore)
+      throws ReplicationException;
+
+  /**
+   * Returns the state of the peer by reading local cache.
    * @return the enabled state
    */
   PeerState getPeerState();
 
   /**
+   * Returns the state of peer, if loadFromBackingStore is true, it will load from backing store
+   * directly and update its local peer state. otherwise, just return the local cached peer state.
+   * @return the enabled state
+   */
+  PeerState getPeerState(boolean loadFromBackingStore) throws ReplicationException;
+
+  /**
    * Get the configuration object required to communicate with this peer
    * @return configuration object
    */
@@ -84,6 +99,15 @@ public interface ReplicationPeer {
    */
   public long getPeerBandwidth();
 
-  void trackPeerConfigChanges(ReplicationPeerConfigListener listener);
+  /**
+   * Register a peer config listener to catch the peer config change event.
+   * @param listener listener to catch the peer config change event.
+   */
+  public void registerPeerConfigListener(ReplicationPeerConfigListener listener);
 
+  /**
+   * Notify all the registered ReplicationPeerConfigListener to update their peer config.
+   * @param newPeerConfig the new peer config.
+   */
+  public void triggerPeerConfigChange(ReplicationPeerConfig newPeerConfig);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5d91fb6e/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
----------------------------------------------------------------------
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 454d09c..5d051a0 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
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.replication;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -33,12 +34,10 @@ 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;
 
@@ -47,14 +46,13 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
     implements ReplicationPeer, Abortable, Closeable {
   private static final Logger LOG = LoggerFactory.getLogger(ReplicationPeerZKImpl.class);
 
-  private ReplicationPeerConfig peerConfig;
+  private volatile ReplicationPeerConfig peerConfig;
   private final String id;
   private volatile PeerState peerState;
   private volatile Map<TableName, List<String>> tableCFs = new HashMap<>();
   private final Configuration conf;
-  private PeerStateTracker peerStateTracker;
-  private PeerConfigTracker peerConfigTracker;
 
+  private final List<ReplicationPeerConfigListener> peerConfigListeners;
 
   /**
    * Constructor that takes all the objects required to communicate with the specified peer, except
@@ -63,62 +61,35 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
    * @param id string representation of this peer's identifier
    * @param peerConfig configuration for the replication peer
    */
-  public ReplicationPeerZKImpl(ZKWatcher zkWatcher, Configuration conf,
-                               String id, ReplicationPeerConfig peerConfig,
-                               Abortable abortable)
-      throws ReplicationException {
+  public ReplicationPeerZKImpl(ZKWatcher zkWatcher, Configuration conf, String id,
+      ReplicationPeerConfig peerConfig, Abortable abortable) throws ReplicationException {
     super(zkWatcher, conf, abortable);
     this.conf = conf;
     this.peerConfig = peerConfig;
     this.id = id;
+    this.peerConfigListeners = new ArrayList<>();
   }
 
-  /**
-   * 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
-   */
-  public void startStateTracker(String peerStateNode)
-      throws KeeperException {
-    ensurePeerEnabled(peerStateNode);
-    this.peerStateTracker = new PeerStateTracker(peerStateNode, zookeeper, this);
-    this.peerStateTracker.start();
+  private PeerState readPeerState() throws ReplicationException {
     try {
-      this.readPeerStateZnode();
-    } catch (DeserializationException e) {
-      throw ZKUtil.convert(e);
+      byte[] data = ZKUtil.getData(zookeeper, this.getPeerStateNode(id));
+      this.peerState = isStateEnabled(data) ? PeerState.ENABLED : PeerState.DISABLED;
+    } catch (DeserializationException | KeeperException | InterruptedException e) {
+      throw new ReplicationException("Get and deserialize peer state data from zookeeper failed: ",
+          e);
     }
+    return this.peerState;
   }
 
-  private void readPeerStateZnode() throws DeserializationException {
-    this.peerState =
-        isStateEnabled(this.peerStateTracker.getData(false))
-          ? PeerState.ENABLED
-          : PeerState.DISABLED;
-  }
-
-  /**
-   * 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 {
-    this.peerConfigTracker = new PeerConfigTracker(peerConfigNode, zookeeper,
-        this);
-    this.peerConfigTracker.start();
-    this.readPeerConfig();
-  }
-
-  private ReplicationPeerConfig readPeerConfig() {
+  private ReplicationPeerConfig readPeerConfig() throws ReplicationException {
     try {
-      byte[] data = peerConfigTracker.getData(false);
+      byte[] data = ZKUtil.getData(zookeeper, this.getPeerNode(id));
       if (data != null) {
         this.peerConfig = ReplicationPeerConfigUtil.parsePeerFrom(data);
       }
-    } catch (DeserializationException e) {
-      LOG.error("", e);
+    } catch (DeserializationException | KeeperException | InterruptedException e) {
+      throw new ReplicationException("Get and deserialize peer config date from zookeeper failed: ",
+          e);
     }
     return this.peerConfig;
   }
@@ -128,6 +99,15 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
     return peerState;
   }
 
+  @Override
+  public PeerState getPeerState(boolean loadFromBackingStore) throws ReplicationException {
+    if (loadFromBackingStore) {
+      return readPeerState();
+    } else {
+      return peerState;
+    }
+  }
+
   /**
    * Get the identifier of this peer
    * @return string representation of the id (short)
@@ -146,6 +126,16 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
     return peerConfig;
   }
 
+  @Override
+  public ReplicationPeerConfig getPeerConfig(boolean loadFromBackingStore)
+      throws ReplicationException {
+    if (loadFromBackingStore) {
+      return readPeerConfig();
+    } else {
+      return peerConfig;
+    }
+  }
+
   /**
    * Get the configuration object required to communicate with this peer
    * @return configuration object
@@ -180,9 +170,14 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
   }
 
   @Override
-  public void trackPeerConfigChanges(ReplicationPeerConfigListener listener) {
-    if (this.peerConfigTracker != null){
-      this.peerConfigTracker.setListener(listener);
+  public void registerPeerConfigListener(ReplicationPeerConfigListener listener) {
+    this.peerConfigListeners.add(listener);
+  }
+
+  @Override
+  public void triggerPeerConfigChange(ReplicationPeerConfig newPeerConfig) {
+    for (ReplicationPeerConfigListener listener : this.peerConfigListeners) {
+      listener.peerConfigUpdated(newPeerConfig);
     }
   }
 
@@ -223,97 +218,16 @@ public class ReplicationPeerZKImpl extends ReplicationStateZKBase
   private static ReplicationProtos.ReplicationState.State parseStateFrom(final byte[] bytes)
       throws DeserializationException {
     ProtobufUtil.expectPBMagicPrefix(bytes);
-    int pblen = ProtobufUtil.lengthOfPBMagic();
+    int pbLen = ProtobufUtil.lengthOfPBMagic();
     ReplicationProtos.ReplicationState.Builder builder =
         ReplicationProtos.ReplicationState.newBuilder();
     ReplicationProtos.ReplicationState state;
     try {
-      ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
+      ProtobufUtil.mergeFrom(builder, bytes, pbLen, bytes.length - pbLen);
       state = builder.build();
       return state.getState();
     } catch (IOException e) {
       throw new DeserializationException(e);
     }
   }
-
-  /**
-   * 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
-   */
-  private boolean ensurePeerEnabled(final String path)
-      throws NodeExistsException, 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.
-      // The peer state data is set as "ENABLED" by default.
-      ZKUtil.createNodeIfNotExistsAndWatch(zookeeper, path,
-        ReplicationStateZKBase.ENABLED_ZNODE_BYTES);
-      return true;
-    }
-    return false;
-  }
-
-  /**
-   * Tracker for state of this peer
-   */
-  public class PeerStateTracker extends ZKNodeTracker {
-
-    public PeerStateTracker(String peerStateZNode, ZKWatcher watcher,
-        Abortable abortable) {
-      super(watcher, peerStateZNode, abortable);
-    }
-
-    @Override
-    public synchronized void nodeDataChanged(String path) {
-      if (path.equals(node)) {
-        super.nodeDataChanged(path);
-        try {
-          readPeerStateZnode();
-        } catch (DeserializationException e) {
-          LOG.warn("Failed deserializing the content of " + path, e);
-        }
-      }
-    }
-  }
-
-  /**
-   * Tracker for PeerConfigNode of this peer
-   */
-  public class PeerConfigTracker extends ZKNodeTracker {
-
-    ReplicationPeerConfigListener listener;
-
-    public PeerConfigTracker(String peerConfigNode, ZKWatcher watcher,
-        Abortable abortable) {
-      super(watcher, peerConfigNode, abortable);
-    }
-
-    public synchronized void setListener(ReplicationPeerConfigListener listener){
-      this.listener = listener;
-    }
-
-    @Override
-    public synchronized void nodeCreated(String path) {
-      if (path.equals(node)) {
-        super.nodeCreated(path);
-        ReplicationPeerConfig config = readPeerConfig();
-        if (listener != null){
-          listener.peerConfigUpdated(config);
-        }
-      }
-    }
-
-    @Override
-    public synchronized void nodeDataChanged(String path) {
-      //superclass calls nodeCreated
-      if (path.equals(node)) {
-        super.nodeDataChanged(path);
-      }
-
-    }
-
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5d91fb6e/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
----------------------------------------------------------------------
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 0b2f08a..8e2c5f4 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
@@ -500,21 +500,12 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
     }
     Configuration peerConf = pair.getSecond();
 
-    ReplicationPeerZKImpl peer = new ReplicationPeerZKImpl(zookeeper,
-        peerConf, peerId, pair.getFirst(), abortable);
-    try {
-      peer.startStateTracker(this.getPeerStateNode(peerId));
-    } catch (KeeperException e) {
-      throw new ReplicationException("Error starting the peer state tracker for peerId=" +
-          peerId, e);
-    }
+    ReplicationPeerZKImpl peer =
+        new ReplicationPeerZKImpl(zookeeper, peerConf, peerId, pair.getFirst(), abortable);
 
-    try {
-      peer.startPeerConfigTracker(this.getPeerNode(peerId));
-    } catch (KeeperException e) {
-      throw new ReplicationException("Error starting the peer tableCFs tracker for peerId=" +
-          peerId, e);
-    }
+    // Load peer state and peer config by reading zookeeper directly.
+    peer.getPeerState(true);
+    peer.getPeerConfig(true);
 
     return peer;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5d91fb6e/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java
----------------------------------------------------------------------
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 9a1d9aa..2c522f6 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
@@ -48,16 +48,12 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
   private final List<ReplicationListener> listeners = new CopyOnWriteArrayList<>();
   // List of all the other region servers in this cluster
   private final ArrayList<String> otherRegionServers = new ArrayList<>();
-  private final ReplicationPeers replicationPeers;
 
-  public ReplicationTrackerZKImpl(ZKWatcher zookeeper,
-      final ReplicationPeers replicationPeers, Configuration conf, Abortable abortable,
-      Stoppable stopper) {
+  public ReplicationTrackerZKImpl(ZKWatcher zookeeper, final ReplicationPeers replicationPeers,
+      Configuration conf, Abortable abortable, Stoppable stopper) {
     super(zookeeper, conf, abortable);
-    this.replicationPeers = replicationPeers;
     this.stopper = stopper;
     this.zookeeper.registerListener(new OtherRegionServerWatcher(this.zookeeper));
-    this.zookeeper.registerListener(new PeersWatcher(this.zookeeper));
   }
 
   @Override
@@ -146,71 +142,6 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
   }
 
   /**
-   * Watcher used to follow the creation and deletion of peer clusters.
-   */
-  public class PeersWatcher extends ZKListener {
-
-    /**
-     * Construct a ZooKeeper event listener.
-     */
-    public PeersWatcher(ZKWatcher watcher) {
-      super(watcher);
-    }
-
-    /**
-     * Called when a node has been deleted
-     * @param path full path of the deleted node
-     */
-    @Override
-    public void nodeDeleted(String path) {
-      List<String> peers = refreshPeersList(path);
-      if (peers == null) {
-        return;
-      }
-      if (isPeerPath(path)) {
-        String id = getZNodeName(path);
-        LOG.info(path + " znode expired, triggering peerRemoved event");
-        for (ReplicationListener rl : listeners) {
-          rl.peerRemoved(id);
-        }
-      }
-    }
-
-    /**
-     * Called when an existing node has a child node added or removed.
-     * @param path full path of the node whose children have changed
-     */
-    @Override
-    public void nodeChildrenChanged(String path) {
-      List<String> peers = refreshPeersList(path);
-      if (peers == null) {
-        return;
-      }
-      LOG.info(path + " znode expired, triggering peerListChanged event");
-      for (ReplicationListener rl : listeners) {
-        rl.peerListChanged(peers);
-      }
-    }
-  }
-
-  /**
-   * Verify if this event is meant for us, and if so then get the latest peers' list from ZK. Also
-   * reset the watches.
-   * @param path path to check against
-   * @return A list of peers' identifiers if the event concerns this watcher, else null.
-   */
-  private List<String> refreshPeersList(String path) {
-    if (!path.startsWith(getPeersZNode())) {
-      return null;
-    }
-    return this.replicationPeers.getAllPeerIds();
-  }
-
-  private String getPeersZNode() {
-    return this.peersZNode;
-  }
-
-  /**
    * Extracts the znode name of a peer cluster from a ZK path
    * @param fullPath Path to extract the id from
    * @return the id or an empty string if path is invalid

http://git-wip-us.apache.org/repos/asf/hbase/blob/5d91fb6e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
index 835ffbf..a82fa3d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReplicationSourceService.java
@@ -18,6 +18,7 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import org.apache.hadoop.hbase.replication.regionserver.PeerProcedureHandler;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 
@@ -29,8 +30,14 @@ import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 @InterfaceAudience.Private
 public interface ReplicationSourceService extends ReplicationService {
   /**
-   * Returns a WALObserver for the service. This is needed to 
+   * Returns a WALObserver for the service. This is needed to
    * observe log rolls and log archival events.
    */
   WALActionsListener getWALActionsListener();
+
+
+  /**
+   * Returns a Handler to handle peer procedures.
+   */
+  PeerProcedureHandler getPeerProcedureHandler();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5d91fb6e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
index 94bcfec..240b0a7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/RSProcedureHandler.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver.handler;
 import org.apache.hadoop.hbase.executor.EventHandler;
 import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.log4j.Logger;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -28,6 +29,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 public class RSProcedureHandler extends EventHandler {
 
+  private static final Logger LOG = Logger.getLogger(RSProcedureHandler.class);
   private final long procId;
 
   private final RSProcedureCallable callable;
@@ -44,6 +46,7 @@ public class RSProcedureHandler extends EventHandler {
     try {
       callable.call();
     } catch (Exception e) {
+      LOG.error("Catch exception when call RSProcedureCallable: ", e);
       error = e;
     }
     ((HRegionServer) server).reportProcedureDone(procId, error);

http://git-wip-us.apache.org/repos/asf/hbase/blob/5d91fb6e/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/BaseReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/BaseReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/BaseReplicationEndpoint.java
index c390d09..2057057 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/BaseReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/BaseReplicationEndpoint.java
@@ -49,7 +49,7 @@ public abstract class BaseReplicationEndpoint extends AbstractService
     if (this.ctx != null){
       ReplicationPeer peer = this.ctx.getReplicationPeer();
       if (peer != null){
-        peer.trackPeerConfigChanges(this);
+        peer.registerPeerConfigListener(this);
       } else {
         LOG.warn("Not tracking replication peer config changes for Peer Id " + this.ctx.getPeerId() +
             " because there's no such peer");

http://git-wip-us.apache.org/repos/asf/hbase/blob/5d91fb6e/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java
new file mode 100644
index 0000000..b392985
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandler.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.replication.regionserver;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public interface PeerProcedureHandler {
+
+  public void addPeer(String peerId) throws ReplicationException, IOException;
+
+  public void removePeer(String peerId) throws ReplicationException, IOException;
+
+  public void disablePeer(String peerId) throws ReplicationException, IOException;
+
+  public void enablePeer(String peerId) throws ReplicationException, IOException;
+
+  public void updatePeerConfig(String peerId) throws ReplicationException, IOException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/5d91fb6e/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
new file mode 100644
index 0000000..9b493d9
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.replication.regionserver;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationPeer;
+import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.log4j.Logger;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
+  private static final Logger LOG = Logger.getLogger(PeerProcedureHandlerImpl.class);
+
+  private ReplicationSourceManager replicationSourceManager;
+
+  public PeerProcedureHandlerImpl(ReplicationSourceManager replicationSourceManager) {
+    this.replicationSourceManager = replicationSourceManager;
+  }
+
+  @Override
+  public void addPeer(String peerId) throws ReplicationException, IOException {
+    replicationSourceManager.addPeer(peerId);
+  }
+
+  @Override
+  public void removePeer(String peerId) throws ReplicationException, IOException {
+    replicationSourceManager.removePeer(peerId);
+  }
+
+  @Override
+  public void disablePeer(String peerId) throws ReplicationException, IOException {
+    ReplicationPeer peer = replicationSourceManager.getReplicationPeers().getConnectedPeer(peerId);
+    if (peer != null) {
+      PeerState peerState = peer.getPeerState(true);
+      LOG.info("disablePeer state, peer id: " + peerId + ", state: " + peerState);
+    } else {
+      throw new ReplicationException("No connected peer found, peerId=" + peerId);
+    }
+  }
+
+  @Override
+  public void enablePeer(String peerId) throws ReplicationException, IOException {
+    ReplicationPeer peer = replicationSourceManager.getReplicationPeers().getConnectedPeer(peerId);
+    if (peer != null) {
+      PeerState peerState = peer.getPeerState(true);
+      LOG.info("enablePeer state, peer id: " + peerId + ", state: " + peerState);
+    } else {
+      throw new ReplicationException("No connected peer found, peerId=" + peerId);
+    }
+  }
+
+  @Override
+  public void updatePeerConfig(String peerId) throws ReplicationException, IOException {
+    ReplicationPeer peer = replicationSourceManager.getReplicationPeers().getConnectedPeer(peerId);
+    if (peer == null) {
+      throw new ReplicationException("No connected peer found, peerId=" + peerId);
+    }
+    ReplicationPeerConfig rpc = peer.getPeerConfig(true);
+    peer.triggerPeerConfigChange(rpc);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/5d91fb6e/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
index a47a483..c3f33aa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
@@ -17,27 +17,29 @@
  */
 package org.apache.hadoop.hbase.replication.regionserver;
 
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.executor.EventType;
 import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationType;
+import org.apache.log4j.Logger;
 import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshPeerParameter;
 
 /**
- * The callable executed at RS side to refresh the peer config/state.
- * <p>
- * TODO: only a dummy implementation for verifying the framework, will add implementation later.
+ * The callable executed at RS side to refresh the peer config/state. <br/>
  */
 @InterfaceAudience.Private
 public class RefreshPeerCallable implements RSProcedureCallable {
 
+  private static final Logger LOG = Logger.getLogger(RefreshPeerCallable.class);
   private HRegionServer rs;
 
   private String peerId;
 
+  private PeerModificationType type;
+
   private Exception initError;
 
   @Override
@@ -45,9 +47,27 @@ public class RefreshPeerCallable implements RSProcedureCallable {
     if (initError != null) {
       throw initError;
     }
-    Path dir = new Path("/" + peerId);
-    if (rs.getFileSystem().exists(dir)) {
-      rs.getFileSystem().create(new Path(dir, rs.getServerName().toString())).close();
+
+    LOG.info("Received a peer change event, peerId=" + peerId + ", type=" + type);
+    PeerProcedureHandler handler = rs.getReplicationSourceService().getPeerProcedureHandler();
+    switch (type) {
+      case ADD_PEER:
+        handler.addPeer(this.peerId);
+        break;
+      case REMOVE_PEER:
+        handler.removePeer(this.peerId);
+        break;
+      case ENABLE_PEER:
+        handler.enablePeer(this.peerId);
+        break;
+      case DISABLE_PEER:
+        handler.disablePeer(this.peerId);
+        break;
+      case UPDATE_PEER_CONFIG:
+        handler.updatePeerConfig(this.peerId);
+        break;
+      default:
+        throw new IllegalArgumentException("Unknown peer modification type: " + type);
     }
     return null;
   }
@@ -56,10 +76,11 @@ public class RefreshPeerCallable implements RSProcedureCallable {
   public void init(byte[] parameter, HRegionServer rs) {
     this.rs = rs;
     try {
-      this.peerId = RefreshPeerParameter.parseFrom(parameter).getPeerId();
+      RefreshPeerParameter param = RefreshPeerParameter.parseFrom(parameter);
+      this.peerId = param.getPeerId();
+      this.type = param.getType();
     } catch (InvalidProtocolBufferException e) {
       initError = e;
-      return;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/5d91fb6e/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
index d8212e9..571ee75 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
@@ -87,6 +87,8 @@ public class Replication implements
   // ReplicationLoad to access replication metrics
   private ReplicationLoad replicationLoad;
 
+  private PeerProcedureHandler peerProcedureHandler;
+
   /**
    * Instantiate the replication management (if rep is enabled).
    * @param server Hosting server
@@ -151,6 +153,8 @@ public class Replication implements
         this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60);
     LOG.debug("ReplicationStatisticsThread " + this.statsThreadPeriod);
     this.replicationLoad = new ReplicationLoad();
+
+    this.peerProcedureHandler = new PeerProcedureHandlerImpl(replicationManager);
   }
 
   /**
@@ -168,6 +172,12 @@ public class Replication implements
   public WALActionsListener getWALActionsListener() {
     return this;
   }
+
+  @Override
+  public PeerProcedureHandler getPeerProcedureHandler() {
+    return peerProcedureHandler;
+  }
+
   /**
    * Stops replication service.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/5d91fb6e/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index f4f35ae..19ea240 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -446,12 +445,10 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
 
   public void terminate(String reason, Exception cause, boolean join) {
     if (cause == null) {
-      LOG.info("Closing source "
-          + this.peerClusterZnode + " because: " + reason);
-
+      LOG.info("Closing source " + this.peerClusterZnode + " because: " + reason);
     } else {
-      LOG.error("Closing source " + this.peerClusterZnode
-          + " because an error occurred: " + reason, cause);
+      LOG.error("Closing source " + this.peerClusterZnode + " because an error occurred: " + reason,
+        cause);
     }
     this.sourceRunning = false;
     Collection<ReplicationSourceShipper> workers = workerThreads.values();

http://git-wip-us.apache.org/repos/asf/hbase/blob/5d91fb6e/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index 07c53e1..a263fc3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -564,6 +564,18 @@ public class ReplicationSourceManager implements ReplicationListener {
     this.walsById.remove(src.getPeerClusterZnode());
   }
 
+  public void addPeer(String id) throws ReplicationException, IOException {
+    LOG.info("Trying to add peer, peerId: " + id);
+    boolean added = this.replicationPeers.peerConnected(id);
+    if (added) {
+      LOG.info("Peer " + id + " connected success, trying to start the replication source thread.");
+      addSource(id);
+      if (replicationForBulkLoadDataEnabled) {
+        this.replicationQueues.addPeerToHFileRefs(id);
+      }
+    }
+  }
+
   /**
    * Thie method first deletes all the recovered sources for the specified
    * id, then deletes the normal source (deleting all related data in ZK).
@@ -611,6 +623,8 @@ public class ReplicationSourceManager implements ReplicationListener {
       }
       deleteSource(id, true);
     }
+    // Remove HFile Refs znode from zookeeper
+    this.replicationQueues.removePeerFromHFileRefs(id);
   }
 
   @Override
@@ -618,29 +632,6 @@ public class ReplicationSourceManager implements ReplicationListener {
     transferQueues(regionserver);
   }
 
-  @Override
-  public void peerRemoved(String peerId) {
-    removePeer(peerId);
-    this.replicationQueues.removePeerFromHFileRefs(peerId);
-  }
-
-  @Override
-  public void peerListChanged(List<String> peerIds) {
-    for (String id : peerIds) {
-      try {
-        boolean added = this.replicationPeers.peerConnected(id);
-        if (added) {
-          addSource(id);
-          if (replicationForBulkLoadDataEnabled) {
-            this.replicationQueues.addPeerToHFileRefs(id);
-          }
-        }
-      } catch (Exception e) {
-        LOG.error("Error while adding a new peer", e);
-      }
-    }
-  }
-
   /**
    * Class responsible to setup new ReplicationSources to take care of the
    * queues from dead region servers.

http://git-wip-us.apache.org/repos/asf/hbase/blob/5d91fb6e/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
index d2a16aa..fb29e9e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
@@ -764,4 +764,4 @@ public class TestReplicationAdmin {
     assertEquals(2097152, admin.getPeerConfig(ID_ONE).getBandwidth());
     admin.removePeer(ID_ONE);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/5d91fb6e/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java
new file mode 100644
index 0000000..b09a8a7
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java
@@ -0,0 +1,226 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.client.replication;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.client.Put;
+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.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.TestReplicationBase;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableMap;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.log4j.Logger;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MediumTests.class, ClientTests.class })
+public class TestReplicationAdminUsingProcedure extends TestReplicationBase {
+
+  private static final String PEER_ID = "2";
+  private static final Logger LOG = Logger.getLogger(TestReplicationAdminUsingProcedure.class);
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    conf1.setInt("hbase.multihconnection.threads.max", 10);
+
+    // Start the master & slave mini cluster.
+    TestReplicationBase.setUpBeforeClass();
+
+    // Remove the replication peer
+    hbaseAdmin.removeReplicationPeer(PEER_ID);
+  }
+
+  private void loadData(int startRowKey, int endRowKey) throws IOException {
+    for (int i = startRowKey; i < endRowKey; i++) {
+      byte[] rowKey = Bytes.add(row, Bytes.toBytes(i));
+      Put put = new Put(rowKey);
+      put.addColumn(famName, null, Bytes.toBytes(i));
+      htable1.put(put);
+    }
+  }
+
+  private void waitForReplication(int expectedRows, int retries)
+      throws IOException, InterruptedException {
+    Scan scan;
+    for (int i = 0; i < retries; i++) {
+      scan = new Scan();
+      if (i == retries - 1) {
+        throw new IOException("Waited too much time for normal batch replication");
+      }
+      try (ResultScanner scanner = htable2.getScanner(scan)) {
+        int count = 0;
+        for (Result res : scanner) {
+          count++;
+        }
+        if (count != expectedRows) {
+          LOG.info("Only got " + count + " rows,  expected rows: " + expectedRows);
+          Thread.sleep(SLEEP_TIME);
+        } else {
+          return;
+        }
+      }
+    }
+  }
+
+  @Before
+  public void setUp() throws IOException {
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(utility2.getClusterKey());
+    hbaseAdmin.addReplicationPeer(PEER_ID, rpc);
+
+    utility1.waitUntilAllRegionsAssigned(tableName);
+    utility2.waitUntilAllRegionsAssigned(tableName);
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    hbaseAdmin.removeReplicationPeer(PEER_ID);
+    truncateBoth();
+  }
+
+  private void truncateBoth() throws IOException {
+    utility1.deleteTableData(tableName);
+    utility2.deleteTableData(tableName);
+  }
+
+  @Test
+  public void testAddPeer() throws Exception {
+    // Load data
+    loadData(0, NB_ROWS_IN_BATCH);
+
+    // Wait the replication finished
+    waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
+  }
+
+  @Test
+  public void testRemovePeer() throws Exception {
+    // prev-check
+    waitForReplication(0, NB_RETRIES);
+
+    // Load data
+    loadData(0, NB_ROWS_IN_BATCH);
+
+    // Wait the replication finished
+    waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
+
+    // Remove the peer id
+    hbaseAdmin.removeReplicationPeer(PEER_ID);
+
+    // Load data again
+    loadData(NB_ROWS_IN_BATCH, 2 * NB_ROWS_IN_BATCH);
+
+    // Wait the replication again
+    boolean foundException = false;
+    try {
+      waitForReplication(NB_ROWS_IN_BATCH * 2, NB_RETRIES);
+    } catch (IOException e) {
+      foundException = true;
+    }
+    Assert.assertTrue(foundException);
+
+    // Truncate the table in source cluster
+    truncateBoth();
+
+    // Add peer again
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(utility2.getClusterKey());
+    hbaseAdmin.addReplicationPeer(PEER_ID, rpc);
+
+    // Load data again
+    loadData(0, NB_ROWS_IN_BATCH);
+
+    // Wait the replication finished
+    waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
+  }
+
+  @Test
+  public void testDisableAndEnablePeer() throws Exception {
+    // disable peer
+    hbaseAdmin.disableReplicationPeer(PEER_ID);
+
+    // Load data
+    loadData(0, NB_ROWS_IN_BATCH);
+
+    // Will failed to wait the replication.
+    boolean foundException = false;
+    try {
+      waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
+    } catch (IOException e) {
+      foundException = true;
+    }
+    Assert.assertTrue(foundException);
+
+    // Enable the peer
+    hbaseAdmin.enableReplicationPeer(PEER_ID);
+    waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
+
+    // Load more data
+    loadData(NB_ROWS_IN_BATCH, NB_ROWS_IN_BATCH * 2);
+
+    // Wait replication again.
+    waitForReplication(NB_ROWS_IN_BATCH * 2, NB_RETRIES);
+  }
+
+  @Test
+  public void testUpdatePeerConfig() throws Exception {
+    ReplicationPeerConfig rpc = new ReplicationPeerConfig();
+    rpc.setClusterKey(utility2.getClusterKey());
+    rpc.setExcludeTableCFsMap(
+      ImmutableMap.of(tableName, ImmutableList.of(Bytes.toString(famName))));
+
+    // Update the peer config to exclude the test table name.
+    hbaseAdmin.updateReplicationPeerConfig(PEER_ID, rpc);
+
+    // Load data
+    loadData(0, NB_ROWS_IN_BATCH);
+
+    // Will failed to wait the replication
+    boolean foundException = false;
+    try {
+      waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
+    } catch (IOException e) {
+      foundException = true;
+    }
+    Assert.assertTrue(foundException);
+
+    // Truncate the table in source cluster
+    truncateBoth();
+
+    // Update the peer config to include the test table name.
+    ReplicationPeerConfig rpc2 = new ReplicationPeerConfig();
+    rpc2.setClusterKey(utility2.getClusterKey());
+    hbaseAdmin.updateReplicationPeerConfig(PEER_ID, rpc2);
+
+    // Load data again
+    loadData(0, NB_ROWS_IN_BATCH);
+
+    // Wait the replication finished
+    waitForReplication(NB_ROWS_IN_BATCH, NB_RETRIES);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/5d91fb6e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
deleted file mode 100644
index ed7c6fa..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.master.replication;
-
-import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-
-public class DummyModifyPeerProcedure extends ModifyPeerProcedure {
-
-  public DummyModifyPeerProcedure() {
-  }
-
-  public DummyModifyPeerProcedure(String peerId) {
-    super(peerId);
-  }
-
-  @Override
-  public PeerOperationType getPeerOperationType() {
-    return PeerOperationType.ADD;
-  }
-
-  @Override
-  protected void prePeerModification(MasterProcedureEnv env) {
-  }
-
-  @Override
-  protected void updatePeerStorage(MasterProcedureEnv env) {
-  }
-
-  @Override
-  protected void postPeerModification(MasterProcedureEnv env) {
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/5d91fb6e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestDummyModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestDummyModifyPeerProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestDummyModifyPeerProcedure.java
deleted file mode 100644
index ec06306..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/TestDummyModifyPeerProcedure.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.master.replication;
-
-import static org.junit.Assert.assertTrue;
-
-import java.util.HashSet;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.testclassification.MasterTests;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({ MasterTests.class, LargeTests.class })
-public class TestDummyModifyPeerProcedure {
-
-  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
-
-  private static String PEER_ID;
-
-  private static Path DIR;
-
-  @BeforeClass
-  public static void setUp() throws Exception {
-    UTIL.startMiniCluster(3);
-    PEER_ID = "testPeer";
-    DIR = new Path("/" + PEER_ID);
-    UTIL.getTestFileSystem().mkdirs(DIR);
-  }
-
-  @AfterClass
-  public static void tearDown() throws Exception {
-    UTIL.shutdownMiniCluster();
-  }
-
-  @Test
-  public void test() throws Exception {
-    ProcedureExecutor<?> executor =
-        UTIL.getMiniHBaseCluster().getMaster().getMasterProcedureExecutor();
-    long procId = executor.submitProcedure(new DummyModifyPeerProcedure(PEER_ID));
-    UTIL.waitFor(30000, new Waiter.Predicate<Exception>() {
-
-      @Override
-      public boolean evaluate() throws Exception {
-        return executor.isFinished(procId);
-      }
-    });
-    Set<String> serverNames = UTIL.getHBaseCluster().getRegionServerThreads().stream()
-        .map(t -> t.getRegionServer().getServerName().toString())
-        .collect(Collectors.toCollection(HashSet::new));
-    for (FileStatus s : UTIL.getTestFileSystem().listStatus(DIR)) {
-      assertTrue(serverNames.remove(s.getPath().getName()));
-    }
-    assertTrue(serverNames.isEmpty());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/5d91fb6e/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
index a04d524..f118ca3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hbase.replication;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
@@ -156,41 +155,6 @@ public class TestReplicationTrackerZKImpl {
   }
 
   @Test(timeout = 30000)
-  public void testPeerRemovedEvent() throws Exception {
-    rp.registerPeer("5", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()));
-    rt.registerListener(new DummyReplicationListener());
-    rp.unregisterPeer("5");
-    // wait for event
-    while (peerRemovedCount.get() < 1) {
-      Thread.sleep(5);
-    }
-    assertEquals("5", peerRemovedData);
-  }
-
-  @Test(timeout = 30000)
-  public void testPeerListChangedEvent() throws Exception {
-    // add a peer
-    rp.registerPeer("5", new ReplicationPeerConfig().setClusterKey(utility.getClusterKey()));
-    zkw.getRecoverableZooKeeper().getZooKeeper().getChildren("/hbase/replication/peers/5", true);
-    rt.registerListener(new DummyReplicationListener());
-    rp.disablePeer("5");
-    int tmp = plChangedCount.get();
-    LOG.info("Peer count=" + tmp);
-    ZKUtil.deleteNode(zkw, "/hbase/replication/peers/5/peer-state");
-    // wait for event
-    while (plChangedCount.get() <= tmp) {
-      Thread.sleep(100);
-      LOG.info("Peer count=" + tmp);
-    }
-    assertEquals(1, plChangedData.size());
-    assertTrue(plChangedData.contains("5"));
-
-    // clean up
-    //ZKUtil.deleteNode(zkw, "/hbase/replication/peers/5");
-    rp.unregisterPeer("5");
-  }
-
-  @Test(timeout = 30000)
   public void testPeerNameControl() throws Exception {
     int exists = 0;
     int hyphen = 0;
@@ -222,21 +186,6 @@ public class TestReplicationTrackerZKImpl {
       rsRemovedCount.getAndIncrement();
       LOG.debug("Received regionServerRemoved event: " + regionServer);
     }
-
-    @Override
-    public void peerRemoved(String peerId) {
-      peerRemovedData = peerId;
-      peerRemovedCount.getAndIncrement();
-      LOG.debug("Received peerDisconnected event: " + peerId);
-    }
-
-    @Override
-    public void peerListChanged(List<String> peerIds) {
-      plChangedData.clear();
-      plChangedData.addAll(peerIds);
-      int count = plChangedCount.getAndIncrement();
-      LOG.debug("Received peerListChanged event " + count);
-    }
   }
 
   private class DummyServer implements Server {

http://git-wip-us.apache.org/repos/asf/hbase/blob/5d91fb6e/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
index 307ea7f..9f234a8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -176,6 +176,12 @@ public abstract class TestReplicationSourceManager {
     replication = new Replication(new DummyServer(), fs, logDir, oldLogDir);
 
     managerOfCluster = getManagerFromCluster();
+    if (managerOfCluster != null) {
+      // After replication procedure, we need to add peer by hand (other than by receiving
+      // notification from zk)
+      managerOfCluster.addPeer(slaveId);
+    }
+
     manager = replication.getReplicationManager();
     manager.addSource(slaveId);
     if (managerOfCluster != null) {
@@ -535,18 +541,16 @@ public abstract class TestReplicationSourceManager {
       final int globalLogQueueSizeInitial = globalSource.getSizeOfLogQueue();
       final long sizeOfLatestPath = getSizeOfLatestPath();
       addPeerAndWait(peerId, peerConfig, true);
-      assertEquals(sizeOfLatestPath + globalLogQueueSizeInitial,
-          globalSource.getSizeOfLogQueue());
+      assertEquals(sizeOfLatestPath + globalLogQueueSizeInitial, globalSource.getSizeOfLogQueue());
       ReplicationSourceInterface source = manager.getSource(peerId);
       // Sanity check
       assertNotNull(source);
       final int sizeOfSingleLogQueue = source.getSourceMetrics().getSizeOfLogQueue();
       // Enqueue log and check if metrics updated
       source.enqueueLog(new Path("abc"));
-      assertEquals(1 + sizeOfSingleLogQueue,
-          source.getSourceMetrics().getSizeOfLogQueue());
-      assertEquals(source.getSourceMetrics().getSizeOfLogQueue()
-              + globalLogQueueSizeInitial, globalSource.getSizeOfLogQueue());
+      assertEquals(1 + sizeOfSingleLogQueue, source.getSourceMetrics().getSizeOfLogQueue());
+      assertEquals(source.getSourceMetrics().getSizeOfLogQueue() + globalLogQueueSizeInitial,
+        globalSource.getSizeOfLogQueue());
 
       // Removing the peer should reset the global metrics
       removePeerAndWait(peerId);
@@ -556,9 +560,8 @@ public abstract class TestReplicationSourceManager {
       addPeerAndWait(peerId, peerConfig, true);
       source = manager.getSource(peerId);
       assertNotNull(source);
-      assertEquals(sizeOfLatestPath, source.getSourceMetrics().getSizeOfLogQueue());
-      assertEquals(source.getSourceMetrics().getSizeOfLogQueue()
-          + globalLogQueueSizeInitial, globalSource.getSizeOfLogQueue());
+      assertEquals(source.getSourceMetrics().getSizeOfLogQueue() + globalLogQueueSizeInitial,
+        globalSource.getSizeOfLogQueue());
     } finally {
       removePeerAndWait(peerId);
     }
@@ -575,8 +578,14 @@ public abstract class TestReplicationSourceManager {
       final boolean waitForSource) throws Exception {
     final ReplicationPeers rp = manager.getReplicationPeers();
     rp.registerPeer(peerId, peerConfig);
+    try {
+      manager.addPeer(peerId);
+    } catch (Exception e) {
+      // ignore the failed exception, because we'll test both success & failed case.
+    }
     waitPeer(peerId, manager, waitForSource);
     if (managerOfCluster != null) {
+      managerOfCluster.addPeer(peerId);
       waitPeer(peerId, managerOfCluster, waitForSource);
     }
   }
@@ -609,6 +618,11 @@ public abstract class TestReplicationSourceManager {
     final ReplicationPeers rp = manager.getReplicationPeers();
     if (rp.getAllPeerIds().contains(peerId)) {
       rp.unregisterPeer(peerId);
+      try {
+        manager.removePeer(peerId);
+      } catch (Exception e) {
+        // ignore the failed exception and continue.
+      }
     }
     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
       @Override public boolean evaluate() throws Exception {


[13/26] hbase git commit: HBASE-19524 Master side changes for moving peer modification from zk watcher to procedure

Posted by zh...@apache.org.
HBASE-19524 Master side changes for moving peer modification from zk watcher to procedure


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/4567906c
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/4567906c
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/4567906c

Branch: refs/heads/HBASE-19397
Commit: 4567906c597d45ffb5036ef6c96956ab5f47632b
Parents: f9ea961
Author: zhangduo <zh...@apache.org>
Authored: Mon Dec 18 15:22:36 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Dec 28 08:51:59 2017 +0800

----------------------------------------------------------------------
 .../procedure2/RemoteProcedureDispatcher.java   |   3 +-
 .../src/main/protobuf/MasterProcedure.proto     |  21 +++-
 .../src/main/protobuf/RegionServerStatus.proto  |   3 +-
 .../src/main/protobuf/Replication.proto         |   5 +
 .../replication/ReplicationPeersZKImpl.java     |   4 +-
 .../org/apache/hadoop/hbase/master/HMaster.java | 100 ++++++++-----------
 .../hadoop/hbase/master/MasterRpcServices.java  |   4 +-
 .../hadoop/hbase/master/MasterServices.java     |  26 +++--
 .../assignment/RegionTransitionProcedure.java   |  11 +-
 .../master/procedure/MasterProcedureEnv.java    |   5 +
 .../master/procedure/ProcedurePrepareLatch.java |   2 +-
 .../master/replication/AddPeerProcedure.java    |  97 ++++++++++++++++++
 .../replication/DisablePeerProcedure.java       |  70 +++++++++++++
 .../master/replication/EnablePeerProcedure.java |  69 +++++++++++++
 .../master/replication/ModifyPeerProcedure.java |  97 +++++++++++++++---
 .../master/replication/RefreshPeerCallable.java |  67 -------------
 .../replication/RefreshPeerProcedure.java       |  28 ++++--
 .../master/replication/RemovePeerProcedure.java |  69 +++++++++++++
 .../master/replication/ReplicationManager.java  |  76 +++++++-------
 .../replication/UpdatePeerConfigProcedure.java  |  92 +++++++++++++++++
 .../hbase/regionserver/HRegionServer.java       |   6 +-
 .../regionserver/RefreshPeerCallable.java       |  70 +++++++++++++
 .../hbase/master/MockNoopMasterServices.java    |  23 +++--
 .../replication/DummyModifyPeerProcedure.java   |  13 ++-
 24 files changed, 736 insertions(+), 225 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4567906c/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
index e9a6906..1235b33 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java
@@ -247,9 +247,8 @@ public abstract class RemoteProcedureDispatcher<TEnv, TRemote extends Comparable
     /**
      * Called when RS tells the remote procedure is failed through the {@code reportProcedureDone}
      * method.
-     * @param error the error message
      */
-    void remoteOperationFailed(TEnv env, String error);
+    void remoteOperationFailed(TEnv env, RemoteProcedureException error);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/4567906c/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
index 0e2bdba..ae676ea 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -27,6 +27,7 @@ option optimize_for = SPEED;
 import "HBase.proto";
 import "RPC.proto";
 import "Snapshot.proto";
+import "Replication.proto";
 
 // ============================================================================
 //  WARNING - Compatibility rules
@@ -367,9 +368,10 @@ message GCMergedRegionsStateData {
 }
 
 enum PeerModificationState {
-  UPDATE_PEER_STORAGE = 1;
-  REFRESH_PEER_ON_RS = 2;
-  POST_PEER_MODIFICATION = 3;
+  PRE_PEER_MODIFICATION = 1;
+  UPDATE_PEER_STORAGE = 2;
+  REFRESH_PEER_ON_RS = 3;
+  POST_PEER_MODIFICATION = 4;
 }
 
 message PeerModificationStateData {
@@ -394,4 +396,17 @@ message RefreshPeerParameter {
   required string peer_id = 1;
   required PeerModificationType type = 2;
   required ServerName target_server = 3;
+}
+
+message ModifyPeerStateData {
+  required string peer_id = 1;
+}
+
+message AddPeerStateData {
+  required ReplicationPeer peer_config = 1;
+  required bool enabled = 2;
+}
+
+message UpdatePeerConfigStateData {
+  required ReplicationPeer peer_config = 1;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/4567906c/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
index eb396ac..4f75941 100644
--- a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto
@@ -28,6 +28,7 @@ option optimize_for = SPEED;
 
 import "HBase.proto";
 import "ClusterStatus.proto";
+import "ErrorHandling.proto";
 
 message RegionServerStartupRequest {
   /** Port number this regionserver is up on */
@@ -152,7 +153,7 @@ message ReportProcedureDoneRequest {
     ERROR = 2;
   }
   required Status status = 2;
-  optional string error = 3;
+  optional ForeignExceptionMessage error = 3;
 }
 
 message ReportProcedureDoneResponse {

http://git-wip-us.apache.org/repos/asf/hbase/blob/4567906c/hbase-protocol-shaded/src/main/protobuf/Replication.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Replication.proto b/hbase-protocol-shaded/src/main/protobuf/Replication.proto
index 8657c25..9f7b4c2 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Replication.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Replication.proto
@@ -84,6 +84,7 @@ message AddReplicationPeerRequest {
 }
 
 message AddReplicationPeerResponse {
+  optional uint64 proc_id = 1;
 }
 
 message RemoveReplicationPeerRequest {
@@ -91,6 +92,7 @@ message RemoveReplicationPeerRequest {
 }
 
 message RemoveReplicationPeerResponse {
+  optional uint64 proc_id = 1;
 }
 
 message EnableReplicationPeerRequest {
@@ -98,6 +100,7 @@ message EnableReplicationPeerRequest {
 }
 
 message EnableReplicationPeerResponse {
+  optional uint64 proc_id = 1;
 }
 
 message DisableReplicationPeerRequest {
@@ -105,6 +108,7 @@ message DisableReplicationPeerRequest {
 }
 
 message DisableReplicationPeerResponse {
+  optional uint64 proc_id = 1;
 }
 
 message GetReplicationPeerConfigRequest {
@@ -122,6 +126,7 @@ message UpdateReplicationPeerConfigRequest {
 }
 
 message UpdateReplicationPeerConfigResponse {
+  optional uint64 proc_id = 1;
 }
 
 message ListReplicationPeersRequest {

http://git-wip-us.apache.org/repos/asf/hbase/blob/4567906c/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
----------------------------------------------------------------------
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 289d2aa..0b2f08a 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
@@ -531,7 +531,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
         for (String queueId : queueIds) {
           ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
           if (queueInfo.getPeerId().equals(peerId)) {
-            throw new ReplicationException("undeleted queue for peerId: " + peerId
+            throw new IllegalArgumentException("undeleted queue for peerId: " + peerId
                 + ", replicator: " + replicator + ", queueId: " + queueId);
           }
         }
@@ -539,7 +539,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
       // Check for hfile-refs queue
       if (-1 != ZKUtil.checkExists(zookeeper, hfileRefsZNode)
           && queuesClient.getAllPeersFromHFileRefsQueue().contains(peerId)) {
-        throw new ReplicationException("Undeleted queue for peerId: " + peerId
+        throw new IllegalArgumentException("Undeleted queue for peerId: " + peerId
             + ", found in hfile-refs node path " + hfileRefsZNode);
       }
     } catch (KeeperException e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/4567906c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 9d0555f..4d3b265 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.master;
 
 import com.google.protobuf.Descriptors;
 import com.google.protobuf.Service;
+
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.lang.reflect.Constructor;
@@ -49,10 +50,12 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Function;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
+
 import javax.servlet.ServletException;
 import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ClusterMetrics.Option;
@@ -127,7 +130,13 @@ import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
 import org.apache.hadoop.hbase.master.procedure.RecoverMetaProcedure;
 import org.apache.hadoop.hbase.master.procedure.TruncateTableProcedure;
+import org.apache.hadoop.hbase.master.replication.AddPeerProcedure;
+import org.apache.hadoop.hbase.master.replication.DisablePeerProcedure;
+import org.apache.hadoop.hbase.master.replication.EnablePeerProcedure;
+import org.apache.hadoop.hbase.master.replication.ModifyPeerProcedure;
+import org.apache.hadoop.hbase.master.replication.RemovePeerProcedure;
 import org.apache.hadoop.hbase.master.replication.ReplicationManager;
+import org.apache.hadoop.hbase.master.replication.UpdatePeerConfigProcedure;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
@@ -140,6 +149,7 @@ import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
 import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.quotas.MasterSpaceQuotaObserver;
@@ -168,7 +178,6 @@ import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CompressionTest;
 import org.apache.hadoop.hbase.util.EncryptionTest;
-import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.hbase.util.HasThread;
 import org.apache.hadoop.hbase.util.IdLock;
@@ -329,15 +338,15 @@ public class HMaster extends HRegionServer implements MasterServices {
   private volatile boolean activeMaster = false;
 
   // flag set after we complete initialization once active
-  private final ProcedureEvent initialized = new ProcedureEvent("master initialized");
+  private final ProcedureEvent<?> initialized = new ProcedureEvent<>("master initialized");
 
   // flag set after master services are started,
   // initialization may have not completed yet.
   volatile boolean serviceStarted = false;
 
   // flag set after we complete assignMeta.
-  private final ProcedureEvent serverCrashProcessingEnabled =
-    new ProcedureEvent("server crash processing");
+  private final ProcedureEvent<?> serverCrashProcessingEnabled =
+    new ProcedureEvent<>("server crash processing");
 
   // Maximum time we should run balancer for
   private final int maxBlancingTime;
@@ -1193,7 +1202,6 @@ public class HMaster extends HRegionServer implements MasterServices {
 
   private void startProcedureExecutor() throws IOException {
     final MasterProcedureEnv procEnv = new MasterProcedureEnv(this);
-    final Path rootDir = FSUtils.getRootDir(conf);
 
     procedureStore = new WALProcedureStore(conf,
         new MasterProcedureEnv.WALStoreLeaseRecovery(this));
@@ -2313,11 +2321,8 @@ public class HMaster extends HRegionServer implements MasterServices {
             return true;
           }
           Pair<RegionInfo, ServerName> pair =
-              new Pair(MetaTableAccessor.getRegionInfo(data),
+              new Pair<>(MetaTableAccessor.getRegionInfo(data),
                   MetaTableAccessor.getServerName(data,0));
-          if (pair == null) {
-            return false;
-          }
           if (!pair.getFirst().getTable().equals(tableName)) {
             return false;
           }
@@ -2743,7 +2748,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
-  public ProcedureEvent getInitializedEvent() {
+  public ProcedureEvent<?> getInitializedEvent() {
     return initialized;
   }
 
@@ -2762,7 +2767,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     procedureExecutor.getEnvironment().setEventReady(serverCrashProcessingEnabled, b);
   }
 
-  public ProcedureEvent getServerCrashProcessingEnabledEvent() {
+  public ProcedureEvent<?> getServerCrashProcessingEnabledEvent() {
     return serverCrashProcessingEnabled;
   }
 
@@ -3313,54 +3318,36 @@ public class HMaster extends HRegionServer implements MasterServices {
     return favoredNodesManager;
   }
 
+  private long executePeerProcedure(ModifyPeerProcedure procedure) throws IOException {
+    long procId = procedureExecutor.submitProcedure(procedure);
+    procedure.getLatch().await();
+    return procId;
+  }
+
   @Override
-  public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
+  public long addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
       throws ReplicationException, IOException {
-    if (cpHost != null) {
-      cpHost.preAddReplicationPeer(peerId, peerConfig);
-    }
-    LOG.info(getClientIdAuditPrefix() + " creating replication peer, id=" + peerId + ", config="
-        + peerConfig + ", state=" + (enabled ? "ENABLED" : "DISABLED"));
-    this.replicationManager.addReplicationPeer(peerId, peerConfig, enabled);
-    if (cpHost != null) {
-      cpHost.postAddReplicationPeer(peerId, peerConfig);
-    }
+    LOG.info(getClientIdAuditPrefix() + " creating replication peer, id=" + peerId + ", config=" +
+      peerConfig + ", state=" + (enabled ? "ENABLED" : "DISABLED"));
+    return executePeerProcedure(new AddPeerProcedure(peerId, peerConfig, enabled));
   }
 
   @Override
-  public void removeReplicationPeer(String peerId) throws ReplicationException, IOException {
-    if (cpHost != null) {
-      cpHost.preRemoveReplicationPeer(peerId);
-    }
+  public long removeReplicationPeer(String peerId) throws ReplicationException, IOException {
     LOG.info(getClientIdAuditPrefix() + " removing replication peer, id=" + peerId);
-    this.replicationManager.removeReplicationPeer(peerId);
-    if (cpHost != null) {
-      cpHost.postRemoveReplicationPeer(peerId);
-    }
+    return executePeerProcedure(new RemovePeerProcedure(peerId));
   }
 
   @Override
-  public void enableReplicationPeer(String peerId) throws ReplicationException, IOException {
-    if (cpHost != null) {
-      cpHost.preEnableReplicationPeer(peerId);
-    }
+  public long enableReplicationPeer(String peerId) throws ReplicationException, IOException {
     LOG.info(getClientIdAuditPrefix() + " enable replication peer, id=" + peerId);
-    this.replicationManager.enableReplicationPeer(peerId);
-    if (cpHost != null) {
-      cpHost.postEnableReplicationPeer(peerId);
-    }
+    return executePeerProcedure(new EnablePeerProcedure(peerId));
   }
 
   @Override
-  public void disableReplicationPeer(String peerId) throws ReplicationException, IOException {
-    if (cpHost != null) {
-      cpHost.preDisableReplicationPeer(peerId);
-    }
+  public long disableReplicationPeer(String peerId) throws ReplicationException, IOException {
     LOG.info(getClientIdAuditPrefix() + " disable replication peer, id=" + peerId);
-    this.replicationManager.disableReplicationPeer(peerId);
-    if (cpHost != null) {
-      cpHost.postDisableReplicationPeer(peerId);
-    }
+    return executePeerProcedure(new DisablePeerProcedure(peerId));
   }
 
   @Override
@@ -3379,17 +3366,11 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
-  public void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
+  public long updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
       throws ReplicationException, IOException {
-    if (cpHost != null) {
-      cpHost.preUpdateReplicationPeerConfig(peerId, peerConfig);
-    }
-    LOG.info(getClientIdAuditPrefix() + " update replication peer config, id=" + peerId
-        + ", config=" + peerConfig);
-    this.replicationManager.updatePeerConfig(peerId, peerConfig);
-    if (cpHost != null) {
-      cpHost.postUpdateReplicationPeerConfig(peerId, peerConfig);
-    }
+    LOG.info(getClientIdAuditPrefix() + " update replication peer config, id=" + peerId +
+      ", config=" + peerConfig);
+    return executePeerProcedure(new UpdatePeerConfigProcedure(peerId, peerConfig));
   }
 
   @Override
@@ -3542,10 +3523,15 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
   }
 
-  public void remoteProcedureFailed(long procId, String error) {
+  public void remoteProcedureFailed(long procId, RemoteProcedureException error) {
     RemoteProcedure<MasterProcedureEnv, ?> procedure = getRemoteProcedure(procId);
     if (procedure != null) {
       procedure.remoteOperationFailed(procedureExecutor.getEnvironment(), error);
     }
   }
-}
\ No newline at end of file
+
+  @Override
+  public ReplicationManager getReplicationManager() {
+    return replicationManager;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/4567906c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 1d6949c..700b363 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -72,6 +72,7 @@ import org.apache.hadoop.hbase.procedure2.LockedResource;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
@@ -2256,7 +2257,8 @@ public class MasterRpcServices extends RSRpcServices
     if (request.getStatus() == ReportProcedureDoneRequest.Status.SUCCESS) {
       master.remoteProcedureCompleted(request.getProcId());
     } else {
-      master.remoteProcedureFailed(request.getProcId(), request.getError());
+      master.remoteProcedureFailed(request.getProcId(),
+        RemoteProcedureException.fromProto(request.getError()));
     }
     return ReportProcedureDoneResponse.getDefaultInstance();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4567906c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index 6b3c212..43df8b1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -18,10 +17,11 @@
  */
 package org.apache.hadoop.hbase.master;
 
+import com.google.protobuf.Service;
+
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableDescriptors;
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.locking.LockManager;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.replication.ReplicationManager;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
 import org.apache.hadoop.hbase.procedure2.LockedResource;
@@ -52,8 +53,6 @@ import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 
-import com.google.protobuf.Service;
-
 /**
  * A curated subset of services provided by {@link HMaster}.
  * For use internally only. Passed to Managers, Services and Chores so can pass less-than-a
@@ -136,7 +135,7 @@ public interface MasterServices extends Server {
    * @return Tripped when Master has finished initialization.
    */
   @VisibleForTesting
-  public ProcedureEvent getInitializedEvent();
+  public ProcedureEvent<?> getInitializedEvent();
 
   /**
    * @return Master's instance of {@link MetricsMaster}
@@ -430,26 +429,26 @@ public interface MasterServices extends Server {
    * @param peerConfig configuration for the replication slave cluster
    * @param enabled peer state, true if ENABLED and false if DISABLED
    */
-  void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
+  long addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
       throws ReplicationException, IOException;
 
   /**
    * Removes a peer and stops the replication
    * @param peerId a short name that identifies the peer
    */
-  void removeReplicationPeer(String peerId) throws ReplicationException, IOException;
+  long removeReplicationPeer(String peerId) throws ReplicationException, IOException;
 
   /**
    * Restart the replication stream to the specified peer
    * @param peerId a short name that identifies the peer
    */
-  void enableReplicationPeer(String peerId) throws ReplicationException, IOException;
+  long enableReplicationPeer(String peerId) throws ReplicationException, IOException;
 
   /**
    * Stop the replication stream to the specified peer
    * @param peerId a short name that identifies the peer
    */
-  void disableReplicationPeer(String peerId) throws ReplicationException, IOException;
+  long disableReplicationPeer(String peerId) throws ReplicationException, IOException;
 
   /**
    * Returns the configured ReplicationPeerConfig for the specified peer
@@ -460,11 +459,16 @@ public interface MasterServices extends Server {
       IOException;
 
   /**
+   * Returns the {@link ReplicationManager}.
+   */
+  ReplicationManager getReplicationManager();
+
+  /**
    * Update the peerConfig for the specified peer
    * @param peerId a short name that identifies the peer
    * @param peerConfig new config for the peer
    */
-  void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
+  long updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
       throws ReplicationException, IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/4567906c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
index 43e75ed..5714cae 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java
@@ -16,7 +16,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.master.assignment;
 
 import java.io.IOException;
@@ -33,13 +32,15 @@ import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
+
 /**
  * Base class for the Assign and Unassign Procedure.
  *
@@ -415,7 +416,7 @@ public abstract class RegionTransitionProcedure
   }
 
   @Override
-  public void remoteOperationFailed(MasterProcedureEnv env, String error) {
+  public void remoteOperationFailed(MasterProcedureEnv env, RemoteProcedureException error) {
     // should not be called for region operation until we modified the open/close region procedure
     throw new UnsupportedOperationException();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4567906c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
index 0a4c97d..fa4d371 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
+import org.apache.hadoop.hbase.master.replication.ReplicationManager;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
@@ -137,6 +138,10 @@ public class MasterProcedureEnv implements ConfigurationObserver {
     return remoteDispatcher;
   }
 
+  public ReplicationManager getReplicationManager() {
+    return master.getReplicationManager();
+  }
+
   public boolean isRunning() {
     if (this.master == null || this.master.getMasterProcedureExecutor() == null) return false;
     return master.getMasterProcedureExecutor().isRunning();

http://git-wip-us.apache.org/repos/asf/hbase/blob/4567906c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java
index 09d05e6..dbea6fa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedurePrepareLatch.java
@@ -64,7 +64,7 @@ public abstract class ProcedurePrepareLatch {
   protected abstract void countDown(final Procedure proc);
   public abstract void await() throws IOException;
 
-  protected static void releaseLatch(final ProcedurePrepareLatch latch, final Procedure proc) {
+  public static void releaseLatch(final ProcedurePrepareLatch latch, final Procedure proc) {
     if (latch != null) {
       latch.countDown(proc);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4567906c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
new file mode 100644
index 0000000..c3862d8
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.replication;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddPeerStateData;
+
+/**
+ * The procedure for adding a new replication peer.
+ */
+@InterfaceAudience.Private
+public class AddPeerProcedure extends ModifyPeerProcedure {
+
+  private static final Log LOG = LogFactory.getLog(AddPeerProcedure.class);
+
+  private ReplicationPeerConfig peerConfig;
+
+  private boolean enabled;
+
+  public AddPeerProcedure() {
+  }
+
+  public AddPeerProcedure(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) {
+    super(peerId);
+    this.peerConfig = peerConfig;
+    this.enabled = enabled;
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.ADD;
+  }
+
+  @Override
+  protected void prePeerModification(MasterProcedureEnv env) throws IOException {
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.preAddReplicationPeer(peerId, peerConfig);
+    }
+  }
+
+  @Override
+  protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
+    env.getReplicationManager().addReplicationPeer(peerId, peerConfig, enabled);
+  }
+
+  @Override
+  protected void postPeerModification(MasterProcedureEnv env) throws IOException {
+    LOG.info("Successfully added " + (enabled ? "ENABLED" : "DISABLED") + " peer " + peerId +
+      ", config " + peerConfig);
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      env.getMasterCoprocessorHost().postAddReplicationPeer(peerId, peerConfig);
+    }
+  }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.serializeStateData(serializer);
+    serializer.serialize(AddPeerStateData.newBuilder()
+        .setPeerConfig(ReplicationPeerConfigUtil.convert(peerConfig)).setEnabled(enabled).build());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.deserializeStateData(serializer);
+    AddPeerStateData data = serializer.deserialize(AddPeerStateData.class);
+    peerConfig = ReplicationPeerConfigUtil.convert(data.getPeerConfig());
+    enabled = data.getEnabled();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/4567906c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
new file mode 100644
index 0000000..0b32db9
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.replication;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * The procedure for disabling a replication peer.
+ */
+@InterfaceAudience.Private
+public class DisablePeerProcedure extends ModifyPeerProcedure {
+
+  private static final Log LOG = LogFactory.getLog(DisablePeerProcedure.class);
+
+  public DisablePeerProcedure() {
+  }
+
+  public DisablePeerProcedure(String peerId) {
+    super(peerId);
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.DISABLE;
+  }
+
+  @Override
+  protected void prePeerModification(MasterProcedureEnv env) throws IOException {
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.preDisableReplicationPeer(peerId);
+    }
+  }
+
+  @Override
+  protected void updatePeerStorage(MasterProcedureEnv env)
+      throws IllegalArgumentException, Exception {
+    env.getReplicationManager().disableReplicationPeer(peerId);
+  }
+
+  @Override
+  protected void postPeerModification(MasterProcedureEnv env) throws IOException {
+    LOG.info("Successfully disabled peer " + peerId);
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.postDisableReplicationPeer(peerId);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/4567906c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
new file mode 100644
index 0000000..92ba000
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
@@ -0,0 +1,69 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.replication;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * The procedure for enabling a replication peer.
+ */
+@InterfaceAudience.Private
+public class EnablePeerProcedure extends ModifyPeerProcedure {
+
+  private static final Log LOG = LogFactory.getLog(EnablePeerProcedure.class);
+
+  public EnablePeerProcedure() {
+  }
+
+  public EnablePeerProcedure(String peerId) {
+    super(peerId);
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.ENABLE;
+  }
+
+  @Override
+  protected void prePeerModification(MasterProcedureEnv env) throws IOException {
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.preEnableReplicationPeer(peerId);
+    }
+  }
+
+  @Override
+  protected void updatePeerStorage(MasterProcedureEnv env) throws Exception {
+    env.getReplicationManager().enableReplicationPeer(peerId);
+  }
+
+  @Override
+  protected void postPeerModification(MasterProcedureEnv env) throws IOException {
+    LOG.info("Successfully enabled peer " + peerId);
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.postEnableReplicationPeer(peerId);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/4567906c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
index fca05a7..7076bab 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
@@ -21,15 +21,22 @@ import java.io.IOException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
+import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
 import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
 import org.apache.yetus.audience.InterfaceAudience;
 
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyPeerStateData;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationState;
 
+/**
+ * The base class for all replication peer related procedure.
+ */
 @InterfaceAudience.Private
 public abstract class ModifyPeerProcedure
     extends StateMachineProcedure<MasterProcedureEnv, PeerModificationState>
@@ -39,11 +46,21 @@ public abstract class ModifyPeerProcedure
 
   protected String peerId;
 
+  // used to keep compatible with old client where we can only returns after updateStorage.
+  protected ProcedurePrepareLatch latch;
+
   protected ModifyPeerProcedure() {
   }
 
   protected ModifyPeerProcedure(String peerId) {
     this.peerId = peerId;
+    // TODO: temporarily set a 4.0 here to always wait for the procedure exection completed. Change
+    // to 3.0 or 2.0 after the client modification is done.
+    this.latch = ProcedurePrepareLatch.createLatch(4, 0);
+  }
+
+  public ProcedurePrepareLatch getLatch() {
+    return latch;
   }
 
   @Override
@@ -52,28 +69,58 @@ public abstract class ModifyPeerProcedure
   }
 
   /**
-   * Return {@code false} means that the operation is invalid and we should give up, otherwise
-   * {@code true}.
+   * Called before we start the actual processing. If an exception is thrown then we will give up
+   * and mark the procedure as failed directly.
+   */
+  protected abstract void prePeerModification(MasterProcedureEnv env) throws IOException;
+
+  /**
+   * We will give up and mark the procedure as failure if {@link IllegalArgumentException} is
+   * thrown, for other type of Exception we will retry.
+   */
+  protected abstract void updatePeerStorage(MasterProcedureEnv env)
+      throws IllegalArgumentException, Exception;
+
+  /**
+   * Called before we finish the procedure. The implementation can do some logging work, and also
+   * call the coprocessor hook if any.
    * <p>
-   * You need to call {@link #setFailure(String, Throwable)} to give the detail failure information.
+   * Notice that, since we have already done the actual work, throwing exception here will not fail
+   * this procedure, we will just ignore it and finish the procedure as suceeded.
    */
-  protected abstract boolean updatePeerStorage() throws IOException;
+  protected abstract void postPeerModification(MasterProcedureEnv env) throws IOException;
 
-  protected void postPeerModification() {
+  private void releaseLatch() {
+    ProcedurePrepareLatch.releaseLatch(latch, this);
   }
 
   @Override
   protected Flow executeFromState(MasterProcedureEnv env, PeerModificationState state)
       throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
     switch (state) {
-      case UPDATE_PEER_STORAGE:
+      case PRE_PEER_MODIFICATION:
         try {
-          if (!updatePeerStorage()) {
-            assert isFailed() : "setFailure is not called";
-            return Flow.NO_MORE_STATE;
-          }
+          prePeerModification(env);
         } catch (IOException e) {
-          LOG.warn("update peer storage failed, retry", e);
+          LOG.warn(getClass().getName() + " failed to call prePeerModification for peer " + peerId +
+            ", mark the procedure as failure and give up", e);
+          setFailure("prePeerModification", e);
+          releaseLatch();
+          return Flow.NO_MORE_STATE;
+        }
+        setNextState(PeerModificationState.UPDATE_PEER_STORAGE);
+        return Flow.HAS_MORE_STATE;
+      case UPDATE_PEER_STORAGE:
+        try {
+          updatePeerStorage(env);
+        } catch (IllegalArgumentException e) {
+          setFailure("master-" + getPeerOperationType().name().toLowerCase() + "-peer",
+            new DoNotRetryIOException(e));
+          releaseLatch();
+          return Flow.NO_MORE_STATE;
+        } catch (Exception e) {
+          LOG.warn(
+            getClass().getName() + " update peer storage for peer " + peerId + " failed, retry", e);
           throw new ProcedureYieldException();
         }
         setNextState(PeerModificationState.REFRESH_PEER_ON_RS);
@@ -85,7 +132,13 @@ public abstract class ModifyPeerProcedure
         setNextState(PeerModificationState.POST_PEER_MODIFICATION);
         return Flow.HAS_MORE_STATE;
       case POST_PEER_MODIFICATION:
-        postPeerModification();
+        try {
+          postPeerModification(env);
+        } catch (IOException e) {
+          LOG.warn(getClass().getName() + " failed to call prePeerModification for peer " + peerId +
+            ", ignore since the procedure has already done", e);
+        }
+        releaseLatch();
         return Flow.NO_MORE_STATE;
       default:
         throw new UnsupportedOperationException("unhandled state=" + state);
@@ -107,6 +160,12 @@ public abstract class ModifyPeerProcedure
   @Override
   protected void rollbackState(MasterProcedureEnv env, PeerModificationState state)
       throws IOException, InterruptedException {
+    if (state == PeerModificationState.PRE_PEER_MODIFICATION ||
+      state == PeerModificationState.UPDATE_PEER_STORAGE) {
+      // actually the peer related operations has no rollback, but if we haven't done any
+      // modifications on the peer storage, we can just return.
+      return;
+    }
     throw new UnsupportedOperationException();
   }
 
@@ -122,6 +181,18 @@ public abstract class ModifyPeerProcedure
 
   @Override
   protected PeerModificationState getInitialState() {
-    return PeerModificationState.UPDATE_PEER_STORAGE;
+    return PeerModificationState.PRE_PEER_MODIFICATION;
+  }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.serializeStateData(serializer);
+    serializer.serialize(ModifyPeerStateData.newBuilder().setPeerId(peerId).build());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.deserializeStateData(serializer);
+    peerId = serializer.deserialize(ModifyPeerStateData.class).getPeerId();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4567906c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerCallable.java
deleted file mode 100644
index 4e09107..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerCallable.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.master.replication;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.executor.EventType;
-import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.yetus.audience.InterfaceAudience;
-
-import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshPeerParameter;
-
-/**
- * The callable executed at RS side to refresh the peer config/state.
- * <p>
- * TODO: only a dummy implementation for verifying the framework, will add implementation later.
- */
-@InterfaceAudience.Private
-public class RefreshPeerCallable implements RSProcedureCallable {
-
-  private HRegionServer rs;
-
-  private String peerId;
-
-  private Exception initError;
-
-  @Override
-  public Void call() throws Exception {
-    if (initError != null) {
-      throw initError;
-    }
-    rs.getFileSystem().create(new Path("/" + peerId + "/" + rs.getServerName().toString())).close();
-    return null;
-  }
-
-  @Override
-  public void init(byte[] parameter, HRegionServer rs) {
-    this.rs = rs;
-    try {
-      this.peerId = RefreshPeerParameter.parseFrom(parameter).getPeerId();
-    } catch (InvalidProtocolBufferException e) {
-      initError = e;
-      return;
-    }
-  }
-
-  @Override
-  public EventType getEventType() {
-    return EventType.RS_REFRESH_PEER;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/4567906c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
index 18da487..ddc2401 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
@@ -32,6 +32,8 @@ import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteOperation;
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
+import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
+import org.apache.hadoop.hbase.replication.regionserver.RefreshPeerCallable;
 import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -118,15 +120,22 @@ public class RefreshPeerProcedure extends Procedure<MasterProcedureEnv>
             .setTargetServer(ProtobufUtil.toServerName(remote)).build().toByteArray());
   }
 
-  private void complete(MasterProcedureEnv env, boolean succ) {
+  private void complete(MasterProcedureEnv env, Throwable error) {
     if (event == null) {
       LOG.warn("procedure event for " + getProcId() +
-          " is null, maybe the procedure is created when recovery", new Exception());
+          " is null, maybe the procedure is created when recovery",
+        new Exception());
       return;
     }
-    LOG.info("Refresh peer " + peerId + " for " + type + " on " + targetServer +
-        (succ ? " suceeded" : " failed"));
-    this.succ = succ;
+    if (error != null) {
+      LOG.warn("Refresh peer " + peerId + " for " + type + " on " + targetServer + " failed",
+        error);
+      this.succ = false;
+    } else {
+      LOG.info("Refresh peer " + peerId + " for " + type + " on " + targetServer + " suceeded");
+      this.succ = true;
+    }
+
     event.wake(env.getProcedureScheduler());
     event = null;
   }
@@ -134,17 +143,18 @@ public class RefreshPeerProcedure extends Procedure<MasterProcedureEnv>
   @Override
   public synchronized void remoteCallFailed(MasterProcedureEnv env, ServerName remote,
       IOException exception) {
-    complete(env, false);
+    complete(env, exception);
   }
 
   @Override
   public synchronized void remoteOperationCompleted(MasterProcedureEnv env) {
-    complete(env, true);
+    complete(env, null);
   }
 
   @Override
-  public synchronized void remoteOperationFailed(MasterProcedureEnv env, String error) {
-    complete(env, false);
+  public synchronized void remoteOperationFailed(MasterProcedureEnv env,
+      RemoteProcedureException error) {
+    complete(env, error);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/4567906c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
new file mode 100644
index 0000000..3daad6d
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
@@ -0,0 +1,69 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.replication;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * The procedure for removing a replication peer.
+ */
+@InterfaceAudience.Private
+public class RemovePeerProcedure extends ModifyPeerProcedure {
+
+  private static final Log LOG = LogFactory.getLog(RemovePeerProcedure.class);
+
+  public RemovePeerProcedure() {
+  }
+
+  public RemovePeerProcedure(String peerId) {
+    super(peerId);
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.REMOVE;
+  }
+
+  @Override
+  protected void prePeerModification(MasterProcedureEnv env) throws IOException {
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.preRemoveReplicationPeer(peerId);
+    }
+  }
+
+  @Override
+  protected void updatePeerStorage(MasterProcedureEnv env) throws Exception {
+    env.getReplicationManager().removeReplicationPeer(peerId);
+  }
+
+  @Override
+  protected void postPeerModification(MasterProcedureEnv env) throws IOException {
+    LOG.info("Successfully removed peer " + peerId);
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.postRemoveReplicationPeer(peerId);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/4567906c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
index f36b2e2..b6f8784 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
@@ -27,10 +27,8 @@ import java.util.regex.Pattern;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.hbase.replication.BaseReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
@@ -39,24 +37,21 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Manages and performs all replication admin operations.
+ * <p>
  * Used to add/remove a replication peer.
  */
 @InterfaceAudience.Private
 public class ReplicationManager {
-
-  private final Configuration conf;
-  private final ZKWatcher zkw;
   private final ReplicationQueuesClient replicationQueuesClient;
   private final ReplicationPeers replicationPeers;
 
   public ReplicationManager(Configuration conf, ZKWatcher zkw, Abortable abortable)
       throws IOException {
-    this.conf = conf;
-    this.zkw = zkw;
     try {
       this.replicationQueuesClient = ReplicationFactory
           .getReplicationQueuesClient(new ReplicationQueuesClientArguments(conf, abortable, zkw));
@@ -70,7 +65,7 @@ public class ReplicationManager {
   }
 
   public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
-      throws ReplicationException, IOException {
+      throws ReplicationException {
     checkPeerConfig(peerConfig);
     replicationPeers.registerPeer(peerId, peerConfig, enabled);
     replicationPeers.peerConnected(peerId);
@@ -89,8 +84,8 @@ public class ReplicationManager {
     this.replicationPeers.disablePeer(peerId);
   }
 
-  public ReplicationPeerConfig getPeerConfig(String peerId) throws ReplicationException,
-      ReplicationPeerNotFoundException {
+  public ReplicationPeerConfig getPeerConfig(String peerId)
+      throws ReplicationException, ReplicationPeerNotFoundException {
     ReplicationPeerConfig peerConfig = replicationPeers.getReplicationPeerConfig(peerId);
     if (peerConfig == null) {
       throw new ReplicationPeerNotFoundException(peerId);
@@ -110,9 +105,9 @@ public class ReplicationManager {
     List<String> peerIds = replicationPeers.getAllPeerIds();
     for (String peerId : peerIds) {
       if (pattern == null || (pattern != null && pattern.matcher(peerId).matches())) {
-        peers.add(new ReplicationPeerDescription(peerId, replicationPeers
-            .getStatusOfPeerFromBackingStore(peerId), replicationPeers
-            .getReplicationPeerConfig(peerId)));
+        peers.add(new ReplicationPeerDescription(peerId,
+            replicationPeers.getStatusOfPeerFromBackingStore(peerId),
+            replicationPeers.getReplicationPeerConfig(peerId)));
       }
     }
     return peers;
@@ -126,13 +121,12 @@ public class ReplicationManager {
    * If replicate_all flag is false, it means all user tables can't be replicated to peer cluster.
    * Then allow to config namespaces or table-cfs which will be replicated to peer cluster.
    */
-  private void checkPeerConfig(ReplicationPeerConfig peerConfig)
-      throws ReplicationException, IOException {
+  private void checkPeerConfig(ReplicationPeerConfig peerConfig) {
     if (peerConfig.replicateAllUserTables()) {
-      if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty())
-          || (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) {
-        throw new ReplicationException("Need clean namespaces or table-cfs config firstly"
-            + " when replicate_all flag is true");
+      if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty()) ||
+        (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) {
+        throw new IllegalArgumentException("Need clean namespaces or table-cfs config firstly " +
+          "when you want replicate all cluster");
       }
       checkNamespacesAndTableCfsConfigConflict(peerConfig.getExcludeNamespaces(),
         peerConfig.getExcludeTableCFsMap());
@@ -141,7 +135,7 @@ public class ReplicationManager {
           && !peerConfig.getExcludeNamespaces().isEmpty())
           || (peerConfig.getExcludeTableCFsMap() != null
               && !peerConfig.getExcludeTableCFsMap().isEmpty())) {
-        throw new ReplicationException(
+        throw new IllegalArgumentException(
             "Need clean exclude-namespaces or exclude-table-cfs config firstly"
                 + " when replicate_all flag is false");
       }
@@ -154,20 +148,24 @@ public class ReplicationManager {
   /**
    * Set a namespace in the peer config means that all tables in this namespace will be replicated
    * to the peer cluster.
-   * 1. If peer config already has a namespace, then not allow set any table of this namespace
-   *    to the peer config.
-   * 2. If peer config already has a table, then not allow set this table's namespace to the peer
-   *    config.
-   *
+   * <ol>
+   * <li>If peer config already has a namespace, then not allow set any table of this namespace to
+   * the peer config.</li>
+   * <li>If peer config already has a table, then not allow set this table's namespace to the peer
+   * config.</li>
+   * </ol>
+   * <p>
    * Set a exclude namespace in the peer config means that all tables in this namespace can't be
    * replicated to the peer cluster.
-   * 1. If peer config already has a exclude namespace, then not allow set any exclude table of
-   *    this namespace to the peer config.
-   * 2. If peer config already has a exclude table, then not allow set this table's namespace
-   *    as a exclude namespace.
+   * <ol>
+   * <li>If peer config already has a exclude namespace, then not allow set any exclude table of
+   * this namespace to the peer config.</li>
+   * <li>If peer config already has a exclude table, then not allow set this table's namespace as a
+   * exclude namespace.</li>
+   * </ol>
    */
   private void checkNamespacesAndTableCfsConfigConflict(Set<String> namespaces,
-      Map<TableName, ? extends Collection<String>> tableCfs) throws ReplicationException {
+      Map<TableName, ? extends Collection<String>> tableCfs) {
     if (namespaces == null || namespaces.isEmpty()) {
       return;
     }
@@ -177,24 +175,22 @@ public class ReplicationManager {
     for (Map.Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
       TableName table = entry.getKey();
       if (namespaces.contains(table.getNamespaceAsString())) {
-        throw new ReplicationException("Table-cfs " + table + " is conflict with namespaces "
+        throw new IllegalArgumentException("Table-cfs " + table + " is conflict with namespaces "
             + table.getNamespaceAsString() + " in peer config");
       }
     }
   }
 
-  private void checkConfiguredWALEntryFilters(ReplicationPeerConfig peerConfig)
-      throws IOException {
-    String filterCSV = peerConfig.getConfiguration().
-        get(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY);
-    if (filterCSV != null && !filterCSV.isEmpty()){
-      String [] filters = filterCSV.split(",");
+  private void checkConfiguredWALEntryFilters(ReplicationPeerConfig peerConfig) {
+    String filterCSV = peerConfig.getConfiguration()
+        .get(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY);
+    if (filterCSV != null && !filterCSV.isEmpty()) {
+      String[] filters = filterCSV.split(",");
       for (String filter : filters) {
         try {
-          Class clazz = Class.forName(filter);
-          Object o = clazz.newInstance();
+          Class.forName(filter).newInstance();
         } catch (Exception e) {
-          throw new DoNotRetryIOException("Configured WALEntryFilter " + filter +
+          throw new IllegalArgumentException("Configured WALEntryFilter " + filter +
               " could not be created. Failing add/update " + "peer operation.", e);
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4567906c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
new file mode 100644
index 0000000..435eefc
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
@@ -0,0 +1,92 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.replication;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.UpdatePeerConfigStateData;
+
+/**
+ * The procedure for updating the config for a replication peer.
+ */
+@InterfaceAudience.Private
+public class UpdatePeerConfigProcedure extends ModifyPeerProcedure {
+
+  private static final Log LOG = LogFactory.getLog(UpdatePeerConfigProcedure.class);
+
+  private ReplicationPeerConfig peerConfig;
+
+  public UpdatePeerConfigProcedure() {
+  }
+
+  public UpdatePeerConfigProcedure(String peerId, ReplicationPeerConfig peerConfig) {
+    super(peerId);
+    this.peerConfig = peerConfig;
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.UPDATE_CONFIG;
+  }
+
+  @Override
+  protected void prePeerModification(MasterProcedureEnv env) throws IOException {
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.preUpdateReplicationPeerConfig(peerId, peerConfig);
+    }
+  }
+
+  @Override
+  protected void updatePeerStorage(MasterProcedureEnv env)
+      throws IllegalArgumentException, Exception {
+    env.getReplicationManager().updatePeerConfig(peerId, peerConfig);
+  }
+
+  @Override
+  protected void postPeerModification(MasterProcedureEnv env) throws IOException {
+    LOG.info("Successfully updated peer config of " + peerId + " to " + peerConfig);
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.postUpdateReplicationPeerConfig(peerId, peerConfig);
+    }
+  }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.serializeStateData(serializer);
+    serializer.serialize(UpdatePeerConfigStateData.newBuilder()
+        .setPeerConfig(ReplicationPeerConfigUtil.convert(peerConfig)).build());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.deserializeStateData(serializer);
+    peerConfig = ReplicationPeerConfigUtil
+        .convert(serializer.deserialize(UpdatePeerConfigStateData.class).getPeerConfig());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/4567906c/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 0799ca6..49489d8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -1,4 +1,4 @@
-/*
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -148,6 +148,7 @@ import org.apache.hadoop.hbase.util.CompressionTest;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
 import org.apache.hadoop.hbase.util.HasThread;
 import org.apache.hadoop.hbase.util.JvmPauseMonitor;
 import org.apache.hadoop.hbase.util.NettyEventLoopGroupConfig;
@@ -178,7 +179,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
@@ -3742,7 +3742,7 @@ public class HRegionServer extends HasThread implements
       ReportProcedureDoneRequest.newBuilder().setProcId(procId);
     if (error != null) {
       builder.setStatus(ReportProcedureDoneRequest.Status.ERROR)
-          .setError(Throwables.getStackTraceAsString(error));
+          .setError(ForeignExceptionUtil.toProtoForeignException(serverName.toString(), error));
     } else {
       builder.setStatus(ReportProcedureDoneRequest.Status.SUCCESS);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/4567906c/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
new file mode 100644
index 0000000..a47a483
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RefreshPeerCallable.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication.regionserver;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RefreshPeerParameter;
+
+/**
+ * The callable executed at RS side to refresh the peer config/state.
+ * <p>
+ * TODO: only a dummy implementation for verifying the framework, will add implementation later.
+ */
+@InterfaceAudience.Private
+public class RefreshPeerCallable implements RSProcedureCallable {
+
+  private HRegionServer rs;
+
+  private String peerId;
+
+  private Exception initError;
+
+  @Override
+  public Void call() throws Exception {
+    if (initError != null) {
+      throw initError;
+    }
+    Path dir = new Path("/" + peerId);
+    if (rs.getFileSystem().exists(dir)) {
+      rs.getFileSystem().create(new Path(dir, rs.getServerName().toString())).close();
+    }
+    return null;
+  }
+
+  @Override
+  public void init(byte[] parameter, HRegionServer rs) {
+    this.rs = rs;
+    try {
+      this.peerId = RefreshPeerParameter.parseFrom(parameter).getPeerId();
+    } catch (InvalidProtocolBufferException e) {
+      initError = e;
+      return;
+    }
+  }
+
+  @Override
+  public EventType getEventType() {
+    return EventType.RS_REFRESH_PEER;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/4567906c/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
index 413abe3..540a67c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.locking.LockManager;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.replication.ReplicationManager;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
 import org.apache.hadoop.hbase.procedure2.LockedResource;
@@ -369,7 +370,6 @@ public class MockNoopMasterServices implements MasterServices, Server {
 
   @Override
   public ClusterConnection getClusterConnection() {
-    // TODO Auto-generated method stub
     return null;
   }
 
@@ -399,20 +399,24 @@ public class MockNoopMasterServices implements MasterServices, Server {
   }
 
   @Override
-  public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
+  public long addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
       throws ReplicationException {
+    return 0;
   }
 
   @Override
-  public void removeReplicationPeer(String peerId) throws ReplicationException {
+  public long removeReplicationPeer(String peerId) throws ReplicationException {
+    return 0;
   }
 
   @Override
-  public void enableReplicationPeer(String peerId) throws ReplicationException, IOException {
+  public long enableReplicationPeer(String peerId) throws ReplicationException, IOException {
+    return 0;
   }
 
   @Override
-  public void disableReplicationPeer(String peerId) throws ReplicationException, IOException {
+  public long disableReplicationPeer(String peerId) throws ReplicationException, IOException {
+    return 0;
   }
 
   @Override
@@ -422,8 +426,9 @@ public class MockNoopMasterServices implements MasterServices, Server {
   }
 
   @Override
-  public void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
+  public long updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
       throws ReplicationException, IOException {
+    return 0;
   }
 
   @Override
@@ -458,7 +463,6 @@ public class MockNoopMasterServices implements MasterServices, Server {
 
   @Override
   public ProcedureEvent getInitializedEvent() {
-    // TODO Auto-generated method stub
     return null;
   }
 
@@ -471,4 +475,9 @@ public class MockNoopMasterServices implements MasterServices, Server {
   public Connection createConnection(Configuration conf) throws IOException {
     return null;
   }
+
+  @Override
+  public ReplicationManager getReplicationManager() {
+    return null;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/4567906c/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
index 44343d7..ed7c6fa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/replication/DummyModifyPeerProcedure.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hbase.master.replication;
 
-import java.io.IOException;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 
 public class DummyModifyPeerProcedure extends ModifyPeerProcedure {
 
@@ -34,8 +34,15 @@ public class DummyModifyPeerProcedure extends ModifyPeerProcedure {
   }
 
   @Override
-  protected boolean updatePeerStorage() throws IOException {
-    return true;
+  protected void prePeerModification(MasterProcedureEnv env) {
+  }
+
+  @Override
+  protected void updatePeerStorage(MasterProcedureEnv env) {
+  }
+
+  @Override
+  protected void postPeerModification(MasterProcedureEnv env) {
   }
 
 }


[17/26] hbase git commit: HBASE-19543 Abstract a replication storage interface to extract the zk specific code

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
deleted file mode 100644
index b6f8784..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java
+++ /dev/null
@@ -1,199 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.master.replication;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.regex.Pattern;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.replication.BaseReplicationEndpoint;
-import org.apache.hadoop.hbase.replication.ReplicationException;
-import org.apache.hadoop.hbase.replication.ReplicationFactory;
-import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
-import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
-import org.apache.hadoop.hbase.replication.ReplicationPeers;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * Manages and performs all replication admin operations.
- * <p>
- * Used to add/remove a replication peer.
- */
-@InterfaceAudience.Private
-public class ReplicationManager {
-  private final ReplicationQueuesClient replicationQueuesClient;
-  private final ReplicationPeers replicationPeers;
-
-  public ReplicationManager(Configuration conf, ZKWatcher zkw, Abortable abortable)
-      throws IOException {
-    try {
-      this.replicationQueuesClient = ReplicationFactory
-          .getReplicationQueuesClient(new ReplicationQueuesClientArguments(conf, abortable, zkw));
-      this.replicationQueuesClient.init();
-      this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf,
-        this.replicationQueuesClient, abortable);
-      this.replicationPeers.init();
-    } catch (Exception e) {
-      throw new IOException("Failed to construct ReplicationManager", e);
-    }
-  }
-
-  public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
-      throws ReplicationException {
-    checkPeerConfig(peerConfig);
-    replicationPeers.registerPeer(peerId, peerConfig, enabled);
-    replicationPeers.peerConnected(peerId);
-  }
-
-  public void removeReplicationPeer(String peerId) throws ReplicationException {
-    replicationPeers.peerDisconnected(peerId);
-    replicationPeers.unregisterPeer(peerId);
-  }
-
-  public void enableReplicationPeer(String peerId) throws ReplicationException {
-    this.replicationPeers.enablePeer(peerId);
-  }
-
-  public void disableReplicationPeer(String peerId) throws ReplicationException {
-    this.replicationPeers.disablePeer(peerId);
-  }
-
-  public ReplicationPeerConfig getPeerConfig(String peerId)
-      throws ReplicationException, ReplicationPeerNotFoundException {
-    ReplicationPeerConfig peerConfig = replicationPeers.getReplicationPeerConfig(peerId);
-    if (peerConfig == null) {
-      throw new ReplicationPeerNotFoundException(peerId);
-    }
-    return peerConfig;
-  }
-
-  public void updatePeerConfig(String peerId, ReplicationPeerConfig peerConfig)
-      throws ReplicationException, IOException {
-    checkPeerConfig(peerConfig);
-    this.replicationPeers.updatePeerConfig(peerId, peerConfig);
-  }
-
-  public List<ReplicationPeerDescription> listReplicationPeers(Pattern pattern)
-      throws ReplicationException {
-    List<ReplicationPeerDescription> peers = new ArrayList<>();
-    List<String> peerIds = replicationPeers.getAllPeerIds();
-    for (String peerId : peerIds) {
-      if (pattern == null || (pattern != null && pattern.matcher(peerId).matches())) {
-        peers.add(new ReplicationPeerDescription(peerId,
-            replicationPeers.getStatusOfPeerFromBackingStore(peerId),
-            replicationPeers.getReplicationPeerConfig(peerId)));
-      }
-    }
-    return peers;
-  }
-
-  /**
-   * If replicate_all flag is true, it means all user tables will be replicated to peer cluster.
-   * Then allow config exclude namespaces or exclude table-cfs which can't be replicated to
-   * peer cluster.
-   *
-   * If replicate_all flag is false, it means all user tables can't be replicated to peer cluster.
-   * Then allow to config namespaces or table-cfs which will be replicated to peer cluster.
-   */
-  private void checkPeerConfig(ReplicationPeerConfig peerConfig) {
-    if (peerConfig.replicateAllUserTables()) {
-      if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty()) ||
-        (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) {
-        throw new IllegalArgumentException("Need clean namespaces or table-cfs config firstly " +
-          "when you want replicate all cluster");
-      }
-      checkNamespacesAndTableCfsConfigConflict(peerConfig.getExcludeNamespaces(),
-        peerConfig.getExcludeTableCFsMap());
-    } else {
-      if ((peerConfig.getExcludeNamespaces() != null
-          && !peerConfig.getExcludeNamespaces().isEmpty())
-          || (peerConfig.getExcludeTableCFsMap() != null
-              && !peerConfig.getExcludeTableCFsMap().isEmpty())) {
-        throw new IllegalArgumentException(
-            "Need clean exclude-namespaces or exclude-table-cfs config firstly"
-                + " when replicate_all flag is false");
-      }
-      checkNamespacesAndTableCfsConfigConflict(peerConfig.getNamespaces(),
-        peerConfig.getTableCFsMap());
-    }
-    checkConfiguredWALEntryFilters(peerConfig);
-  }
-
-  /**
-   * Set a namespace in the peer config means that all tables in this namespace will be replicated
-   * to the peer cluster.
-   * <ol>
-   * <li>If peer config already has a namespace, then not allow set any table of this namespace to
-   * the peer config.</li>
-   * <li>If peer config already has a table, then not allow set this table's namespace to the peer
-   * config.</li>
-   * </ol>
-   * <p>
-   * Set a exclude namespace in the peer config means that all tables in this namespace can't be
-   * replicated to the peer cluster.
-   * <ol>
-   * <li>If peer config already has a exclude namespace, then not allow set any exclude table of
-   * this namespace to the peer config.</li>
-   * <li>If peer config already has a exclude table, then not allow set this table's namespace as a
-   * exclude namespace.</li>
-   * </ol>
-   */
-  private void checkNamespacesAndTableCfsConfigConflict(Set<String> namespaces,
-      Map<TableName, ? extends Collection<String>> tableCfs) {
-    if (namespaces == null || namespaces.isEmpty()) {
-      return;
-    }
-    if (tableCfs == null || tableCfs.isEmpty()) {
-      return;
-    }
-    for (Map.Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
-      TableName table = entry.getKey();
-      if (namespaces.contains(table.getNamespaceAsString())) {
-        throw new IllegalArgumentException("Table-cfs " + table + " is conflict with namespaces "
-            + table.getNamespaceAsString() + " in peer config");
-      }
-    }
-  }
-
-  private void checkConfiguredWALEntryFilters(ReplicationPeerConfig peerConfig) {
-    String filterCSV = peerConfig.getConfiguration()
-        .get(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY);
-    if (filterCSV != null && !filterCSV.isEmpty()) {
-      String[] filters = filterCSV.split(",");
-      for (String filter : filters) {
-        try {
-          Class.forName(filter).newInstance();
-        } catch (Exception e) {
-          throw new IllegalArgumentException("Configured WALEntryFilter " + filter +
-              " could not be created. Failing add/update " + "peer operation.", e);
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
new file mode 100644
index 0000000..5abd874
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
@@ -0,0 +1,331 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.replication;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.replication.BaseReplicationEndpoint;
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
+import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Manages and performs all replication admin operations.
+ * <p>
+ * Used to add/remove a replication peer.
+ */
+@InterfaceAudience.Private
+public final class ReplicationPeerManager {
+
+  private final ReplicationPeerStorage peerStorage;
+
+  private final ReplicationQueueStorage queueStorage;
+
+  private final ConcurrentMap<String, ReplicationPeerDescription> peers;
+
+  private ReplicationPeerManager(ReplicationPeerStorage peerStorage,
+      ReplicationQueueStorage queueStorage,
+      ConcurrentMap<String, ReplicationPeerDescription> peers) {
+    this.peerStorage = peerStorage;
+    this.queueStorage = queueStorage;
+    this.peers = peers;
+  }
+
+  private void checkQueuesDeleted(String peerId)
+      throws ReplicationException, DoNotRetryIOException {
+    for (ServerName replicator : queueStorage.getListOfReplicators()) {
+      List<String> queueIds = queueStorage.getAllQueues(replicator);
+      for (String queueId : queueIds) {
+        ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
+        if (queueInfo.getPeerId().equals(peerId)) {
+          throw new DoNotRetryIOException("undeleted queue for peerId: " + peerId +
+            ", replicator: " + replicator + ", queueId: " + queueId);
+        }
+      }
+    }
+    if (queueStorage.getAllPeersFromHFileRefsQueue().contains(peerId)) {
+      throw new DoNotRetryIOException("Undeleted queue for peer " + peerId + " in hfile-refs");
+    }
+  }
+
+  public void preAddPeer(String peerId, ReplicationPeerConfig peerConfig)
+      throws DoNotRetryIOException, ReplicationException {
+    if (peerId.contains("-")) {
+      throw new DoNotRetryIOException("Found invalid peer name: " + peerId);
+    }
+    checkPeerConfig(peerConfig);
+    if (peers.containsKey(peerId)) {
+      throw new DoNotRetryIOException("Replication peer " + peerId + " already exists");
+    }
+    // make sure that there is no queues with the same peer id. This may happen when we create a
+    // peer with the same id with a old deleted peer. If the replication queues for the old peer
+    // have not been cleaned up yet then we should not create the new peer, otherwise the old wal
+    // file may also be replicated.
+    checkQueuesDeleted(peerId);
+  }
+
+  private ReplicationPeerDescription checkPeerExists(String peerId) throws DoNotRetryIOException {
+    ReplicationPeerDescription desc = peers.get(peerId);
+    if (desc == null) {
+      throw new DoNotRetryIOException("Replication peer " + peerId + " does not exist");
+    }
+    return desc;
+  }
+
+  public void preRemovePeer(String peerId) throws DoNotRetryIOException {
+    checkPeerExists(peerId);
+  }
+
+  public void preEnablePeer(String peerId) throws DoNotRetryIOException {
+    ReplicationPeerDescription desc = checkPeerExists(peerId);
+    if (desc.isEnabled()) {
+      throw new DoNotRetryIOException("Replication peer " + peerId + " has already been enabled");
+    }
+  }
+
+  public void preDisablePeer(String peerId) throws DoNotRetryIOException {
+    ReplicationPeerDescription desc = checkPeerExists(peerId);
+    if (!desc.isEnabled()) {
+      throw new DoNotRetryIOException("Replication peer " + peerId + " has already been disabled");
+    }
+  }
+
+  public void preUpdatePeerConfig(String peerId, ReplicationPeerConfig peerConfig)
+      throws DoNotRetryIOException {
+    checkPeerConfig(peerConfig);
+    ReplicationPeerDescription desc = checkPeerExists(peerId);
+    ReplicationPeerConfig oldPeerConfig = desc.getPeerConfig();
+    if (!StringUtils.isBlank(peerConfig.getClusterKey()) &&
+      !peerConfig.getClusterKey().equals(oldPeerConfig.getClusterKey())) {
+      throw new DoNotRetryIOException(
+          "Changing the cluster key on an existing peer is not allowed. Existing key '" +
+            oldPeerConfig.getClusterKey() + "' for peer " + peerId + " does not match new key '" +
+            peerConfig.getClusterKey() + "'");
+    }
+
+    if (!StringUtils.isBlank(peerConfig.getReplicationEndpointImpl()) &&
+      !peerConfig.getReplicationEndpointImpl().equals(oldPeerConfig.getReplicationEndpointImpl())) {
+      throw new DoNotRetryIOException("Changing the replication endpoint implementation class " +
+        "on an existing peer is not allowed. Existing class '" +
+        oldPeerConfig.getReplicationEndpointImpl() + "' for peer " + peerId +
+        " does not match new class '" + peerConfig.getReplicationEndpointImpl() + "'");
+    }
+  }
+
+  private ReplicationPeerConfig copy(ReplicationPeerConfig peerConfig) {
+    ReplicationPeerConfig copiedPeerConfig = new ReplicationPeerConfig();
+    copiedPeerConfig.getConfiguration().putAll(peerConfig.getConfiguration());
+    copiedPeerConfig.getPeerData().putAll(peerConfig.getPeerData());
+    copiedPeerConfig.setTableCFsMap(peerConfig.getTableCFsMap());
+    copiedPeerConfig.setNamespaces(peerConfig.getNamespaces());
+    copiedPeerConfig.setExcludeTableCFsMap(peerConfig.getExcludeTableCFsMap());
+    copiedPeerConfig.setExcludeNamespaces(peerConfig.getExcludeNamespaces());
+    copiedPeerConfig.setBandwidth(peerConfig.getBandwidth());
+    copiedPeerConfig.setReplicateAllUserTables(peerConfig.replicateAllUserTables());
+    copiedPeerConfig.setClusterKey(peerConfig.getClusterKey());
+    copiedPeerConfig.setReplicationEndpointImpl(peerConfig.getReplicationEndpointImpl());
+    return copiedPeerConfig;
+  }
+
+  public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
+      throws ReplicationException {
+    if (peers.containsKey(peerId)) {
+      // this should be a retry, just return
+      return;
+    }
+    ReplicationPeerConfig copiedPeerConfig = copy(peerConfig);
+    peerStorage.addPeer(peerId, copiedPeerConfig, enabled);
+    peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, copiedPeerConfig));
+  }
+
+  public void removePeer(String peerId) throws ReplicationException {
+    if (!peers.containsKey(peerId)) {
+      // this should be a retry, just return
+      return;
+    }
+    peerStorage.removePeer(peerId);
+    peers.remove(peerId);
+  }
+
+  private void setPeerState(String peerId, boolean enabled) throws ReplicationException {
+    ReplicationPeerDescription desc = peers.get(peerId);
+    if (desc.isEnabled() == enabled) {
+      // this should be a retry, just return
+      return;
+    }
+    peerStorage.setPeerState(peerId, enabled);
+    peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, desc.getPeerConfig()));
+  }
+
+  public void enablePeer(String peerId) throws ReplicationException {
+    setPeerState(peerId, true);
+  }
+
+  public void disablePeer(String peerId) throws ReplicationException {
+    setPeerState(peerId, false);
+  }
+
+  public void updatePeerConfig(String peerId, ReplicationPeerConfig peerConfig)
+      throws ReplicationException {
+    // the checking rules are too complicated here so we give up checking whether this is a retry.
+    ReplicationPeerDescription desc = peers.get(peerId);
+    ReplicationPeerConfig oldPeerConfig = desc.getPeerConfig();
+    ReplicationPeerConfig newPeerConfig = copy(peerConfig);
+    // we need to use the new conf to overwrite the old one.
+    newPeerConfig.getConfiguration().putAll(oldPeerConfig.getConfiguration());
+    newPeerConfig.getConfiguration().putAll(peerConfig.getConfiguration());
+    newPeerConfig.getPeerData().putAll(oldPeerConfig.getPeerData());
+    newPeerConfig.getPeerData().putAll(peerConfig.getPeerData());
+
+    peerStorage.updatePeerConfig(peerId, newPeerConfig);
+    peers.put(peerId, new ReplicationPeerDescription(peerId, desc.isEnabled(), newPeerConfig));
+  }
+
+  public List<ReplicationPeerDescription> listPeers(Pattern pattern) {
+    if (pattern == null) {
+      return new ArrayList<>(peers.values());
+    }
+    return peers.values().stream().filter(r -> pattern.matcher(r.getPeerId()).matches())
+        .collect(Collectors.toList());
+  }
+
+  public Optional<ReplicationPeerConfig> getPeerConfig(String peerId) {
+    ReplicationPeerDescription desc = peers.get(peerId);
+    return desc != null ? Optional.of(desc.getPeerConfig()) : Optional.empty();
+  }
+
+  /**
+   * If replicate_all flag is true, it means all user tables will be replicated to peer cluster.
+   * Then allow config exclude namespaces or exclude table-cfs which can't be replicated to peer
+   * cluster.
+   * <p>
+   * If replicate_all flag is false, it means all user tables can't be replicated to peer cluster.
+   * Then allow to config namespaces or table-cfs which will be replicated to peer cluster.
+   */
+  private static void checkPeerConfig(ReplicationPeerConfig peerConfig)
+      throws DoNotRetryIOException {
+    if (peerConfig.replicateAllUserTables()) {
+      if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty()) ||
+        (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) {
+        throw new DoNotRetryIOException("Need clean namespaces or table-cfs config firstly " +
+          "when you want replicate all cluster");
+      }
+      checkNamespacesAndTableCfsConfigConflict(peerConfig.getExcludeNamespaces(),
+        peerConfig.getExcludeTableCFsMap());
+    } else {
+      if ((peerConfig.getExcludeNamespaces() != null &&
+        !peerConfig.getExcludeNamespaces().isEmpty()) ||
+        (peerConfig.getExcludeTableCFsMap() != null &&
+          !peerConfig.getExcludeTableCFsMap().isEmpty())) {
+        throw new DoNotRetryIOException(
+            "Need clean exclude-namespaces or exclude-table-cfs config firstly" +
+              " when replicate_all flag is false");
+      }
+      checkNamespacesAndTableCfsConfigConflict(peerConfig.getNamespaces(),
+        peerConfig.getTableCFsMap());
+    }
+    checkConfiguredWALEntryFilters(peerConfig);
+  }
+
+  /**
+   * Set a namespace in the peer config means that all tables in this namespace will be replicated
+   * to the peer cluster.
+   * <ol>
+   * <li>If peer config already has a namespace, then not allow set any table of this namespace to
+   * the peer config.</li>
+   * <li>If peer config already has a table, then not allow set this table's namespace to the peer
+   * config.</li>
+   * </ol>
+   * <p>
+   * Set a exclude namespace in the peer config means that all tables in this namespace can't be
+   * replicated to the peer cluster.
+   * <ol>
+   * <li>If peer config already has a exclude namespace, then not allow set any exclude table of
+   * this namespace to the peer config.</li>
+   * <li>If peer config already has a exclude table, then not allow set this table's namespace as a
+   * exclude namespace.</li>
+   * </ol>
+   */
+  private static void checkNamespacesAndTableCfsConfigConflict(Set<String> namespaces,
+      Map<TableName, ? extends Collection<String>> tableCfs) throws DoNotRetryIOException {
+    if (namespaces == null || namespaces.isEmpty()) {
+      return;
+    }
+    if (tableCfs == null || tableCfs.isEmpty()) {
+      return;
+    }
+    for (Map.Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
+      TableName table = entry.getKey();
+      if (namespaces.contains(table.getNamespaceAsString())) {
+        throw new DoNotRetryIOException("Table-cfs " + table + " is conflict with namespaces " +
+          table.getNamespaceAsString() + " in peer config");
+      }
+    }
+  }
+
+  private static void checkConfiguredWALEntryFilters(ReplicationPeerConfig peerConfig)
+      throws DoNotRetryIOException {
+    String filterCSV = peerConfig.getConfiguration()
+        .get(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY);
+    if (filterCSV != null && !filterCSV.isEmpty()) {
+      String[] filters = filterCSV.split(",");
+      for (String filter : filters) {
+        try {
+          Class.forName(filter).newInstance();
+        } catch (Exception e) {
+          throw new DoNotRetryIOException("Configured WALEntryFilter " + filter +
+            " could not be created. Failing add/update " + "peer operation.", e);
+        }
+      }
+    }
+  }
+
+  public static ReplicationPeerManager create(ZKWatcher zk, Configuration conf)
+      throws ReplicationException {
+    ReplicationPeerStorage peerStorage =
+      ReplicationStorageFactory.getReplicationPeerStorage(zk, conf);
+    ConcurrentMap<String, ReplicationPeerDescription> peers = new ConcurrentHashMap<>();
+    for (String peerId : peerStorage.listPeerIds()) {
+      Optional<ReplicationPeerConfig> peerConfig = peerStorage.getPeerConfig(peerId);
+      boolean enabled = peerStorage.isPeerEnabled(peerId);
+      peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, peerConfig.get()));
+    }
+    return new ReplicationPeerManager(peerStorage,
+        ReplicationStorageFactory.getReplicationQueueStorage(zk, conf), peers);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
index d8154dc..a43532d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -59,12 +60,12 @@ public class UpdatePeerConfigProcedure extends ModifyPeerProcedure {
     if (cpHost != null) {
       cpHost.preUpdateReplicationPeerConfig(peerId, peerConfig);
     }
+    env.getReplicationPeerManager().preUpdatePeerConfig(peerId, peerConfig);
   }
 
   @Override
-  protected void updatePeerStorage(MasterProcedureEnv env)
-      throws IllegalArgumentException, Exception {
-    env.getReplicationManager().updatePeerConfig(peerId, peerConfig);
+  protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
+    env.getReplicationPeerManager().updatePeerConfig(peerId, peerConfig);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
index fb29e9e..c2fcd8c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
@@ -17,6 +17,12 @@
  */
 package org.apache.hadoop.hbase.client.replication;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -31,18 +37,17 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.replication.ReplicationException;
-import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfigBuilder;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -50,15 +55,6 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 /**
  * Unit testing of ReplicationAdmin
@@ -66,8 +62,6 @@ import static org.junit.Assert.fail;
 @Category({MediumTests.class, ClientTests.class})
 public class TestReplicationAdmin {
 
-  private static final Logger LOG =
-      LoggerFactory.getLogger(TestReplicationAdmin.class);
   private final static HBaseTestingUtility TEST_UTIL =
       new HBaseTestingUtility();
 
@@ -102,16 +96,17 @@ public class TestReplicationAdmin {
   }
 
   @After
-  public void cleanupPeer() {
-    try {
-      hbaseAdmin.removeReplicationPeer(ID_ONE);
-    } catch (Exception e) {
-      LOG.debug("Replication peer " + ID_ONE + " may already be removed");
+  public void tearDown() throws Exception {
+    for (ReplicationPeerDescription desc : hbaseAdmin.listReplicationPeers()) {
+      hbaseAdmin.removeReplicationPeer(desc.getPeerId());
     }
-    try {
-      hbaseAdmin.removeReplicationPeer(ID_SECOND);
-    } catch (Exception e) {
-      LOG.debug("Replication peer " + ID_SECOND + " may already be removed");
+    ReplicationQueueStorage queueStorage = ReplicationStorageFactory
+        .getReplicationQueueStorage(TEST_UTIL.getZooKeeperWatcher(), TEST_UTIL.getConfiguration());
+    for (ServerName serverName : queueStorage.getListOfReplicators()) {
+      for (String queue : queueStorage.getAllQueues(serverName)) {
+        queueStorage.removeQueue(serverName, queue);
+      }
+      queueStorage.removeReplicatorIfQueueIsEmpty(serverName);
     }
   }
 
@@ -201,32 +196,29 @@ public class TestReplicationAdmin {
     ReplicationPeerConfig rpc2 = new ReplicationPeerConfig();
     rpc2.setClusterKey(KEY_SECOND);
     Configuration conf = TEST_UTIL.getConfiguration();
-    ZKWatcher zkw = new ZKWatcher(conf, "Test HBaseAdmin", null);
-    ReplicationQueues repQueues =
-        ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, null, zkw));
-    repQueues.init("server1");
+    ReplicationQueueStorage queueStorage =
+      ReplicationStorageFactory.getReplicationQueueStorage(TEST_UTIL.getZooKeeperWatcher(), conf);
 
+    ServerName serverName = ServerName.valueOf("server1", 8000, 1234);
     // add queue for ID_ONE
-    repQueues.addLog(ID_ONE, "file1");
+    queueStorage.addWAL(serverName, ID_ONE, "file1");
     try {
       admin.addPeer(ID_ONE, rpc1, null);
       fail();
     } catch (Exception e) {
       // OK!
     }
-    repQueues.removeQueue(ID_ONE);
-    assertEquals(0, repQueues.getAllQueues().size());
+    queueStorage.removeQueue(serverName, ID_ONE);
+    assertEquals(0, queueStorage.getAllQueues(serverName).size());
 
     // add recovered queue for ID_ONE
-    repQueues.addLog(ID_ONE + "-server2", "file1");
+    queueStorage.addWAL(serverName, ID_ONE + "-server2", "file1");
     try {
       admin.addPeer(ID_ONE, rpc2, null);
       fail();
     } catch (Exception e) {
       // OK!
     }
-    repQueues.removeAllQueues();
-    zkw.close();
   }
 
   /**
@@ -422,7 +414,7 @@ public class TestReplicationAdmin {
       tableCFs.clear();
       tableCFs.put(tableName2, null);
       admin.removePeerTableCFs(ID_ONE, tableCFs);
-      assertTrue(false);
+      fail();
     } catch (ReplicationException e) {
     }
     tableCFs.clear();

http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
index 540a67c..7196b7c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java
@@ -19,6 +19,8 @@ package org.apache.hadoop.hbase.master;
 
 import static org.mockito.Mockito.mock;
 
+import com.google.protobuf.Service;
+
 import java.io.IOException;
 import java.util.List;
 
@@ -42,7 +44,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.locking.LockManager;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.master.replication.ReplicationManager;
+import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
 import org.apache.hadoop.hbase.procedure2.LockedResource;
@@ -56,8 +58,6 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 
-import com.google.protobuf.Service;
-
 public class MockNoopMasterServices implements MasterServices, Server {
   private final Configuration conf;
   private final MetricsMaster metricsMaster;
@@ -462,7 +462,7 @@ public class MockNoopMasterServices implements MasterServices, Server {
   }
 
   @Override
-  public ProcedureEvent getInitializedEvent() {
+  public ProcedureEvent<?> getInitializedEvent() {
     return null;
   }
 
@@ -477,7 +477,7 @@ public class MockNoopMasterServices implements MasterServices, Server {
   }
 
   @Override
-  public ReplicationManager getReplicationManager() {
+  public ReplicationPeerManager getReplicationPeerManager() {
     return null;
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
index 6119e43..eaace03 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
@@ -44,6 +44,8 @@ import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
+import org.apache.hadoop.hbase.replication.ReplicationException;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -270,7 +272,7 @@ public class TestMasterNoCluster {
 
       @Override
       void initializeZKBasedSystemTrackers() throws IOException, InterruptedException,
-          KeeperException, CoordinatedStateException {
+          KeeperException, CoordinatedStateException, ReplicationException {
         super.initializeZKBasedSystemTrackers();
         // Record a newer server in server manager at first
         getServerManager().recordNewServerWithLock(newServer,

http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java
index 1675496..24bb4d0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDisableInactivePeer.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -47,9 +46,6 @@ public class TestReplicationDisableInactivePeer extends TestReplicationBase {
    */
   @Test(timeout = 600000)
   public void testDisableInactivePeer() throws Exception {
-
-    // enabling and shutdown the peer
-    admin.enablePeer("2");
     utility2.shutdownMiniHBaseCluster();
 
     byte[] rowkey = Bytes.toBytes("disable inactive peer");


[22/26] hbase git commit: HBASE-19520 Add UTs for the new lock type PEER

Posted by zh...@apache.org.
HBASE-19520 Add UTs for the new lock type PEER

Signed-off-by: zhangduo <zh...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/0d0b2098
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/0d0b2098
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/0d0b2098

Branch: refs/heads/HBASE-19397
Commit: 0d0b2098a489d0069e5f05b3381886d7046df620
Parents: 5224858
Author: Guanghao Zhang <zg...@apache.org>
Authored: Wed Dec 20 16:43:38 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Dec 28 08:51:59 2017 +0800

----------------------------------------------------------------------
 .../procedure/MasterProcedureScheduler.java     |   9 +-
 .../procedure/TestMasterProcedureScheduler.java |  65 ++++++++-
 ...TestMasterProcedureSchedulerConcurrency.java | 135 +++++++++++++++++++
 3 files changed, 201 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0d0b2098/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
index dd85f5c..5f4665c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
@@ -389,6 +389,13 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     while (tableIter.hasNext()) {
       count += tableIter.next().size();
     }
+
+    // Peer queues
+    final AvlTreeIterator<PeerQueue> peerIter = new AvlTreeIterator<>(peerMap);
+    while (peerIter.hasNext()) {
+      count += peerIter.next().size();
+    }
+
     return count;
   }
 
@@ -1041,7 +1048,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
    * @see #wakePeerExclusiveLock(Procedure, String)
    * @param procedure the procedure trying to acquire the lock
    * @param peerId peer to lock
-   * @return true if the procedure has to wait for the per to be available
+   * @return true if the procedure has to wait for the peer to be available
    */
   public boolean waitPeerExclusiveLock(Procedure<?> procedure, String peerId) {
     schedLock();

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d0b2098/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
index 0291165..fd77e1f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java
@@ -905,6 +905,27 @@ public class TestMasterProcedureScheduler {
     }
   }
 
+  public static class TestPeerProcedure extends TestProcedure implements PeerProcedureInterface {
+    private final String peerId;
+    private final PeerOperationType opType;
+
+    public TestPeerProcedure(long procId, String peerId, PeerOperationType opType) {
+      super(procId);
+      this.peerId = peerId;
+      this.opType = opType;
+    }
+
+    @Override
+    public String getPeerId() {
+      return peerId;
+    }
+
+    @Override
+    public PeerOperationType getPeerOperationType() {
+      return opType;
+    }
+  }
+
   private static LockProcedure createLockProcedure(LockType lockType, long procId) throws Exception {
     LockProcedure procedure = new LockProcedure();
 
@@ -927,22 +948,19 @@ public class TestMasterProcedureScheduler {
     return createLockProcedure(LockType.SHARED, procId);
   }
 
-  private static void assertLockResource(LockedResource resource,
-      LockedResourceType resourceType, String resourceName)
-  {
+  private static void assertLockResource(LockedResource resource, LockedResourceType resourceType,
+      String resourceName) {
     assertEquals(resourceType, resource.getResourceType());
     assertEquals(resourceName, resource.getResourceName());
   }
 
-  private static void assertExclusiveLock(LockedResource resource, Procedure<?> procedure)
-  {
+  private static void assertExclusiveLock(LockedResource resource, Procedure<?> procedure) {
     assertEquals(LockType.EXCLUSIVE, resource.getLockType());
     assertEquals(procedure, resource.getExclusiveLockOwnerProcedure());
     assertEquals(0, resource.getSharedLockCount());
   }
 
-  private static void assertSharedLock(LockedResource resource, int lockCount)
-  {
+  private static void assertSharedLock(LockedResource resource, int lockCount) {
     assertEquals(LockType.SHARED, resource.getLockType());
     assertEquals(lockCount, resource.getSharedLockCount());
   }
@@ -1027,6 +1045,39 @@ public class TestMasterProcedureScheduler {
   }
 
   @Test
+  public void testListLocksPeer() throws Exception {
+    String peerId = "1";
+    LockProcedure procedure = createExclusiveLockProcedure(4);
+    queue.waitPeerExclusiveLock(procedure, peerId);
+
+    List<LockedResource> locks = queue.getLocks();
+    assertEquals(1, locks.size());
+
+    LockedResource resource = locks.get(0);
+    assertLockResource(resource, LockedResourceType.PEER, peerId);
+    assertExclusiveLock(resource, procedure);
+    assertTrue(resource.getWaitingProcedures().isEmpty());
+
+    // Try to acquire the exclusive lock again with same procedure
+    assertFalse(queue.waitPeerExclusiveLock(procedure, peerId));
+
+    // Try to acquire the exclusive lock again with new procedure
+    LockProcedure procedure2 = createExclusiveLockProcedure(5);
+    assertTrue(queue.waitPeerExclusiveLock(procedure2, peerId));
+
+    // Same peerId, still only has 1 LockedResource
+    locks = queue.getLocks();
+    assertEquals(1, locks.size());
+
+    resource = locks.get(0);
+    assertLockResource(resource, LockedResourceType.PEER, peerId);
+    // LockedResource owner still is the origin procedure
+    assertExclusiveLock(resource, procedure);
+    // The new procedure should in the waiting list
+    assertEquals(1, resource.getWaitingProcedures().size());
+  }
+
+  @Test
   public void testListLocksWaiting() throws Exception {
     LockProcedure procedure1 = createExclusiveLockProcedure(1);
     queue.waitTableExclusiveLock(procedure1, TableName.valueOf("ns4", "table4"));

http://git-wip-us.apache.org/repos/asf/hbase/blob/0d0b2098/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureSchedulerConcurrency.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureSchedulerConcurrency.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureSchedulerConcurrency.java
index 2e8e52a..4e67a63 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureSchedulerConcurrency.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureSchedulerConcurrency.java
@@ -26,6 +26,8 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface.PeerOperationType;
+import org.apache.hadoop.hbase.master.procedure.TestMasterProcedureScheduler.TestPeerProcedure;
 import org.apache.hadoop.hbase.master.procedure.TestMasterProcedureScheduler.TestTableProcedure;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -63,6 +65,85 @@ public class TestMasterProcedureSchedulerConcurrency {
     queue.clear();
   }
 
+  @Test
+  public void testConcurrentPeerOperations() throws Exception {
+    TestPeerProcedureSet procSet = new TestPeerProcedureSet(queue);
+
+    int NUM_ITEMS = 10;
+    int NUM_PEERS = 5;
+    AtomicInteger opsCount = new AtomicInteger(0);
+    for (int i = 0; i < NUM_PEERS; ++i) {
+      String peerId = String.format("test-peer-%04d", i);
+      for (int j = 1; j < NUM_ITEMS; ++j) {
+        procSet.addBack(new TestPeerProcedure(i * 100 + j, peerId, PeerOperationType.ADD));
+        opsCount.incrementAndGet();
+      }
+    }
+    assertEquals(opsCount.get(), queue.size());
+
+    Thread[] threads = new Thread[NUM_PEERS * 2];
+    HashSet<String> concurrentPeers = new HashSet<>();
+    ArrayList<String> failures = new ArrayList<>();
+    AtomicInteger concurrentCount = new AtomicInteger(0);
+    for (int i = 0; i < threads.length; ++i) {
+      threads[i] = new Thread() {
+        @Override
+        public void run() {
+          while (opsCount.get() > 0) {
+            try {
+              TestPeerProcedure proc = procSet.acquire();
+              if (proc == null) {
+                queue.signalAll();
+                if (opsCount.get() > 0) {
+                  continue;
+                }
+                break;
+              }
+
+              String peerId = proc.getPeerId();
+              synchronized (concurrentPeers) {
+                assertTrue("unexpected concurrency on " + peerId, concurrentPeers.add(peerId));
+              }
+              assertTrue(opsCount.decrementAndGet() >= 0);
+
+              try {
+                long procId = proc.getProcId();
+                int concurrent = concurrentCount.incrementAndGet();
+                assertTrue("inc-concurrent="+ concurrent +" 1 <= concurrent <= "+ NUM_PEERS,
+                  concurrent >= 1 && concurrent <= NUM_PEERS);
+                LOG.debug("[S] peerId="+ peerId +" procId="+ procId +" concurrent="+ concurrent);
+                Thread.sleep(2000);
+                concurrent = concurrentCount.decrementAndGet();
+                LOG.debug("[E] peerId="+ peerId +" procId="+ procId +" concurrent="+ concurrent);
+                assertTrue("dec-concurrent=" + concurrent, concurrent < NUM_PEERS);
+              } finally {
+                synchronized (concurrentPeers) {
+                  assertTrue(concurrentPeers.remove(peerId));
+                }
+                procSet.release(proc);
+              }
+            } catch (Throwable e) {
+              LOG.error("Failed " + e.getMessage(), e);
+              synchronized (failures) {
+                failures.add(e.getMessage());
+              }
+            } finally {
+              queue.signalAll();
+            }
+          }
+        }
+      };
+      threads[i].start();
+    }
+
+    for (int i = 0; i < threads.length; ++i) {
+      threads[i].join();
+    }
+    assertTrue(failures.toString(), failures.isEmpty());
+    assertEquals(0, opsCount.get());
+    assertEquals(0, queue.size());
+  }
+
   /**
    * Verify that "write" operations for a single table are serialized,
    * but different tables can be executed in parallel.
@@ -221,4 +302,58 @@ public class TestMasterProcedureSchedulerConcurrency {
       return ((TableProcedureInterface)proc).getTableOperationType();
     }
   }
+
+  public static class TestPeerProcedureSet {
+    private final MasterProcedureScheduler queue;
+
+    public TestPeerProcedureSet(final MasterProcedureScheduler queue) {
+      this.queue = queue;
+    }
+
+    public void addBack(TestPeerProcedure proc) {
+      queue.addBack(proc);
+    }
+
+    public TestPeerProcedure acquire() {
+      TestPeerProcedure proc = null;
+      boolean waiting = true;
+      while (waiting && queue.size() > 0) {
+        proc = (TestPeerProcedure) queue.poll(100000000L);
+        if (proc == null) {
+          continue;
+        }
+        switch (proc.getPeerOperationType()) {
+          case ADD:
+          case REMOVE:
+          case ENABLE:
+          case DISABLE:
+          case UPDATE_CONFIG:
+            waiting = queue.waitPeerExclusiveLock(proc, proc.getPeerId());
+            break;
+          case REFRESH:
+            waiting = false;
+            break;
+          default:
+            throw new UnsupportedOperationException();
+        }
+      }
+      return proc;
+    }
+
+    public void release(TestPeerProcedure proc) {
+      switch (proc.getPeerOperationType()) {
+        case ADD:
+        case REMOVE:
+        case ENABLE:
+        case DISABLE:
+        case UPDATE_CONFIG:
+          queue.wakePeerExclusiveLock(proc, proc.getPeerId());
+          break;
+        case REFRESH:
+          break;
+        default:
+          throw new UnsupportedOperationException();
+      }
+    }
+  }
 }


[19/26] hbase git commit: HBASE-19580 Use slf4j instead of commons-logging in new, just-added Peer Procedure classes

Posted by zh...@apache.org.
HBASE-19580 Use slf4j instead of commons-logging in new, just-added Peer Procedure classes


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/729b6a78
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/729b6a78
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/729b6a78

Branch: refs/heads/HBASE-19397
Commit: 729b6a788c1180eb259e441e5511f4b3cda649d2
Parents: 0d0b209
Author: zhangduo <zh...@apache.org>
Authored: Thu Dec 21 21:59:46 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Dec 28 08:51:59 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/master/replication/AddPeerProcedure.java      | 6 +++---
 .../hadoop/hbase/master/replication/DisablePeerProcedure.java  | 6 +++---
 .../hadoop/hbase/master/replication/EnablePeerProcedure.java   | 6 +++---
 .../hadoop/hbase/master/replication/ModifyPeerProcedure.java   | 6 +++---
 .../hadoop/hbase/master/replication/RefreshPeerProcedure.java  | 6 +++---
 .../hadoop/hbase/master/replication/RemovePeerProcedure.java   | 6 +++---
 .../hbase/master/replication/UpdatePeerConfigProcedure.java    | 6 +++---
 7 files changed, 21 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/729b6a78/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
index c3862d8..066c3e7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
@@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
@@ -28,6 +26,8 @@ import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddPeerStateData;
 
@@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.A
 @InterfaceAudience.Private
 public class AddPeerProcedure extends ModifyPeerProcedure {
 
-  private static final Log LOG = LogFactory.getLog(AddPeerProcedure.class);
+  private static final Logger LOG = LoggerFactory.getLogger(AddPeerProcedure.class);
 
   private ReplicationPeerConfig peerConfig;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/729b6a78/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
index 0b32db9..9a28de6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
@@ -19,11 +19,11 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The procedure for disabling a replication peer.
@@ -31,7 +31,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 public class DisablePeerProcedure extends ModifyPeerProcedure {
 
-  private static final Log LOG = LogFactory.getLog(DisablePeerProcedure.class);
+  private static final Logger LOG = LoggerFactory.getLogger(DisablePeerProcedure.class);
 
   public DisablePeerProcedure() {
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/729b6a78/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
index 92ba000..4855901 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
@@ -19,11 +19,11 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The procedure for enabling a replication peer.
@@ -31,7 +31,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 public class EnablePeerProcedure extends ModifyPeerProcedure {
 
-  private static final Log LOG = LogFactory.getLog(EnablePeerProcedure.class);
+  private static final Logger LOG = LoggerFactory.getLogger(EnablePeerProcedure.class);
 
   public EnablePeerProcedure() {
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/729b6a78/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
index 23f6f87..c4552ed 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
@@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
@@ -30,6 +28,8 @@ import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
 import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyPeerStateData;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationState;
@@ -42,7 +42,7 @@ public abstract class ModifyPeerProcedure
     extends StateMachineProcedure<MasterProcedureEnv, PeerModificationState>
     implements PeerProcedureInterface {
 
-  private static final Log LOG = LogFactory.getLog(ModifyPeerProcedure.class);
+  private static final Logger LOG = LoggerFactory.getLogger(ModifyPeerProcedure.class);
 
   protected String peerId;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/729b6a78/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
index ddc2401..ba4285f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RefreshPeerProcedure.java
@@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
@@ -35,6 +33,8 @@ import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProced
 import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
 import org.apache.hadoop.hbase.replication.regionserver.RefreshPeerCallable;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationType;
@@ -45,7 +45,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.R
 public class RefreshPeerProcedure extends Procedure<MasterProcedureEnv>
     implements PeerProcedureInterface, RemoteProcedure<MasterProcedureEnv, ServerName> {
 
-  private static final Log LOG = LogFactory.getLog(RefreshPeerProcedure.class);
+  private static final Logger LOG = LoggerFactory.getLogger(RefreshPeerProcedure.class);
 
   private String peerId;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/729b6a78/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
index 3daad6d..d40df02 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
@@ -19,11 +19,11 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The procedure for removing a replication peer.
@@ -31,7 +31,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 @InterfaceAudience.Private
 public class RemovePeerProcedure extends ModifyPeerProcedure {
 
-  private static final Log LOG = LogFactory.getLog(RemovePeerProcedure.class);
+  private static final Logger LOG = LoggerFactory.getLogger(RemovePeerProcedure.class);
 
   public RemovePeerProcedure() {
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/729b6a78/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
index 435eefc..d8154dc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
@@ -19,14 +19,14 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.UpdatePeerConfigStateData;
 
@@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.U
 @InterfaceAudience.Private
 public class UpdatePeerConfigProcedure extends ModifyPeerProcedure {
 
-  private static final Log LOG = LogFactory.getLog(UpdatePeerConfigProcedure.class);
+  private static final Logger LOG = LoggerFactory.getLogger(UpdatePeerConfigProcedure.class);
 
   private ReplicationPeerConfig peerConfig;
 


[21/26] hbase git commit: HBASE-19630 Add peer cluster key check when add new replication peer

Posted by zh...@apache.org.
HBASE-19630 Add peer cluster key check when add new replication peer

Signed-off-by: zhangduo <zh...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/09249a20
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/09249a20
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/09249a20

Branch: refs/heads/HBASE-19397
Commit: 09249a20e4274ef452ba9c870d416ee7ccd6fa1e
Parents: 76539be
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Dec 26 21:10:00 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Dec 28 08:51:59 2017 +0800

----------------------------------------------------------------------
 .../replication/ReplicationPeerManager.java     | 54 ++++++++++++--------
 .../replication/TestReplicationAdmin.java       | 22 ++++++++
 2 files changed, 54 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/09249a20/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
index 84abfeb..b78cbce 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.master.replication;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
@@ -42,6 +43,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
+import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -216,36 +218,36 @@ public final class ReplicationPeerManager {
     return desc != null ? Optional.of(desc.getPeerConfig()) : Optional.empty();
   }
 
-  /**
-   * If replicate_all flag is true, it means all user tables will be replicated to peer cluster.
-   * Then allow config exclude namespaces or exclude table-cfs which can't be replicated to peer
-   * cluster.
-   * <p>
-   * If replicate_all flag is false, it means all user tables can't be replicated to peer cluster.
-   * Then allow to config namespaces or table-cfs which will be replicated to peer cluster.
-   */
-  private static void checkPeerConfig(ReplicationPeerConfig peerConfig)
-      throws DoNotRetryIOException {
+  private void checkPeerConfig(ReplicationPeerConfig peerConfig) throws DoNotRetryIOException {
+    checkClusterKey(peerConfig.getClusterKey());
+
     if (peerConfig.replicateAllUserTables()) {
-      if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty()) ||
-        (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) {
-        throw new DoNotRetryIOException("Need clean namespaces or table-cfs config firstly " +
-          "when you want replicate all cluster");
+      // If replicate_all flag is true, it means all user tables will be replicated to peer cluster.
+      // Then allow config exclude namespaces or exclude table-cfs which can't be replicated to peer
+      // cluster.
+      if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty())
+          || (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) {
+        throw new DoNotRetryIOException("Need clean namespaces or table-cfs config firstly "
+            + "when you want replicate all cluster");
       }
       checkNamespacesAndTableCfsConfigConflict(peerConfig.getExcludeNamespaces(),
         peerConfig.getExcludeTableCFsMap());
     } else {
-      if ((peerConfig.getExcludeNamespaces() != null &&
-        !peerConfig.getExcludeNamespaces().isEmpty()) ||
-        (peerConfig.getExcludeTableCFsMap() != null &&
-          !peerConfig.getExcludeTableCFsMap().isEmpty())) {
+      // If replicate_all flag is false, it means all user tables can't be replicated to peer
+      // cluster. Then allow to config namespaces or table-cfs which will be replicated to peer
+      // cluster.
+      if ((peerConfig.getExcludeNamespaces() != null
+          && !peerConfig.getExcludeNamespaces().isEmpty())
+          || (peerConfig.getExcludeTableCFsMap() != null
+              && !peerConfig.getExcludeTableCFsMap().isEmpty())) {
         throw new DoNotRetryIOException(
-            "Need clean exclude-namespaces or exclude-table-cfs config firstly" +
-              " when replicate_all flag is false");
+            "Need clean exclude-namespaces or exclude-table-cfs config firstly"
+                + " when replicate_all flag is false");
       }
       checkNamespacesAndTableCfsConfigConflict(peerConfig.getNamespaces(),
         peerConfig.getTableCFsMap());
     }
+
     checkConfiguredWALEntryFilters(peerConfig);
   }
 
@@ -268,7 +270,7 @@ public final class ReplicationPeerManager {
    * exclude namespace.</li>
    * </ol>
    */
-  private static void checkNamespacesAndTableCfsConfigConflict(Set<String> namespaces,
+  private void checkNamespacesAndTableCfsConfigConflict(Set<String> namespaces,
       Map<TableName, ? extends Collection<String>> tableCfs) throws DoNotRetryIOException {
     if (namespaces == null || namespaces.isEmpty()) {
       return;
@@ -285,7 +287,7 @@ public final class ReplicationPeerManager {
     }
   }
 
-  private static void checkConfiguredWALEntryFilters(ReplicationPeerConfig peerConfig)
+  private void checkConfiguredWALEntryFilters(ReplicationPeerConfig peerConfig)
       throws DoNotRetryIOException {
     String filterCSV = peerConfig.getConfiguration()
         .get(BaseReplicationEndpoint.REPLICATION_WALENTRYFILTER_CONFIG_KEY);
@@ -302,6 +304,14 @@ public final class ReplicationPeerManager {
     }
   }
 
+  private void checkClusterKey(String clusterKey) throws DoNotRetryIOException {
+    try {
+      ZKConfig.validateClusterKey(clusterKey);
+    } catch (IOException e) {
+      throw new DoNotRetryIOException("Invalid cluster key: " + clusterKey, e);
+    }
+  }
+
   public static ReplicationPeerManager create(ZKWatcher zk, Configuration conf)
       throws ReplicationException {
     ReplicationPeerStorage peerStorage =

http://git-wip-us.apache.org/repos/asf/hbase/blob/09249a20/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
index 8bb3230..a198d20 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
@@ -112,6 +112,28 @@ public class TestReplicationAdmin {
     }
   }
 
+  @Test
+  public void testAddInvalidPeer() {
+    ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder();
+    builder.setClusterKey(KEY_ONE);
+    try {
+      String invalidPeerId = "1-2";
+      hbaseAdmin.addReplicationPeer(invalidPeerId, builder.build());
+      fail("Should fail as the peer id: " + invalidPeerId + " is invalid");
+    } catch (Exception e) {
+      // OK
+    }
+
+    try {
+      String invalidClusterKey = "2181:/hbase";
+      builder.setClusterKey(invalidClusterKey);
+      hbaseAdmin.addReplicationPeer(ID_ONE, builder.build());
+      fail("Should fail as the peer cluster key: " + invalidClusterKey + " is invalid");
+    } catch (Exception e) {
+      // OK
+    }
+  }
+
   /**
    * Simple testing of adding and removing peers, basically shows that
    * all interactions with ZK work


[10/26] hbase git commit: HBASE-19599 Remove ReplicationQueuesClient, use ReplicationQueueStorage directly

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
index 93b8649..1faaae3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
@@ -21,6 +21,7 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -48,17 +49,18 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
+import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.AtomicLongMap;
 
 /**
@@ -303,57 +305,53 @@ public class DumpReplicationQueues extends Configured implements Tool {
   }
 
   public String dumpQueues(ClusterConnection connection, ZKWatcher zkw, Set<String> peerIds,
-                           boolean hdfs) throws Exception {
-    ReplicationQueuesClient queuesClient;
+      boolean hdfs) throws Exception {
+    ReplicationQueueStorage queueStorage;
     ReplicationPeers replicationPeers;
     ReplicationQueues replicationQueues;
     ReplicationTracker replicationTracker;
-    ReplicationQueuesClientArguments replicationArgs =
-        new ReplicationQueuesClientArguments(getConf(), new WarnOnlyAbortable(), zkw);
+    ReplicationQueuesArguments replicationArgs =
+        new ReplicationQueuesArguments(getConf(), new WarnOnlyAbortable(), zkw);
     StringBuilder sb = new StringBuilder();
 
-    queuesClient = ReplicationFactory.getReplicationQueuesClient(replicationArgs);
-    queuesClient.init();
+    queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, getConf());
     replicationQueues = ReplicationFactory.getReplicationQueues(replicationArgs);
-    replicationPeers = ReplicationFactory.getReplicationPeers(zkw, getConf(), queuesClient, connection);
+    replicationPeers =
+        ReplicationFactory.getReplicationPeers(zkw, getConf(), queueStorage, connection);
     replicationTracker = ReplicationFactory.getReplicationTracker(zkw, replicationPeers, getConf(),
       new WarnOnlyAbortable(), new WarnOnlyStoppable());
-    List<String> liveRegionServers = replicationTracker.getListOfRegionServers();
+    Set<String> liveRegionServers = new HashSet<>(replicationTracker.getListOfRegionServers());
 
     // Loops each peer on each RS and dumps the queues
-    try {
-      List<String> regionservers = queuesClient.getListOfReplicators();
-      if (regionservers == null || regionservers.isEmpty()) {
-        return sb.toString();
+    List<ServerName> regionservers = queueStorage.getListOfReplicators();
+    if (regionservers == null || regionservers.isEmpty()) {
+      return sb.toString();
+    }
+    for (ServerName regionserver : regionservers) {
+      List<String> queueIds = queueStorage.getAllQueues(regionserver);
+      replicationQueues.init(regionserver.getServerName());
+      if (!liveRegionServers.contains(regionserver.getServerName())) {
+        deadRegionServers.add(regionserver.getServerName());
       }
-      for (String regionserver : regionservers) {
-        List<String> queueIds = queuesClient.getAllQueues(regionserver);
-        replicationQueues.init(regionserver);
-        if (!liveRegionServers.contains(regionserver)) {
-          deadRegionServers.add(regionserver);
-        }
-        for (String queueId : queueIds) {
-          ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
-          List<String> wals = queuesClient.getLogsInQueue(regionserver, queueId);
-          if (!peerIds.contains(queueInfo.getPeerId())) {
-            deletedQueues.add(regionserver + "/" + queueId);
-            sb.append(formatQueue(regionserver, replicationQueues, queueInfo, queueId, wals, true,
-              hdfs));
-          } else {
-            sb.append(formatQueue(regionserver, replicationQueues, queueInfo, queueId, wals, false,
-              hdfs));
-          }
+      for (String queueId : queueIds) {
+        ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
+        List<String> wals = queueStorage.getWALsInQueue(regionserver, queueId);
+        if (!peerIds.contains(queueInfo.getPeerId())) {
+          deletedQueues.add(regionserver + "/" + queueId);
+          sb.append(
+            formatQueue(regionserver, replicationQueues, queueInfo, queueId, wals, true, hdfs));
+        } else {
+          sb.append(
+            formatQueue(regionserver, replicationQueues, queueInfo, queueId, wals, false, hdfs));
         }
       }
-    } catch (KeeperException ke) {
-      throw new IOException(ke);
     }
     return sb.toString();
   }
 
-  private String formatQueue(String regionserver, ReplicationQueues replicationQueues, ReplicationQueueInfo queueInfo,
-                           String queueId, List<String> wals, boolean isDeleted, boolean hdfs) throws Exception {
-
+  private String formatQueue(ServerName regionserver, ReplicationQueues replicationQueues,
+      ReplicationQueueInfo queueInfo, String queueId, List<String> wals, boolean isDeleted,
+      boolean hdfs) throws Exception {
     StringBuilder sb = new StringBuilder();
 
     List<ServerName> deadServers;
@@ -389,13 +387,14 @@ public class DumpReplicationQueues extends Configured implements Tool {
   /**
    *  return total size in bytes from a list of WALs
    */
-  private long getTotalWALSize(FileSystem fs, List<String> wals, String server) throws IOException {
+  private long getTotalWALSize(FileSystem fs, List<String> wals, ServerName server)
+      throws IOException {
     long size = 0;
     FileStatus fileStatus;
 
     for (String wal : wals) {
       try {
-        fileStatus = (new WALLink(getConf(), server, wal)).getFileStatus(fs);
+        fileStatus = (new WALLink(getConf(), server.getServerName(), wal)).getFileStatus(fs);
       } catch (IOException e) {
         if (e instanceof FileNotFoundException) {
           numWalsNotFound++;

http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java
index 839b5ad..85fa729 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/hbck/ReplicationChecker.java
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.util.hbck;
 
 import java.io.IOException;
@@ -27,22 +26,23 @@ import java.util.Map.Entry;
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.master.cleaner.ReplicationZKNodeCleaner;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.util.HBaseFsck;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
 
-/*
+/**
  * Check and fix undeleted replication queues for removed peerId.
  */
 @InterfaceAudience.Private
 public class ReplicationChecker {
   private final ErrorReporter errorReporter;
   // replicator with its queueIds for removed peers
-  private Map<String, List<String>> undeletedQueueIds = new HashMap<>();
+  private Map<ServerName, List<String>> undeletedQueueIds = new HashMap<>();
   // replicator with its undeleted queueIds for removed peers in hfile-refs queue
   private Set<String> undeletedHFileRefsQueueIds = new HashSet<>();
   private final ReplicationZKNodeCleaner cleaner;
@@ -60,8 +60,8 @@ public class ReplicationChecker {
 
   public void checkUnDeletedQueues() throws IOException {
     undeletedQueueIds = cleaner.getUnDeletedQueues();
-    for (Entry<String, List<String>> replicatorAndQueueIds : undeletedQueueIds.entrySet()) {
-      String replicator = replicatorAndQueueIds.getKey();
+    for (Entry<ServerName, List<String>> replicatorAndQueueIds : undeletedQueueIds.entrySet()) {
+      ServerName replicator = replicatorAndQueueIds.getKey();
       for (String queueId : replicatorAndQueueIds.getValue()) {
         ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
         String msg = "Undeleted replication queue for removed peer found: "

http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java
index 28a7562..b28eaaf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncReplicationAdminApi.java
@@ -25,6 +25,7 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -34,12 +35,16 @@ import java.util.Set;
 import java.util.concurrent.CompletionException;
 
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.zookeeper.ReadOnlyZKClient;
 import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -56,8 +61,8 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
 
   private final String ID_ONE = "1";
   private final String KEY_ONE = "127.0.0.1:2181:/hbase";
-  private final String ID_SECOND = "2";
-  private final String KEY_SECOND = "127.0.0.1:2181:/hbase2";
+  private final String ID_TWO = "2";
+  private final String KEY_TWO = "127.0.0.1:2181:/hbase2";
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
@@ -65,21 +70,27 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
     TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, 120000);
     TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
     TEST_UTIL.getConfiguration().setInt(START_LOG_ERRORS_AFTER_COUNT_KEY, 0);
+    TEST_UTIL.getConfiguration().setInt(ReadOnlyZKClient.RECOVERY_RETRY, 1);
     TEST_UTIL.startMiniCluster();
     ASYNC_CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
   }
 
   @After
-  public void cleanupPeer() {
+  public void clearPeerAndQueues() throws IOException, ReplicationException {
     try {
       admin.removeReplicationPeer(ID_ONE).join();
     } catch (Exception e) {
-      LOG.debug("Replication peer " + ID_ONE + " may already be removed");
     }
     try {
-      admin.removeReplicationPeer(ID_SECOND).join();
+      admin.removeReplicationPeer(ID_TWO).join();
     } catch (Exception e) {
-      LOG.debug("Replication peer " + ID_SECOND + " may already be removed");
+    }
+    ReplicationQueueStorage queueStorage = ReplicationStorageFactory
+        .getReplicationQueueStorage(TEST_UTIL.getZooKeeperWatcher(), TEST_UTIL.getConfiguration());
+    for (ServerName serverName : queueStorage.getListOfReplicators()) {
+      for (String queue : queueStorage.getAllQueues(serverName)) {
+        queueStorage.removeQueue(serverName, queue);
+      }
     }
   }
 
@@ -88,7 +99,7 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
     ReplicationPeerConfig rpc1 = new ReplicationPeerConfig();
     rpc1.setClusterKey(KEY_ONE);
     ReplicationPeerConfig rpc2 = new ReplicationPeerConfig();
-    rpc2.setClusterKey(KEY_SECOND);
+    rpc2.setClusterKey(KEY_TWO);
     // Add a valid peer
     admin.addReplicationPeer(ID_ONE, rpc1).join();
     // try adding the same (fails)
@@ -101,19 +112,19 @@ public class TestAsyncReplicationAdminApi extends TestAsyncAdminBase {
     assertEquals(1, admin.listReplicationPeers().get().size());
     // Try to remove an inexisting peer
     try {
-      admin.removeReplicationPeer(ID_SECOND).join();
+      admin.removeReplicationPeer(ID_TWO).join();
       fail("Test case should fail as removing a inexisting peer.");
     } catch (CompletionException e) {
       // OK!
     }
     assertEquals(1, admin.listReplicationPeers().get().size());
     // Add a second since multi-slave is supported
-    admin.addReplicationPeer(ID_SECOND, rpc2).join();
+    admin.addReplicationPeer(ID_TWO, rpc2).join();
     assertEquals(2, admin.listReplicationPeers().get().size());
     // Remove the first peer we added
     admin.removeReplicationPeer(ID_ONE).join();
     assertEquals(1, admin.listReplicationPeers().get().size());
-    admin.removeReplicationPeer(ID_SECOND).join();
+    admin.removeReplicationPeer(ID_TWO).join();
     assertEquals(0, admin.listReplicationPeers().get().size());
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
index c2fcd8c..8bb3230 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.zookeeper.ReadOnlyZKClient;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -82,6 +83,7 @@ public class TestReplicationAdmin {
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
+    TEST_UTIL.getConfiguration().setInt(ReadOnlyZKClient.RECOVERY_RETRY, 1);
     TEST_UTIL.startMiniCluster();
     admin = new ReplicationAdmin(TEST_UTIL.getConfiguration());
     hbaseAdmin = TEST_UTIL.getAdmin();

http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
index 2d517c4..e678e1b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
@@ -1,4 +1,4 @@
-/*
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -24,16 +24,13 @@ import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.spy;
 
 import java.io.IOException;
-import java.lang.reflect.Field;
 import java.net.URLEncoder;
 import java.util.Iterator;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Random;
 
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -51,7 +48,6 @@ import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClientZKImpl;
 import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -65,10 +61,11 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+
 @Category({MasterTests.class, MediumTests.class})
 public class TestLogsCleaner {
 
@@ -195,24 +192,6 @@ public class TestLogsCleaner {
     }
   }
 
-  @Test(timeout=5000)
-  public void testZnodeCversionChange() throws Exception {
-    Configuration conf = TEST_UTIL.getConfiguration();
-    ReplicationLogCleaner cleaner = new ReplicationLogCleaner();
-    cleaner.setConf(conf);
-
-    ReplicationQueuesClientZKImpl rqcMock = Mockito.mock(ReplicationQueuesClientZKImpl.class);
-    Mockito.when(rqcMock.getQueuesZNodeCversion()).thenReturn(1, 2, 3, 4);
-
-    Field rqc = ReplicationLogCleaner.class.getDeclaredField("replicationQueues");
-    rqc.setAccessible(true);
-
-    rqc.set(cleaner, rqcMock);
-
-    // This should return eventually when cversion stabilizes
-    cleaner.getDeletableFiles(new LinkedList<>());
-  }
-
   /**
    * ReplicationLogCleaner should be able to ride over ZooKeeper errors without aborting.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
index 2f23301..cfad645 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
@@ -1,12 +1,19 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
- * law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
- * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
- * for the specific language governing permissions and limitations under the License.
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 package org.apache.hadoop.hbase.master.cleaner;
 
@@ -17,10 +24,7 @@ import static org.junit.Assert.fail;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.spy;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-
 import java.io.IOException;
-import java.lang.reflect.Field;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
@@ -45,7 +49,6 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl;
 import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
@@ -63,10 +66,11 @@ import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+
 @Category({ MasterTests.class, SmallTests.class })
 public class TestReplicationHFileCleaner {
   private static final Logger LOG = LoggerFactory.getLogger(ReplicationQueuesZKImpl.class);
@@ -188,32 +192,6 @@ public class TestReplicationHFileCleaner {
     assertTrue(deletableFilesIterator.next().getPath().equals(deletablefile));
   }
 
-  /*
-   * Test for HBASE-14621. This test will not assert directly anything. Without the fix the test
-   * will end up in a infinite loop, so it will timeout.
-   */
-  @Test(timeout = 15000)
-  public void testForDifferntHFileRefsZnodeVersion() throws Exception {
-    // 1. Create a file
-    Path file = new Path(root, "testForDifferntHFileRefsZnodeVersion");
-    fs.createNewFile(file);
-    // 2. Assert file is successfully created
-    assertTrue("Test file not created!", fs.exists(file));
-    ReplicationHFileCleaner cleaner = new ReplicationHFileCleaner();
-    cleaner.setConf(conf);
-
-    ReplicationQueuesClient replicationQueuesClient = Mockito.mock(ReplicationQueuesClient.class);
-    //Return different znode version for each call
-    Mockito.when(replicationQueuesClient.getHFileRefsNodeChangeVersion()).thenReturn(1, 2);
-
-    Class<? extends ReplicationHFileCleaner> cleanerClass = cleaner.getClass();
-    Field rqc = cleanerClass.getDeclaredField("rqc");
-    rqc.setAccessible(true);
-    rqc.set(cleaner, replicationQueuesClient);
-
-    cleaner.isFileDeletable(fs.getFileStatus(file));
-  }
-
   /**
    * ReplicationHFileCleaner should be able to ride over ZooKeeper errors without aborting.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java
index 6aa59cb..8178266 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.master.cleaner;
 
 import static org.junit.Assert.assertEquals;
@@ -26,6 +25,7 @@ import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
@@ -43,9 +43,9 @@ public class TestReplicationZKNodeCleaner {
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   private final String ID_ONE = "1";
-  private final String SERVER_ONE = "server1";
+  private final ServerName SERVER_ONE = ServerName.valueOf("server1", 8000, 1234);
   private final String ID_TWO = "2";
-  private final String SERVER_TWO = "server2";
+  private final ServerName SERVER_TWO = ServerName.valueOf("server2", 8000, 1234);
 
   private final Configuration conf;
   private final ZKWatcher zkw;
@@ -72,12 +72,12 @@ public class TestReplicationZKNodeCleaner {
 
   @Test
   public void testReplicationZKNodeCleaner() throws Exception {
-    repQueues.init(SERVER_ONE);
+    repQueues.init(SERVER_ONE.getServerName());
     // add queue for ID_ONE which isn't exist
     repQueues.addLog(ID_ONE, "file1");
 
     ReplicationZKNodeCleaner cleaner = new ReplicationZKNodeCleaner(conf, zkw, null);
-    Map<String, List<String>> undeletedQueues = cleaner.getUnDeletedQueues();
+    Map<ServerName, List<String>> undeletedQueues = cleaner.getUnDeletedQueues();
     assertEquals(1, undeletedQueues.size());
     assertTrue(undeletedQueues.containsKey(SERVER_ONE));
     assertEquals(1, undeletedQueues.get(SERVER_ONE).size());
@@ -100,7 +100,7 @@ public class TestReplicationZKNodeCleaner {
 
   @Test
   public void testReplicationZKNodeCleanerChore() throws Exception {
-    repQueues.init(SERVER_ONE);
+    repQueues.init(SERVER_ONE.getServerName());
     // add queue for ID_ONE which isn't exist
     repQueues.addLog(ID_ONE, "file1");
     // add a recovery queue for ID_TWO which isn't exist

http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
deleted file mode 100644
index 29c0930..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
+++ /dev/null
@@ -1,378 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.replication;
-
-import static org.junit.Assert.*;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.zookeeper.ZKConfig;
-import org.apache.zookeeper.KeeperException;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * White box testing for replication state interfaces. Implementations should extend this class, and
- * initialize the interfaces properly.
- */
-public abstract class TestReplicationStateBasic {
-
-  protected ReplicationQueues rq1;
-  protected ReplicationQueues rq2;
-  protected ReplicationQueues rq3;
-  protected ReplicationQueuesClient rqc;
-  protected String server1 = ServerName.valueOf("hostname1.example.org", 1234, -1L).toString();
-  protected String server2 = ServerName.valueOf("hostname2.example.org", 1234, -1L).toString();
-  protected String server3 = ServerName.valueOf("hostname3.example.org", 1234, -1L).toString();
-  protected ReplicationPeers rp;
-  protected static final String ID_ONE = "1";
-  protected static final String ID_TWO = "2";
-  protected static String KEY_ONE;
-  protected static String KEY_TWO;
-
-  // For testing when we try to replicate to ourself
-  protected String OUR_ID = "3";
-  protected String OUR_KEY;
-
-  protected static int zkTimeoutCount;
-  protected static final int ZK_MAX_COUNT = 300;
-  protected static final int ZK_SLEEP_INTERVAL = 100; // millis
-
-  private static final Logger LOG = LoggerFactory.getLogger(TestReplicationStateBasic.class);
-
-  @Before
-  public void setUp() {
-    zkTimeoutCount = 0;
-  }
-
-  @Test
-  public void testReplicationQueuesClient() throws ReplicationException, KeeperException {
-    rqc.init();
-    // Test methods with empty state
-    assertEquals(0, rqc.getListOfReplicators().size());
-    assertNull(rqc.getLogsInQueue(server1, "qId1"));
-    assertNull(rqc.getAllQueues(server1));
-
-    /*
-     * Set up data Two replicators: -- server1: three queues with 0, 1 and 2 log files each --
-     * server2: zero queues
-     */
-    rq1.init(server1);
-    rq2.init(server2);
-    rq1.addLog("qId1", "trash");
-    rq1.removeLog("qId1", "trash");
-    rq1.addLog("qId2", "filename1");
-    rq1.addLog("qId3", "filename2");
-    rq1.addLog("qId3", "filename3");
-    rq2.addLog("trash", "trash");
-    rq2.removeQueue("trash");
-
-    List<String> reps = rqc.getListOfReplicators();
-    assertEquals(2, reps.size());
-    assertTrue(server1, reps.contains(server1));
-    assertTrue(server2, reps.contains(server2));
-
-    assertNull(rqc.getLogsInQueue("bogus", "bogus"));
-    assertNull(rqc.getLogsInQueue(server1, "bogus"));
-    assertEquals(0, rqc.getLogsInQueue(server1, "qId1").size());
-    assertEquals(1, rqc.getLogsInQueue(server1, "qId2").size());
-    assertEquals("filename1", rqc.getLogsInQueue(server1, "qId2").get(0));
-
-    assertNull(rqc.getAllQueues("bogus"));
-    assertEquals(0, rqc.getAllQueues(server2).size());
-    List<String> list = rqc.getAllQueues(server1);
-    assertEquals(3, list.size());
-    assertTrue(list.contains("qId2"));
-    assertTrue(list.contains("qId3"));
-  }
-
-  @Test
-  public void testReplicationQueues() throws ReplicationException {
-    rq1.init(server1);
-    rq2.init(server2);
-    rq3.init(server3);
-    //Initialize ReplicationPeer so we can add peers (we don't transfer lone queues)
-    rp.init();
-
-    // 3 replicators should exist
-    assertEquals(3, rq1.getListOfReplicators().size());
-    rq1.removeQueue("bogus");
-    rq1.removeLog("bogus", "bogus");
-    rq1.removeAllQueues();
-    assertEquals(0, rq1.getAllQueues().size());
-    assertEquals(0, rq1.getLogPosition("bogus", "bogus"));
-    assertNull(rq1.getLogsInQueue("bogus"));
-    assertNull(rq1.getUnClaimedQueueIds(
-        ServerName.valueOf("bogus", 1234, -1L).toString()));
-
-    rq1.setLogPosition("bogus", "bogus", 5L);
-
-    populateQueues();
-
-    assertEquals(3, rq1.getListOfReplicators().size());
-    assertEquals(0, rq2.getLogsInQueue("qId1").size());
-    assertEquals(5, rq3.getLogsInQueue("qId5").size());
-    assertEquals(0, rq3.getLogPosition("qId1", "filename0"));
-    rq3.setLogPosition("qId5", "filename4", 354L);
-    assertEquals(354L, rq3.getLogPosition("qId5", "filename4"));
-
-    assertEquals(5, rq3.getLogsInQueue("qId5").size());
-    assertEquals(0, rq2.getLogsInQueue("qId1").size());
-    assertEquals(0, rq1.getAllQueues().size());
-    assertEquals(1, rq2.getAllQueues().size());
-    assertEquals(5, rq3.getAllQueues().size());
-
-    assertEquals(0, rq3.getUnClaimedQueueIds(server1).size());
-    rq3.removeReplicatorIfQueueIsEmpty(server1);
-    assertEquals(2, rq3.getListOfReplicators().size());
-
-    List<String> queues = rq2.getUnClaimedQueueIds(server3);
-    assertEquals(5, queues.size());
-    for(String queue: queues) {
-      rq2.claimQueue(server3, queue);
-    }
-    rq2.removeReplicatorIfQueueIsEmpty(server3);
-    assertEquals(1, rq2.getListOfReplicators().size());
-
-    // Try to claim our own queues
-    assertNull(rq2.getUnClaimedQueueIds(server2));
-    rq2.removeReplicatorIfQueueIsEmpty(server2);
-
-    assertEquals(6, rq2.getAllQueues().size());
-
-    rq2.removeAllQueues();
-
-    assertEquals(0, rq2.getListOfReplicators().size());
-  }
-
-  @Test
-  public void testInvalidClusterKeys() throws ReplicationException, KeeperException {
-    rp.init();
-
-    try {
-      rp.registerPeer(ID_ONE,
-        new ReplicationPeerConfig().setClusterKey("hostname1.example.org:1234:hbase"));
-      fail("Should throw an IllegalArgumentException because "
-            + "zookeeper.znode.parent is missing leading '/'.");
-    } catch (IllegalArgumentException e) {
-      // Expected.
-    }
-
-    try {
-      rp.registerPeer(ID_ONE,
-        new ReplicationPeerConfig().setClusterKey("hostname1.example.org:1234:/"));
-      fail("Should throw an IllegalArgumentException because zookeeper.znode.parent is missing.");
-    } catch (IllegalArgumentException e) {
-      // Expected.
-    }
-
-    try {
-      rp.registerPeer(ID_ONE,
-        new ReplicationPeerConfig().setClusterKey("hostname1.example.org::/hbase"));
-      fail("Should throw an IllegalArgumentException because "
-          + "hbase.zookeeper.property.clientPort is missing.");
-    } catch (IllegalArgumentException e) {
-      // Expected.
-    }
-  }
-
-  @Test
-  public void testHfileRefsReplicationQueues() throws ReplicationException, KeeperException {
-    rp.init();
-    rq1.init(server1);
-    rqc.init();
-
-    List<Pair<Path, Path>> files1 = new ArrayList<>(3);
-    files1.add(new Pair<>(null, new Path("file_1")));
-    files1.add(new Pair<>(null, new Path("file_2")));
-    files1.add(new Pair<>(null, new Path("file_3")));
-    assertNull(rqc.getReplicableHFiles(ID_ONE));
-    assertEquals(0, rqc.getAllPeersFromHFileRefsQueue().size());
-    rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
-    rq1.addPeerToHFileRefs(ID_ONE);
-    rq1.addHFileRefs(ID_ONE, files1);
-    assertEquals(1, rqc.getAllPeersFromHFileRefsQueue().size());
-    assertEquals(3, rqc.getReplicableHFiles(ID_ONE).size());
-    List<String> hfiles2 = new ArrayList<>(files1.size());
-    for (Pair<Path, Path> p : files1) {
-      hfiles2.add(p.getSecond().getName());
-    }
-    String removedString = hfiles2.remove(0);
-    rq1.removeHFileRefs(ID_ONE, hfiles2);
-    assertEquals(1, rqc.getReplicableHFiles(ID_ONE).size());
-    hfiles2 = new ArrayList<>(1);
-    hfiles2.add(removedString);
-    rq1.removeHFileRefs(ID_ONE, hfiles2);
-    assertEquals(0, rqc.getReplicableHFiles(ID_ONE).size());
-    rp.unregisterPeer(ID_ONE);
-  }
-
-  @Test
-  public void testRemovePeerForHFileRefs() throws ReplicationException, KeeperException {
-    rq1.init(server1);
-    rqc.init();
-
-    rp.init();
-    rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
-    rq1.addPeerToHFileRefs(ID_ONE);
-    rp.registerPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO));
-    rq1.addPeerToHFileRefs(ID_TWO);
-
-    List<Pair<Path, Path>> files1 = new ArrayList<>(3);
-    files1.add(new Pair<>(null, new Path("file_1")));
-    files1.add(new Pair<>(null, new Path("file_2")));
-    files1.add(new Pair<>(null, new Path("file_3")));
-    rq1.addHFileRefs(ID_ONE, files1);
-    rq1.addHFileRefs(ID_TWO, files1);
-    assertEquals(2, rqc.getAllPeersFromHFileRefsQueue().size());
-    assertEquals(3, rqc.getReplicableHFiles(ID_ONE).size());
-    assertEquals(3, rqc.getReplicableHFiles(ID_TWO).size());
-
-    rp.unregisterPeer(ID_ONE);
-    rq1.removePeerFromHFileRefs(ID_ONE);
-    assertEquals(1, rqc.getAllPeersFromHFileRefsQueue().size());
-    assertNull(rqc.getReplicableHFiles(ID_ONE));
-    assertEquals(3, rqc.getReplicableHFiles(ID_TWO).size());
-
-    rp.unregisterPeer(ID_TWO);
-    rq1.removePeerFromHFileRefs(ID_TWO);
-    assertEquals(0, rqc.getAllPeersFromHFileRefsQueue().size());
-    assertNull(rqc.getReplicableHFiles(ID_TWO));
-  }
-
-  @Test
-  public void testReplicationPeers() throws Exception {
-    rp.init();
-
-    // Test methods with non-existent peer ids
-    try {
-      rp.unregisterPeer("bogus");
-      fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
-    } catch (IllegalArgumentException e) {
-    }
-    try {
-      rp.enablePeer("bogus");
-      fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
-    } catch (IllegalArgumentException e) {
-    }
-    try {
-      rp.disablePeer("bogus");
-      fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
-    } catch (IllegalArgumentException e) {
-    }
-    try {
-      rp.getStatusOfPeer("bogus");
-      fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
-    } catch (IllegalArgumentException e) {
-    }
-    assertFalse(rp.peerConnected("bogus"));
-    rp.peerDisconnected("bogus");
-
-    assertNull(rp.getPeerConf("bogus"));
-    assertNumberOfPeers(0);
-
-    // Add some peers
-    rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
-    assertNumberOfPeers(1);
-    rp.registerPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO));
-    assertNumberOfPeers(2);
-
-    // Test methods with a peer that is added but not connected
-    try {
-      rp.getStatusOfPeer(ID_ONE);
-      fail("There are no connected peers, should have thrown an IllegalArgumentException");
-    } catch (IllegalArgumentException e) {
-    }
-    assertEquals(KEY_ONE, ZKConfig.getZooKeeperClusterKey(rp.getPeerConf(ID_ONE).getSecond()));
-    rp.unregisterPeer(ID_ONE);
-    rp.peerDisconnected(ID_ONE);
-    assertNumberOfPeers(1);
-
-    // Add one peer
-    rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
-    rp.peerConnected(ID_ONE);
-    assertNumberOfPeers(2);
-    assertTrue(rp.getStatusOfPeer(ID_ONE));
-    rp.disablePeer(ID_ONE);
-    assertConnectedPeerStatus(false, ID_ONE);
-    rp.enablePeer(ID_ONE);
-    assertConnectedPeerStatus(true, ID_ONE);
-
-    // Disconnect peer
-    rp.peerDisconnected(ID_ONE);
-    assertNumberOfPeers(2);
-    try {
-      rp.getStatusOfPeer(ID_ONE);
-      fail("There are no connected peers, should have thrown an IllegalArgumentException");
-    } catch (IllegalArgumentException e) {
-    }
-  }
-
-  protected void assertConnectedPeerStatus(boolean status, String peerId) throws Exception {
-    // we can first check if the value was changed in the store, if it wasn't then fail right away
-    if (status != rp.getStatusOfPeerFromBackingStore(peerId)) {
-      fail("ConnectedPeerStatus was " + !status + " but expected " + status + " in ZK");
-    }
-    while (true) {
-      if (status == rp.getStatusOfPeer(peerId)) {
-        return;
-      }
-      if (zkTimeoutCount < ZK_MAX_COUNT) {
-        LOG.debug("ConnectedPeerStatus was " + !status + " but expected " + status
-            + ", sleeping and trying again.");
-        Thread.sleep(ZK_SLEEP_INTERVAL);
-      } else {
-        fail("Timed out waiting for ConnectedPeerStatus to be " + status);
-      }
-    }
-  }
-
-  protected void assertNumberOfPeers(int total) {
-    assertEquals(total, rp.getAllPeerConfigs().size());
-    assertEquals(total, rp.getAllPeerIds().size());
-    assertEquals(total, rp.getAllPeerIds().size());
-  }
-
-  /*
-   * three replicators: rq1 has 0 queues, rq2 has 1 queue with no logs, rq3 has 5 queues with 1, 2,
-   * 3, 4, 5 log files respectively
-   */
-  protected void populateQueues() throws ReplicationException {
-    rq1.addLog("trash", "trash");
-    rq1.removeQueue("trash");
-
-    rq2.addLog("qId1", "trash");
-    rq2.removeLog("qId1", "trash");
-
-    for (int i = 1; i < 6; i++) {
-      for (int j = 0; j < i; j++) {
-        rq3.addLog("qId" + i, "filename" + j);
-      }
-      //Add peers for the corresponding queues so they are not orphans
-      rp.registerPeer("qId" + i, new ReplicationPeerConfig().setClusterKey("localhost:2818:/bogus" + i));
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
deleted file mode 100644
index 231d655..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
+++ /dev/null
@@ -1,227 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.replication;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hbase.ChoreService;
-import org.apache.hadoop.hbase.ClusterId;
-import org.apache.hadoop.hbase.CoordinatedStateManager;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.ClusterConnection;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.ReplicationTests;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
-import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
-import org.apache.hadoop.hbase.zookeeper.ZKConfig;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.zookeeper.KeeperException;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@Category({ReplicationTests.class, MediumTests.class})
-public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
-
-  private static final Logger LOG = LoggerFactory.getLogger(TestReplicationStateZKImpl.class);
-
-  private static Configuration conf;
-  private static HBaseTestingUtility utility;
-  private static ZKWatcher zkw;
-  private static String replicationZNode;
-  private ReplicationQueuesZKImpl rqZK;
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    utility = new HBaseTestingUtility();
-    utility.startMiniZKCluster();
-    conf = utility.getConfiguration();
-    conf.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
-    zkw = HBaseTestingUtility.getZooKeeperWatcher(utility);
-    String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
-    replicationZNode = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, replicationZNodeName);
-    KEY_ONE = initPeerClusterState("/hbase1");
-    KEY_TWO = initPeerClusterState("/hbase2");
-  }
-
-  private static String initPeerClusterState(String baseZKNode)
-      throws IOException, KeeperException {
-    // Add a dummy region server and set up the cluster id
-    Configuration testConf = new Configuration(conf);
-    testConf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, baseZKNode);
-    ZKWatcher zkw1 = new ZKWatcher(testConf, "test1", null);
-    String fakeRs = ZNodePaths.joinZNode(zkw1.znodePaths.rsZNode, "hostname1.example.org:1234");
-    ZKUtil.createWithParents(zkw1, fakeRs);
-    ZKClusterId.setClusterId(zkw1, new ClusterId());
-    return ZKConfig.getZooKeeperClusterKey(testConf);
-  }
-
-  @Before
-  @Override
-  public void setUp() {
-    super.setUp();
-    DummyServer ds1 = new DummyServer(server1);
-    DummyServer ds2 = new DummyServer(server2);
-    DummyServer ds3 = new DummyServer(server3);
-    try {
-      rq1 = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, ds1, zkw));
-      rq2 = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, ds2, zkw));
-      rq3 = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, ds3, zkw));
-      rqc = ReplicationFactory.getReplicationQueuesClient(
-        new ReplicationQueuesClientArguments(conf, ds1, zkw));
-    } catch (Exception e) {
-      // This should not occur, because getReplicationQueues() only throws for
-      // TableBasedReplicationQueuesImpl
-      fail("ReplicationFactory.getReplicationQueues() threw an IO Exception");
-    }
-    rp = ReplicationFactory.getReplicationPeers(zkw, conf, zkw);
-    OUR_KEY = ZKConfig.getZooKeeperClusterKey(conf);
-    rqZK = new ReplicationQueuesZKImpl(zkw, conf, ds1);
-  }
-
-  @After
-  public void tearDown() throws KeeperException, IOException {
-    ZKUtil.deleteNodeRecursively(zkw, replicationZNode);
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    utility.shutdownMiniZKCluster();
-  }
-
-  @Test
-  public void testIsPeerPath_PathToParentOfPeerNode() {
-    assertFalse(rqZK.isPeerPath(rqZK.peersZNode));
-  }
-
-  @Test
-  public void testIsPeerPath_PathToChildOfPeerNode() {
-    String peerChild = ZNodePaths.joinZNode(ZNodePaths.joinZNode(rqZK.peersZNode, "1"), "child");
-    assertFalse(rqZK.isPeerPath(peerChild));
-  }
-
-  @Test
-  public void testIsPeerPath_ActualPeerPath() {
-    String peerPath = ZNodePaths.joinZNode(rqZK.peersZNode, "1");
-    assertTrue(rqZK.isPeerPath(peerPath));
-  }
-
-  static class DummyServer implements Server {
-    private String serverName;
-    private boolean isAborted = false;
-    private boolean isStopped = false;
-
-    public DummyServer(String serverName) {
-      this.serverName = serverName;
-    }
-
-    @Override
-    public Configuration getConfiguration() {
-      return conf;
-    }
-
-    @Override
-    public ZKWatcher getZooKeeper() {
-      return zkw;
-    }
-
-    @Override
-    public CoordinatedStateManager getCoordinatedStateManager() {
-      return null;
-    }
-
-    @Override
-    public ClusterConnection getConnection() {
-      return null;
-    }
-
-    @Override
-    public MetaTableLocator getMetaTableLocator() {
-      return null;
-    }
-
-    @Override
-    public ServerName getServerName() {
-      return ServerName.valueOf(this.serverName);
-    }
-
-    @Override
-    public void abort(String why, Throwable e) {
-      LOG.info("Aborting " + serverName);
-      this.isAborted = true;
-    }
-
-    @Override
-    public boolean isAborted() {
-      return this.isAborted;
-    }
-
-    @Override
-    public void stop(String why) {
-      this.isStopped = true;
-    }
-
-    @Override
-    public boolean isStopped() {
-      return this.isStopped;
-    }
-
-    @Override
-    public ChoreService getChoreService() {
-      return null;
-    }
-
-    @Override
-    public ClusterConnection getClusterConnection() {
-      // TODO Auto-generated method stub
-      return null;
-    }
-
-    @Override
-    public FileSystem getFileSystem() {
-      return null;
-    }
-
-    @Override
-    public boolean isStopping() {
-      return false;
-    }
-
-    @Override
-    public Connection createConnection(Configuration conf) throws IOException {
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java
index b47a8d3..aeab8b0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java
@@ -1,34 +1,34 @@
-/*
-*
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.hadoop.hbase.replication.regionserver;
 
+import static org.junit.Assert.assertTrue;
+
+import java.util.List;
+
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.ReplicationQueues;
 import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClientZKImpl;
 import org.apache.hadoop.hbase.replication.ReplicationSourceDummy;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
@@ -36,11 +36,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
 /**
  * Tests the ReplicationSourceManager with ReplicationQueueZkImpl's and
  * ReplicationQueuesClientZkImpl. Also includes extra tests outside of those in
@@ -114,41 +109,4 @@ public class TestReplicationSourceManagerZkImpl extends TestReplicationSourceMan
 
     server.stop("");
   }
-
-  @Test
-  public void testFailoverDeadServerCversionChange() throws Exception {
-    final Server s0 = new DummyServer("cversion-change0.example.org");
-    ReplicationQueues repQueues =
-      ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, s0,
-        s0.getZooKeeper()));
-    repQueues.init(s0.getServerName().toString());
-    // populate some znodes in the peer znode
-    files.add("log1");
-    files.add("log2");
-    for (String file : files) {
-      repQueues.addLog("1", file);
-    }
-    // simulate queue transfer
-    Server s1 = new DummyServer("cversion-change1.example.org");
-    ReplicationQueues rq1 =
-      ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(s1.getConfiguration(), s1,
-        s1.getZooKeeper()));
-    rq1.init(s1.getServerName().toString());
-
-    ReplicationQueuesClientZKImpl client =
-      (ReplicationQueuesClientZKImpl)ReplicationFactory.getReplicationQueuesClient(
-        new ReplicationQueuesClientArguments(s1.getConfiguration(), s1, s1.getZooKeeper()));
-
-    int v0 = client.getQueuesZNodeCversion();
-    List<String> queues = rq1.getUnClaimedQueueIds(s0.getServerName().getServerName());
-    for(String queue : queues) {
-      rq1.claimQueue(s0.getServerName().getServerName(), queue);
-    }
-    rq1.removeReplicatorIfQueueIsEmpty(s0.getServerName().getServerName());
-    int v1 = client.getQueuesZNodeCversion();
-    // cversion should increase by 1 since a child node is deleted
-    assertEquals(v0 + 1, v1);
-
-    s0.stop("");
-  }
 }


[15/26] hbase git commit: HBASE-19564 Procedure id is missing in the response of peer related operations

Posted by zh...@apache.org.
HBASE-19564 Procedure id is missing in the response of peer related operations


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/52248585
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/52248585
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/52248585

Branch: refs/heads/HBASE-19397
Commit: 522485854dbd033ca0cfa4888650369b5261130d
Parents: d77c744
Author: zhangduo <zh...@apache.org>
Authored: Wed Dec 20 20:57:37 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Dec 28 08:51:59 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/master/MasterRpcServices.java  | 24 ++++++++++----------
 .../master/replication/ModifyPeerProcedure.java |  4 +---
 2 files changed, 13 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/52248585/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 700b363..9f71bab 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -1886,10 +1886,10 @@ public class MasterRpcServices extends RSRpcServices
   public AddReplicationPeerResponse addReplicationPeer(RpcController controller,
       AddReplicationPeerRequest request) throws ServiceException {
     try {
-      master.addReplicationPeer(request.getPeerId(),
-        ReplicationPeerConfigUtil.convert(request.getPeerConfig()), request.getPeerState()
-            .getState().equals(ReplicationState.State.ENABLED));
-      return AddReplicationPeerResponse.newBuilder().build();
+      long procId = master.addReplicationPeer(request.getPeerId(),
+        ReplicationPeerConfigUtil.convert(request.getPeerConfig()),
+        request.getPeerState().getState().equals(ReplicationState.State.ENABLED));
+      return AddReplicationPeerResponse.newBuilder().setProcId(procId).build();
     } catch (ReplicationException | IOException e) {
       throw new ServiceException(e);
     }
@@ -1899,8 +1899,8 @@ public class MasterRpcServices extends RSRpcServices
   public RemoveReplicationPeerResponse removeReplicationPeer(RpcController controller,
       RemoveReplicationPeerRequest request) throws ServiceException {
     try {
-      master.removeReplicationPeer(request.getPeerId());
-      return RemoveReplicationPeerResponse.newBuilder().build();
+      long procId = master.removeReplicationPeer(request.getPeerId());
+      return RemoveReplicationPeerResponse.newBuilder().setProcId(procId).build();
     } catch (ReplicationException | IOException e) {
       throw new ServiceException(e);
     }
@@ -1910,8 +1910,8 @@ public class MasterRpcServices extends RSRpcServices
   public EnableReplicationPeerResponse enableReplicationPeer(RpcController controller,
       EnableReplicationPeerRequest request) throws ServiceException {
     try {
-      master.enableReplicationPeer(request.getPeerId());
-      return EnableReplicationPeerResponse.newBuilder().build();
+      long procId = master.enableReplicationPeer(request.getPeerId());
+      return EnableReplicationPeerResponse.newBuilder().setProcId(procId).build();
     } catch (ReplicationException | IOException e) {
       throw new ServiceException(e);
     }
@@ -1921,8 +1921,8 @@ public class MasterRpcServices extends RSRpcServices
   public DisableReplicationPeerResponse disableReplicationPeer(RpcController controller,
       DisableReplicationPeerRequest request) throws ServiceException {
     try {
-      master.disableReplicationPeer(request.getPeerId());
-      return DisableReplicationPeerResponse.newBuilder().build();
+      long procId = master.disableReplicationPeer(request.getPeerId());
+      return DisableReplicationPeerResponse.newBuilder().setProcId(procId).build();
     } catch (ReplicationException | IOException e) {
       throw new ServiceException(e);
     }
@@ -1948,9 +1948,9 @@ public class MasterRpcServices extends RSRpcServices
   public UpdateReplicationPeerConfigResponse updateReplicationPeerConfig(RpcController controller,
       UpdateReplicationPeerConfigRequest request) throws ServiceException {
     try {
-      master.updateReplicationPeerConfig(request.getPeerId(),
+      long procId = master.updateReplicationPeerConfig(request.getPeerId(),
         ReplicationPeerConfigUtil.convert(request.getPeerConfig()));
-      return UpdateReplicationPeerConfigResponse.newBuilder().build();
+      return UpdateReplicationPeerConfigResponse.newBuilder().setProcId(procId).build();
     } catch (ReplicationException | IOException e) {
       throw new ServiceException(e);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/52248585/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
index 7076bab..23f6f87 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
@@ -54,9 +54,7 @@ public abstract class ModifyPeerProcedure
 
   protected ModifyPeerProcedure(String peerId) {
     this.peerId = peerId;
-    // TODO: temporarily set a 4.0 here to always wait for the procedure exection completed. Change
-    // to 3.0 or 2.0 after the client modification is done.
-    this.latch = ProcedurePrepareLatch.createLatch(4, 0);
+    this.latch = ProcedurePrepareLatch.createLatch(2, 0);
   }
 
   public ProcedurePrepareLatch getLatch() {


[26/26] hbase git commit: HBASE-19617 Remove ReplicationQueues, use ReplicationQueueStorage directly

Posted by zh...@apache.org.
HBASE-19617 Remove ReplicationQueues, use ReplicationQueueStorage directly


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/264890a9
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/264890a9
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/264890a9

Branch: refs/heads/HBASE-19397
Commit: 264890a91d83b30fcbda91cc0350968972ede288
Parents: 28b93f4
Author: zhangduo <zh...@apache.org>
Authored: Wed Dec 27 22:03:51 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Dec 28 08:52:14 2017 +0800

----------------------------------------------------------------------
 .../hbase/replication/ReplicationFactory.java   |   9 +-
 .../hbase/replication/ReplicationQueues.java    | 160 -------
 .../replication/ReplicationQueuesArguments.java |  70 ---
 .../replication/ReplicationQueuesZKImpl.java    | 407 -----------------
 .../hbase/replication/ReplicationTableBase.java | 442 -------------------
 .../replication/ReplicationTrackerZKImpl.java   |  21 +-
 .../replication/ZKReplicationQueueStorage.java  |  22 +
 .../replication/TestReplicationStateBasic.java  | 131 +++---
 .../replication/TestReplicationStateZKImpl.java |  41 +-
 .../regionserver/DumpReplicationQueues.java     |  15 +-
 .../RecoveredReplicationSource.java             |  17 +-
 .../RecoveredReplicationSourceShipper.java      |  22 +-
 .../replication/regionserver/Replication.java   |  38 +-
 .../regionserver/ReplicationSource.java         |  22 +-
 .../ReplicationSourceInterface.java             |  11 +-
 .../regionserver/ReplicationSourceManager.java  | 260 ++++++-----
 .../regionserver/ReplicationSyncUp.java         |  29 +-
 .../hbase/master/cleaner/TestLogsCleaner.java   |  12 +-
 .../cleaner/TestReplicationHFileCleaner.java    |  26 +-
 .../cleaner/TestReplicationZKNodeCleaner.java   |  22 +-
 .../replication/ReplicationSourceDummy.java     |   6 +-
 .../replication/TestReplicationSyncUpTool.java  |   6 +-
 .../TestReplicationSourceManager.java           |  98 ++--
 .../TestReplicationSourceManagerZkImpl.java     |  58 +--
 24 files changed, 381 insertions(+), 1564 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/264890a9/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
----------------------------------------------------------------------
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 6c1c213..5e70e57 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
@@ -17,12 +17,11 @@
  */
 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.
@@ -30,12 +29,6 @@ import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 @InterfaceAudience.Private
 public class ReplicationFactory {
 
-  public static ReplicationQueues getReplicationQueues(ReplicationQueuesArguments args)
-      throws Exception {
-    return (ReplicationQueues) ConstructorUtils.invokeConstructor(ReplicationQueuesZKImpl.class,
-      args);
-  }
-
   public static ReplicationPeers getReplicationPeers(ZKWatcher zk, Configuration conf,
       Abortable abortable) {
     return getReplicationPeers(zk, conf, null, abortable);

http://git-wip-us.apache.org/repos/asf/hbase/blob/264890a9/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 7f440b1..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java
+++ /dev/null
@@ -1,160 +0,0 @@
-/*
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.replication;
-
-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;
-
-/**
- * This provides an interface for maintaining a region server's replication queues. These queues
- * keep track of the WALs and HFile references (if hbase.replication.bulkload.enabled is enabled)
- * that still need to be replicated to remote clusters.
- */
-@InterfaceAudience.Private
-public interface ReplicationQueues {
-
-  /**
-   * Initialize the region server replication queue interface.
-   * @param serverName The server name of the region server that owns the replication queues this
-   *          interface manages.
-   */
-  void init(String serverName) throws ReplicationException;
-
-  /**
-   * Remove a replication queue.
-   * @param queueId a String that identifies the queue.
-   */
-  void removeQueue(String queueId);
-
-  /**
-   * Add a new WAL file to the given queue. If the queue does not exist it is created.
-   * @param queueId a String that identifies the queue.
-   * @param filename name of the WAL
-   */
-  void addLog(String queueId, String filename) throws ReplicationException;
-
-  /**
-   * Remove an WAL file from the given queue.
-   * @param queueId a String that identifies the queue.
-   * @param filename name of the WAL
-   */
-  void removeLog(String queueId, String filename);
-
-  /**
-   * Set the current position for a specific WAL in a given queue.
-   * @param queueId a String that identifies the queue
-   * @param filename name of the WAL
-   * @param position the current position in the file
-   */
-  void setLogPosition(String queueId, String filename, long position);
-
-  /**
-   * Get the current position for a specific WAL in a given queue.
-   * @param queueId a String that identifies the queue
-   * @param filename name of the WAL
-   * @return the current position in the file
-   */
-  long getLogPosition(String queueId, String filename) throws ReplicationException;
-
-  /**
-   * Remove all replication queues for this region server.
-   */
-  void removeAllQueues();
-
-  /**
-   * Get a list of all WALs in the given queue.
-   * @param queueId a String that identifies the queue
-   * @return a list of WALs, null if no such queue exists for this server
-   */
-  List<String> getLogsInQueue(String queueId);
-
-  /**
-   * 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
-   */
-  List<String> getAllQueues();
-
-  /**
-   * Get queueIds from a dead region server, whose queues has not been claimed by other region
-   * servers.
-   * @return empty if the queue exists but no children, null if the queue does not exist.
-  */
-  List<String> getUnClaimedQueueIds(String regionserver);
-
-  /**
-   * Take ownership for the queue identified by queueId and belongs to a dead region server.
-   * @param regionserver the id of the dead region server
-   * @param queueId the id of the queue
-   * @return the new PeerId and A SortedSet of WALs in its queue, and null if no unclaimed queue.
-   */
-  Pair<String, SortedSet<String>> claimQueue(String regionserver, String queueId);
-
-  /**
-   * Remove the znode of region server if the queue is empty.
-   * @param regionserver
-   */
-  void removeReplicatorIfQueueIsEmpty(String regionserver);
-
-  /**
-   * Get a list of all region servers that have outstanding replication queues. These servers could
-   * be alive, dead or from a previous run of the cluster.
-   * @return a list of server names
-   */
-  List<String> getListOfReplicators();
-
-  /**
-   * Checks if the provided znode is the same as this region server's
-   * @param regionserver the id of the region server
-   * @return if this is this rs's znode
-   */
-  boolean isThisOurRegionServer(String regionserver);
-
-  /**
-   * Add a peer to hfile reference queue if peer does not exist.
-   * @param peerId peer cluster id to be added
-   * @throws ReplicationException if fails to add a peer id to hfile reference queue
-   */
-  void addPeerToHFileRefs(String peerId) throws ReplicationException;
-
-  /**
-   * Remove a peer from hfile reference queue.
-   * @param peerId peer cluster id to be removed
-   */
-  void removePeerFromHFileRefs(String peerId);
-
-  /**
-   * Add new hfile references to the queue.
-   * @param peerId peer cluster id to which the hfiles need to be replicated
-   * @param pairs list of pairs of { HFile location in staging dir, HFile path in region dir which
-   *          will be added in the queue }
-   * @throws ReplicationException if fails to add a hfile reference
-   */
-  void addHFileRefs(String peerId, List<Pair<Path, Path>> pairs) throws ReplicationException;
-
-  /**
-   * Remove hfile references from the queue.
-   * @param peerId peer cluster id from which this hfile references needs to be removed
-   * @param files list of hfile references to be removed
-   */
-  void removeHFileRefs(String peerId, List<String> files);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/264890a9/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java
deleted file mode 100644
index c2a5df3..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesArguments.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.replication;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * Wrapper around common arguments used to construct ReplicationQueues. Used to construct various
- * ReplicationQueues Implementations with different constructor arguments by reflection.
- */
-@InterfaceAudience.Private
-public class ReplicationQueuesArguments {
-
-  private ZKWatcher zk;
-  private Configuration conf;
-  private Abortable abort;
-
-  public ReplicationQueuesArguments(Configuration conf, Abortable abort) {
-    this.conf = conf;
-    this.abort = abort;
-  }
-
-  public ReplicationQueuesArguments(Configuration conf, Abortable abort, ZKWatcher zk) {
-    this(conf, abort);
-    setZk(zk);
-  }
-
-  public ZKWatcher getZk() {
-    return zk;
-  }
-
-  public void setZk(ZKWatcher zk) {
-    this.zk = zk;
-  }
-
-  public Configuration getConf() {
-    return conf;
-  }
-
-  public void setConf(Configuration conf) {
-    this.conf = conf;
-  }
-
-  public Abortable getAbortable() {
-    return abort;
-  }
-
-  public void setAbortable(Abortable abort) {
-    this.abort = abort;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/264890a9/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 7551cb7..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesZKImpl.java
+++ /dev/null
@@ -1,407 +0,0 @@
-/*
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.replication;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This class provides an implementation of the
- * interface using ZooKeeper. The
- * base znode that this class works at is the myQueuesZnode. The myQueuesZnode contains a list of
- * all outstanding WAL files on this region server that need to be replicated. The myQueuesZnode is
- * the regionserver name (a concatenation of the region server’s hostname, client port and start
- * code). For example:
- *
- * /hbase/replication/rs/hostname.example.org,6020,1234
- *
- * Within this znode, the region server maintains a set of WAL replication queues. These queues are
- * represented by child znodes named using there give queue id. For example:
- *
- * /hbase/replication/rs/hostname.example.org,6020,1234/1
- * /hbase/replication/rs/hostname.example.org,6020,1234/2
- *
- * Each queue has one child znode for every WAL that still needs to be replicated. The value of
- * these WAL child znodes is the latest position that has been replicated. This position is updated
- * every time a WAL entry is replicated. For example:
- *
- * /hbase/replication/rs/hostname.example.org,6020,1234/1/23522342.23422 [VALUE: 254]
- */
-@InterfaceAudience.Private
-public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements ReplicationQueues {
-
-  /** Znode containing all replication queues for this region server. */
-  private String myQueuesZnode;
-
-  private static final Logger LOG = LoggerFactory.getLogger(ReplicationQueuesZKImpl.class);
-
-  public ReplicationQueuesZKImpl(ReplicationQueuesArguments args) {
-    this(args.getZk(), args.getConf(), args.getAbortable());
-  }
-
-  public ReplicationQueuesZKImpl(final ZKWatcher zk, Configuration conf,
-                                 Abortable abortable) {
-    super(zk, conf, abortable);
-  }
-
-  @Override
-  public void init(String serverName) throws ReplicationException {
-    this.myQueuesZnode = ZNodePaths.joinZNode(this.queuesZNode, serverName);
-    try {
-      if (ZKUtil.checkExists(this.zookeeper, this.myQueuesZnode) < 0) {
-        ZKUtil.createWithParents(this.zookeeper, this.myQueuesZnode);
-      }
-    } catch (KeeperException e) {
-      throw new ReplicationException("Could not initialize replication queues.", e);
-    }
-    if (conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
-      HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT)) {
-      try {
-        if (ZKUtil.checkExists(this.zookeeper, this.hfileRefsZNode) < 0) {
-          ZKUtil.createWithParents(this.zookeeper, this.hfileRefsZNode);
-        }
-      } catch (KeeperException e) {
-        throw new ReplicationException("Could not initialize hfile references replication queue.",
-            e);
-      }
-    }
-  }
-
-  @Override
-  public void removeQueue(String queueId) {
-    try {
-      ZKUtil.deleteNodeRecursively(this.zookeeper,
-        ZNodePaths.joinZNode(this.myQueuesZnode, queueId));
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to delete queue (queueId=" + queueId + ")", e);
-    }
-  }
-
-  @Override
-  public void addLog(String queueId, String filename) throws ReplicationException {
-    String znode = ZNodePaths.joinZNode(this.myQueuesZnode, queueId);
-    znode = ZNodePaths.joinZNode(znode, filename);
-    try {
-      ZKUtil.createWithParents(this.zookeeper, znode);
-    } catch (KeeperException e) {
-      throw new ReplicationException(
-          "Could not add log because znode could not be created. queueId=" + queueId
-              + ", filename=" + filename);
-    }
-  }
-
-  @Override
-  public void removeLog(String queueId, String filename) {
-    try {
-      String znode = ZNodePaths.joinZNode(this.myQueuesZnode, queueId);
-      znode = ZNodePaths.joinZNode(znode, filename);
-      ZKUtil.deleteNode(this.zookeeper, znode);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to remove wal from queue (queueId=" + queueId + ", filename="
-          + filename + ")", e);
-    }
-  }
-
-  @Override
-  public void setLogPosition(String queueId, String filename, long position) {
-    try {
-      String znode = ZNodePaths.joinZNode(this.myQueuesZnode, queueId);
-      znode = ZNodePaths.joinZNode(znode, filename);
-      // Why serialize String of Long and not Long as bytes?
-      ZKUtil.setData(this.zookeeper, znode, ZKUtil.positionToByteArray(position));
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to write replication wal position (filename=" + filename
-          + ", position=" + position + ")", e);
-    }
-  }
-
-  @Override
-  public long getLogPosition(String queueId, String filename) throws ReplicationException {
-    String clusterZnode = ZNodePaths.joinZNode(this.myQueuesZnode, queueId);
-    String znode = ZNodePaths.joinZNode(clusterZnode, filename);
-    byte[] bytes = null;
-    try {
-      bytes = ZKUtil.getData(this.zookeeper, znode);
-    } catch (KeeperException e) {
-      throw new ReplicationException("Internal Error: could not get position in log for queueId="
-          + queueId + ", filename=" + filename, e);
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      return 0;
-    }
-    try {
-      return ZKUtil.parseWALPositionFrom(bytes);
-    } catch (DeserializationException de) {
-      LOG.warn("Failed to parse WALPosition for queueId=" + queueId + " and wal=" + filename
-          + " znode content, continuing.");
-    }
-    // if we can not parse the position, start at the beginning of the wal file
-    // again
-    return 0;
-  }
-
-  @Override
-  public boolean isThisOurRegionServer(String regionserver) {
-    return ZNodePaths.joinZNode(this.queuesZNode, regionserver).equals(this.myQueuesZnode);
-  }
-
-  @Override
-  public List<String> getUnClaimedQueueIds(String regionserver) {
-    if (isThisOurRegionServer(regionserver)) {
-      return null;
-    }
-    String rsZnodePath = ZNodePaths.joinZNode(this.queuesZNode, regionserver);
-    List<String> queues = null;
-    try {
-      queues = ZKUtil.listChildrenNoWatch(this.zookeeper, rsZnodePath);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to getUnClaimedQueueIds for RS" + regionserver, e);
-    }
-    return queues;
-  }
-
-  @Override
-  public Pair<String, SortedSet<String>> claimQueue(String regionserver, String queueId) {
-    LOG.info("Atomically moving " + regionserver + "/" + queueId + "'s WALs to my queue");
-    return moveQueueUsingMulti(regionserver, queueId);
-  }
-
-  @Override
-  public void removeReplicatorIfQueueIsEmpty(String regionserver) {
-    String rsPath = ZNodePaths.joinZNode(this.queuesZNode, regionserver);
-    try {
-      List<String> list = ZKUtil.listChildrenNoWatch(this.zookeeper, rsPath);
-      if (list != null && list.isEmpty()){
-        ZKUtil.deleteNode(this.zookeeper, rsPath);
-      }
-    } catch (KeeperException e) {
-      LOG.warn("Got error while removing replicator", e);
-    }
-  }
-
-  @Override
-  public void removeAllQueues() {
-    try {
-      ZKUtil.deleteNodeRecursively(this.zookeeper, this.myQueuesZnode);
-    } catch (KeeperException e) {
-      // if the znode is already expired, don't bother going further
-      if (e instanceof KeeperException.SessionExpiredException) {
-        return;
-      }
-      this.abortable.abort("Failed to delete replication queues for region server: "
-          + this.myQueuesZnode, e);
-    }
-  }
-
-  @Override
-  public List<String> getLogsInQueue(String queueId) {
-    String znode = ZNodePaths.joinZNode(this.myQueuesZnode, queueId);
-    List<String> result = null;
-    try {
-      result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to get list of wals for queueId=" + queueId, e);
-    }
-    return result;
-  }
-
-  @Override
-  public List<String> getAllQueues() {
-    List<String> listOfQueues = null;
-    try {
-      listOfQueues = ZKUtil.listChildrenNoWatch(this.zookeeper, this.myQueuesZnode);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to get a list of queues for region server: "
-          + this.myQueuesZnode, e);
-    }
-    return listOfQueues == null ? new ArrayList<>() : listOfQueues;
-  }
-
-  /**
-   * It "atomically" copies one peer's wals queue from another dead region server and returns them
-   * all sorted. The new peer id is equal to the old peer id appended with the dead server's znode.
-   * @param znode pertaining to the region server to copy the queues from
-   * @peerId peerId pertaining to the queue need to be copied
-   */
-  private Pair<String, SortedSet<String>> moveQueueUsingMulti(String znode, String peerId) {
-    try {
-      // hbase/replication/rs/deadrs
-      String deadRSZnodePath = ZNodePaths.joinZNode(this.queuesZNode, znode);
-      List<ZKUtilOp> listOfOps = new ArrayList<>();
-      ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId);
-
-      String newPeerId = peerId + "-" + znode;
-      String newPeerZnode = ZNodePaths.joinZNode(this.myQueuesZnode, newPeerId);
-      // check the logs queue for the old peer cluster
-      String oldClusterZnode = ZNodePaths.joinZNode(deadRSZnodePath, peerId);
-      List<String> wals = ZKUtil.listChildrenNoWatch(this.zookeeper, oldClusterZnode);
-
-      if (!peerExists(replicationQueueInfo.getPeerId())) {
-        LOG.warn("Peer " + replicationQueueInfo.getPeerId() +
-                " didn't exist, will move its queue to avoid the failure of multi op");
-        for (String wal : wals) {
-          String oldWalZnode = ZNodePaths.joinZNode(oldClusterZnode, wal);
-          listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldWalZnode));
-        }
-        listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
-        ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false);
-        return null;
-      }
-
-      SortedSet<String> logQueue = new TreeSet<>();
-      if (wals == null || wals.isEmpty()) {
-        listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
-      } else {
-        // create the new cluster znode
-        ZKUtilOp op = ZKUtilOp.createAndFailSilent(newPeerZnode, HConstants.EMPTY_BYTE_ARRAY);
-        listOfOps.add(op);
-        // get the offset of the logs and set it to new znodes
-        for (String wal : wals) {
-          String oldWalZnode = ZNodePaths.joinZNode(oldClusterZnode, wal);
-          byte[] logOffset = ZKUtil.getData(this.zookeeper, oldWalZnode);
-          LOG.debug("Creating " + wal + " with data " + Bytes.toString(logOffset));
-          String newLogZnode = ZNodePaths.joinZNode(newPeerZnode, wal);
-          listOfOps.add(ZKUtilOp.createAndFailSilent(newLogZnode, logOffset));
-          listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldWalZnode));
-          logQueue.add(wal);
-        }
-        // add delete op for peer
-        listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
-
-        if (LOG.isTraceEnabled())
-          LOG.trace(" The multi list size is: " + listOfOps.size());
-      }
-      ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false);
-
-      LOG.info("Atomically moved " + znode + "/" + peerId + "'s WALs to my queue");
-      return new Pair<>(newPeerId, logQueue);
-    } catch (KeeperException e) {
-      // Multi call failed; it looks like some other regionserver took away the logs.
-      LOG.warn("Got exception in copyQueuesFromRSUsingMulti: ", e);
-    } catch (InterruptedException e) {
-      LOG.warn("Got exception in copyQueuesFromRSUsingMulti: ", e);
-      Thread.currentThread().interrupt();
-    }
-    return null;
-  }
-
-  @Override
-  public void addHFileRefs(String peerId, List<Pair<Path, Path>> pairs)
-      throws ReplicationException {
-    String peerZnode = ZNodePaths.joinZNode(this.hfileRefsZNode, peerId);
-    boolean debugEnabled = LOG.isDebugEnabled();
-    if (debugEnabled) {
-      LOG.debug("Adding hfile references " + pairs + " in queue " + peerZnode);
-    }
-
-    int size = pairs.size();
-    List<ZKUtilOp> listOfOps = new ArrayList<>(size);
-
-    for (int i = 0; i < size; i++) {
-      listOfOps.add(ZKUtilOp.createAndFailSilent(
-        ZNodePaths.joinZNode(peerZnode, pairs.get(i).getSecond().getName()),
-        HConstants.EMPTY_BYTE_ARRAY));
-    }
-    if (debugEnabled) {
-      LOG.debug(" The multi list size for adding hfile references in zk for node " + peerZnode
-          + " is " + listOfOps.size());
-    }
-    try {
-      ZKUtil.multiOrSequential(this.zookeeper, listOfOps, true);
-    } catch (KeeperException e) {
-      throw new ReplicationException("Failed to create hfile reference znode=" + e.getPath(), e);
-    }
-  }
-
-  @Override
-  public void removeHFileRefs(String peerId, List<String> files) {
-    String peerZnode = ZNodePaths.joinZNode(this.hfileRefsZNode, peerId);
-    boolean debugEnabled = LOG.isDebugEnabled();
-    if (debugEnabled) {
-      LOG.debug("Removing hfile references " + files + " from queue " + peerZnode);
-    }
-
-    int size = files.size();
-    List<ZKUtilOp> listOfOps = new ArrayList<>(size);
-
-    for (int i = 0; i < size; i++) {
-      listOfOps.add(ZKUtilOp.deleteNodeFailSilent(ZNodePaths.joinZNode(peerZnode, files.get(i))));
-    }
-    if (debugEnabled) {
-      LOG.debug(" The multi list size for removing hfile references in zk for node " + peerZnode
-          + " is " + listOfOps.size());
-    }
-    try {
-      ZKUtil.multiOrSequential(this.zookeeper, listOfOps, true);
-    } catch (KeeperException e) {
-      LOG.error("Failed to remove hfile reference znode=" + e.getPath(), e);
-    }
-  }
-
-  @Override
-  public void addPeerToHFileRefs(String peerId) throws ReplicationException {
-    String peerZnode = ZNodePaths.joinZNode(this.hfileRefsZNode, peerId);
-    try {
-      if (ZKUtil.checkExists(this.zookeeper, peerZnode) == -1) {
-        LOG.info("Adding peer " + peerId + " to hfile reference queue.");
-        ZKUtil.createWithParents(this.zookeeper, peerZnode);
-      }
-    } catch (KeeperException e) {
-      throw new ReplicationException("Failed to add peer " + peerId + " to hfile reference queue.",
-          e);
-    }
-  }
-
-  @Override
-  public void removePeerFromHFileRefs(String peerId) {
-    final String peerZnode = ZNodePaths.joinZNode(this.hfileRefsZNode, peerId);
-    try {
-      if (ZKUtil.checkExists(this.zookeeper, peerZnode) == -1) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Peer " + peerZnode + " not found in hfile reference queue.");
-        }
-        return;
-      } else {
-        LOG.info("Removing peer " + peerZnode + " from hfile reference queue.");
-        ZKUtil.deleteNodeRecursively(this.zookeeper, peerZnode);
-      }
-    } catch (KeeperException e) {
-      LOG.error("Ignoring the exception to remove peer " + peerId + " from hfile reference queue.",
-        e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/264890a9/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTableBase.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 86124bc..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTableBase.java
+++ /dev/null
@@ -1,442 +0,0 @@
-/*
-*
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-package org.apache.hadoop.hbase.replication;
-
-import org.apache.hadoop.hbase.CompareOperator;
-import org.apache.hadoop.hbase.shaded.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;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.Executor;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-/*
- * Abstract class that provides an interface to the Replication Table. Which is currently
- * being used for WAL offset tracking.
- * The basic schema of this table will store each individual queue as a
- * seperate row. The row key will be a unique identifier of the creating server's name and the
- * queueId. Each queue must have the following two columns:
- *  COL_QUEUE_OWNER: tracks which server is currently responsible for tracking the queue
- *  COL_QUEUE_OWNER_HISTORY: a "|" delimited list of the previous server's that have owned this
- *    queue. The most recent previous owner is the leftmost entry.
- * They will also have columns mapping [WAL filename : offset]
- * The most flexible method of interacting with the Replication Table is by calling
- * getOrBlockOnReplicationTable() which will return a new copy of the Replication Table. It is up
- * to the caller to close the returned table.
- */
-@InterfaceAudience.Private
-abstract class ReplicationTableBase {
-
-  /** Name of the HBase Table used for tracking replication*/
-  public static final TableName REPLICATION_TABLE_NAME =
-    TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "replication");
-
-  // Column family and column names for Queues in the Replication Table
-  public static final byte[] CF_QUEUE = Bytes.toBytes("q");
-  public static final byte[] COL_QUEUE_OWNER = Bytes.toBytes("o");
-  public static final byte[] COL_QUEUE_OWNER_HISTORY = Bytes.toBytes("h");
-
-  // Column Descriptor for the Replication Table
-  private static final HColumnDescriptor REPLICATION_COL_DESCRIPTOR =
-    new HColumnDescriptor(CF_QUEUE).setMaxVersions(1)
-      .setInMemory(true)
-      .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
-        // TODO: Figure out which bloom filter to use
-      .setBloomFilterType(BloomType.NONE);
-
-  // The value used to delimit the queueId and server name inside of a queue's row key. Currently a
-  // hyphen, because it is guaranteed that queueId (which is a cluster id) cannot contain hyphens.
-  // See HBASE-11394.
-  public static final String ROW_KEY_DELIMITER = "-";
-
-  // The value used to delimit server names in the queue history list
-  public static final String QUEUE_HISTORY_DELIMITER = "|";
-
-  /*
-  * Make sure that HBase table operations for replication have a high number of retries. This is
-  * because the server is aborted if any HBase table operation fails. Each RPC will be attempted
-  * 3600 times before exiting. This provides each operation with 2 hours of retries
-  * before the server is aborted.
-  */
-  private static final int CLIENT_RETRIES = 3600;
-  private static final int RPC_TIMEOUT = 2000;
-  private static final int OPERATION_TIMEOUT = CLIENT_RETRIES * RPC_TIMEOUT;
-
-  // We only need a single thread to initialize the Replication Table
-  private static final int NUM_INITIALIZE_WORKERS = 1;
-
-  protected final Configuration conf;
-  protected final Abortable abortable;
-  private final Connection connection;
-  private final Executor executor;
-  private volatile CountDownLatch replicationTableInitialized;
-
-  public ReplicationTableBase(Configuration conf, Abortable abort) throws IOException {
-    this.conf = new Configuration(conf);
-    this.abortable = abort;
-    decorateConf();
-    this.connection = ConnectionFactory.createConnection(this.conf);
-    this.executor = setUpExecutor();
-    this.replicationTableInitialized = new CountDownLatch(1);
-    createReplicationTableInBackground();
-  }
-
-  /**
-   * Modify the connection's config so that operations run on the Replication Table have longer and
-   * a larger number of retries
-   */
-  private void decorateConf() {
-    this.conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, CLIENT_RETRIES);
-  }
-
-  /**
-   * Sets up the thread pool executor used to build the Replication Table in the background
-   * @return the configured executor
-   */
-  private Executor setUpExecutor() {
-    ThreadPoolExecutor tempExecutor = new ThreadPoolExecutor(NUM_INITIALIZE_WORKERS,
-        NUM_INITIALIZE_WORKERS, 100, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>());
-    ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
-    tfb.setNameFormat("ReplicationTableExecutor-%d");
-    tfb.setDaemon(true);
-    tempExecutor.setThreadFactory(tfb.build());
-    return tempExecutor;
-  }
-
-  /**
-   * Get whether the Replication Table has been successfully initialized yet
-   * @return whether the Replication Table is initialized
-   */
-  public boolean getInitializationStatus() {
-    return replicationTableInitialized.getCount() == 0;
-  }
-
-  /**
-   * Increases the RPC and operations timeouts for the Replication Table
-   */
-  private Table setReplicationTableTimeOuts(Table replicationTable) {
-    replicationTable.setRpcTimeout(RPC_TIMEOUT);
-    replicationTable.setOperationTimeout(OPERATION_TIMEOUT);
-    return replicationTable;
-  }
-
-  /**
-   * Build the row key for the given queueId. This will uniquely identify it from all other queues
-   * in the cluster.
-   * @param serverName The owner of the queue
-   * @param queueId String identifier of the queue
-   * @return String representation of the queue's row key
-   */
-  protected String buildQueueRowKey(String serverName, String queueId) {
-    return queueId + ROW_KEY_DELIMITER + serverName;
-  }
-
-  /**
-   * Parse the original queueId from a row key
-   * @param rowKey String representation of a queue's row key
-   * @return the original queueId
-   */
-  protected String getRawQueueIdFromRowKey(String rowKey) {
-    return rowKey.split(ROW_KEY_DELIMITER)[0];
-  }
-
-  /**
-   * Returns a queue's row key given either its raw or reclaimed queueId
-   *
-   * @param queueId queueId of the queue
-   * @return byte representation of the queue's row key
-   */
-  protected byte[] queueIdToRowKey(String serverName, String queueId) {
-    // Cluster id's are guaranteed to have no hyphens, so if the passed in queueId has no hyphen
-    // then this is not a reclaimed queue.
-    if (!queueId.contains(ROW_KEY_DELIMITER)) {
-      return Bytes.toBytes(buildQueueRowKey(serverName, queueId));
-      // If the queueId contained some hyphen it was reclaimed. In this case, the queueId is the
-      // queue's row key
-    } else {
-      return Bytes.toBytes(queueId);
-    }
-  }
-
-  /**
-   * Creates a "|" delimited record of the queue's past region server owners.
-   *
-   * @param originalHistory the queue's original owner history
-   * @param oldServer the name of the server that used to own the queue
-   * @return the queue's new owner history
-   */
-  protected String buildClaimedQueueHistory(String originalHistory, String oldServer) {
-    return oldServer + QUEUE_HISTORY_DELIMITER + originalHistory;
-  }
-
-  /**
-   * Get a list of all region servers that have outstanding replication queues. These servers could
-   * be alive, dead or from a previous run of the cluster.
-   * @return a list of server names
-   */
-  protected List<String> getListOfReplicators() {
-    // scan all of the queues and return a list of all unique OWNER values
-    Set<String> peerServers = new HashSet<>();
-    ResultScanner allQueuesInCluster = null;
-    try (Table replicationTable = getOrBlockOnReplicationTable()){
-      Scan scan = new Scan();
-      scan.addColumn(CF_QUEUE, COL_QUEUE_OWNER);
-      allQueuesInCluster = replicationTable.getScanner(scan);
-      for (Result queue : allQueuesInCluster) {
-        peerServers.add(Bytes.toString(queue.getValue(CF_QUEUE, COL_QUEUE_OWNER)));
-      }
-    } catch (IOException e) {
-      String errMsg = "Failed getting list of replicators";
-      abortable.abort(errMsg, e);
-    } finally {
-      if (allQueuesInCluster != null) {
-        allQueuesInCluster.close();
-      }
-    }
-    return new ArrayList<>(peerServers);
-  }
-
-  protected List<String> getAllQueues(String serverName) {
-    List<String> allQueues = new ArrayList<>();
-    ResultScanner queueScanner = null;
-    try {
-      queueScanner = getQueuesBelongingToServer(serverName);
-      for (Result queue : queueScanner) {
-        String rowKey =  Bytes.toString(queue.getRow());
-        // If the queue does not have a Owner History, then we must be its original owner. So we
-        // want to return its queueId in raw form
-        if (Bytes.toString(queue.getValue(CF_QUEUE, COL_QUEUE_OWNER_HISTORY)).length() == 0) {
-          allQueues.add(getRawQueueIdFromRowKey(rowKey));
-        } else {
-          allQueues.add(rowKey);
-        }
-      }
-      return allQueues;
-    } catch (IOException e) {
-      String errMsg = "Failed getting list of all replication queues for serverName=" + serverName;
-      abortable.abort(errMsg, e);
-      return null;
-    } finally {
-      if (queueScanner != null) {
-        queueScanner.close();
-      }
-    }
-  }
-
-  protected List<String> getLogsInQueue(String serverName, String queueId) {
-    String rowKey = queueId;
-    if (!queueId.contains(ROW_KEY_DELIMITER)) {
-      rowKey = buildQueueRowKey(serverName, queueId);
-    }
-    return getLogsInQueue(Bytes.toBytes(rowKey));
-  }
-
-  protected List<String> getLogsInQueue(byte[] rowKey) {
-    String errMsg = "Failed getting logs in queue queueId=" + Bytes.toString(rowKey);
-    try (Table replicationTable = getOrBlockOnReplicationTable()) {
-      Get getQueue = new Get(rowKey);
-      Result queue = replicationTable.get(getQueue);
-      if (queue == null || queue.isEmpty()) {
-        abortable.abort(errMsg, new ReplicationException(errMsg));
-        return null;
-      }
-      return readWALsFromResult(queue);
-    } catch (IOException e) {
-      abortable.abort(errMsg, e);
-      return null;
-    }
-  }
-
-  /**
-   * Read all of the WAL's from a queue into a list
-   *
-   * @param queue HBase query result containing the queue
-   * @return a list of all the WAL filenames
-   */
-  protected List<String> readWALsFromResult(Result queue) {
-    List<String> wals = new ArrayList<>();
-    Map<byte[], byte[]> familyMap = queue.getFamilyMap(CF_QUEUE);
-    for (byte[] cQualifier : familyMap.keySet()) {
-      // Ignore the meta data fields of the queue
-      if (Arrays.equals(cQualifier, COL_QUEUE_OWNER) || Arrays.equals(cQualifier,
-          COL_QUEUE_OWNER_HISTORY)) {
-        continue;
-      }
-      wals.add(Bytes.toString(cQualifier));
-    }
-    return wals;
-  }
-
-  /**
-   * Get the queue id's and meta data (Owner and History) for the queues belonging to the named
-   * server
-   *
-   * @param server name of the server
-   * @return a ResultScanner over the QueueIds belonging to the server
-   * @throws IOException
-   */
-  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));
-    scan.setFilter(filterMyQueues);
-    scan.addColumn(CF_QUEUE, COL_QUEUE_OWNER);
-    scan.addColumn(CF_QUEUE, COL_QUEUE_OWNER_HISTORY);
-    try (Table replicationTable = getOrBlockOnReplicationTable()) {
-      ResultScanner results = replicationTable.getScanner(scan);
-      return results;
-    }
-  }
-
-  /**
-   * Attempts to acquire the Replication Table. This operation will block until it is assigned by
-   * 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
-   */
-  protected Table getOrBlockOnReplicationTable() throws IOException {
-    // Sleep until the Replication Table becomes available
-    try {
-      replicationTableInitialized.await();
-    } catch (InterruptedException e) {
-      String errMsg = "Unable to acquire the Replication Table due to InterruptedException: " +
-          e.getMessage();
-      throw new InterruptedIOException(errMsg);
-    }
-    return getAndSetUpReplicationTable();
-  }
-
-  /**
-   * Creates a new copy of the Replication Table and sets up the proper Table time outs for it
-   *
-   * @return the Replication Table
-   * @throws IOException
-   */
-  private Table getAndSetUpReplicationTable() throws IOException {
-    Table replicationTable = connection.getTable(REPLICATION_TABLE_NAME);
-    setReplicationTableTimeOuts(replicationTable);
-    return replicationTable;
-  }
-
-  /**
-   * Builds the Replication Table in a background thread. Any method accessing the Replication Table
-   * should do so through getOrBlockOnReplicationTable()
-   *
-   * @return the Replication Table
-   * @throws IOException if the Replication Table takes too long to build
-   */
-  private void createReplicationTableInBackground() throws IOException {
-    executor.execute(new CreateReplicationTableWorker());
-  }
-
-  /**
-   * Attempts to build the Replication Table. Will continue blocking until we have a valid
-   * Table for the Replication Table.
-   */
-  private class CreateReplicationTableWorker implements Runnable {
-
-    private Admin admin;
-
-    @Override
-    public void run() {
-      try {
-        admin = connection.getAdmin();
-        if (!replicationTableExists()) {
-          createReplicationTable();
-        }
-        int maxRetries = conf.getInt("hbase.replication.queues.createtable.retries.number",
-            CLIENT_RETRIES);
-        RetryCounterFactory counterFactory = new RetryCounterFactory(maxRetries, RPC_TIMEOUT);
-        RetryCounter retryCounter = counterFactory.create();
-        while (!replicationTableExists()) {
-          retryCounter.sleepUntilNextRetry();
-          if (!retryCounter.shouldRetry()) {
-            throw new IOException("Unable to acquire the Replication Table");
-          }
-        }
-        replicationTableInitialized.countDown();
-      } catch (IOException | InterruptedException e) {
-        abortable.abort("Failed building Replication Table", e);
-      }
-    }
-
-    /**
-     * Create the replication table with the provided HColumnDescriptor REPLICATION_COL_DESCRIPTOR
-     * in TableBasedReplicationQueuesImpl
-     *
-     * @throws IOException
-     */
-    private void createReplicationTable() throws IOException {
-      HTableDescriptor replicationTableDescriptor = new HTableDescriptor(REPLICATION_TABLE_NAME);
-      replicationTableDescriptor.addFamily(REPLICATION_COL_DESCRIPTOR);
-      try {
-        admin.createTable(replicationTableDescriptor);
-      } catch (TableExistsException e) {
-        // In this case we can just continue as normal
-      }
-    }
-
-    /**
-     * Checks whether the Replication Table exists yet
-     *
-     * @return whether the Replication Table exists
-     * @throws IOException
-     */
-    private boolean replicationTableExists() {
-      try {
-        return admin.tableExists(REPLICATION_TABLE_NAME);
-      } catch (IOException e) {
-        return false;
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/264890a9/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationTrackerZKImpl.java
----------------------------------------------------------------------
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 2c522f6..5659e4b 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
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -54,6 +53,8 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
     super(zookeeper, conf, abortable);
     this.stopper = stopper;
     this.zookeeper.registerListener(new OtherRegionServerWatcher(this.zookeeper));
+    // watch the changes
+    refreshOtherRegionServersList(true);
   }
 
   @Override
@@ -71,7 +72,7 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
    */
   @Override
   public List<String> getListOfRegionServers() {
-    refreshOtherRegionServersList();
+    refreshOtherRegionServersList(false);
 
     List<String> list = null;
     synchronized (otherRegionServers) {
@@ -137,7 +138,7 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
       if (!path.startsWith(this.watcher.znodePaths.rsZNode)) {
         return false;
       }
-      return refreshOtherRegionServersList();
+      return refreshOtherRegionServersList(true);
     }
   }
 
@@ -157,8 +158,8 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
    * @return true if the local list of the other region servers was updated with the ZK data (even
    *         if it was empty), false if the data was missing in ZK
    */
-  private boolean refreshOtherRegionServersList() {
-    List<String> newRsList = getRegisteredRegionServers();
+  private boolean refreshOtherRegionServersList(boolean watch) {
+    List<String> newRsList = getRegisteredRegionServers(watch);
     if (newRsList == null) {
       return false;
     } else {
@@ -174,10 +175,14 @@ public class ReplicationTrackerZKImpl extends ReplicationStateZKBase implements
    * Get a list of all the other region servers in this cluster and set a watch
    * @return a list of server nanes
    */
-  private List<String> getRegisteredRegionServers() {
+  private List<String> getRegisteredRegionServers(boolean watch) {
     List<String> result = null;
     try {
-      result = ZKUtil.listChildrenAndWatchThem(this.zookeeper, this.zookeeper.znodePaths.rsZNode);
+      if (watch) {
+        result = ZKUtil.listChildrenAndWatchThem(this.zookeeper, this.zookeeper.znodePaths.rsZNode);
+      } else {
+        result = ZKUtil.listChildrenNoWatch(this.zookeeper, this.zookeeper.znodePaths.rsZNode);
+      }
     } catch (KeeperException e) {
       this.abortable.abort("Get list of registered region servers", e);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/264890a9/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
index 0275d52..41f50d8 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
@@ -54,6 +54,28 @@ import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTe
 
 /**
  * ZK based replication queue storage.
+ * <p>
+ * The base znode for each regionserver is the regionserver name. For example:
+ *
+ * <pre>
+ * /hbase/replication/rs/hostname.example.org,6020,1234
+ * </pre>
+ *
+ * Within this znode, the region server maintains a set of WAL replication queues. These queues are
+ * represented by child znodes named using there give queue id. For example:
+ *
+ * <pre>
+ * /hbase/replication/rs/hostname.example.org,6020,1234/1
+ * /hbase/replication/rs/hostname.example.org,6020,1234/2
+ * </pre>
+ *
+ * Each queue has one child znode for every WAL that still needs to be replicated. The value of
+ * these WAL child znodes is the latest position that has been replicated. This position is updated
+ * every time a WAL entry is replicated. For example:
+ *
+ * <pre>
+ * /hbase/replication/rs/hostname.example.org,6020,1234/1/23522342.23422 [VALUE: 254]
+ * </pre>
  */
 @InterfaceAudience.Private
 class ZKReplicationQueueStorage extends ZKReplicationStorageBase

http://git-wip-us.apache.org/repos/asf/hbase/blob/264890a9/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
index 8905d43..4afda5d 100644
--- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
@@ -42,9 +42,8 @@ import org.slf4j.LoggerFactory;
  */
 public abstract class TestReplicationStateBasic {
 
-  protected ReplicationQueues rq1;
-  protected ReplicationQueues rq2;
-  protected ReplicationQueues rq3;
+  private static final Logger LOG = LoggerFactory.getLogger(TestReplicationStateBasic.class);
+
   protected ReplicationQueueStorage rqs;
   protected ServerName server1 = ServerName.valueOf("hostname1.example.org", 1234, 12345);
   protected ServerName server2 = ServerName.valueOf("hostname2.example.org", 1234, 12345);
@@ -63,8 +62,6 @@ public abstract class TestReplicationStateBasic {
   protected static final int ZK_MAX_COUNT = 300;
   protected static final int ZK_SLEEP_INTERVAL = 100; // millis
 
-  private static final Logger LOG = LoggerFactory.getLogger(TestReplicationStateBasic.class);
-
   @Test
   public void testReplicationQueueStorage() throws ReplicationException {
     // Test methods with empty state
@@ -76,15 +73,13 @@ public abstract class TestReplicationStateBasic {
      * Set up data Two replicators: -- server1: three queues with 0, 1 and 2 log files each --
      * server2: zero queues
      */
-    rq1.init(server1.getServerName());
-    rq2.init(server2.getServerName());
-    rq1.addLog("qId1", "trash");
-    rq1.removeLog("qId1", "trash");
-    rq1.addLog("qId2", "filename1");
-    rq1.addLog("qId3", "filename2");
-    rq1.addLog("qId3", "filename3");
-    rq2.addLog("trash", "trash");
-    rq2.removeQueue("trash");
+    rqs.addWAL(server1, "qId1", "trash");
+    rqs.removeWAL(server1, "qId1", "trash");
+    rqs.addWAL(server1,"qId2", "filename1");
+    rqs.addWAL(server1,"qId3", "filename2");
+    rqs.addWAL(server1,"qId3", "filename3");
+    rqs.addWAL(server2,"trash", "trash");
+    rqs.removeQueue(server2,"trash");
 
     List<ServerName> reps = rqs.getListOfReplicators();
     assertEquals(2, reps.size());
@@ -105,62 +100,55 @@ public abstract class TestReplicationStateBasic {
     assertTrue(list.contains("qId3"));
   }
 
+  private void removeAllQueues(ServerName serverName) throws ReplicationException {
+    for (String queue: rqs.getAllQueues(serverName)) {
+      rqs.removeQueue(serverName, queue);
+    }
+  }
   @Test
   public void testReplicationQueues() throws ReplicationException {
-    rq1.init(server1.getServerName());
-    rq2.init(server2.getServerName());
-    rq3.init(server3.getServerName());
     // Initialize ReplicationPeer so we can add peers (we don't transfer lone queues)
     rp.init();
 
-    // 3 replicators should exist
-    assertEquals(3, rq1.getListOfReplicators().size());
-    rq1.removeQueue("bogus");
-    rq1.removeLog("bogus", "bogus");
-    rq1.removeAllQueues();
-    assertEquals(0, rq1.getAllQueues().size());
-    assertEquals(0, rq1.getLogPosition("bogus", "bogus"));
-    assertNull(rq1.getLogsInQueue("bogus"));
-    assertNull(rq1.getUnClaimedQueueIds(ServerName.valueOf("bogus", 1234, -1L).toString()));
-
-    rq1.setLogPosition("bogus", "bogus", 5L);
+    rqs.removeQueue(server1, "bogus");
+    rqs.removeWAL(server1, "bogus", "bogus");
+    removeAllQueues(server1);
+    assertEquals(0, rqs.getAllQueues(server1).size());
+    assertEquals(0, rqs.getWALPosition(server1, "bogus", "bogus"));
+    assertTrue(rqs.getWALsInQueue(server1, "bogus").isEmpty());
+    assertTrue(rqs.getAllQueues(ServerName.valueOf("bogus", 1234, 12345)).isEmpty());
 
     populateQueues();
 
-    assertEquals(3, rq1.getListOfReplicators().size());
-    assertEquals(0, rq2.getLogsInQueue("qId1").size());
-    assertEquals(5, rq3.getLogsInQueue("qId5").size());
-    assertEquals(0, rq3.getLogPosition("qId1", "filename0"));
-    rq3.setLogPosition("qId5", "filename4", 354L);
-    assertEquals(354L, rq3.getLogPosition("qId5", "filename4"));
+    assertEquals(3, rqs.getListOfReplicators().size());
+    assertEquals(0, rqs.getWALsInQueue(server2, "qId1").size());
+    assertEquals(5, rqs.getWALsInQueue(server3, "qId5").size());
+    assertEquals(0, rqs.getWALPosition(server3, "qId1", "filename0"));
+    rqs.setWALPosition(server3, "qId5", "filename4", 354L);
+    assertEquals(354L, rqs.getWALPosition(server3, "qId5", "filename4"));
 
-    assertEquals(5, rq3.getLogsInQueue("qId5").size());
-    assertEquals(0, rq2.getLogsInQueue("qId1").size());
-    assertEquals(0, rq1.getAllQueues().size());
-    assertEquals(1, rq2.getAllQueues().size());
-    assertEquals(5, rq3.getAllQueues().size());
+    assertEquals(5, rqs.getWALsInQueue(server3, "qId5").size());
+    assertEquals(0, rqs.getWALsInQueue(server2, "qId1").size());
+    assertEquals(0, rqs.getAllQueues(server1).size());
+    assertEquals(1, rqs.getAllQueues(server2).size());
+    assertEquals(5, rqs.getAllQueues(server3).size());
 
-    assertEquals(0, rq3.getUnClaimedQueueIds(server1.getServerName()).size());
-    rq3.removeReplicatorIfQueueIsEmpty(server1.getServerName());
-    assertEquals(2, rq3.getListOfReplicators().size());
+    assertEquals(0, rqs.getAllQueues(server1).size());
+    rqs.removeReplicatorIfQueueIsEmpty(server1);
+    assertEquals(2, rqs.getListOfReplicators().size());
 
-    List<String> queues = rq2.getUnClaimedQueueIds(server3.getServerName());
+    List<String> queues = rqs.getAllQueues(server3);
     assertEquals(5, queues.size());
     for (String queue : queues) {
-      rq2.claimQueue(server3.getServerName(), queue);
+      rqs.claimQueue(server3, queue, server2);
     }
-    rq2.removeReplicatorIfQueueIsEmpty(server3.getServerName());
-    assertEquals(1, rq2.getListOfReplicators().size());
-
-    // Try to claim our own queues
-    assertNull(rq2.getUnClaimedQueueIds(server2.getServerName()));
-    rq2.removeReplicatorIfQueueIsEmpty(server2.getServerName());
-
-    assertEquals(6, rq2.getAllQueues().size());
+    rqs.removeReplicatorIfQueueIsEmpty(server3);
+    assertEquals(1, rqs.getListOfReplicators().size());
 
-    rq2.removeAllQueues();
-
-    assertEquals(0, rq2.getListOfReplicators().size());
+    assertEquals(6, rqs.getAllQueues(server2).size());
+    removeAllQueues(server2);
+    rqs.removeReplicatorIfQueueIsEmpty(server2);
+    assertEquals(0, rqs.getListOfReplicators().size());
   }
 
   @Test
@@ -197,7 +185,6 @@ public abstract class TestReplicationStateBasic {
   @Test
   public void testHfileRefsReplicationQueues() throws ReplicationException, KeeperException {
     rp.init();
-    rq1.init(server1.getServerName());
 
     List<Pair<Path, Path>> files1 = new ArrayList<>(3);
     files1.add(new Pair<>(null, new Path("file_1")));
@@ -206,8 +193,8 @@ public abstract class TestReplicationStateBasic {
     assertTrue(rqs.getReplicableHFiles(ID_ONE).isEmpty());
     assertEquals(0, rqs.getAllPeersFromHFileRefsQueue().size());
     rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
-    rq1.addPeerToHFileRefs(ID_ONE);
-    rq1.addHFileRefs(ID_ONE, files1);
+    rqs.addPeerToHFileRefs(ID_ONE);
+    rqs.addHFileRefs(ID_ONE, files1);
     assertEquals(1, rqs.getAllPeersFromHFileRefsQueue().size());
     assertEquals(3, rqs.getReplicableHFiles(ID_ONE).size());
     List<String> hfiles2 = new ArrayList<>(files1.size());
@@ -215,43 +202,41 @@ public abstract class TestReplicationStateBasic {
       hfiles2.add(p.getSecond().getName());
     }
     String removedString = hfiles2.remove(0);
-    rq1.removeHFileRefs(ID_ONE, hfiles2);
+    rqs.removeHFileRefs(ID_ONE, hfiles2);
     assertEquals(1, rqs.getReplicableHFiles(ID_ONE).size());
     hfiles2 = new ArrayList<>(1);
     hfiles2.add(removedString);
-    rq1.removeHFileRefs(ID_ONE, hfiles2);
+    rqs.removeHFileRefs(ID_ONE, hfiles2);
     assertEquals(0, rqs.getReplicableHFiles(ID_ONE).size());
     rp.unregisterPeer(ID_ONE);
   }
 
   @Test
   public void testRemovePeerForHFileRefs() throws ReplicationException, KeeperException {
-    rq1.init(server1.getServerName());
-
     rp.init();
     rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
-    rq1.addPeerToHFileRefs(ID_ONE);
+    rqs.addPeerToHFileRefs(ID_ONE);
     rp.registerPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO));
-    rq1.addPeerToHFileRefs(ID_TWO);
+    rqs.addPeerToHFileRefs(ID_TWO);
 
     List<Pair<Path, Path>> files1 = new ArrayList<>(3);
     files1.add(new Pair<>(null, new Path("file_1")));
     files1.add(new Pair<>(null, new Path("file_2")));
     files1.add(new Pair<>(null, new Path("file_3")));
-    rq1.addHFileRefs(ID_ONE, files1);
-    rq1.addHFileRefs(ID_TWO, files1);
+    rqs.addHFileRefs(ID_ONE, files1);
+    rqs.addHFileRefs(ID_TWO, files1);
     assertEquals(2, rqs.getAllPeersFromHFileRefsQueue().size());
     assertEquals(3, rqs.getReplicableHFiles(ID_ONE).size());
     assertEquals(3, rqs.getReplicableHFiles(ID_TWO).size());
 
     rp.unregisterPeer(ID_ONE);
-    rq1.removePeerFromHFileRefs(ID_ONE);
+    rqs.removePeerFromHFileRefs(ID_ONE);
     assertEquals(1, rqs.getAllPeersFromHFileRefsQueue().size());
     assertTrue(rqs.getReplicableHFiles(ID_ONE).isEmpty());
     assertEquals(3, rqs.getReplicableHFiles(ID_TWO).size());
 
     rp.unregisterPeer(ID_TWO);
-    rq1.removePeerFromHFileRefs(ID_TWO);
+    rqs.removePeerFromHFileRefs(ID_TWO);
     assertEquals(0, rqs.getAllPeersFromHFileRefsQueue().size());
     assertTrue(rqs.getReplicableHFiles(ID_TWO).isEmpty());
   }
@@ -363,15 +348,15 @@ public abstract class TestReplicationStateBasic {
    * 3, 4, 5 log files respectively
    */
   protected void populateQueues() throws ReplicationException {
-    rq1.addLog("trash", "trash");
-    rq1.removeQueue("trash");
+    rqs.addWAL(server1, "trash", "trash");
+    rqs.removeQueue(server1, "trash");
 
-    rq2.addLog("qId1", "trash");
-    rq2.removeLog("qId1", "trash");
+    rqs.addWAL(server2, "qId1", "trash");
+    rqs.removeWAL(server2, "qId1", "trash");
 
     for (int i = 1; i < 6; i++) {
       for (int j = 0; j < i; j++) {
-        rq3.addLog("qId" + i, "filename" + j);
+        rqs.addWAL(server3, "qId" + i, "filename" + j);
       }
       // Add peers for the corresponding queues so they are not orphans
       rp.registerPeer("qId" + i,

http://git-wip-us.apache.org/repos/asf/hbase/blob/264890a9/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
index 5fe7c55..ac869d9 100644
--- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
@@ -17,10 +17,6 @@
  */
 package org.apache.hadoop.hbase.replication;
 
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
@@ -40,7 +36,6 @@ import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -54,7 +49,6 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
   private static HBaseZKTestingUtility utility;
   private static ZKWatcher zkw;
   private static String replicationZNode;
-  private ReplicationQueuesZKImpl rqZK;
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
@@ -84,23 +78,9 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
   @Before
   public void setUp() {
     zkTimeoutCount = 0;
-    WarnOnlyAbortable abortable = new WarnOnlyAbortable();
-    try {
-      rq1 = ReplicationFactory
-          .getReplicationQueues(new ReplicationQueuesArguments(conf, abortable, zkw));
-      rq2 = ReplicationFactory
-          .getReplicationQueues(new ReplicationQueuesArguments(conf, abortable, zkw));
-      rq3 = ReplicationFactory
-          .getReplicationQueues(new ReplicationQueuesArguments(conf, abortable, zkw));
-      rqs = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf);
-    } catch (Exception e) {
-      // This should not occur, because getReplicationQueues() only throws for
-      // TableBasedReplicationQueuesImpl
-      fail("ReplicationFactory.getReplicationQueues() threw an IO Exception");
-    }
-    rp = ReplicationFactory.getReplicationPeers(zkw, conf, zkw);
+    rqs = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf);
+    rp = ReplicationFactory.getReplicationPeers(zkw, conf, new WarnOnlyAbortable());
     OUR_KEY = ZKConfig.getZooKeeperClusterKey(conf);
-    rqZK = new ReplicationQueuesZKImpl(zkw, conf, abortable);
   }
 
   @After
@@ -113,23 +93,6 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
     utility.shutdownMiniZKCluster();
   }
 
-  @Test
-  public void testIsPeerPath_PathToParentOfPeerNode() {
-    assertFalse(rqZK.isPeerPath(rqZK.peersZNode));
-  }
-
-  @Test
-  public void testIsPeerPath_PathToChildOfPeerNode() {
-    String peerChild = ZNodePaths.joinZNode(ZNodePaths.joinZNode(rqZK.peersZNode, "1"), "child");
-    assertFalse(rqZK.isPeerPath(peerChild));
-  }
-
-  @Test
-  public void testIsPeerPath_ActualPeerPath() {
-    String peerPath = ZNodePaths.joinZNode(rqZK.peersZNode, "1");
-    assertTrue(rqZK.isPeerPath(peerPath));
-  }
-
   private static class WarnOnlyAbortable implements Abortable {
 
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/264890a9/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
index 1faaae3..ce03f1c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java
@@ -50,8 +50,6 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
 import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@@ -308,14 +306,10 @@ public class DumpReplicationQueues extends Configured implements Tool {
       boolean hdfs) throws Exception {
     ReplicationQueueStorage queueStorage;
     ReplicationPeers replicationPeers;
-    ReplicationQueues replicationQueues;
     ReplicationTracker replicationTracker;
-    ReplicationQueuesArguments replicationArgs =
-        new ReplicationQueuesArguments(getConf(), new WarnOnlyAbortable(), zkw);
     StringBuilder sb = new StringBuilder();
 
     queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, getConf());
-    replicationQueues = ReplicationFactory.getReplicationQueues(replicationArgs);
     replicationPeers =
         ReplicationFactory.getReplicationPeers(zkw, getConf(), queueStorage, connection);
     replicationTracker = ReplicationFactory.getReplicationTracker(zkw, replicationPeers, getConf(),
@@ -329,7 +323,6 @@ public class DumpReplicationQueues extends Configured implements Tool {
     }
     for (ServerName regionserver : regionservers) {
       List<String> queueIds = queueStorage.getAllQueues(regionserver);
-      replicationQueues.init(regionserver.getServerName());
       if (!liveRegionServers.contains(regionserver.getServerName())) {
         deadRegionServers.add(regionserver.getServerName());
       }
@@ -339,17 +332,17 @@ public class DumpReplicationQueues extends Configured implements Tool {
         if (!peerIds.contains(queueInfo.getPeerId())) {
           deletedQueues.add(regionserver + "/" + queueId);
           sb.append(
-            formatQueue(regionserver, replicationQueues, queueInfo, queueId, wals, true, hdfs));
+            formatQueue(regionserver, queueStorage, queueInfo, queueId, wals, true, hdfs));
         } else {
           sb.append(
-            formatQueue(regionserver, replicationQueues, queueInfo, queueId, wals, false, hdfs));
+            formatQueue(regionserver, queueStorage, queueInfo, queueId, wals, false, hdfs));
         }
       }
     }
     return sb.toString();
   }
 
-  private String formatQueue(ServerName regionserver, ReplicationQueues replicationQueues,
+  private String formatQueue(ServerName regionserver, ReplicationQueueStorage queueStorage,
       ReplicationQueueInfo queueInfo, String queueId, List<String> wals, boolean isDeleted,
       boolean hdfs) throws Exception {
     StringBuilder sb = new StringBuilder();
@@ -371,7 +364,7 @@ public class DumpReplicationQueues extends Configured implements Tool {
     peersQueueSize.addAndGet(queueInfo.getPeerId(), wals.size());
 
     for (String wal : wals) {
-      long position = replicationQueues.getLogPosition(queueInfo.getPeerId(), wal);
+      long position = queueStorage.getWALPosition(regionserver, queueInfo.getPeerId(), wal);
       sb.append("    Replication position for " + wal + ": " + (position > 0 ? position : "0"
           + " (not started or nothing to replicate)") + "\n");
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/264890a9/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
index bd191e3..e0c45d5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -29,15 +28,15 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Server;
 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.replication.ReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Class that handles the recovered source of a replication stream, which is transfered from
@@ -52,10 +51,10 @@ public class RecoveredReplicationSource extends ReplicationSource {
 
   @Override
   public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
-      ReplicationQueues replicationQueues, ReplicationPeers replicationPeers, Server server,
+      ReplicationQueueStorage queueStorage, ReplicationPeers replicationPeers, Server server,
       String peerClusterZnode, UUID clusterId, ReplicationEndpoint replicationEndpoint,
       WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException {
-    super.init(conf, fs, manager, replicationQueues, replicationPeers, server, peerClusterZnode,
+    super.init(conf, fs, manager, queueStorage, replicationPeers, server, peerClusterZnode,
       clusterId, replicationEndpoint, walFileLengthProvider, metrics);
     this.actualPeerId = this.replicationQueueInfo.getPeerId();
   }
@@ -64,7 +63,7 @@ public class RecoveredReplicationSource extends ReplicationSource {
   protected void tryStartNewShipper(String walGroupId, PriorityBlockingQueue<Path> queue) {
     final RecoveredReplicationSourceShipper worker =
         new RecoveredReplicationSourceShipper(conf, walGroupId, queue, this,
-            this.replicationQueues);
+            this.queueStorage);
     ReplicationSourceShipper extant = workerThreads.putIfAbsent(walGroupId, worker);
     if (extant != null) {
       LOG.debug("Someone has beat us to start a worker thread for wal group " + walGroupId);

http://git-wip-us.apache.org/repos/asf/hbase/blob/264890a9/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
index 630b90b..fb365bc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
@@ -23,13 +23,13 @@ import java.util.concurrent.PriorityBlockingQueue;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.replication.ReplicationException;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.WALEntryBatch;
 import org.apache.hadoop.hbase.util.Threads;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  *  Used by a {@link RecoveredReplicationSource}.
@@ -40,14 +40,14 @@ public class RecoveredReplicationSourceShipper extends ReplicationSourceShipper
       LoggerFactory.getLogger(RecoveredReplicationSourceShipper.class);
 
   protected final RecoveredReplicationSource source;
-  private final ReplicationQueues replicationQueues;
+  private final ReplicationQueueStorage replicationQueues;
 
   public RecoveredReplicationSourceShipper(Configuration conf, String walGroupId,
       PriorityBlockingQueue<Path> queue, RecoveredReplicationSource source,
-      ReplicationQueues replicationQueues) {
+      ReplicationQueueStorage queueStorage) {
     super(conf, walGroupId, queue, source);
     this.source = source;
-    this.replicationQueues = replicationQueues;
+    this.replicationQueues = queueStorage;
   }
 
   @Override
@@ -116,11 +116,11 @@ public class RecoveredReplicationSourceShipper extends ReplicationSourceShipper
     long startPosition = 0;
     String peerClusterZnode = source.getPeerClusterZnode();
     try {
-      startPosition = this.replicationQueues.getLogPosition(peerClusterZnode,
-        this.queue.peek().getName());
+      startPosition = this.replicationQueues.getWALPosition(source.getServerWALsBelongTo(),
+        peerClusterZnode, this.queue.peek().getName());
       if (LOG.isTraceEnabled()) {
-        LOG.trace("Recovered queue started with log " + this.queue.peek() + " at position "
-            + startPosition);
+        LOG.trace("Recovered queue started with log " + this.queue.peek() + " at position " +
+          startPosition);
       }
     } catch (ReplicationException e) {
       terminate("Couldn't get the position of this recovered queue " + peerClusterZnode, e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/264890a9/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
index 571ee75..6d871cc 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/Replication.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -29,11 +28,6 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.hadoop.hbase.wal.WALKey;
-import org.apache.hadoop.hbase.wal.WALKeyImpl;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -43,26 +37,32 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
-import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.regionserver.ReplicationSinkService;
 import org.apache.hadoop.hbase.regionserver.ReplicationSourceService;
+import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
-import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
-import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
 import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
+import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.WALEntry;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
 
 /**
  * Gateway to Replication.  Used by {@link org.apache.hadoop.hbase.regionserver.HRegionServer}.
@@ -74,7 +74,7 @@ public class Replication implements
       LoggerFactory.getLogger(Replication.class);
   private boolean replicationForBulkLoadData;
   private ReplicationSourceManager replicationManager;
-  private ReplicationQueues replicationQueues;
+  private ReplicationQueueStorage queueStorage;
   private ReplicationPeers replicationPeers;
   private ReplicationTracker replicationTracker;
   private Configuration conf;
@@ -127,10 +127,8 @@ public class Replication implements
     }
 
     try {
-      this.replicationQueues =
-          ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, this.server,
-            server.getZooKeeper()));
-      this.replicationQueues.init(this.server.getServerName().toString());
+      this.queueStorage =
+          ReplicationStorageFactory.getReplicationQueueStorage(server.getZooKeeper(), conf);
       this.replicationPeers =
           ReplicationFactory.getReplicationPeers(server.getZooKeeper(), this.conf, this.server);
       this.replicationPeers.init();
@@ -147,7 +145,7 @@ public class Replication implements
       throw new IOException("Could not read cluster id", ke);
     }
     this.replicationManager =
-        new ReplicationSourceManager(replicationQueues, replicationPeers, replicationTracker, conf,
+        new ReplicationSourceManager(queueStorage, replicationPeers, replicationTracker, conf,
             this.server, fs, logDir, oldLogDir, clusterId, walFileLengthProvider);
     this.statsThreadPeriod =
         this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60);

http://git-wip-us.apache.org/repos/asf/hbase/blob/264890a9/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index 19ea240..fef9054 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -41,9 +40,6 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.regionserver.RSRpcServices;
 import org.apache.hadoop.hbase.replication.ChainWALEntryFilter;
 import org.apache.hadoop.hbase.replication.ClusterMarkingEntryFilter;
@@ -52,7 +48,7 @@ import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeer;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.SystemTableWALEntryFilter;
 import org.apache.hadoop.hbase.replication.WALEntryFilter;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -60,6 +56,10 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 
 
@@ -83,7 +83,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
   private Map<String, PriorityBlockingQueue<Path>> queues = new HashMap<>();
   // per group queue size, keep no more than this number of logs in each wal group
   protected int queueSizePerGroup;
-  protected ReplicationQueues replicationQueues;
+  protected ReplicationQueueStorage queueStorage;
   private ReplicationPeers replicationPeers;
 
   protected Configuration conf;
@@ -148,7 +148,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
    */
   @Override
   public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
-      ReplicationQueues replicationQueues, ReplicationPeers replicationPeers, Server server,
+      ReplicationQueueStorage queueStorage, ReplicationPeers replicationPeers, Server server,
       String peerClusterZnode, UUID clusterId, ReplicationEndpoint replicationEndpoint,
       WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException {
     this.server = server;
@@ -161,7 +161,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
     this.maxRetriesMultiplier =
         this.conf.getInt("replication.source.maxretriesmultiplier", 300); // 5 minutes @ 1 sec per
     this.queueSizePerGroup = this.conf.getInt("hbase.regionserver.maxlogs", 32);
-    this.replicationQueues = replicationQueues;
+    this.queueStorage = queueStorage;
     this.replicationPeers = replicationPeers;
     this.manager = manager;
     this.fs = fs;
@@ -229,7 +229,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
       List<String> tableCfs = tableCFMap.get(tableName);
       if (tableCFMap.containsKey(tableName)
           && (tableCfs == null || tableCfs.contains(Bytes.toString(family)))) {
-        this.replicationQueues.addHFileRefs(peerId, pairs);
+        this.queueStorage.addHFileRefs(peerId, pairs);
         metrics.incrSizeOfHFileRefsQueue(pairs.size());
       } else {
         LOG.debug("HFiles will not be replicated belonging to the table " + tableName + " family "
@@ -238,7 +238,7 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf
     } else {
       // user has explicitly not defined any table cfs for replication, means replicate all the
       // data
-      this.replicationQueues.addHFileRefs(peerId, pairs);
+      this.queueStorage.addHFileRefs(peerId, pairs);
       metrics.incrSizeOfHFileRefsQueue(pairs.size());
     }
   }


[16/26] hbase git commit: HBASE-19536 Client side changes for moving peer modification from zk watcher to procedure

Posted by zh...@apache.org.
HBASE-19536 Client side changes for moving peer modification from zk watcher to procedure

Signed-off-by: zhangduo <zh...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/d77c7449
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/d77c7449
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/d77c7449

Branch: refs/heads/HBASE-19397
Commit: d77c7449aa34aa24cfe0333050899381b230dc0d
Parents: 4567906
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Dec 19 15:50:57 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Dec 28 08:51:59 2017 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Admin.java   |  87 ++++++++++-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  | 149 ++++++++++++++-----
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java |  82 +++++-----
 3 files changed, 238 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d77c7449/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
index ff2722e..cf8e198 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java
@@ -2463,7 +2463,7 @@ public interface Admin extends Abortable, Closeable {
   /**
    * Add a new replication peer for replicating data to slave cluster.
    * @param peerId a short name that identifies the peer
-   * @param peerConfig configuration for the replication slave cluster
+   * @param peerConfig configuration for the replication peer
    * @throws IOException if a remote or network exception occurs
    */
   default void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig)
@@ -2474,7 +2474,7 @@ public interface Admin extends Abortable, Closeable {
   /**
    * Add a new replication peer for replicating data to slave cluster.
    * @param peerId a short name that identifies the peer
-   * @param peerConfig configuration for the replication slave cluster
+   * @param peerConfig configuration for the replication peer
    * @param enabled peer state, true if ENABLED and false if DISABLED
    * @throws IOException if a remote or network exception occurs
    */
@@ -2482,6 +2482,37 @@ public interface Admin extends Abortable, Closeable {
       throws IOException;
 
   /**
+   * Add a new replication peer but does not block and wait for it.
+   * <p>
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
+   * ExecutionException if there was an error while executing the operation or TimeoutException in
+   * case the wait timeout was not long enough to allow the operation to complete.
+   * @param peerId a short name that identifies the peer
+   * @param peerConfig configuration for the replication peer
+   * @return the result of the async operation
+   * @throws IOException IOException if a remote or network exception occurs
+   */
+  default Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig)
+      throws IOException {
+    return addReplicationPeerAsync(peerId, peerConfig, true);
+  }
+
+  /**
+   * Add a new replication peer but does not block and wait for it.
+   * <p>
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
+   * ExecutionException if there was an error while executing the operation or TimeoutException in
+   * case the wait timeout was not long enough to allow the operation to complete.
+   * @param peerId a short name that identifies the peer
+   * @param peerConfig configuration for the replication peer
+   * @param enabled peer state, true if ENABLED and false if DISABLED
+   * @return the result of the async operation
+   * @throws IOException IOException if a remote or network exception occurs
+   */
+  Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig,
+      boolean enabled) throws IOException;
+
+  /**
    * Remove a peer and stop the replication.
    * @param peerId a short name that identifies the peer
    * @throws IOException if a remote or network exception occurs
@@ -2489,6 +2520,18 @@ public interface Admin extends Abortable, Closeable {
   void removeReplicationPeer(String peerId) throws IOException;
 
   /**
+   * Remove a replication peer but does not block and wait for it.
+   * <p>
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
+   * ExecutionException if there was an error while executing the operation or TimeoutException in
+   * case the wait timeout was not long enough to allow the operation to complete.
+   * @param peerId a short name that identifies the peer
+   * @return the result of the async operation
+   * @throws IOException IOException if a remote or network exception occurs
+   */
+  Future<Void> removeReplicationPeerAsync(String peerId) throws IOException;
+
+  /**
    * Restart the replication stream to the specified peer.
    * @param peerId a short name that identifies the peer
    * @throws IOException if a remote or network exception occurs
@@ -2496,6 +2539,18 @@ public interface Admin extends Abortable, Closeable {
   void enableReplicationPeer(String peerId) throws IOException;
 
   /**
+   * Enable a replication peer but does not block and wait for it.
+   * <p>
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
+   * ExecutionException if there was an error while executing the operation or TimeoutException in
+   * case the wait timeout was not long enough to allow the operation to complete.
+   * @param peerId a short name that identifies the peer
+   * @return the result of the async operation
+   * @throws IOException IOException if a remote or network exception occurs
+   */
+  Future<Void> enableReplicationPeerAsync(String peerId) throws IOException;
+
+  /**
    * Stop the replication stream to the specified peer.
    * @param peerId a short name that identifies the peer
    * @throws IOException if a remote or network exception occurs
@@ -2503,6 +2558,18 @@ public interface Admin extends Abortable, Closeable {
   void disableReplicationPeer(String peerId) throws IOException;
 
   /**
+   * Disable a replication peer but does not block and wait for it.
+   * <p>
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
+   * ExecutionException if there was an error while executing the operation or TimeoutException in
+   * case the wait timeout was not long enough to allow the operation to complete.
+   * @param peerId a short name that identifies the peer
+   * @return the result of the async operation
+   * @throws IOException IOException if a remote or network exception occurs
+   */
+  Future<Void> disableReplicationPeerAsync(String peerId) throws IOException;
+
+  /**
    * Returns the configured ReplicationPeerConfig for the specified peer.
    * @param peerId a short name that identifies the peer
    * @return ReplicationPeerConfig for the peer
@@ -2513,13 +2580,27 @@ public interface Admin extends Abortable, Closeable {
   /**
    * Update the peerConfig for the specified peer.
    * @param peerId a short name that identifies the peer
-   * @param peerConfig new config for the peer
+   * @param peerConfig new config for the replication peer
    * @throws IOException if a remote or network exception occurs
    */
   void updateReplicationPeerConfig(String peerId,
       ReplicationPeerConfig peerConfig) throws IOException;
 
   /**
+   * Update the peerConfig for the specified peer but does not block and wait for it.
+   * <p>
+   * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw
+   * ExecutionException if there was an error while executing the operation or TimeoutException in
+   * case the wait timeout was not long enough to allow the operation to complete.
+   * @param peerId a short name that identifies the peer
+   * @param peerConfig new config for the replication peer
+   * @return the result of the async operation
+   * @throws IOException IOException if a remote or network exception occurs
+   */
+  Future<Void> updateReplicationPeerConfigAsync(String peerId, ReplicationPeerConfig peerConfig)
+      throws IOException;
+
+  /**
    * Append the replicable table column family config from the specified peer.
    * @param id a short that identifies the cluster
    * @param tableCfs A map from tableName to column family names

http://git-wip-us.apache.org/repos/asf/hbase/blob/d77c7449/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 63310e6..b42b556 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -1,4 +1,4 @@
-/*
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -39,6 +39,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
@@ -198,7 +199,12 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTa
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.TruncateTableResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.UnassignRegionRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.AddReplicationPeerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.DisableReplicationPeerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.EnableReplicationPeerResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
 
 /**
@@ -3778,6 +3784,25 @@ public class HBaseAdmin implements Admin {
     }
   }
 
+  @InterfaceAudience.Private
+  @InterfaceStability.Evolving
+  private static class ReplicationFuture extends ProcedureFuture<Void> {
+    private final String peerId;
+    private final Supplier<String> getOperation;
+
+    public ReplicationFuture(HBaseAdmin admin, String peerId, Long procId,
+        Supplier<String> getOperation) {
+      super(admin, procId);
+      this.peerId = peerId;
+      this.getOperation = getOperation;
+    }
+
+    @Override
+    public String toString() {
+      return "Operation: " + getOperation.get() + ", peerId: " + peerId;
+    }
+  }
+
   @Override
   public List<SecurityCapability> getSecurityCapabilities() throws IOException {
     try {
@@ -3850,50 +3875,82 @@ public class HBaseAdmin implements Admin {
   @Override
   public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
       throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        master.addReplicationPeer(getRpcController(),
-          RequestConverter.buildAddReplicationPeerRequest(peerId, peerConfig, enabled));
-        return null;
-      }
-    });
+    get(addReplicationPeerAsync(peerId, peerConfig, enabled), this.syncWaitTimeout,
+      TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig,
+      boolean enabled) throws IOException {
+    AddReplicationPeerResponse response = executeCallable(
+      new MasterCallable<AddReplicationPeerResponse>(getConnection(), getRpcControllerFactory()) {
+        @Override
+        protected AddReplicationPeerResponse rpcCall() throws Exception {
+          return master.addReplicationPeer(getRpcController(),
+            RequestConverter.buildAddReplicationPeerRequest(peerId, peerConfig, enabled));
+        }
+      });
+    return new ReplicationFuture(this, peerId, response.getProcId(), () -> "ADD_REPLICATION_PEER");
   }
 
   @Override
   public void removeReplicationPeer(String peerId) throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        master.removeReplicationPeer(getRpcController(),
-          RequestConverter.buildRemoveReplicationPeerRequest(peerId));
-        return null;
-      }
-    });
+    get(removeReplicationPeerAsync(peerId), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> removeReplicationPeerAsync(String peerId) throws IOException {
+    RemoveReplicationPeerResponse response =
+      executeCallable(new MasterCallable<RemoveReplicationPeerResponse>(getConnection(),
+          getRpcControllerFactory()) {
+        @Override
+        protected RemoveReplicationPeerResponse rpcCall() throws Exception {
+          return master.removeReplicationPeer(getRpcController(),
+            RequestConverter.buildRemoveReplicationPeerRequest(peerId));
+        }
+      });
+    return new ReplicationFuture(this, peerId, response.getProcId(),
+      () -> "REMOVE_REPLICATION_PEER");
   }
 
   @Override
   public void enableReplicationPeer(final String peerId) throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        master.enableReplicationPeer(getRpcController(),
-          RequestConverter.buildEnableReplicationPeerRequest(peerId));
-        return null;
-      }
-    });
+    get(enableReplicationPeerAsync(peerId), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> enableReplicationPeerAsync(final String peerId) throws IOException {
+    EnableReplicationPeerResponse response =
+      executeCallable(new MasterCallable<EnableReplicationPeerResponse>(getConnection(),
+          getRpcControllerFactory()) {
+        @Override
+        protected EnableReplicationPeerResponse rpcCall() throws Exception {
+          return master.enableReplicationPeer(getRpcController(),
+            RequestConverter.buildEnableReplicationPeerRequest(peerId));
+        }
+      });
+    return new ReplicationFuture(this, peerId, response.getProcId(),
+      () -> "ENABLE_REPLICATION_PEER");
   }
 
   @Override
   public void disableReplicationPeer(final String peerId) throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        master.disableReplicationPeer(getRpcController(),
-          RequestConverter.buildDisableReplicationPeerRequest(peerId));
-        return null;
-      }
-    });
+    get(disableReplicationPeerAsync(peerId), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> disableReplicationPeerAsync(final String peerId) throws IOException {
+    DisableReplicationPeerResponse response =
+      executeCallable(new MasterCallable<DisableReplicationPeerResponse>(getConnection(),
+          getRpcControllerFactory()) {
+        @Override
+        protected DisableReplicationPeerResponse rpcCall() throws Exception {
+          return master.disableReplicationPeer(getRpcController(),
+            RequestConverter.buildDisableReplicationPeerRequest(peerId));
+        }
+      });
+    return new ReplicationFuture(this, peerId, response.getProcId(),
+      () -> "DISABLE_REPLICATION_PEER");
   }
 
   @Override
@@ -3912,14 +3969,24 @@ public class HBaseAdmin implements Admin {
   @Override
   public void updateReplicationPeerConfig(final String peerId,
       final ReplicationPeerConfig peerConfig) throws IOException {
-    executeCallable(new MasterCallable<Void>(getConnection(), getRpcControllerFactory()) {
-      @Override
-      protected Void rpcCall() throws Exception {
-        master.updateReplicationPeerConfig(getRpcController(),
-          RequestConverter.buildUpdateReplicationPeerConfigRequest(peerId, peerConfig));
-        return null;
-      }
-    });
+    get(updateReplicationPeerConfigAsync(peerId, peerConfig), this.syncWaitTimeout,
+      TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> updateReplicationPeerConfigAsync(final String peerId,
+      final ReplicationPeerConfig peerConfig) throws IOException {
+    UpdateReplicationPeerConfigResponse response =
+      executeCallable(new MasterCallable<UpdateReplicationPeerConfigResponse>(getConnection(),
+          getRpcControllerFactory()) {
+        @Override
+        protected UpdateReplicationPeerConfigResponse rpcCall() throws Exception {
+          return master.updateReplicationPeerConfig(getRpcController(),
+            RequestConverter.buildUpdateReplicationPeerConfigRequest(peerId, peerConfig));
+        }
+      });
+    return new ReplicationFuture(this, peerId, response.getProcId(),
+      () -> "UPDATE_REPLICATION_PEER_CONFIG");
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/d77c7449/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index 37d5330..3bd38fd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -36,6 +36,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.BiConsumer;
 import java.util.function.Function;
+import java.util.function.Supplier;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
@@ -1522,47 +1523,34 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   @Override
   public CompletableFuture<Void> addReplicationPeer(String peerId,
       ReplicationPeerConfig peerConfig, boolean enabled) {
-    return this
-        .<Void> newMasterCaller()
-        .action(
-          (controller, stub) -> this
-              .<AddReplicationPeerRequest, AddReplicationPeerResponse, Void> call(controller, stub,
-                RequestConverter.buildAddReplicationPeerRequest(peerId, peerConfig, enabled), (s,
-                    c, req, done) -> s.addReplicationPeer(c, req, done), (resp) -> null)).call();
+    return this.<AddReplicationPeerRequest, AddReplicationPeerResponse> procedureCall(
+      RequestConverter.buildAddReplicationPeerRequest(peerId, peerConfig, enabled),
+      (s, c, req, done) -> s.addReplicationPeer(c, req, done), (resp) -> resp.getProcId(),
+      new ReplicationProcedureBiConsumer(peerId, () -> "ADD_REPLICATION_PEER"));
   }
 
   @Override
   public CompletableFuture<Void> removeReplicationPeer(String peerId) {
-    return this
-        .<Void> newMasterCaller()
-        .action(
-          (controller, stub) -> this
-              .<RemoveReplicationPeerRequest, RemoveReplicationPeerResponse, Void> call(controller,
-                stub, RequestConverter.buildRemoveReplicationPeerRequest(peerId),
-                (s, c, req, done) -> s.removeReplicationPeer(c, req, done), (resp) -> null)).call();
+    return this.<RemoveReplicationPeerRequest, RemoveReplicationPeerResponse> procedureCall(
+      RequestConverter.buildRemoveReplicationPeerRequest(peerId),
+      (s, c, req, done) -> s.removeReplicationPeer(c, req, done), (resp) -> resp.getProcId(),
+      new ReplicationProcedureBiConsumer(peerId, () -> "REMOVE_REPLICATION_PEER"));
   }
 
   @Override
   public CompletableFuture<Void> enableReplicationPeer(String peerId) {
-    return this
-        .<Void> newMasterCaller()
-        .action(
-          (controller, stub) -> this
-              .<EnableReplicationPeerRequest, EnableReplicationPeerResponse, Void> call(controller,
-                stub, RequestConverter.buildEnableReplicationPeerRequest(peerId),
-                (s, c, req, done) -> s.enableReplicationPeer(c, req, done), (resp) -> null)).call();
+    return this.<EnableReplicationPeerRequest, EnableReplicationPeerResponse> procedureCall(
+      RequestConverter.buildEnableReplicationPeerRequest(peerId),
+      (s, c, req, done) -> s.enableReplicationPeer(c, req, done), (resp) -> resp.getProcId(),
+      new ReplicationProcedureBiConsumer(peerId, () -> "ENABLE_REPLICATION_PEER"));
   }
 
   @Override
   public CompletableFuture<Void> disableReplicationPeer(String peerId) {
-    return this
-        .<Void> newMasterCaller()
-        .action(
-          (controller, stub) -> this
-              .<DisableReplicationPeerRequest, DisableReplicationPeerResponse, Void> call(
-                controller, stub, RequestConverter.buildDisableReplicationPeerRequest(peerId), (s,
-                    c, req, done) -> s.disableReplicationPeer(c, req, done), (resp) -> null))
-        .call();
+    return this.<DisableReplicationPeerRequest, DisableReplicationPeerResponse> procedureCall(
+      RequestConverter.buildDisableReplicationPeerRequest(peerId),
+      (s, c, req, done) -> s.disableReplicationPeer(c, req, done), (resp) -> resp.getProcId(),
+      new ReplicationProcedureBiConsumer(peerId, () -> "DISABLE_REPLICATION_PEER"));
   }
 
   @Override
@@ -1581,13 +1569,11 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   public CompletableFuture<Void> updateReplicationPeerConfig(String peerId,
       ReplicationPeerConfig peerConfig) {
     return this
-        .<Void> newMasterCaller()
-        .action(
-          (controller, stub) -> this
-              .<UpdateReplicationPeerConfigRequest, UpdateReplicationPeerConfigResponse, Void> call(
-                controller, stub, RequestConverter.buildUpdateReplicationPeerConfigRequest(peerId,
-                  peerConfig), (s, c, req, done) -> s.updateReplicationPeerConfig(c, req, done), (
-                    resp) -> null)).call();
+        .<UpdateReplicationPeerConfigRequest, UpdateReplicationPeerConfigResponse> procedureCall(
+          RequestConverter.buildUpdateReplicationPeerConfigRequest(peerId, peerConfig),
+          (s, c, req, done) -> s.updateReplicationPeerConfig(c, req, done),
+          (resp) -> resp.getProcId(),
+          new ReplicationProcedureBiConsumer(peerId, () -> "UPDATE_REPLICATION_PEER_CONFIG"));
   }
 
   @Override
@@ -2546,6 +2532,30 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
   }
 
+  private class ReplicationProcedureBiConsumer extends ProcedureBiConsumer {
+    private final String peerId;
+    private final Supplier<String> getOperation;
+
+    ReplicationProcedureBiConsumer(String peerId, Supplier<String> getOperation) {
+      this.peerId = peerId;
+      this.getOperation = getOperation;
+    }
+
+    String getDescription() {
+      return "Operation: " + getOperation.get() + ", peerId: " + peerId;
+    }
+
+    @Override
+    void onFinished() {
+      LOG.info(getDescription() + " completed");
+    }
+
+    @Override
+    void onError(Throwable error) {
+      LOG.info(getDescription() + " failed with " + error.getMessage());
+    }
+  }
+
   private CompletableFuture<Void> waitProcedureResult(CompletableFuture<Long> procFuture) {
     CompletableFuture<Void> future = new CompletableFuture<>();
     procFuture.whenComplete((procId, error) -> {


[07/26] hbase git commit: HBASE-19644 add the checkstyle rule to reject the illegal imports

Posted by zh...@apache.org.
HBASE-19644 add the checkstyle rule to reject the illegal imports


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/01b1f48c
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/01b1f48c
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/01b1f48c

Branch: refs/heads/HBASE-19397
Commit: 01b1f48ccd93e1b5173c3a6593b428ed5a014961
Parents: 16cd916
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Thu Dec 28 04:10:42 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Thu Dec 28 04:10:42 2017 +0800

----------------------------------------------------------------------
 .../TestIncrementalBackupMergeWithFailures.java |  3 +-
 .../src/main/resources/hbase/checkstyle.xml     |  4 ++-
 .../client/AsyncBufferedMutatorBuilderImpl.java |  5 ++--
 .../hbase/client/ConnectionImplementation.java  |  6 ++--
 .../apache/hadoop/hbase/client/MultiAction.java |  7 +----
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java |  4 +--
 .../client/replication/ReplicationAdmin.java    | 10 +++----
 .../apache/hadoop/hbase/master/RegionState.java |  3 +-
 .../hadoop/hbase/util/ByteBufferArray.java      |  3 +-
 .../regionserver/MetricsTableLatenciesImpl.java |  3 +-
 .../apache/hadoop/hbase/http/HttpServer.java    | 29 +++++++++-----------
 .../hbase/procedure2/ProcedureScheduler.java    |  3 +-
 .../hadoop/hbase/rsgroup/TestRSGroupsBase.java  |  2 +-
 .../hbase/coprocessor/ObserverContextImpl.java  |  4 +--
 .../hbase/io/hfile/bucket/BucketCache.java      | 10 +++----
 .../hbase/io/hfile/bucket/FileIOEngine.java     | 15 +++++-----
 .../master/balancer/StochasticLoadBalancer.java |  4 +--
 .../master/procedure/RecoverMetaProcedure.java  |  3 +-
 .../regionserver/VersionedSegmentsList.java     |  5 ++--
 .../hbase/wal/NettyAsyncFSWALConfigHelper.java  | 10 +++----
 .../hbase/client/TestAsyncClusterAdminApi.java  |  5 ++--
 .../hadoop/hbase/filter/TestFilterList.java     |  4 +--
 .../hbase/io/hfile/bucket/TestBucketCache.java  |  9 +++---
 .../spark/IntegrationTestSparkBulkLoad.java     |  9 ++----
 24 files changed, 69 insertions(+), 91 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/01b1f48c/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java
index 50ca1d9..0aa7f98 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/TestIncrementalBackupMergeWithFailures.java
@@ -25,7 +25,6 @@ import static org.junit.Assert.assertTrue;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -49,7 +48,7 @@ import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 
 @Category(LargeTests.class)
 public class TestIncrementalBackupMergeWithFailures extends TestBackupBase {

http://git-wip-us.apache.org/repos/asf/hbase/blob/01b1f48c/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml
----------------------------------------------------------------------
diff --git a/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml b/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml
index 2240096..bdfea28 100644
--- a/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml
+++ b/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml
@@ -76,7 +76,9 @@
     <module name="UnusedImports">
       <property name="processJavadoc" value="true"/>
     </module>
-
+    <module name="IllegalImport">
+      <property name="illegalPkgs" value="com.google.common, io.netty, org.apache.curator.shaded, org.apache.htrace.shaded"/>
+    </module>
     <!-- Javadoc Checks
     http://checkstyle.sourceforge.net/config_javadoc.html -->
     <module name="JavadocTagContinuationIndentation">

http://git-wip-us.apache.org/repos/asf/hbase/blob/01b1f48c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorBuilderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorBuilderImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorBuilderImpl.java
index a44bafa..9880ba5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorBuilderImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutatorBuilderImpl.java
@@ -17,12 +17,11 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import com.google.common.base.Preconditions;
-
 import java.util.concurrent.TimeUnit;
-
 import org.apache.yetus.audience.InterfaceAudience;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
+
 /**
  * The implementation of {@link AsyncBufferedMutatorBuilder}.
  */

http://git-wip-us.apache.org/repos/asf/hbase/blob/01b1f48c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 1f34dba..f6c207a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -25,10 +25,7 @@ import static org.apache.hadoop.hbase.client.MetricsConnection.CLIENT_SIDE_METRI
 import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
 import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsentEx;
 
-import com.google.common.base.Throwables;
-
 import edu.umd.cs.findbugs.annotations.Nullable;
-
 import java.io.Closeable;
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -48,7 +45,6 @@ import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.ReentrantLock;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CallQueueTooBigException;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
@@ -83,7 +79,9 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Throwables;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/01b1f48c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiAction.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiAction.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiAction.java
index e893099..1934021 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiAction.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiAction.java
@@ -20,19 +20,14 @@ package org.apache.hadoop.hbase.client;
 
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
 import java.util.TreeMap;
-import java.util.function.Consumer;
-import java.util.function.Predicate;
-
-import com.google.common.collect.Iterables;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Container for Actions (i.e. Get, Delete, or Put), which are grouped by

http://git-wip-us.apache.org/repos/asf/hbase/blob/01b1f48c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index dac83f3..37d5330 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.client;
 
 import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
 import com.google.protobuf.Message;
 import com.google.protobuf.RpcChannel;
 import java.io.IOException;
@@ -85,6 +83,8 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
 import org.apache.hadoop.hbase.shaded.io.netty.util.HashedWheelTimer;
 import org.apache.hadoop.hbase.shaded.io.netty.util.Timeout;

http://git-wip-us.apache.org/repos/asf/hbase/blob/01b1f48c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
index ccfc444..52a68fe 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java
@@ -27,22 +27,22 @@ import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 import java.util.regex.Pattern;
-
-import org.apache.curator.shaded.com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 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.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 
 /**
  * <p>

http://git-wip-us.apache.org/repos/asf/hbase/blob/01b1f48c/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
index 55a68fc..4055242 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
@@ -17,14 +17,13 @@
  */
 package org.apache.hadoop.hbase.master;
 
-import com.google.common.annotations.VisibleForTesting;
 import java.util.Date;
-
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/01b1f48c/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java
index 6782de6..aec7a18 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferArray.java
@@ -27,7 +27,6 @@ import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
-
 import org.apache.hadoop.hbase.nio.ByteBuff;
 import org.apache.hadoop.hbase.nio.MultiByteBuff;
 import org.apache.hadoop.hbase.nio.SingleByteBuff;
@@ -36,7 +35,7 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 
 /**
  * This class manages an array of ByteBuffers with a default size 4MB. These

http://git-wip-us.apache.org/repos/asf/hbase/blob/01b1f48c/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableLatenciesImpl.java
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableLatenciesImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableLatenciesImpl.java
index 30b3720..1396b04 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableLatenciesImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableLatenciesImpl.java
@@ -17,14 +17,13 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import java.util.HashMap;
-
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
 import org.apache.hadoop.metrics2.MetricHistogram;
 import org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry;
 import org.apache.yetus.audience.InterfaceAudience;
 
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 
 /**
  * Implementation of {@link MetricsTableLatencies} to track latencies for one table in a

http://git-wip-us.apache.org/repos/asf/hbase/blob/01b1f48c/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
index 087a33f..54396ad 100644
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
+++ b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hbase.http;
 
-import com.google.common.annotations.VisibleForTesting;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -34,7 +33,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
-
 import javax.servlet.Filter;
 import javax.servlet.FilterChain;
 import javax.servlet.FilterConfig;
@@ -46,54 +44,53 @@ import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletRequestWrapper;
 import javax.servlet.http.HttpServletResponse;
-
 import org.apache.hadoop.HadoopIllegalArgumentException;
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.http.conf.ConfServlet;
 import org.apache.hadoop.hbase.http.jmx.JMXJsonServlet;
 import org.apache.hadoop.hbase.http.log.LogLevel;
-import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.util.Shell;
-
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
 import org.eclipse.jetty.http.HttpVersion;
-import org.eclipse.jetty.server.Server;
 import org.eclipse.jetty.server.Handler;
 import org.eclipse.jetty.server.HttpConfiguration;
 import org.eclipse.jetty.server.HttpConnectionFactory;
-import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.server.RequestLog;
 import org.eclipse.jetty.server.SecureRequestCustomizer;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
 import org.eclipse.jetty.server.SslConnectionFactory;
 import org.eclipse.jetty.server.handler.ContextHandlerCollection;
 import org.eclipse.jetty.server.handler.HandlerCollection;
-import org.eclipse.jetty.server.RequestLog;
 import org.eclipse.jetty.server.handler.RequestLogHandler;
-import org.eclipse.jetty.servlet.FilterMapping;
-import org.eclipse.jetty.servlet.ServletHandler;
+import org.eclipse.jetty.servlet.DefaultServlet;
 import org.eclipse.jetty.servlet.FilterHolder;
+import org.eclipse.jetty.servlet.FilterMapping;
 import org.eclipse.jetty.servlet.ServletContextHandler;
-import org.eclipse.jetty.servlet.DefaultServlet;
+import org.eclipse.jetty.servlet.ServletHandler;
 import org.eclipse.jetty.servlet.ServletHolder;
 import org.eclipse.jetty.util.MultiException;
 import org.eclipse.jetty.util.ssl.SslContextFactory;
 import org.eclipse.jetty.util.thread.QueuedThreadPool;
 import org.eclipse.jetty.webapp.WebAppContext;
-
 import org.glassfish.jersey.server.ResourceConfig;
 import org.glassfish.jersey.servlet.ServletContainer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+
 /**
  * Create a Jetty embedded server to answer http requests. The primary goal
  * is to serve up status information for the server.

http://git-wip-us.apache.org/repos/asf/hbase/blob/01b1f48c/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java
index 79367f3..b1f5212 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureScheduler.java
@@ -18,12 +18,13 @@
 
 package org.apache.hadoop.hbase.procedure2;
 
-import com.google.common.annotations.VisibleForTesting;
 import java.util.Iterator;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 import org.apache.yetus.audience.InterfaceAudience;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+
 /**
  * Keep track of the runnable procedures
  */

http://git-wip-us.apache.org/repos/asf/hbase/blob/01b1f48c/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
index 7fa264b..79cb77e 100644
--- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
+++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBase.java
@@ -23,7 +23,6 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import com.google.common.collect.Lists;
 import java.io.IOException;
 import java.security.SecureRandom;
 import java.util.EnumSet;
@@ -59,6 +58,7 @@ import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;

http://git-wip-us.apache.org/repos/asf/hbase/blob/01b1f48c/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContextImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContextImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContextImpl.java
index b52c231..8ad8ea2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContextImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContextImpl.java
@@ -18,13 +18,13 @@
 package org.apache.hadoop.hbase.coprocessor;
 
 import java.util.Optional;
-
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.yetus.audience.InterfaceAudience;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+
 /**
  * This is the only implementation of {@link ObserverContext}, which serves as the interface for
  * third-party Coprocessor developers.

http://git-wip-us.apache.org/repos/asf/hbase/blob/01b1f48c/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
index ee6eca4..472f647 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
@@ -52,14 +52,8 @@ import java.util.concurrent.atomic.LongAdder;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import com.google.common.base.Preconditions;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
 import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
@@ -79,8 +73,12 @@ import org.apache.hadoop.hbase.util.HasThread;
 import org.apache.hadoop.hbase.util.IdReadWriteLock;
 import org.apache.hadoop.hbase.util.IdReadWriteLock.ReferenceType;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
 import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/01b1f48c/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java
index 9c19c88..ffc4dfb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/FileIOEngine.java
@@ -26,19 +26,18 @@ import java.nio.ByteBuffer;
 import java.nio.channels.ClosedChannelException;
 import java.nio.channels.FileChannel;
 import java.util.Arrays;
-
-import com.google.common.annotations.VisibleForTesting;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.io.hfile.Cacheable;
-import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer;
 import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType;
+import org.apache.hadoop.hbase.io.hfile.CacheableDeserializer;
 import org.apache.hadoop.hbase.nio.ByteBuff;
 import org.apache.hadoop.hbase.nio.SingleByteBuff;
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
 
 /**
  * IO engine that stores data to a file on the local file system.

http://git-wip-us.apache.org/repos/asf/hbase/blob/01b1f48c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
index 7123155..3b2f9b0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java
@@ -29,7 +29,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Random;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
@@ -52,10 +51,11 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Optional;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 
-import com.google.common.annotations.VisibleForTesting;
 
 /**
  * <p>This is a best effort load balancer. Given a Cost function F(C) =&gt; x It will

http://git-wip-us.apache.org/repos/asf/hbase/blob/01b1f48c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RecoverMetaProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RecoverMetaProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RecoverMetaProcedure.java
index 90dfff0..fbf23d2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RecoverMetaProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RecoverMetaProcedure.java
@@ -36,11 +36,12 @@ import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RecoverMetaState;
 
-import com.google.common.base.Preconditions;
 
 /**
  * This procedure recovers meta from prior shutdown/ crash of a server, and brings meta online by

http://git-wip-us.apache.org/repos/asf/hbase/blob/01b1f48c/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/VersionedSegmentsList.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/VersionedSegmentsList.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/VersionedSegmentsList.java
index a697912..fb1dfa5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/VersionedSegmentsList.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/VersionedSegmentsList.java
@@ -18,12 +18,11 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import com.google.common.annotations.VisibleForTesting;
-
 import java.util.List;
-
 import org.apache.yetus.audience.InterfaceAudience;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+
 /**
  * A list of segment managers coupled with the version of the memstore (version at the time it was
  * created).

http://git-wip-us.apache.org/repos/asf/hbase/blob/01b1f48c/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/NettyAsyncFSWALConfigHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/NettyAsyncFSWALConfigHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/NettyAsyncFSWALConfigHelper.java
index 34ef3f0..a6790c1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/NettyAsyncFSWALConfigHelper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/NettyAsyncFSWALConfigHelper.java
@@ -17,18 +17,16 @@
  */
 package org.apache.hadoop.hbase.wal;
 
-import com.google.common.base.Preconditions;
-
-import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
-import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
-
 import java.util.HashMap;
 import java.util.Map;
-
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.util.Pair;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.Channel;
+import org.apache.hadoop.hbase.shaded.io.netty.channel.EventLoopGroup;
+
 /**
  * Helper class for passing netty event loop config to {@link AsyncFSWALProvider}.
  * @since 2.0.0

http://git-wip-us.apache.org/repos/asf/hbase/blob/01b1f48c/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java
index 26f45bc..824fcd1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java
@@ -21,8 +21,6 @@ import static org.apache.hadoop.hbase.client.AsyncProcess.START_LOG_ERRORS_AFTER
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 import java.io.IOException;
 import java.nio.file.FileSystems;
 import java.nio.file.Files;
@@ -54,6 +52,9 @@ import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
+
 @RunWith(Parameterized.class)
 @Category({ ClientTests.class, MediumTests.class })
 public class TestAsyncClusterAdminApi extends TestAsyncAdminBase {

http://git-wip-us.apache.org/repos/asf/hbase/blob/01b1f48c/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
index fa5981e..7421b49 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
@@ -24,13 +24,10 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import com.google.common.collect.Lists;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
-
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CompareOperator;
@@ -46,6 +43,7 @@ import org.junit.Assert;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/01b1f48c/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java
index 83edf8a..ff4e4c5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java
@@ -31,16 +31,14 @@ import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import com.google.common.collect.ImmutableMap;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
 import org.apache.hadoop.hbase.io.hfile.CacheTestUtils;
-import org.apache.hadoop.hbase.io.hfile.Cacheable;
 import org.apache.hadoop.hbase.io.hfile.CacheTestUtils.HFileBlockPair;
+import org.apache.hadoop.hbase.io.hfile.Cacheable;
 import org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator.BucketSizeInfo;
 import org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator.IndexStatistics;
 import org.apache.hadoop.hbase.testclassification.IOTests;
@@ -52,6 +50,8 @@ import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableMap;
+
 /**
  * Basic test of BucketCache.Puts and gets.
  * <p>
@@ -336,7 +336,8 @@ public class TestBucketCache {
     float[] configValues = {-1f, 0f, 0.96f, 1.05f};
     //throws due to <0, in expected range, minFactor > acceptableFactor, > 1.0
     boolean[] expectedOutcomes = {false, true, false, false};
-    Map<String, float[]> configMappings = ImmutableMap.of(BucketCache.MIN_FACTOR_CONFIG_NAME, configValues);
+    Map<String, float[]> configMappings = ImmutableMap
+      .of(BucketCache.MIN_FACTOR_CONFIG_NAME, configValues);
     Configuration conf = HBaseConfiguration.create();
     checkConfigValues(conf, configMappings, expectedOutcomes);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/01b1f48c/hbase-spark-it/src/test/java/org/apache/hadoop/hbase/spark/IntegrationTestSparkBulkLoad.java
----------------------------------------------------------------------
diff --git a/hbase-spark-it/src/test/java/org/apache/hadoop/hbase/spark/IntegrationTestSparkBulkLoad.java b/hbase-spark-it/src/test/java/org/apache/hadoop/hbase/spark/IntegrationTestSparkBulkLoad.java
index 4e4f7f6..4baf0d3 100644
--- a/hbase-spark-it/src/test/java/org/apache/hadoop/hbase/spark/IntegrationTestSparkBulkLoad.java
+++ b/hbase-spark-it/src/test/java/org/apache/hadoop/hbase/spark/IntegrationTestSparkBulkLoad.java
@@ -18,8 +18,6 @@
  */
 package org.apache.hadoop.hbase.spark;
 
-import com.google.common.collect.Sets;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -30,12 +28,10 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
-
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
@@ -62,7 +58,6 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.RegionSplitter;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ToolRunner;
-
 import org.apache.spark.Partitioner;
 import org.apache.spark.SerializableWritable;
 import org.apache.spark.SparkConf;
@@ -72,13 +67,13 @@ import org.apache.spark.api.java.function.Function;
 import org.apache.spark.api.java.function.Function2;
 import org.apache.spark.api.java.function.PairFlatMapFunction;
 import org.apache.spark.api.java.function.VoidFunction;
-
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import scala.Tuple2;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
+
 /**
  * Test Bulk Load and Spark on a distributed cluster.
  * It starts an Spark job that creates linked chains.


[05/26] hbase git commit: HBASE-19609 Fixed Checkstyle errors in hbase-metrics and enabled Checkstyle to fail on violations

Posted by zh...@apache.org.
HBASE-19609 Fixed Checkstyle errors in hbase-metrics and enabled Checkstyle to fail on violations


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/6d9081b8
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/6d9081b8
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/6d9081b8

Branch: refs/heads/HBASE-19397
Commit: 6d9081b8645f9b0ff4cd4bc50129a585d5518ef4
Parents: 3ed68fd
Author: Jan Hentschel <ja...@ultratendency.com>
Authored: Sun Dec 24 13:27:25 2017 +0100
Committer: Jan Hentschel <ja...@ultratendency.com>
Committed: Wed Dec 27 17:34:13 2017 +0100

----------------------------------------------------------------------
 hbase-metrics/pom.xml                               | 16 ++++++++++++++++
 .../hadoop/hbase/metrics/impl/CounterImpl.java      |  2 +-
 .../hadoop/hbase/metrics/impl/DropwizardMeter.java  |  4 ++--
 .../hbase/metrics/impl/FastLongHistogram.java       |  4 ++--
 .../hadoop/hbase/metrics/impl/HistogramImpl.java    |  4 +---
 .../hbase/metrics/impl/MetricRegistriesImpl.java    |  4 +---
 .../metrics/impl/MetricRegistryFactoryImpl.java     |  4 +---
 .../hbase/metrics/impl/MetricRegistryImpl.java      |  2 +-
 .../hadoop/hbase/metrics/impl/RefCountingMap.java   |  4 +---
 .../apache/hadoop/hbase/metrics/impl/TimerImpl.java |  2 +-
 .../hbase/metrics/impl/TestDropwizardMeter.java     |  4 ++--
 .../hbase/metrics/impl/TestRefCountingMap.java      |  8 +++-----
 12 files changed, 32 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6d9081b8/hbase-metrics/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-metrics/pom.xml b/hbase-metrics/pom.xml
index f82ad54..b3ba39d 100644
--- a/hbase-metrics/pom.xml
+++ b/hbase-metrics/pom.xml
@@ -68,6 +68,22 @@
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-checkstyle-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>checkstyle</id>
+            <phase>validate</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+            <configuration>
+              <failOnViolation>true</failOnViolation>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
   </build>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6d9081b8/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/CounterImpl.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/CounterImpl.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/CounterImpl.java
index 848aa5b..8021c06 100644
--- a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/CounterImpl.java
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/CounterImpl.java
@@ -19,8 +19,8 @@ package org.apache.hadoop.hbase.metrics.impl;
 
 import java.util.concurrent.atomic.LongAdder;
 
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.metrics.Counter;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Custom implementation of {@link org.apache.hadoop.hbase.metrics.Counter} using LongAdder.

http://git-wip-us.apache.org/repos/asf/hbase/blob/6d9081b8/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/DropwizardMeter.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/DropwizardMeter.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/DropwizardMeter.java
index 8b3bb80..b5c52cf 100644
--- a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/DropwizardMeter.java
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/DropwizardMeter.java
@@ -17,12 +17,12 @@
  */
 package org.apache.hadoop.hbase.metrics.impl;
 
+import com.codahale.metrics.Meter;
+
 import java.util.Objects;
 
 import org.apache.yetus.audience.InterfaceAudience;
 
-import com.codahale.metrics.Meter;
-
 /**
  * Dropwizard metrics implementation of {@link org.apache.hadoop.hbase.metrics.Meter}.
  */

http://git-wip-us.apache.org/repos/asf/hbase/blob/6d9081b8/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/FastLongHistogram.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/FastLongHistogram.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/FastLongHistogram.java
index a341d21..69f4ae5 100644
--- a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/FastLongHistogram.java
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/FastLongHistogram.java
@@ -22,10 +22,10 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.LongAdder;
 import java.util.stream.Stream;
 
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
 import org.apache.hadoop.hbase.metrics.Snapshot;
 import org.apache.hadoop.hbase.util.AtomicUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
 
 /**
  * FastLongHistogram is a thread-safe class that estimate distribution of data and computes the

http://git-wip-us.apache.org/repos/asf/hbase/blob/6d9081b8/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/HistogramImpl.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/HistogramImpl.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/HistogramImpl.java
index 910d1cd..6871f7e 100644
--- a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/HistogramImpl.java
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/HistogramImpl.java
@@ -16,13 +16,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-
 package org.apache.hadoop.hbase.metrics.impl;
 
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.metrics.Histogram;
 import org.apache.hadoop.hbase.metrics.Snapshot;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Custom histogram implementation based on FastLongHistogram. Dropwizard-based histograms are

http://git-wip-us.apache.org/repos/asf/hbase/blob/6d9081b8/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistriesImpl.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistriesImpl.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistriesImpl.java
index 28bdbb4..d72c68f 100644
--- a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistriesImpl.java
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistriesImpl.java
@@ -16,8 +16,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-
 package org.apache.hadoop.hbase.metrics.impl;
 
 import java.util.Collection;
@@ -25,11 +23,11 @@ import java.util.Collections;
 import java.util.Optional;
 import java.util.Set;
 
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.metrics.MetricRegistries;
 import org.apache.hadoop.hbase.metrics.MetricRegistry;
 import org.apache.hadoop.hbase.metrics.MetricRegistryFactory;
 import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Implementation of MetricRegistries that does ref-counting.

http://git-wip-us.apache.org/repos/asf/hbase/blob/6d9081b8/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryFactoryImpl.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryFactoryImpl.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryFactoryImpl.java
index e5dbc74..6f9e163 100644
--- a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryFactoryImpl.java
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryFactoryImpl.java
@@ -16,14 +16,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-
 package org.apache.hadoop.hbase.metrics.impl;
 
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.metrics.MetricRegistry;
 import org.apache.hadoop.hbase.metrics.MetricRegistryFactory;
 import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
+import org.apache.yetus.audience.InterfaceAudience;
 
 @InterfaceAudience.Private
 public class MetricRegistryFactoryImpl implements MetricRegistryFactory {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6d9081b8/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryImpl.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryImpl.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryImpl.java
index c5dc849..03c669f 100644
--- a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryImpl.java
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/MetricRegistryImpl.java
@@ -22,7 +22,6 @@ import java.util.Optional;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.metrics.Counter;
 import org.apache.hadoop.hbase.metrics.Gauge;
 import org.apache.hadoop.hbase.metrics.Histogram;
@@ -33,6 +32,7 @@ import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
 import org.apache.hadoop.hbase.metrics.MetricSet;
 import org.apache.hadoop.hbase.metrics.Timer;
 import org.apache.hadoop.hbase.util.CollectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Custom implementation of {@link MetricRegistry}.

http://git-wip-us.apache.org/repos/asf/hbase/blob/6d9081b8/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/RefCountingMap.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/RefCountingMap.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/RefCountingMap.java
index 567ffa8..63131a1 100644
--- a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/RefCountingMap.java
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/RefCountingMap.java
@@ -16,8 +16,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-
 package org.apache.hadoop.hbase.metrics.impl;
 
 import java.util.Collection;
@@ -69,7 +67,7 @@ class RefCountingMap<K, V> {
    * @return the value associated with the specified key or null if key is removed from map.
    */
   V remove(K k) {
-    Payload<V> p = map.computeIfPresent(k, (k1, v) -> --v.refCount <= 0 ? null : v );
+    Payload<V> p = map.computeIfPresent(k, (k1, v) -> --v.refCount <= 0 ? null : v);
     return p == null ? null : p.v;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6d9081b8/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/TimerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/TimerImpl.java b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/TimerImpl.java
index 495bc78..03a8c65 100644
--- a/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/TimerImpl.java
+++ b/hbase-metrics/src/main/java/org/apache/hadoop/hbase/metrics/impl/TimerImpl.java
@@ -19,8 +19,8 @@ package org.apache.hadoop.hbase.metrics.impl;
 
 import java.util.concurrent.TimeUnit;
 
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.metrics.Timer;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Custom implementation of {@link Timer}.

http://git-wip-us.apache.org/repos/asf/hbase/blob/6d9081b8/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestDropwizardMeter.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestDropwizardMeter.java b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestDropwizardMeter.java
index 55f5df3..e48d051 100644
--- a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestDropwizardMeter.java
+++ b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestDropwizardMeter.java
@@ -17,14 +17,14 @@
  */
 package org.apache.hadoop.hbase.metrics.impl;
 
+import com.codahale.metrics.Meter;
+
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
 
-import com.codahale.metrics.Meter;
-
 /**
  * Test class for {@link DropwizardMeter}.
  */

http://git-wip-us.apache.org/repos/asf/hbase/blob/6d9081b8/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestRefCountingMap.java
----------------------------------------------------------------------
diff --git a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestRefCountingMap.java b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestRefCountingMap.java
index d4c249c..a776ccd 100644
--- a/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestRefCountingMap.java
+++ b/hbase-metrics/src/test/java/org/apache/hadoop/hbase/metrics/impl/TestRefCountingMap.java
@@ -16,8 +16,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-
 package org.apache.hadoop.hbase.metrics.impl;
 
 import static org.junit.Assert.assertEquals;
@@ -135,7 +133,7 @@ public class TestRefCountingMap {
     Set<String> keys = map.keySet();
     assertEquals(3, keys.size());
 
-    Lists.newArrayList("foo", "bar", "baz").stream().forEach( v -> assertTrue(keys.contains(v)));
+    Lists.newArrayList("foo", "bar", "baz").stream().forEach(v -> assertTrue(keys.contains(v)));
   }
 
   @Test
@@ -148,7 +146,7 @@ public class TestRefCountingMap {
     Collection<String> values = map.values();
     assertEquals(3, values.size());
 
-    Lists.newArrayList("foovalue", "foovalue3", "foovalue4").stream().forEach(
-        v -> assertTrue(values.contains(v)));
+    Lists.newArrayList("foovalue", "foovalue3", "foovalue4").stream()
+            .forEach(v -> assertTrue(values.contains(v)));
   }
 }


[18/26] hbase git commit: HBASE-19543 Abstract a replication storage interface to extract the zk specific code

Posted by zh...@apache.org.
HBASE-19543 Abstract a replication storage interface to extract the zk specific code


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/19b20af4
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/19b20af4
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/19b20af4

Branch: refs/heads/HBASE-19397
Commit: 19b20af467396cb802d2cd10f1d4465ee0edfc05
Parents: 5d91fb6
Author: zhangduo <zh...@apache.org>
Authored: Fri Dec 22 14:37:28 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Dec 28 08:51:59 2017 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/util/CollectionUtils.java      |   3 +
 hbase-replication/pom.xml                       |  12 +
 .../replication/ReplicationPeerStorage.java     |  74 ++++
 .../replication/ReplicationQueueStorage.java    | 164 +++++++
 .../replication/ReplicationStateZKBase.java     |   1 -
 .../replication/ReplicationStorageFactory.java  |  49 +++
 .../replication/ZKReplicationPeerStorage.java   | 164 +++++++
 .../replication/ZKReplicationQueueStorage.java  | 425 +++++++++++++++++++
 .../replication/ZKReplicationStorageBase.java   |  75 ++++
 .../TestZKReplicationPeerStorage.java           | 171 ++++++++
 .../TestZKReplicationQueueStorage.java          | 171 ++++++++
 .../org/apache/hadoop/hbase/master/HMaster.java |  36 +-
 .../hadoop/hbase/master/MasterServices.java     |   6 +-
 .../master/procedure/MasterProcedureEnv.java    |  24 +-
 .../master/replication/AddPeerProcedure.java    |   6 +-
 .../replication/DisablePeerProcedure.java       |   7 +-
 .../master/replication/EnablePeerProcedure.java |   6 +-
 .../master/replication/ModifyPeerProcedure.java |  41 +-
 .../master/replication/RemovePeerProcedure.java |   6 +-
 .../master/replication/ReplicationManager.java  | 199 ---------
 .../replication/ReplicationPeerManager.java     | 331 +++++++++++++++
 .../replication/UpdatePeerConfigProcedure.java  |   7 +-
 .../replication/TestReplicationAdmin.java       |  62 ++-
 .../hbase/master/MockNoopMasterServices.java    |  10 +-
 .../hbase/master/TestMasterNoCluster.java       |   4 +-
 .../TestReplicationDisableInactivePeer.java     |   6 +-
 26 files changed, 1749 insertions(+), 311 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java
index 875b124..8bbb6f1 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java
@@ -107,6 +107,9 @@ public class CollectionUtils {
     return list.get(list.size() - 1);
   }
 
+  public static <T> List<T> nullToEmpty(List<T> list) {
+    return list != null ? list : Collections.emptyList();
+  }
   /**
    * In HBASE-16648 we found that ConcurrentHashMap.get is much faster than computeIfAbsent if the
    * value already exists. Notice that the implementation does not guarantee that the supplier will

http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-replication/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-replication/pom.xml b/hbase-replication/pom.xml
index ab22199..4e3cea0 100644
--- a/hbase-replication/pom.xml
+++ b/hbase-replication/pom.xml
@@ -121,6 +121,18 @@
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-zookeeper</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-zookeeper</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
     <!-- General dependencies -->
     <dependency>
       <groupId>org.apache.commons</groupId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java
new file mode 100644
index 0000000..e00cd0d
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import java.util.List;
+import java.util.Optional;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Perform read/write to the replication peer storage.
+ */
+@InterfaceAudience.Private
+public interface ReplicationPeerStorage {
+
+  /**
+   * Add a replication peer.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
+      throws ReplicationException;
+
+  /**
+   * Remove a replication peer.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  void removePeer(String peerId) throws ReplicationException;
+
+  /**
+   * Set the state of peer, {@code true} to {@code ENABLED}, otherwise to {@code DISABLED}.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  void setPeerState(String peerId, boolean enabled) throws ReplicationException;
+
+  /**
+   * Update the config a replication peer.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  void updatePeerConfig(String peerId, ReplicationPeerConfig peerConfig)
+      throws ReplicationException;
+
+  /**
+   * Return the peer ids of all replication peers.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  List<String> listPeerIds() throws ReplicationException;
+
+  /**
+   * Test whether a replication peer is enabled.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  boolean isPeerEnabled(String peerId) throws ReplicationException;
+
+  /**
+   * Get the peer config of a replication peer.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  Optional<ReplicationPeerConfig> getPeerConfig(String peerId) throws ReplicationException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
new file mode 100644
index 0000000..7210d9a
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
@@ -0,0 +1,164 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import java.util.List;
+import java.util.Set;
+import java.util.SortedSet;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Perform read/write to the replication queue storage.
+ */
+@InterfaceAudience.Private
+public interface ReplicationQueueStorage {
+
+  /**
+   * Remove a replication queue for a given regionserver.
+   * @param serverName the name of the regionserver
+   * @param queueId a String that identifies the queue.
+   */
+  void removeQueue(ServerName serverName, String queueId) throws ReplicationException;
+
+  /**
+   * Add a new WAL file to the given queue for a given regionserver. If the queue does not exist it
+   * is created.
+   * @param serverName the name of the regionserver
+   * @param queueId a String that identifies the queue.
+   * @param fileName name of the WAL
+   */
+  void addWAL(ServerName serverName, String queueId, String fileName) throws ReplicationException;
+
+  /**
+   * Remove an WAL file from the given queue for a given regionserver.
+   * @param serverName the name of the regionserver
+   * @param queueId a String that identifies the queue.
+   * @param fileName name of the WAL
+   */
+  void removeWAL(ServerName serverName, String queueId, String fileName)
+      throws ReplicationException;
+
+  /**
+   * Set the current position for a specific WAL in a given queue for a given regionserver.
+   * @param serverName the name of the regionserver
+   * @param queueId a String that identifies the queue
+   * @param fileName name of the WAL
+   * @param position the current position in the file
+   */
+  void setWALPosition(ServerName serverName, String queueId, String fileName, long position)
+      throws ReplicationException;
+
+  /**
+   * Get the current position for a specific WAL in a given queue for a given regionserver.
+   * @param serverName the name of the regionserver
+   * @param queueId a String that identifies the queue
+   * @param fileName name of the WAL
+   * @return the current position in the file
+   */
+  long getWALPosition(ServerName serverName, String queueId, String fileName)
+      throws ReplicationException;
+
+  /**
+   * Get a list of all queues for the specified region server.
+   * @param serverName the server name of the region server that owns the set of queues
+   * @return a list of queueIds
+   */
+  List<String> getAllQueues(ServerName serverName) throws ReplicationException;
+
+  /**
+   * Change ownership for the queue identified by queueId and belongs to a dead region server.
+   * @param sourceServerName the name of the dead region server
+   * @param destServerName the name of the target region server
+   * @param queueId the id of the queue
+   * @return the new PeerId and A SortedSet of WALs in its queue
+   */
+  Pair<String, SortedSet<String>> claimQueue(ServerName sourceServerName, String queueId,
+      ServerName destServerName) throws ReplicationException;
+
+  /**
+   * Remove the record of region server if the queue is empty.
+   */
+  void removeReplicatorIfQueueIsEmpty(ServerName serverName) throws ReplicationException;
+
+  /**
+   * Get a list of all region servers that have outstanding replication queues. These servers could
+   * be alive, dead or from a previous run of the cluster.
+   * @return a list of server names
+   */
+  List<ServerName> getListOfReplicators() throws ReplicationException;
+
+  /**
+   * Load all wals in all replication queues. This method guarantees to return a snapshot which
+   * contains all WALs in the zookeeper at the start of this call even there is concurrent queue
+   * failover. However, some newly created WALs during the call may not be included.
+   */
+  Set<String> getAllWALs() throws ReplicationException;
+
+  /**
+   * Add a peer to hfile reference queue if peer does not exist.
+   * @param peerId peer cluster id to be added
+   * @throws ReplicationException if fails to add a peer id to hfile reference queue
+   */
+  void addPeerToHFileRefs(String peerId) throws ReplicationException;
+
+  /**
+   * Remove a peer from hfile reference queue.
+   * @param peerId peer cluster id to be removed
+   */
+  void removePeerFromHFileRefs(String peerId) throws ReplicationException;
+
+  /**
+   * Add new hfile references to the queue.
+   * @param peerId peer cluster id to which the hfiles need to be replicated
+   * @param pairs list of pairs of { HFile location in staging dir, HFile path in region dir which
+   *          will be added in the queue }
+   * @throws ReplicationException if fails to add a hfile reference
+   */
+  void addHFileRefs(String peerId, List<Pair<Path, Path>> pairs) throws ReplicationException;
+
+  /**
+   * Remove hfile references from the queue.
+   * @param peerId peer cluster id from which this hfile references needs to be removed
+   * @param files list of hfile references to be removed
+   */
+  void removeHFileRefs(String peerId, List<String> files) throws ReplicationException;
+
+  /**
+   * Get the change version number of replication hfile references node. This can be used as
+   * optimistic locking to get a consistent snapshot of the replication queues of hfile references.
+   * @return change version number of hfile references node
+   */
+  int getHFileRefsNodeChangeVersion() throws ReplicationException;
+
+  /**
+   * Get list of all peers from hfile reference queue.
+   * @return a list of peer ids
+   */
+  List<String> getAllPeersFromHFileRefsQueue() throws ReplicationException;
+
+  /**
+   * Get a list of all hfile references in the given peer.
+   * @param peerId a String that identifies the peer
+   * @return a list of hfile references
+   */
+  List<String> getReplicableHFiles(String peerId) throws ReplicationException;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java
----------------------------------------------------------------------
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 ad970c6..edbbd4b 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
@@ -63,7 +63,6 @@ public abstract class ReplicationStateZKBase {
   protected final Configuration conf;
   protected final Abortable abortable;
 
-  // Public for testing
   public static final byte[] ENABLED_ZNODE_BYTES =
       toByteArray(ReplicationProtos.ReplicationState.State.ENABLED);
   public static final byte[] DISABLED_ZNODE_BYTES =

http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java
new file mode 100644
index 0000000..60d0749
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Used to create replication storage(peer, queue) classes.
+ * <p>
+ * For now we only have zk based implementation.
+ */
+@InterfaceAudience.Private
+public class ReplicationStorageFactory {
+
+  private ReplicationStorageFactory() {
+  }
+
+  /**
+   * Create a new {@link ReplicationPeerStorage}.
+   */
+  public static ReplicationPeerStorage getReplicationPeerStorage(ZKWatcher zk, Configuration conf) {
+    return new ZKReplicationPeerStorage(zk, conf);
+  }
+
+  /**
+   * Create a new {@link ReplicationQueueStorage}.
+   */
+  public static ReplicationQueueStorage getReplicationQueueStorage(ZKWatcher zk,
+      Configuration conf) {
+    return new ZKReplicationQueueStorage(zk, conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
new file mode 100644
index 0000000..49af4c3
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationPeerStorage.java
@@ -0,0 +1,164 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.util.CollectionUtils;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+
+/**
+ * ZK based replication peer storage.
+ */
+@InterfaceAudience.Private
+class ZKReplicationPeerStorage extends ZKReplicationStorageBase implements ReplicationPeerStorage {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ZKReplicationPeerStorage.class);
+
+  public static final byte[] ENABLED_ZNODE_BYTES =
+    toByteArray(ReplicationProtos.ReplicationState.State.ENABLED);
+  public static final byte[] DISABLED_ZNODE_BYTES =
+    toByteArray(ReplicationProtos.ReplicationState.State.DISABLED);
+
+  /**
+   * The name of the znode that contains the replication status of a remote slave (i.e. peer)
+   * cluster.
+   */
+  private final String peerStateNodeName;
+
+  /**
+   * The name of the znode that contains a list of all remote slave (i.e. peer) clusters.
+   */
+  private final String peersZNode;
+
+  public ZKReplicationPeerStorage(ZKWatcher zookeeper, Configuration conf) {
+    super(zookeeper, conf);
+    this.peerStateNodeName = conf.get("zookeeper.znode.replication.peers.state", "peer-state");
+    String peersZNodeName = conf.get("zookeeper.znode.replication.peers", "peers");
+    this.peersZNode = ZNodePaths.joinZNode(replicationZNode, peersZNodeName);
+  }
+
+  private String getPeerStateNode(String peerId) {
+    return ZNodePaths.joinZNode(getPeerNode(peerId), peerStateNodeName);
+  }
+
+  private String getPeerNode(String peerId) {
+    return ZNodePaths.joinZNode(peersZNode, peerId);
+  }
+
+  @Override
+  public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
+      throws ReplicationException {
+    try {
+      ZKUtil.createWithParents(zookeeper, peersZNode);
+      ZKUtil.multiOrSequential(zookeeper,
+        Arrays.asList(
+          ZKUtilOp.createAndFailSilent(getPeerNode(peerId),
+            ReplicationPeerConfigUtil.toByteArray(peerConfig)),
+          ZKUtilOp.createAndFailSilent(getPeerStateNode(peerId),
+            enabled ? ENABLED_ZNODE_BYTES : DISABLED_ZNODE_BYTES)),
+        false);
+    } catch (KeeperException e) {
+      throw new ReplicationException("Could not add peer with id=" + peerId + ", peerConfif=>" +
+        peerConfig + ", state=" + (enabled ? "ENABLED" : "DISABLED"), e);
+    }
+  }
+
+  @Override
+  public void removePeer(String peerId) throws ReplicationException {
+    try {
+      ZKUtil.deleteNodeRecursively(zookeeper, getPeerNode(peerId));
+    } catch (KeeperException e) {
+      throw new ReplicationException("Could not remove peer with id=" + peerId, e);
+    }
+  }
+
+  @Override
+  public void setPeerState(String peerId, boolean enabled) throws ReplicationException {
+    byte[] stateBytes = enabled ? ENABLED_ZNODE_BYTES : DISABLED_ZNODE_BYTES;
+    try {
+      ZKUtil.setData(zookeeper, getPeerStateNode(peerId), stateBytes);
+    } catch (KeeperException e) {
+      throw new ReplicationException("Unable to change state of the peer with id=" + peerId, e);
+    }
+  }
+
+  @Override
+  public void updatePeerConfig(String peerId, ReplicationPeerConfig peerConfig)
+      throws ReplicationException {
+    try {
+      ZKUtil.setData(this.zookeeper, getPeerNode(peerId),
+        ReplicationPeerConfigUtil.toByteArray(peerConfig));
+    } catch (KeeperException e) {
+      throw new ReplicationException(
+          "There was a problem trying to save changes to the " + "replication peer " + peerId, e);
+    }
+  }
+
+  @Override
+  public List<String> listPeerIds() throws ReplicationException {
+    try {
+      return CollectionUtils.nullToEmpty(ZKUtil.listChildrenAndWatchThem(zookeeper, peersZNode));
+    } catch (KeeperException e) {
+      throw new ReplicationException("Cannot get the list of peers", e);
+    }
+  }
+
+  @Override
+  public boolean isPeerEnabled(String peerId) throws ReplicationException {
+    try {
+      return Arrays.equals(ENABLED_ZNODE_BYTES,
+        ZKUtil.getData(zookeeper, getPeerStateNode(peerId)));
+    } catch (KeeperException | InterruptedException e) {
+      throw new ReplicationException("Unable to get status of the peer with id=" + peerId, e);
+    }
+  }
+
+  @Override
+  public Optional<ReplicationPeerConfig> getPeerConfig(String peerId) throws ReplicationException {
+    byte[] data;
+    try {
+      data = ZKUtil.getData(zookeeper, getPeerNode(peerId));
+    } catch (KeeperException | InterruptedException e) {
+      throw new ReplicationException("Error getting configuration for peer with id=" + peerId, e);
+    }
+    if (data == null || data.length == 0) {
+      return Optional.empty();
+    }
+    try {
+      return Optional.of(ReplicationPeerConfigUtil.parsePeerFrom(data));
+    } catch (DeserializationException e) {
+      LOG.warn("Failed to parse replication peer config for peer with id=" + peerId, e);
+      return Optional.empty();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
new file mode 100644
index 0000000..7015d7f
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
@@ -0,0 +1,425 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.hadoop.hbase.util.CollectionUtils.nullToEmpty;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CollectionUtils;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.BadVersionException;
+import org.apache.zookeeper.KeeperException.NoNodeException;
+import org.apache.zookeeper.KeeperException.NodeExistsException;
+import org.apache.zookeeper.KeeperException.NotEmptyException;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
+
+/**
+ * ZK based replication queue storage.
+ */
+@InterfaceAudience.Private
+class ZKReplicationQueueStorage extends ZKReplicationStorageBase
+    implements ReplicationQueueStorage {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ZKReplicationQueueStorage.class);
+
+  public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY =
+    "zookeeper.znode.replication.hfile.refs";
+  public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT = "hfile-refs";
+
+  /**
+   * The name of the znode that contains all replication queues
+   */
+  private final String queuesZNode;
+
+  /**
+   * The name of the znode that contains queues of hfile references to be replicated
+   */
+  private final String hfileRefsZNode;
+
+  public ZKReplicationQueueStorage(ZKWatcher zookeeper, Configuration conf) {
+    super(zookeeper, conf);
+
+    String queuesZNodeName = conf.get("zookeeper.znode.replication.rs", "rs");
+    String hfileRefsZNodeName = conf.get(ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY,
+      ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT);
+    this.queuesZNode = ZNodePaths.joinZNode(replicationZNode, queuesZNodeName);
+    this.hfileRefsZNode = ZNodePaths.joinZNode(replicationZNode, hfileRefsZNodeName);
+  }
+
+  private String getRsNode(ServerName serverName) {
+    return ZNodePaths.joinZNode(queuesZNode, serverName.getServerName());
+  }
+
+  private String getQueueNode(ServerName serverName, String queueId) {
+    return ZNodePaths.joinZNode(getRsNode(serverName), queueId);
+  }
+
+  private String getFileNode(String queueNode, String fileName) {
+    return ZNodePaths.joinZNode(queueNode, fileName);
+  }
+
+  private String getFileNode(ServerName serverName, String queueId, String fileName) {
+    return getFileNode(getQueueNode(serverName, queueId), fileName);
+  }
+
+  @Override
+  public void removeQueue(ServerName serverName, String queueId) throws ReplicationException {
+    try {
+      ZKUtil.deleteNodeRecursively(zookeeper, getQueueNode(serverName, queueId));
+    } catch (KeeperException e) {
+      throw new ReplicationException(
+          "Failed to delete queue (serverName=" + serverName + ", queueId=" + queueId + ")", e);
+    }
+  }
+
+  @Override
+  public void addWAL(ServerName serverName, String queueId, String fileName)
+      throws ReplicationException {
+    try {
+      ZKUtil.createWithParents(zookeeper, getFileNode(serverName, queueId, fileName));
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to add wal to queue (serverName=" + serverName +
+        ", queueId=" + queueId + ", fileName=" + fileName + ")", e);
+    }
+  }
+
+  @Override
+  public void removeWAL(ServerName serverName, String queueId, String fileName)
+      throws ReplicationException {
+    String fileNode = getFileNode(serverName, queueId, fileName);
+    try {
+      ZKUtil.deleteNode(zookeeper, fileNode);
+    } catch (NoNodeException e) {
+      LOG.warn(fileNode + " has already been deleted when removing log");
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to remove wal from queue (serverName=" + serverName +
+        ", queueId=" + queueId + ", fileName=" + fileName + ")", e);
+    }
+  }
+
+  @Override
+  public void setWALPosition(ServerName serverName, String queueId, String fileName, long position)
+      throws ReplicationException {
+    try {
+      ZKUtil.setData(zookeeper, getFileNode(serverName, queueId, fileName),
+        ZKUtil.positionToByteArray(position));
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to set log position (serverName=" + serverName +
+        ", queueId=" + queueId + ", fileName=" + fileName + ", position=" + position + ")", e);
+    }
+  }
+
+  @Override
+  public long getWALPosition(ServerName serverName, String queueId, String fileName)
+      throws ReplicationException {
+    byte[] bytes;
+    try {
+      bytes = ZKUtil.getData(zookeeper, getFileNode(serverName, queueId, fileName));
+    } catch (KeeperException | InterruptedException e) {
+      throw new ReplicationException("Failed to get log position (serverName=" + serverName +
+        ", queueId=" + queueId + ", fileName=" + fileName + ")", e);
+    }
+    try {
+      return ZKUtil.parseWALPositionFrom(bytes);
+    } catch (DeserializationException de) {
+      LOG.warn("Failed to parse log position (serverName=" + serverName + ", queueId=" + queueId +
+        ", fileName=" + fileName + ")");
+    }
+    // if we can not parse the position, start at the beginning of the wal file again
+    return 0;
+  }
+
+  @Override
+  public Pair<String, SortedSet<String>> claimQueue(ServerName sourceServerName, String queueId,
+      ServerName destServerName) throws ReplicationException {
+    LOG.info(
+      "Atomically moving " + sourceServerName + "/" + queueId + "'s WALs to " + destServerName);
+    try {
+      ZKUtil.createWithParents(zookeeper, getRsNode(destServerName));
+    } catch (KeeperException e) {
+      throw new ReplicationException(
+          "Claim queue queueId=" + queueId + " from " + sourceServerName + " to " + destServerName +
+            " failed when creating the node for " + destServerName,
+          e);
+    }
+    try {
+      String oldQueueNode = getQueueNode(sourceServerName, queueId);
+      List<String> wals = ZKUtil.listChildrenNoWatch(zookeeper, oldQueueNode);
+      String newQueueId = queueId + "-" + sourceServerName;
+      if (CollectionUtils.isEmpty(wals)) {
+        ZKUtil.deleteNodeFailSilent(zookeeper, oldQueueNode);
+        LOG.info("Removed " + sourceServerName + "/" + queueId + " since it's empty");
+        return new Pair<>(newQueueId, Collections.emptySortedSet());
+      }
+      String newQueueNode = getQueueNode(destServerName, newQueueId);
+      List<ZKUtilOp> listOfOps = new ArrayList<>();
+      SortedSet<String> logQueue = new TreeSet<>();
+      // create the new cluster znode
+      listOfOps.add(ZKUtilOp.createAndFailSilent(newQueueNode, HConstants.EMPTY_BYTE_ARRAY));
+      // get the offset of the logs and set it to new znodes
+      for (String wal : wals) {
+        String oldWalNode = getFileNode(oldQueueNode, wal);
+        byte[] logOffset = ZKUtil.getData(this.zookeeper, oldWalNode);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Creating " + wal + " with data " + Bytes.toStringBinary(logOffset));
+        }
+        String newWalNode = getFileNode(newQueueNode, wal);
+        listOfOps.add(ZKUtilOp.createAndFailSilent(newWalNode, logOffset));
+        listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldWalNode));
+        logQueue.add(wal);
+      }
+      // add delete op for peer
+      listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldQueueNode));
+
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("The multi list size is: " + listOfOps.size());
+      }
+      ZKUtil.multiOrSequential(zookeeper, listOfOps, false);
+
+      LOG.info(
+        "Atomically moved " + sourceServerName + "/" + queueId + "'s WALs to " + destServerName);
+      return new Pair<>(newQueueId, logQueue);
+    } catch (NoNodeException | NodeExistsException | NotEmptyException | BadVersionException e) {
+      // Multi call failed; it looks like some other regionserver took away the logs.
+      // These exceptions mean that zk tells us the request can not be execute so it is safe to just
+      // return a null. For other types of exception should be thrown out to notify the upper layer.
+      LOG.info(
+        "Claim queue queueId=" + queueId + " from " + sourceServerName + " to " + destServerName +
+          " failed with " + e.toString() + ", maybe someone else has already took away the logs");
+      return null;
+    } catch (KeeperException | InterruptedException e) {
+      throw new ReplicationException("Claim queue queueId=" + queueId + " from " +
+        sourceServerName + " to " + destServerName + " failed", e);
+    }
+  }
+
+  @Override
+  public void removeReplicatorIfQueueIsEmpty(ServerName serverName) throws ReplicationException {
+    try {
+      ZKUtil.deleteNodeFailSilent(zookeeper, getRsNode(serverName));
+    } catch (NotEmptyException e) {
+      // keep silence to avoid logging too much.
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to remove replicator for " + serverName, e);
+    }
+  }
+
+  private List<ServerName> getListOfReplicators0() throws KeeperException {
+    return nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, queuesZNode)).stream()
+        .map(ServerName::parseServerName).collect(toList());
+  }
+
+  @Override
+  public List<ServerName> getListOfReplicators() throws ReplicationException {
+    try {
+      return getListOfReplicators0();
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to get list of replicators", e);
+    }
+  }
+
+  private List<String> getLogsInQueue0(ServerName serverName, String queueId)
+      throws KeeperException {
+    return nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, getQueueNode(serverName, queueId)));
+  }
+
+  private List<String> getAllQueues0(ServerName serverName) throws KeeperException {
+    return nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, getRsNode(serverName)));
+  }
+
+  @Override
+  public List<String> getAllQueues(ServerName serverName) throws ReplicationException {
+    try {
+      return getAllQueues0(serverName);
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to get all queues (serverName=" + serverName + ")", e);
+    }
+  }
+
+  private int getQueuesZNodeCversion() throws KeeperException {
+    Stat stat = new Stat();
+    ZKUtil.getDataNoWatch(this.zookeeper, this.queuesZNode, stat);
+    return stat.getCversion();
+  }
+
+  @Override
+  public Set<String> getAllWALs() throws ReplicationException {
+    try {
+      for (int retry = 0;; retry++) {
+        int v0 = getQueuesZNodeCversion();
+        List<ServerName> rss = getListOfReplicators0();
+        if (rss.isEmpty()) {
+          LOG.debug("Didn't find any region server that replicates, won't prevent any deletions.");
+          return Collections.emptySet();
+        }
+        Set<String> wals = Sets.newHashSet();
+        for (ServerName rs : rss) {
+          for (String queueId : getAllQueues0(rs)) {
+            wals.addAll(getLogsInQueue0(rs, queueId));
+          }
+        }
+        int v1 = getQueuesZNodeCversion();
+        if (v0 == v1) {
+          return wals;
+        }
+        LOG.info(String.format("Replication queue node cversion changed from %d to %d, retry = %d",
+          v0, v1, retry));
+      }
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to get all wals", e);
+    }
+  }
+
+  private String getHFileRefsPeerNode(String peerId) {
+    return ZNodePaths.joinZNode(hfileRefsZNode, peerId);
+  }
+
+  private String getHFileNode(String peerNode, String fileName) {
+    return ZNodePaths.joinZNode(peerNode, fileName);
+  }
+
+  @Override
+  public void addPeerToHFileRefs(String peerId) throws ReplicationException {
+    String peerNode = getHFileRefsPeerNode(peerId);
+    try {
+      if (ZKUtil.checkExists(zookeeper, peerNode) == -1) {
+        LOG.info("Adding peer " + peerId + " to hfile reference queue.");
+        ZKUtil.createWithParents(zookeeper, peerNode);
+      }
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to add peer " + peerId + " to hfile reference queue.",
+          e);
+    }
+  }
+
+  @Override
+  public void removePeerFromHFileRefs(String peerId) throws ReplicationException {
+    String peerNode = getHFileRefsPeerNode(peerId);
+    try {
+      if (ZKUtil.checkExists(zookeeper, peerNode) == -1) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Peer " + peerNode + " not found in hfile reference queue.");
+        }
+      } else {
+        LOG.info("Removing peer " + peerNode + " from hfile reference queue.");
+        ZKUtil.deleteNodeRecursively(zookeeper, peerNode);
+      }
+    } catch (KeeperException e) {
+      throw new ReplicationException(
+          "Failed to remove peer " + peerId + " from hfile reference queue.", e);
+    }
+  }
+
+  @Override
+  public void addHFileRefs(String peerId, List<Pair<Path, Path>> pairs)
+      throws ReplicationException {
+    String peerNode = getHFileRefsPeerNode(peerId);
+    boolean debugEnabled = LOG.isDebugEnabled();
+    if (debugEnabled) {
+      LOG.debug("Adding hfile references " + pairs + " in queue " + peerNode);
+    }
+    List<ZKUtilOp> listOfOps =
+      pairs.stream().map(p -> p.getSecond().getName()).map(n -> getHFileNode(peerNode, n))
+          .map(f -> ZKUtilOp.createAndFailSilent(f, HConstants.EMPTY_BYTE_ARRAY)).collect(toList());
+    if (debugEnabled) {
+      LOG.debug("The multi list size for adding hfile references in zk for node " + peerNode +
+        " is " + listOfOps.size());
+    }
+    try {
+      ZKUtil.multiOrSequential(this.zookeeper, listOfOps, true);
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to add hfile reference to peer " + peerId, e);
+    }
+  }
+
+  @Override
+  public void removeHFileRefs(String peerId, List<String> files) throws ReplicationException {
+    String peerNode = getHFileRefsPeerNode(peerId);
+    boolean debugEnabled = LOG.isDebugEnabled();
+    if (debugEnabled) {
+      LOG.debug("Removing hfile references " + files + " from queue " + peerNode);
+    }
+
+    List<ZKUtilOp> listOfOps = files.stream().map(n -> getHFileNode(peerNode, n))
+        .map(ZKUtilOp::deleteNodeFailSilent).collect(toList());
+    if (debugEnabled) {
+      LOG.debug("The multi list size for removing hfile references in zk for node " + peerNode +
+        " is " + listOfOps.size());
+    }
+    try {
+      ZKUtil.multiOrSequential(this.zookeeper, listOfOps, true);
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to remove hfile reference from peer " + peerId, e);
+    }
+  }
+
+  @Override
+  public int getHFileRefsNodeChangeVersion() throws ReplicationException {
+    Stat stat = new Stat();
+    try {
+      ZKUtil.getDataNoWatch(zookeeper, hfileRefsZNode, stat);
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to get stat of replication hfile references node.", e);
+    }
+    return stat.getCversion();
+  }
+
+  @Override
+  public List<String> getAllPeersFromHFileRefsQueue() throws ReplicationException {
+    try {
+      return nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, hfileRefsZNode));
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to get list of all peers in hfile references node.",
+          e);
+    }
+  }
+
+  @Override
+  public List<String> getReplicableHFiles(String peerId) throws ReplicationException {
+    try {
+      return nullToEmpty(ZKUtil.listChildrenNoWatch(this.zookeeper, getHFileRefsPeerNode(peerId)));
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to get list of hfile references for peer " + peerId,
+          e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
new file mode 100644
index 0000000..b8a2044
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationStorageBase.java
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hadoop.hbase.shaded.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 related data,for example, peer, queue, etc, in
+ * zookeeper.
+ */
+@InterfaceAudience.Private
+class ZKReplicationStorageBase {
+
+  /** The name of the base znode that contains all replication state. */
+  protected final String replicationZNode;
+
+  protected final ZKWatcher zookeeper;
+  protected final Configuration conf;
+
+  protected ZKReplicationStorageBase(ZKWatcher zookeeper, Configuration conf) {
+    this.zookeeper = zookeeper;
+    this.conf = conf;
+    String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
+
+    this.replicationZNode =
+      ZNodePaths.joinZNode(this.zookeeper.znodePaths.baseZNode, replicationZNodeName);
+
+  }
+
+  /**
+   * Serialized protobuf of <code>state</code> 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.
+   */
+  protected static byte[] toByteArray(final ReplicationProtos.ReplicationState.State state) {
+    ReplicationProtos.ReplicationState msg =
+      ReplicationProtos.ReplicationState.newBuilder().setState(state).build();
+    // There is no toByteArray on this pb Message?
+    // 32 bytes is default which seems fair enough here.
+    try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
+      CodedOutputStream cos = CodedOutputStream.newInstance(baos, 16);
+      msg.writeTo(cos);
+      cos.flush();
+      baos.flush();
+      return ProtobufUtil.prependPBMagic(baos.toByteArray());
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java
new file mode 100644
index 0000000..a3be1e6
--- /dev/null
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java
@@ -0,0 +1,171 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Collectors.toSet;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Stream;
+
+import org.apache.hadoop.hbase.HBaseZKTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ ReplicationTests.class, MediumTests.class })
+public class TestZKReplicationPeerStorage {
+
+  private static final HBaseZKTestingUtility UTIL = new HBaseZKTestingUtility();
+
+  private static ZKReplicationPeerStorage STORAGE;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.startMiniZKCluster();
+    STORAGE = new ZKReplicationPeerStorage(UTIL.getZooKeeperWatcher(), UTIL.getConfiguration());
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    UTIL.shutdownMiniZKCluster();
+  }
+
+  private Set<String> randNamespaces(Random rand) {
+    return Stream.generate(() -> Long.toHexString(rand.nextLong())).limit(rand.nextInt(5))
+        .collect(toSet());
+  }
+
+  private Map<TableName, List<String>> randTableCFs(Random rand) {
+    int size = rand.nextInt(5);
+    Map<TableName, List<String>> map = new HashMap<>();
+    for (int i = 0; i < size; i++) {
+      TableName tn = TableName.valueOf(Long.toHexString(rand.nextLong()));
+      List<String> cfs = Stream.generate(() -> Long.toHexString(rand.nextLong()))
+          .limit(rand.nextInt(5)).collect(toList());
+      map.put(tn, cfs);
+    }
+    return map;
+  }
+
+  private ReplicationPeerConfig getConfig(int seed) {
+    Random rand = new Random(seed);
+    ReplicationPeerConfig config = new ReplicationPeerConfig();
+    config.setClusterKey(Long.toHexString(rand.nextLong()));
+    config.setReplicationEndpointImpl(Long.toHexString(rand.nextLong()));
+    config.setNamespaces(randNamespaces(rand));
+    config.setExcludeNamespaces(randNamespaces(rand));
+    config.setTableCFsMap(randTableCFs(rand));
+    config.setReplicateAllUserTables(rand.nextBoolean());
+    config.setBandwidth(rand.nextInt(1000));
+    return config;
+  }
+
+  private void assertSetEquals(Set<String> expected, Set<String> actual) {
+    if (expected == null || expected.size() == 0) {
+      assertTrue(actual == null || actual.size() == 0);
+      return;
+    }
+    assertEquals(expected.size(), actual.size());
+    expected.forEach(s -> assertTrue(actual.contains(s)));
+  }
+
+  private void assertMapEquals(Map<TableName, List<String>> expected,
+      Map<TableName, List<String>> actual) {
+    if (expected == null || expected.size() == 0) {
+      assertTrue(actual == null || actual.size() == 0);
+      return;
+    }
+    assertEquals(expected.size(), actual.size());
+    expected.forEach((expectedTn, expectedCFs) -> {
+      List<String> actualCFs = actual.get(expectedTn);
+      if (expectedCFs == null || expectedCFs.size() == 0) {
+        assertTrue(actual.containsKey(expectedTn));
+        assertTrue(actualCFs == null || actualCFs.size() == 0);
+      } else {
+        assertNotNull(actualCFs);
+        assertEquals(expectedCFs.size(), actualCFs.size());
+        for (Iterator<String> expectedIt = expectedCFs.iterator(), actualIt = actualCFs.iterator();
+          expectedIt.hasNext();) {
+          assertEquals(expectedIt.next(), actualIt.next());
+        }
+      }
+    });
+  }
+
+  private void assertConfigEquals(ReplicationPeerConfig expected, ReplicationPeerConfig actual) {
+    assertEquals(expected.getClusterKey(), actual.getClusterKey());
+    assertEquals(expected.getReplicationEndpointImpl(), actual.getReplicationEndpointImpl());
+    assertSetEquals(expected.getNamespaces(), actual.getNamespaces());
+    assertSetEquals(expected.getExcludeNamespaces(), actual.getExcludeNamespaces());
+    assertMapEquals(expected.getTableCFsMap(), actual.getTableCFsMap());
+    assertMapEquals(expected.getExcludeTableCFsMap(), actual.getExcludeTableCFsMap());
+    assertEquals(expected.replicateAllUserTables(), actual.replicateAllUserTables());
+    assertEquals(expected.getBandwidth(), actual.getBandwidth());
+  }
+
+  @Test
+  public void test() throws ReplicationException {
+    int peerCount = 10;
+    for (int i = 0; i < peerCount; i++) {
+      STORAGE.addPeer(Integer.toString(i), getConfig(i), i % 2 == 0);
+    }
+    List<String> peerIds = STORAGE.listPeerIds();
+    assertEquals(peerCount, peerIds.size());
+    for (String peerId : peerIds) {
+      int seed = Integer.parseInt(peerId);
+      assertConfigEquals(getConfig(seed), STORAGE.getPeerConfig(peerId).get());
+    }
+    for (int i = 0; i < peerCount; i++) {
+      STORAGE.updatePeerConfig(Integer.toString(i), getConfig(i + 1));
+    }
+    for (String peerId : peerIds) {
+      int seed = Integer.parseInt(peerId);
+      assertConfigEquals(getConfig(seed + 1), STORAGE.getPeerConfig(peerId).get());
+    }
+    for (int i = 0; i < peerCount; i++) {
+      assertEquals(i % 2 == 0, STORAGE.isPeerEnabled(Integer.toString(i)));
+    }
+    for (int i = 0; i < peerCount; i++) {
+      STORAGE.setPeerState(Integer.toString(i), i % 2 != 0);
+    }
+    for (int i = 0; i < peerCount; i++) {
+      assertEquals(i % 2 != 0, STORAGE.isPeerEnabled(Integer.toString(i)));
+    }
+    String toRemove = Integer.toString(peerCount / 2);
+    STORAGE.removePeer(toRemove);
+    peerIds = STORAGE.listPeerIds();
+    assertEquals(peerCount - 1, peerIds.size());
+    assertFalse(peerIds.contains(toRemove));
+    assertFalse(STORAGE.getPeerConfig(toRemove).isPresent());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
new file mode 100644
index 0000000..d5bba0d
--- /dev/null
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
@@ -0,0 +1,171 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import static org.hamcrest.CoreMatchers.hasItems;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+import java.util.SortedSet;
+
+import org.apache.hadoop.hbase.HBaseZKTestingUtility;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Pair;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ ReplicationTests.class, MediumTests.class })
+public class TestZKReplicationQueueStorage {
+  private static final HBaseZKTestingUtility UTIL = new HBaseZKTestingUtility();
+
+  private static ZKReplicationQueueStorage STORAGE;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.startMiniZKCluster();
+    STORAGE = new ZKReplicationQueueStorage(UTIL.getZooKeeperWatcher(), UTIL.getConfiguration());
+  }
+
+  @AfterClass
+  public static void tearDown() throws IOException {
+    UTIL.shutdownMiniZKCluster();
+  }
+
+  @After
+  public void tearDownAfterTest() throws ReplicationException {
+    for (ServerName serverName : STORAGE.getListOfReplicators()) {
+      for (String queue : STORAGE.getAllQueues(serverName)) {
+        STORAGE.removeQueue(serverName, queue);
+      }
+      STORAGE.removeReplicatorIfQueueIsEmpty(serverName);
+    }
+    for (String peerId : STORAGE.getAllPeersFromHFileRefsQueue()) {
+      STORAGE.removePeerFromHFileRefs(peerId);
+    }
+  }
+
+  private ServerName getServerName(int i) {
+    return ServerName.valueOf("127.0.0.1", 8000 + i, 10000 + i);
+  }
+
+  @Test
+  public void testReplicator() throws ReplicationException {
+    assertTrue(STORAGE.getListOfReplicators().isEmpty());
+    String queueId = "1";
+    for (int i = 0; i < 10; i++) {
+      STORAGE.addWAL(getServerName(i), queueId, "file" + i);
+    }
+    List<ServerName> replicators = STORAGE.getListOfReplicators();
+    assertEquals(10, replicators.size());
+    for (int i = 0; i < 10; i++) {
+      assertThat(replicators, hasItems(getServerName(i)));
+    }
+    for (int i = 0; i < 5; i++) {
+      STORAGE.removeQueue(getServerName(i), queueId);
+    }
+    for (int i = 0; i < 10; i++) {
+      STORAGE.removeReplicatorIfQueueIsEmpty(getServerName(i));
+    }
+    replicators = STORAGE.getListOfReplicators();
+    assertEquals(5, replicators.size());
+    for (int i = 5; i < 10; i++) {
+      assertThat(replicators, hasItems(getServerName(i)));
+    }
+  }
+
+  private String getFileName(String base, int i) {
+    return String.format(base + "-%04d", i);
+  }
+
+  @Test
+  public void testAddRemoveLog() throws ReplicationException {
+    ServerName serverName1 = ServerName.valueOf("127.0.0.1", 8000, 10000);
+    assertTrue(STORAGE.getAllQueues(serverName1).isEmpty());
+    String queue1 = "1";
+    String queue2 = "2";
+    for (int i = 0; i < 10; i++) {
+      STORAGE.addWAL(serverName1, queue1, getFileName("file1", i));
+      STORAGE.addWAL(serverName1, queue2, getFileName("file2", i));
+    }
+    List<String> queueIds = STORAGE.getAllQueues(serverName1);
+    assertEquals(2, queueIds.size());
+    assertThat(queueIds, hasItems("1", "2"));
+
+    for (int i = 0; i < 10; i++) {
+      assertEquals(0, STORAGE.getWALPosition(serverName1, queue1, getFileName("file1", i)));
+      assertEquals(0, STORAGE.getWALPosition(serverName1, queue2, getFileName("file2", i)));
+      STORAGE.setWALPosition(serverName1, queue1, getFileName("file1", i), (i + 1) * 100);
+      STORAGE.setWALPosition(serverName1, queue2, getFileName("file2", i), (i + 1) * 100 + 10);
+    }
+
+    for (int i = 0; i < 10; i++) {
+      assertEquals((i + 1) * 100,
+        STORAGE.getWALPosition(serverName1, queue1, getFileName("file1", i)));
+      assertEquals((i + 1) * 100 + 10,
+        STORAGE.getWALPosition(serverName1, queue2, getFileName("file2", i)));
+    }
+
+    for (int i = 0; i < 10; i++) {
+      if (i % 2 == 0) {
+        STORAGE.removeWAL(serverName1, queue1, getFileName("file1", i));
+      } else {
+        STORAGE.removeWAL(serverName1, queue2, getFileName("file2", i));
+      }
+    }
+
+    queueIds = STORAGE.getAllQueues(serverName1);
+    assertEquals(2, queueIds.size());
+    assertThat(queueIds, hasItems("1", "2"));
+
+    ServerName serverName2 = ServerName.valueOf("127.0.0.1", 8001, 10001);
+    Pair<String, SortedSet<String>> peer1 = STORAGE.claimQueue(serverName1, "1", serverName2);
+
+    assertEquals("1-" + serverName1.getServerName(), peer1.getFirst());
+    assertEquals(5, peer1.getSecond().size());
+    int i = 1;
+    for (String wal : peer1.getSecond()) {
+      assertEquals(getFileName("file1", i), wal);
+      assertEquals((i + 1) * 100,
+        STORAGE.getWALPosition(serverName2, peer1.getFirst(), getFileName("file1", i)));
+      i += 2;
+    }
+
+    queueIds = STORAGE.getAllQueues(serverName1);
+    assertEquals(1, queueIds.size());
+    assertThat(queueIds, hasItems("2"));
+
+    queueIds = STORAGE.getAllQueues(serverName2);
+    assertEquals(1, queueIds.size());
+    assertThat(queueIds, hasItems(peer1.getFirst()));
+
+    Set<String> allWals = STORAGE.getAllWALs();
+    assertEquals(10, allWals.size());
+    for (i = 0; i < 10; i++) {
+      assertThat(allWals, hasItems(i % 2 == 0 ? getFileName("file2", i) : getFileName("file1", i)));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 4d3b265..468ff89 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -39,6 +39,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Objects;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
@@ -135,7 +136,7 @@ import org.apache.hadoop.hbase.master.replication.DisablePeerProcedure;
 import org.apache.hadoop.hbase.master.replication.EnablePeerProcedure;
 import org.apache.hadoop.hbase.master.replication.ModifyPeerProcedure;
 import org.apache.hadoop.hbase.master.replication.RemovePeerProcedure;
-import org.apache.hadoop.hbase.master.replication.ReplicationManager;
+import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
 import org.apache.hadoop.hbase.master.replication.UpdatePeerConfigProcedure;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.mob.MobConstants;
@@ -327,7 +328,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   private AssignmentManager assignmentManager;
 
   // manager of replication
-  private ReplicationManager replicationManager;
+  private ReplicationPeerManager replicationPeerManager;
 
   // buffer for "fatal error" notices from region servers
   // in the cluster. This is only used for assisting
@@ -698,8 +699,8 @@ public class HMaster extends HRegionServer implements MasterServices {
   /**
    * Initialize all ZK based system trackers.
    */
-  void initializeZKBasedSystemTrackers() throws IOException,
-      InterruptedException, KeeperException, CoordinatedStateException {
+  void initializeZKBasedSystemTrackers() throws IOException, InterruptedException, KeeperException,
+      CoordinatedStateException, ReplicationException {
     this.balancer = LoadBalancerFactory.getLoadBalancer(conf);
     this.normalizer = RegionNormalizerFactory.getRegionNormalizer(conf);
     this.normalizer.setMasterServices(this);
@@ -717,7 +718,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     this.assignmentManager = new AssignmentManager(this);
     this.assignmentManager.start();
 
-    this.replicationManager = new ReplicationManager(conf, zooKeeper, this);
+    this.replicationPeerManager = ReplicationPeerManager.create(zooKeeper, conf);
 
     this.regionServerTracker = new RegionServerTracker(zooKeeper, this, this.serverManager);
     this.regionServerTracker.start();
@@ -762,9 +763,8 @@ public class HMaster extends HRegionServer implements MasterServices {
    * <li>Handle either fresh cluster start or master failover</li>
    * </ol>
    */
-  private void finishActiveMasterInitialization(MonitoredTask status)
-      throws IOException, InterruptedException, KeeperException, CoordinatedStateException {
-
+  private void finishActiveMasterInitialization(MonitoredTask status) throws IOException,
+      InterruptedException, KeeperException, CoordinatedStateException, ReplicationException {
     activeMaster = true;
     Thread zombieDetector = new Thread(new InitializationMonitor(this),
         "ActiveMasterInitializationMonitor-" + System.currentTimeMillis());
@@ -3351,18 +3351,19 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
-  public ReplicationPeerConfig getReplicationPeerConfig(String peerId) throws ReplicationException,
-      IOException {
+  public ReplicationPeerConfig getReplicationPeerConfig(String peerId)
+      throws ReplicationException, IOException {
     if (cpHost != null) {
       cpHost.preGetReplicationPeerConfig(peerId);
     }
-    final ReplicationPeerConfig peerConfig = this.replicationManager.getPeerConfig(peerId);
-    LOG.info(getClientIdAuditPrefix() + " get replication peer config, id=" + peerId + ", config="
-        + peerConfig);
+    LOG.info(getClientIdAuditPrefix() + " get replication peer config, id=" + peerId);
+    Optional<ReplicationPeerConfig> peerConfig =
+      this.replicationPeerManager.getPeerConfig(peerId);
+
     if (cpHost != null) {
       cpHost.postGetReplicationPeerConfig(peerId);
     }
-    return peerConfig;
+    return peerConfig.orElse(null);
   }
 
   @Override
@@ -3381,7 +3382,8 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
     LOG.info(getClientIdAuditPrefix() + " list replication peers, regex=" + regex);
     Pattern pattern = regex == null ? null : Pattern.compile(regex);
-    List<ReplicationPeerDescription> peers = this.replicationManager.listReplicationPeers(pattern);
+    List<ReplicationPeerDescription> peers =
+      this.replicationPeerManager.listPeers(pattern);
     if (cpHost != null) {
       cpHost.postListReplicationPeers(regex);
     }
@@ -3531,7 +3533,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   @Override
-  public ReplicationManager getReplicationManager() {
-    return replicationManager;
+  public ReplicationPeerManager getReplicationPeerManager() {
+    return replicationPeerManager;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index 43df8b1..67baf14 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.locking.LockManager;
 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
-import org.apache.hadoop.hbase.master.replication.ReplicationManager;
+import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
 import org.apache.hadoop.hbase.procedure2.LockedResource;
@@ -459,9 +459,9 @@ public interface MasterServices extends Server {
       IOException;
 
   /**
-   * Returns the {@link ReplicationManager}.
+   * Returns the {@link ReplicationPeerManager}.
    */
-  ReplicationManager getReplicationManager();
+  ReplicationPeerManager getReplicationPeerManager();
 
   /**
    * Update the peerConfig for the specified peer

http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
index fa4d371..ae038a5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureEnv.java
@@ -24,24 +24,24 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
-import org.apache.hadoop.hbase.master.replication.ReplicationManager;
+import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
 import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
 import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
-import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.Superusers;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
@@ -138,8 +138,8 @@ public class MasterProcedureEnv implements ConfigurationObserver {
     return remoteDispatcher;
   }
 
-  public ReplicationManager getReplicationManager() {
-    return master.getReplicationManager();
+  public ReplicationPeerManager getReplicationPeerManager() {
+    return master.getReplicationPeerManager();
   }
 
   public boolean isRunning() {
@@ -151,22 +151,22 @@ public class MasterProcedureEnv implements ConfigurationObserver {
     return master.isInitialized();
   }
 
-  public boolean waitInitialized(Procedure proc) {
+  public boolean waitInitialized(Procedure<?> proc) {
     return master.getInitializedEvent().suspendIfNotReady(proc);
   }
 
-  public boolean waitServerCrashProcessingEnabled(Procedure proc) {
+  public boolean waitServerCrashProcessingEnabled(Procedure<?> proc) {
     if (master instanceof HMaster) {
       return ((HMaster)master).getServerCrashProcessingEnabledEvent().suspendIfNotReady(proc);
     }
     return false;
   }
 
-  public boolean waitFailoverCleanup(Procedure proc) {
+  public boolean waitFailoverCleanup(Procedure<?> proc) {
     return master.getAssignmentManager().getFailoverCleanupEvent().suspendIfNotReady(proc);
   }
 
-  public void setEventReady(ProcedureEvent event, boolean isReady) {
+  public void setEventReady(ProcedureEvent<?> event, boolean isReady) {
     if (isReady) {
       event.wake(procSched);
     } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
index 066c3e7..a4f9b32 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
@@ -58,16 +58,18 @@ public class AddPeerProcedure extends ModifyPeerProcedure {
   }
 
   @Override
-  protected void prePeerModification(MasterProcedureEnv env) throws IOException {
+  protected void prePeerModification(MasterProcedureEnv env)
+      throws IOException, ReplicationException {
     MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
     if (cpHost != null) {
       cpHost.preAddReplicationPeer(peerId, peerConfig);
     }
+    env.getReplicationPeerManager().preAddPeer(peerId, peerConfig);
   }
 
   @Override
   protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
-    env.getReplicationManager().addReplicationPeer(peerId, peerConfig, enabled);
+    env.getReplicationPeerManager().addPeer(peerId, peerConfig, enabled);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
index 9a28de6..10e35a8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -51,12 +52,12 @@ public class DisablePeerProcedure extends ModifyPeerProcedure {
     if (cpHost != null) {
       cpHost.preDisableReplicationPeer(peerId);
     }
+    env.getReplicationPeerManager().preDisablePeer(peerId);
   }
 
   @Override
-  protected void updatePeerStorage(MasterProcedureEnv env)
-      throws IllegalArgumentException, Exception {
-    env.getReplicationManager().disableReplicationPeer(peerId);
+  protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
+    env.getReplicationPeerManager().disablePeer(peerId);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
index 4855901..f2a9f01 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -51,11 +52,12 @@ public class EnablePeerProcedure extends ModifyPeerProcedure {
     if (cpHost != null) {
       cpHost.preEnableReplicationPeer(peerId);
     }
+    env.getReplicationPeerManager().preEnablePeer(peerId);
   }
 
   @Override
-  protected void updatePeerStorage(MasterProcedureEnv env) throws Exception {
-    env.getReplicationManager().enableReplicationPeer(peerId);
+  protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
+    env.getReplicationPeerManager().enablePeer(peerId);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
index c4552ed..279fbc7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
-import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface;
 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
@@ -27,6 +26,7 @@ import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
 import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -67,17 +67,16 @@ public abstract class ModifyPeerProcedure
   }
 
   /**
-   * Called before we start the actual processing. If an exception is thrown then we will give up
-   * and mark the procedure as failed directly.
+   * Called before we start the actual processing. The implementation should call the pre CP hook,
+   * and also the pre-check for the peer modification.
+   * <p>
+   * If an IOException is thrown then we will give up and mark the procedure as failed directly. If
+   * all checks passes then the procedure can not be rolled back any more.
    */
-  protected abstract void prePeerModification(MasterProcedureEnv env) throws IOException;
+  protected abstract void prePeerModification(MasterProcedureEnv env)
+      throws IOException, ReplicationException;
 
-  /**
-   * We will give up and mark the procedure as failure if {@link IllegalArgumentException} is
-   * thrown, for other type of Exception we will retry.
-   */
-  protected abstract void updatePeerStorage(MasterProcedureEnv env)
-      throws IllegalArgumentException, Exception;
+  protected abstract void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException;
 
   /**
    * Called before we finish the procedure. The implementation can do some logging work, and also
@@ -100,23 +99,24 @@ public abstract class ModifyPeerProcedure
         try {
           prePeerModification(env);
         } catch (IOException e) {
-          LOG.warn(getClass().getName() + " failed to call prePeerModification for peer " + peerId +
-            ", mark the procedure as failure and give up", e);
-          setFailure("prePeerModification", e);
+          LOG.warn(
+            getClass().getName() + " failed to call CP hook or the pre check is failed for peer " +
+              peerId + ", mark the procedure as failure and give up",
+            e);
+          setFailure("master-" + getPeerOperationType().name().toLowerCase() + "-peer", e);
           releaseLatch();
           return Flow.NO_MORE_STATE;
+        } catch (ReplicationException e) {
+          LOG.warn(getClass().getName() + " failed to call prePeerModification for peer " + peerId +
+            ", retry", e);
+          throw new ProcedureYieldException();
         }
         setNextState(PeerModificationState.UPDATE_PEER_STORAGE);
         return Flow.HAS_MORE_STATE;
       case UPDATE_PEER_STORAGE:
         try {
           updatePeerStorage(env);
-        } catch (IllegalArgumentException e) {
-          setFailure("master-" + getPeerOperationType().name().toLowerCase() + "-peer",
-            new DoNotRetryIOException(e));
-          releaseLatch();
-          return Flow.NO_MORE_STATE;
-        } catch (Exception e) {
+        } catch (ReplicationException e) {
           LOG.warn(
             getClass().getName() + " update peer storage for peer " + peerId + " failed, retry", e);
           throw new ProcedureYieldException();
@@ -158,8 +158,7 @@ public abstract class ModifyPeerProcedure
   @Override
   protected void rollbackState(MasterProcedureEnv env, PeerModificationState state)
       throws IOException, InterruptedException {
-    if (state == PeerModificationState.PRE_PEER_MODIFICATION ||
-      state == PeerModificationState.UPDATE_PEER_STORAGE) {
+    if (state == PeerModificationState.PRE_PEER_MODIFICATION) {
       // actually the peer related operations has no rollback, but if we haven't done any
       // modifications on the peer storage, we can just return.
       return;

http://git-wip-us.apache.org/repos/asf/hbase/blob/19b20af4/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
index d40df02..6e9c384 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -51,11 +52,12 @@ public class RemovePeerProcedure extends ModifyPeerProcedure {
     if (cpHost != null) {
       cpHost.preRemoveReplicationPeer(peerId);
     }
+    env.getReplicationPeerManager().preRemovePeer(peerId);
   }
 
   @Override
-  protected void updatePeerStorage(MasterProcedureEnv env) throws Exception {
-    env.getReplicationManager().removeReplicationPeer(peerId);
+  protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
+    env.getReplicationPeerManager().removePeer(peerId);
   }
 
   @Override


[24/26] hbase git commit: HBASE-19573 Rewrite ReplicationPeer with the new replication storage interface

Posted by zh...@apache.org.
HBASE-19573 Rewrite ReplicationPeer with the new replication storage interface


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/76539be4
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/76539be4
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/76539be4

Branch: refs/heads/HBASE-19397
Commit: 76539be479206129d6fafdef7c20b4009bf22b02
Parents: 08f51e8
Author: Guanghao Zhang <zg...@apache.org>
Authored: Tue Dec 26 11:39:34 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Dec 28 08:51:59 2017 +0800

----------------------------------------------------------------------
 .../replication/VerifyReplication.java          |   5 -
 .../hbase/replication/ReplicationPeer.java      |  42 ++--
 .../hbase/replication/ReplicationPeerImpl.java  | 170 ++++++++++++++
 .../replication/ReplicationPeerZKImpl.java      | 233 -------------------
 .../hbase/replication/ReplicationPeers.java     |   4 +-
 .../replication/ReplicationPeersZKImpl.java     |  23 +-
 .../replication/TestReplicationStateBasic.java  |   7 +-
 .../regionserver/PeerProcedureHandlerImpl.java  |  29 +--
 8 files changed, 217 insertions(+), 296 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/76539be4/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
index 01df2bd..da231e6 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java
@@ -49,7 +49,6 @@ import org.apache.hadoop.hbase.mapreduce.TableSplit;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
-import org.apache.hadoop.hbase.replication.ReplicationPeerZKImpl;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -330,7 +329,6 @@ public class VerifyReplication extends Configured implements Tool {
   private static Pair<ReplicationPeerConfig, Configuration> getPeerQuorumConfig(
       final Configuration conf, String peerId) throws IOException {
     ZKWatcher localZKW = null;
-    ReplicationPeerZKImpl peer = null;
     try {
       localZKW = new ZKWatcher(conf, "VerifyReplication",
           new Abortable() {
@@ -351,9 +349,6 @@ public class VerifyReplication extends Configured implements Tool {
       throw new IOException(
           "An error occurred while trying to connect to the remove peer cluster", e);
     } finally {
-      if (peer != null) {
-        peer.close();
-      }
       if (localZKW != null) {
         localZKW.close();
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/76539be4/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
index b66d76d..4846018 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
@@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.yetus.audience.InterfaceAudience;
 
-
 /**
  * ReplicationPeer manages enabled / disabled state for the peer.
  */
@@ -49,65 +48,52 @@ public interface ReplicationPeer {
   String getId();
 
   /**
-   * Get the peer config object
-   * @return the ReplicationPeerConfig for this peer
-   */
-  public ReplicationPeerConfig getPeerConfig();
-
-  /**
-   * Get the peer config object. if loadFromBackingStore is true, it will load from backing store
-   * directly and update its load peer config. otherwise, just return the local cached peer config.
-   * @return the ReplicationPeerConfig for this peer
-   */
-  public ReplicationPeerConfig getPeerConfig(boolean loadFromBackingStore)
-      throws ReplicationException;
-
-  /**
    * Returns the state of the peer by reading local cache.
    * @return the enabled state
    */
   PeerState getPeerState();
 
   /**
-   * Returns the state of peer, if loadFromBackingStore is true, it will load from backing store
-   * directly and update its local peer state. otherwise, just return the local cached peer state.
-   * @return the enabled state
+   * Get the peer config object
+   * @return the ReplicationPeerConfig for this peer
    */
-  PeerState getPeerState(boolean loadFromBackingStore) throws ReplicationException;
+  ReplicationPeerConfig getPeerConfig();
 
   /**
    * Get the configuration object required to communicate with this peer
    * @return configuration object
    */
-  public Configuration getConfiguration();
+  Configuration getConfiguration();
 
   /**
    * Get replicable (table, cf-list) map of this peer
    * @return the replicable (table, cf-list) map
    */
-  public Map<TableName, List<String>> getTableCFs();
+  Map<TableName, List<String>> getTableCFs();
 
   /**
    * Get replicable namespace set of this peer
    * @return the replicable namespaces set
    */
-  public Set<String> getNamespaces();
+  Set<String> getNamespaces();
 
   /**
    * Get the per node bandwidth upper limit for this peer
    * @return the bandwidth upper limit
    */
-  public long getPeerBandwidth();
+  long getPeerBandwidth();
 
   /**
    * Register a peer config listener to catch the peer config change event.
    * @param listener listener to catch the peer config change event.
    */
-  public void registerPeerConfigListener(ReplicationPeerConfigListener listener);
+  void registerPeerConfigListener(ReplicationPeerConfigListener listener);
 
   /**
-   * Notify all the registered ReplicationPeerConfigListener to update their peer config.
-   * @param newPeerConfig the new peer config.
+   * @deprecated Use {@link #registerPeerConfigListener(ReplicationPeerConfigListener)} instead.
    */
-  public void triggerPeerConfigChange(ReplicationPeerConfig newPeerConfig);
-}
+  @Deprecated
+  default void trackPeerConfigChanges(ReplicationPeerConfigListener listener) {
+    registerPeerConfigListener(listener);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/76539be4/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
new file mode 100644
index 0000000..ed9fe14
--- /dev/null
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java
@@ -0,0 +1,170 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.yetus.audience.InterfaceAudience;
+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 ReplicationPeerImpl implements ReplicationPeer {
+  private static final Logger LOG = LoggerFactory.getLogger(ReplicationPeerImpl.class);
+
+  private final ReplicationPeerStorage peerStorage;
+
+  private final Configuration conf;
+
+  private final String id;
+
+  private volatile ReplicationPeerConfig peerConfig;
+
+  private volatile PeerState peerState;
+
+  private final List<ReplicationPeerConfigListener> peerConfigListeners;
+
+  /**
+   * Constructor that takes all the objects required to communicate with the specified peer, except
+   * for the region server addresses.
+   * @param conf configuration object to this peer
+   * @param id string representation of this peer's identifier
+   * @param peerConfig configuration for the replication peer
+   */
+  public ReplicationPeerImpl(ZKWatcher zkWatcher, Configuration conf, String id,
+      ReplicationPeerConfig peerConfig) {
+    this.peerStorage = ReplicationStorageFactory.getReplicationPeerStorage(zkWatcher, conf);
+    this.conf = conf;
+    this.peerConfig = peerConfig;
+    this.id = id;
+    this.peerConfigListeners = new ArrayList<>();
+  }
+
+  public void refreshPeerState() throws ReplicationException {
+    this.peerState = peerStorage.isPeerEnabled(id) ? PeerState.ENABLED : PeerState.DISABLED;
+  }
+
+  public void refreshPeerConfig() throws ReplicationException {
+    this.peerConfig = peerStorage.getPeerConfig(id).orElse(peerConfig);
+    peerConfigListeners.forEach(listener -> listener.peerConfigUpdated(peerConfig));
+  }
+
+  /**
+   * Get the identifier of this peer
+   * @return string representation of the id (short)
+   */
+  @Override
+  public String getId() {
+    return id;
+  }
+
+  @Override
+  public PeerState getPeerState() {
+    return peerState;
+  }
+
+  /**
+   * Get the peer config object
+   * @return the ReplicationPeerConfig for this peer
+   */
+  @Override
+  public ReplicationPeerConfig getPeerConfig() {
+    return peerConfig;
+  }
+
+  /**
+   * Get the configuration object required to communicate with this peer
+   * @return configuration object
+   */
+  @Override
+  public Configuration getConfiguration() {
+    return conf;
+  }
+
+  /**
+   * Get replicable (table, cf-list) map of this peer
+   * @return the replicable (table, cf-list) map
+   */
+  @Override
+  public Map<TableName, List<String>> getTableCFs() {
+    return this.peerConfig.getTableCFsMap();
+  }
+
+  /**
+   * Get replicable namespace set of this peer
+   * @return the replicable namespaces set
+   */
+  @Override
+  public Set<String> getNamespaces() {
+    return this.peerConfig.getNamespaces();
+  }
+
+  @Override
+  public long getPeerBandwidth() {
+    return this.peerConfig.getBandwidth();
+  }
+
+  @Override
+  public void registerPeerConfigListener(ReplicationPeerConfigListener listener) {
+    this.peerConfigListeners.add(listener);
+  }
+
+  /**
+   * Parse the raw data from ZK to get a peer's state
+   * @param bytes raw ZK data
+   * @return True if the passed in <code>bytes</code> are those of a pb serialized ENABLED state.
+   * @throws DeserializationException
+   */
+  public static boolean isStateEnabled(final byte[] bytes) throws DeserializationException {
+    ReplicationProtos.ReplicationState.State state = parseStateFrom(bytes);
+    return ReplicationProtos.ReplicationState.State.ENABLED == state;
+  }
+
+  /**
+   * @param bytes Content of a state znode.
+   * @return State parsed from the passed bytes.
+   * @throws DeserializationException
+   */
+  private static ReplicationProtos.ReplicationState.State parseStateFrom(final byte[] bytes)
+      throws DeserializationException {
+    ProtobufUtil.expectPBMagicPrefix(bytes);
+    int pbLen = ProtobufUtil.lengthOfPBMagic();
+    ReplicationProtos.ReplicationState.Builder builder =
+        ReplicationProtos.ReplicationState.newBuilder();
+    ReplicationProtos.ReplicationState state;
+    try {
+      ProtobufUtil.mergeFrom(builder, bytes, pbLen, bytes.length - pbLen);
+      state = builder.build();
+      return state.getState();
+    } catch (IOException e) {
+      throw new DeserializationException(e);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/76539be4/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 5d051a0..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
+++ /dev/null
@@ -1,233 +0,0 @@
-/*
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.replication;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-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.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;
-
-@InterfaceAudience.Private
-public class ReplicationPeerZKImpl extends ReplicationStateZKBase
-    implements ReplicationPeer, Abortable, Closeable {
-  private static final Logger LOG = LoggerFactory.getLogger(ReplicationPeerZKImpl.class);
-
-  private volatile ReplicationPeerConfig peerConfig;
-  private final String id;
-  private volatile PeerState peerState;
-  private volatile Map<TableName, List<String>> tableCFs = new HashMap<>();
-  private final Configuration conf;
-
-  private final List<ReplicationPeerConfigListener> peerConfigListeners;
-
-  /**
-   * Constructor that takes all the objects required to communicate with the specified peer, except
-   * for the region server addresses.
-   * @param conf configuration object to this peer
-   * @param id string representation of this peer's identifier
-   * @param peerConfig configuration for the replication peer
-   */
-  public ReplicationPeerZKImpl(ZKWatcher zkWatcher, Configuration conf, String id,
-      ReplicationPeerConfig peerConfig, Abortable abortable) throws ReplicationException {
-    super(zkWatcher, conf, abortable);
-    this.conf = conf;
-    this.peerConfig = peerConfig;
-    this.id = id;
-    this.peerConfigListeners = new ArrayList<>();
-  }
-
-  private PeerState readPeerState() throws ReplicationException {
-    try {
-      byte[] data = ZKUtil.getData(zookeeper, this.getPeerStateNode(id));
-      this.peerState = isStateEnabled(data) ? PeerState.ENABLED : PeerState.DISABLED;
-    } catch (DeserializationException | KeeperException | InterruptedException e) {
-      throw new ReplicationException("Get and deserialize peer state data from zookeeper failed: ",
-          e);
-    }
-    return this.peerState;
-  }
-
-  private ReplicationPeerConfig readPeerConfig() throws ReplicationException {
-    try {
-      byte[] data = ZKUtil.getData(zookeeper, this.getPeerNode(id));
-      if (data != null) {
-        this.peerConfig = ReplicationPeerConfigUtil.parsePeerFrom(data);
-      }
-    } catch (DeserializationException | KeeperException | InterruptedException e) {
-      throw new ReplicationException("Get and deserialize peer config date from zookeeper failed: ",
-          e);
-    }
-    return this.peerConfig;
-  }
-
-  @Override
-  public PeerState getPeerState() {
-    return peerState;
-  }
-
-  @Override
-  public PeerState getPeerState(boolean loadFromBackingStore) throws ReplicationException {
-    if (loadFromBackingStore) {
-      return readPeerState();
-    } else {
-      return peerState;
-    }
-  }
-
-  /**
-   * Get the identifier of this peer
-   * @return string representation of the id (short)
-   */
-  @Override
-  public String getId() {
-    return id;
-  }
-
-  /**
-   * Get the peer config object
-   * @return the ReplicationPeerConfig for this peer
-   */
-  @Override
-  public ReplicationPeerConfig getPeerConfig() {
-    return peerConfig;
-  }
-
-  @Override
-  public ReplicationPeerConfig getPeerConfig(boolean loadFromBackingStore)
-      throws ReplicationException {
-    if (loadFromBackingStore) {
-      return readPeerConfig();
-    } else {
-      return peerConfig;
-    }
-  }
-
-  /**
-   * Get the configuration object required to communicate with this peer
-   * @return configuration object
-   */
-  @Override
-  public Configuration getConfiguration() {
-    return conf;
-  }
-
-  /**
-   * Get replicable (table, cf-list) map of this peer
-   * @return the replicable (table, cf-list) map
-   */
-  @Override
-  public Map<TableName, List<String>> getTableCFs() {
-    this.tableCFs = peerConfig.getTableCFsMap();
-    return this.tableCFs;
-  }
-
-  /**
-   * Get replicable namespace set of this peer
-   * @return the replicable namespaces set
-   */
-  @Override
-  public Set<String> getNamespaces() {
-    return this.peerConfig.getNamespaces();
-  }
-
-  @Override
-  public long getPeerBandwidth() {
-    return this.peerConfig.getBandwidth();
-  }
-
-  @Override
-  public void registerPeerConfigListener(ReplicationPeerConfigListener listener) {
-    this.peerConfigListeners.add(listener);
-  }
-
-  @Override
-  public void triggerPeerConfigChange(ReplicationPeerConfig newPeerConfig) {
-    for (ReplicationPeerConfigListener listener : this.peerConfigListeners) {
-      listener.peerConfigUpdated(newPeerConfig);
-    }
-  }
-
-  @Override
-  public void abort(String why, Throwable e) {
-    LOG.error(HBaseMarkers.FATAL, "The ReplicationPeer corresponding to peer " +
-        peerConfig + " was aborted for the following reason(s):" + why, e);
-  }
-
-  @Override
-  public boolean isAborted() {
-    // Currently the replication peer is never "Aborted", we just log when the
-    // abort method is called.
-    return false;
-  }
-
-  @Override
-  public void close() throws IOException {
-    // TODO: stop zkw?
-  }
-
-  /**
-   * Parse the raw data from ZK to get a peer's state
-   * @param bytes raw ZK data
-   * @return True if the passed in <code>bytes</code> are those of a pb serialized ENABLED state.
-   * @throws DeserializationException
-   */
-  public static boolean isStateEnabled(final byte[] bytes) throws DeserializationException {
-    ReplicationProtos.ReplicationState.State state = parseStateFrom(bytes);
-    return ReplicationProtos.ReplicationState.State.ENABLED == state;
-  }
-
-  /**
-   * @param bytes Content of a state znode.
-   * @return State parsed from the passed bytes.
-   * @throws DeserializationException
-   */
-  private static ReplicationProtos.ReplicationState.State parseStateFrom(final byte[] bytes)
-      throws DeserializationException {
-    ProtobufUtil.expectPBMagicPrefix(bytes);
-    int pbLen = ProtobufUtil.lengthOfPBMagic();
-    ReplicationProtos.ReplicationState.Builder builder =
-        ReplicationProtos.ReplicationState.newBuilder();
-    ReplicationProtos.ReplicationState state;
-    try {
-      ProtobufUtil.mergeFrom(builder, bytes, pbLen, bytes.length - pbLen);
-      state = builder.build();
-      return state.getState();
-    } catch (IOException e) {
-      throw new DeserializationException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/76539be4/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java
----------------------------------------------------------------------
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 10936bf..afc19bd 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
@@ -116,13 +116,13 @@ public interface ReplicationPeers {
       throws ReplicationException;
 
   /**
-   * Returns the ReplicationPeer for the specified connected peer. This ReplicationPeer will
+   * Returns the ReplicationPeerImpl for the specified connected peer. This ReplicationPeer will
    * continue to track changes to the Peer's state and config. This method returns null if no
    * peer has been connected with the given peerId.
    * @param peerId id for the peer
    * @return ReplicationPeer object
    */
-  ReplicationPeer getConnectedPeer(String peerId);
+  ReplicationPeerImpl getConnectedPeer(String peerId);
 
   /**
    * Returns the set of peerIds of the clusters that have been connected and have an underlying

http://git-wip-us.apache.org/repos/asf/hbase/blob/76539be4/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
----------------------------------------------------------------------
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 f2e5647..268ba87 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
@@ -80,7 +80,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements ReplicationPeers {
 
   // Map of peer clusters keyed by their id
-  private Map<String, ReplicationPeerZKImpl> peerClusters;
+  private ConcurrentMap<String, ReplicationPeerImpl> peerClusters;
   private final ReplicationQueueStorage queueStorage;
   private Abortable abortable;
 
@@ -232,7 +232,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
       }
       String peerStateZNode = getPeerStateNode(id);
       try {
-        return ReplicationPeerZKImpl.isStateEnabled(ZKUtil.getData(this.zookeeper, peerStateZNode));
+        return ReplicationPeerImpl.isStateEnabled(ZKUtil.getData(this.zookeeper, peerStateZNode));
       } catch (KeeperException e) {
         throw new ReplicationException(e);
       } catch (DeserializationException e) {
@@ -270,7 +270,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
   }
 
   @Override
-  public ReplicationPeer getConnectedPeer(String peerId) {
+  public ReplicationPeerImpl getConnectedPeer(String peerId) {
     return peerClusters.get(peerId);
   }
 
@@ -425,7 +425,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
   public void peerDisconnected(String peerId) {
     ReplicationPeer rp = this.peerClusters.get(peerId);
     if (rp != null) {
-      ((ConcurrentMap<String, ReplicationPeerZKImpl>) peerClusters).remove(peerId, rp);
+      peerClusters.remove(peerId, rp);
     }
   }
 
@@ -442,7 +442,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
       return false;
     }
 
-    ReplicationPeerZKImpl peer = null;
+    ReplicationPeerImpl peer = null;
     try {
       peer = createPeer(peerId);
     } catch (Exception e) {
@@ -451,8 +451,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
     if (peer == null) {
       return false;
     }
-    ReplicationPeerZKImpl previous =
-      ((ConcurrentMap<String, ReplicationPeerZKImpl>) peerClusters).putIfAbsent(peerId, peer);
+    ReplicationPeerImpl previous = peerClusters.putIfAbsent(peerId, peer);
     if (previous == null) {
       LOG.info("Added new peer cluster=" + peer.getPeerConfig().getClusterKey());
     } else {
@@ -495,19 +494,19 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
    * @return object representing the peer
    * @throws ReplicationException
    */
-  private ReplicationPeerZKImpl createPeer(String peerId) throws ReplicationException {
+  private ReplicationPeerImpl createPeer(String peerId) throws ReplicationException {
     Pair<ReplicationPeerConfig, Configuration> pair = getPeerConf(peerId);
     if (pair == null) {
       return null;
     }
     Configuration peerConf = pair.getSecond();
 
-    ReplicationPeerZKImpl peer =
-        new ReplicationPeerZKImpl(zookeeper, peerConf, peerId, pair.getFirst(), abortable);
+    ReplicationPeerImpl peer =
+        new ReplicationPeerImpl(zookeeper, peerConf, peerId, pair.getFirst());
 
     // Load peer state and peer config by reading zookeeper directly.
-    peer.getPeerState(true);
-    peer.getPeerConfig(true);
+    peer.refreshPeerState();
+    peer.refreshPeerConfig();
 
     return peer;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/76539be4/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
index 6fe869c..8905d43 100644
--- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
@@ -312,12 +312,15 @@ public abstract class TestReplicationStateBasic {
     rp.disablePeer(ID_ONE);
     // now we do not rely on zk watcher to trigger the state change so we need to trigger it
     // manually...
-    assertEquals(PeerState.DISABLED, rp.getConnectedPeer(ID_ONE).getPeerState(true));
+    ReplicationPeerImpl peer = rp.getConnectedPeer(ID_ONE);
+    peer.refreshPeerState();
+    assertEquals(PeerState.DISABLED, peer.getPeerState());
     assertConnectedPeerStatus(false, ID_ONE);
     rp.enablePeer(ID_ONE);
     // now we do not rely on zk watcher to trigger the state change so we need to trigger it
     // manually...
-    assertEquals(PeerState.ENABLED, rp.getConnectedPeer(ID_ONE).getPeerState(true));
+    peer.refreshPeerState();
+    assertEquals(PeerState.ENABLED, peer.getPeerState());
     assertConnectedPeerStatus(true, ID_ONE);
 
     // Disconnect peer

http://git-wip-us.apache.org/repos/asf/hbase/blob/76539be4/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
index 9b493d9..598357c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
@@ -21,15 +21,14 @@ package org.apache.hadoop.hbase.replication.regionserver;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.replication.ReplicationException;
-import org.apache.hadoop.hbase.replication.ReplicationPeer;
-import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
-import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
-import org.apache.log4j.Logger;
+import org.apache.hadoop.hbase.replication.ReplicationPeerImpl;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @InterfaceAudience.Private
 public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
-  private static final Logger LOG = Logger.getLogger(PeerProcedureHandlerImpl.class);
+  private static final Logger LOG = LoggerFactory.getLogger(PeerProcedureHandlerImpl.class);
 
   private ReplicationSourceManager replicationSourceManager;
 
@@ -49,10 +48,11 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
 
   @Override
   public void disablePeer(String peerId) throws ReplicationException, IOException {
-    ReplicationPeer peer = replicationSourceManager.getReplicationPeers().getConnectedPeer(peerId);
+    ReplicationPeerImpl peer =
+        replicationSourceManager.getReplicationPeers().getConnectedPeer(peerId);
     if (peer != null) {
-      PeerState peerState = peer.getPeerState(true);
-      LOG.info("disablePeer state, peer id: " + peerId + ", state: " + peerState);
+      peer.refreshPeerState();
+      LOG.info("disable replication peer, id: " + peerId + ", new state: " + peer.getPeerState());
     } else {
       throw new ReplicationException("No connected peer found, peerId=" + peerId);
     }
@@ -60,10 +60,11 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
 
   @Override
   public void enablePeer(String peerId) throws ReplicationException, IOException {
-    ReplicationPeer peer = replicationSourceManager.getReplicationPeers().getConnectedPeer(peerId);
+    ReplicationPeerImpl peer =
+        replicationSourceManager.getReplicationPeers().getConnectedPeer(peerId);
     if (peer != null) {
-      PeerState peerState = peer.getPeerState(true);
-      LOG.info("enablePeer state, peer id: " + peerId + ", state: " + peerState);
+      peer.refreshPeerState();
+      LOG.info("enable replication peer, id: " + peerId + ", new state: " + peer.getPeerState());
     } else {
       throw new ReplicationException("No connected peer found, peerId=" + peerId);
     }
@@ -71,11 +72,11 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
 
   @Override
   public void updatePeerConfig(String peerId) throws ReplicationException, IOException {
-    ReplicationPeer peer = replicationSourceManager.getReplicationPeers().getConnectedPeer(peerId);
+    ReplicationPeerImpl peer =
+        replicationSourceManager.getReplicationPeers().getConnectedPeer(peerId);
     if (peer == null) {
       throw new ReplicationException("No connected peer found, peerId=" + peerId);
     }
-    ReplicationPeerConfig rpc = peer.getPeerConfig(true);
-    peer.triggerPeerConfigChange(rpc);
+    peer.refreshPeerConfig();
   }
 }


[04/26] hbase git commit: HBASE-19610 Fixed Checkstyle errors in hbase-protocol and enabled Checkstyle to fail on violations

Posted by zh...@apache.org.
HBASE-19610 Fixed Checkstyle errors in hbase-protocol and enabled Checkstyle to fail on violations


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/3ed68fd7
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/3ed68fd7
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/3ed68fd7

Branch: refs/heads/HBASE-19397
Commit: 3ed68fd70d7ee187553627998ee39581309296e2
Parents: 6b39062
Author: Jan Hentschel <ja...@ultratendency.com>
Authored: Sun Dec 24 13:59:27 2017 +0100
Committer: Jan Hentschel <ja...@ultratendency.com>
Committed: Wed Dec 27 17:26:54 2017 +0100

----------------------------------------------------------------------
 hbase-protocol/pom.xml                              | 16 ++++++++++++++++
 .../google/protobuf/HBaseZeroCopyByteString.java    |  2 +-
 .../org/apache/hadoop/hbase/util/ByteStringer.java  |  8 ++++----
 3 files changed, 21 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3ed68fd7/hbase-protocol/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-protocol/pom.xml b/hbase-protocol/pom.xml
index d07a031..ec7f66a 100644
--- a/hbase-protocol/pom.xml
+++ b/hbase-protocol/pom.xml
@@ -112,6 +112,22 @@
             </replacements>
         </configuration>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-checkstyle-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>checkstyle</id>
+            <phase>validate</phase>
+            <goals>
+              <goal>check</goal>
+            </goals>
+            <configuration>
+              <failOnViolation>true</failOnViolation>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
     <pluginManagement>
       <plugins>

http://git-wip-us.apache.org/repos/asf/hbase/blob/3ed68fd7/hbase-protocol/src/main/java/com/google/protobuf/HBaseZeroCopyByteString.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/com/google/protobuf/HBaseZeroCopyByteString.java b/hbase-protocol/src/main/java/com/google/protobuf/HBaseZeroCopyByteString.java
index 9d75612..3d1953e 100644
--- a/hbase-protocol/src/main/java/com/google/protobuf/HBaseZeroCopyByteString.java
+++ b/hbase-protocol/src/main/java/com/google/protobuf/HBaseZeroCopyByteString.java
@@ -61,7 +61,7 @@ public final class HBaseZeroCopyByteString extends LiteralByteString {
   /**
    * Extracts the byte array from the given {@link ByteString} without copy.
    * @param buf A buffer from which to extract the array.  This buffer must be
-   * actually an instance of a {@code LiteralByteString}.
+   *            actually an instance of a {@code LiteralByteString}.
    * @return byte[] representation
    */
   public static byte[] zeroCopyGetBytes(final ByteString buf) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/3ed68fd7/hbase-protocol/src/main/java/org/apache/hadoop/hbase/util/ByteStringer.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/util/ByteStringer.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/util/ByteStringer.java
index 65f1cc6..581741d 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/util/ByteStringer.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/util/ByteStringer.java
@@ -17,18 +17,18 @@
  */
 package org.apache.hadoop.hbase.util;
 
+import com.google.protobuf.ByteString;
+import com.google.protobuf.HBaseZeroCopyByteString;
+
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.protobuf.ByteString;
-import com.google.protobuf.HBaseZeroCopyByteString;
-
 /**
  * Hack to workaround HBASE-10304 issue that keeps bubbling up when a mapreduce context.
  */
 @InterfaceAudience.Private
-public class ByteStringer {
+public final class ByteStringer {
   private static final Logger LOG = LoggerFactory.getLogger(ByteStringer.class);
 
   /**


[09/26] hbase git commit: HBASE-19592 Add UTs to test retry on update zk failure

Posted by zh...@apache.org.
HBASE-19592 Add UTs to test retry on update zk failure


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/4850dbdc
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/4850dbdc
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/4850dbdc

Branch: refs/heads/HBASE-19397
Commit: 4850dbdc0069e65ba8018bf854eae572b0cf763b
Parents: 09249a2
Author: zhangduo <zh...@apache.org>
Authored: Tue Dec 26 20:39:00 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Dec 28 08:51:59 2017 +0800

----------------------------------------------------------------------
 .../replication/ReplicationPeerManager.java     |   5 +-
 .../TestReplicationProcedureRetry.java          | 200 +++++++++++++++++++
 2 files changed, 202 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4850dbdc/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
index b78cbce..f4ccce8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
@@ -53,7 +53,7 @@ import org.apache.yetus.audience.InterfaceAudience;
  * Used to add/remove a replication peer.
  */
 @InterfaceAudience.Private
-public final class ReplicationPeerManager {
+public class ReplicationPeerManager {
 
   private final ReplicationPeerStorage peerStorage;
 
@@ -61,8 +61,7 @@ public final class ReplicationPeerManager {
 
   private final ConcurrentMap<String, ReplicationPeerDescription> peers;
 
-  private ReplicationPeerManager(ReplicationPeerStorage peerStorage,
-      ReplicationQueueStorage queueStorage,
+  ReplicationPeerManager(ReplicationPeerStorage peerStorage, ReplicationQueueStorage queueStorage,
       ConcurrentMap<String, ReplicationPeerDescription> peers) {
     this.peerStorage = peerStorage;
     this.queueStorage = queueStorage;

http://git-wip-us.apache.org/repos/asf/hbase/blob/4850dbdc/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java
new file mode 100644
index 0000000..ab35b46
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java
@@ -0,0 +1,200 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyBoolean;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.spy;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.zookeeper.KeeperException;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.invocation.InvocationOnMock;
+
+/**
+ * All the modification method will fail once in the test and should finally succeed.
+ */
+@Category({ ReplicationTests.class, MediumTests.class })
+public class TestReplicationProcedureRetry {
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.getConfiguration().setClass(HConstants.MASTER_IMPL, MockHMaster.class, HMaster.class);
+    UTIL.startMiniCluster(3);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @After
+  public void tearDownAfterTest() throws IOException {
+    for (ReplicationPeerDescription desc : UTIL.getAdmin().listReplicationPeers()) {
+      UTIL.getAdmin().removeReplicationPeer(desc.getPeerId());
+    }
+  }
+
+  private void doTest() throws IOException {
+    Admin admin = UTIL.getAdmin();
+    String peerId = "1";
+    ReplicationPeerConfig peerConfig = ReplicationPeerConfig.newBuilder()
+        .setClusterKey("localhost:" + UTIL.getZkCluster().getClientPort() + ":/hbase2").build();
+    admin.addReplicationPeer(peerId, peerConfig, true);
+
+    assertEquals(peerConfig.getClusterKey(),
+      admin.getReplicationPeerConfig(peerId).getClusterKey());
+    ReplicationPeerConfig newPeerConfig =
+        ReplicationPeerConfig.newBuilder(peerConfig).setBandwidth(123456).build();
+    admin.updateReplicationPeerConfig(peerId, newPeerConfig);
+    assertEquals(newPeerConfig.getBandwidth(),
+      admin.getReplicationPeerConfig(peerId).getBandwidth());
+
+    admin.disableReplicationPeer(peerId);
+    assertFalse(admin.listReplicationPeers().get(0).isEnabled());
+
+    admin.enableReplicationPeer(peerId);
+    assertTrue(admin.listReplicationPeers().get(0).isEnabled());
+
+    admin.removeReplicationPeer(peerId);
+    assertTrue(admin.listReplicationPeers().isEmpty());
+
+    // make sure that we have run into the mocked method
+    MockHMaster master = (MockHMaster) UTIL.getHBaseCluster().getMaster();
+    assertTrue(master.addPeerCalled);
+    assertTrue(master.removePeerCalled);
+    assertTrue(master.updatePeerConfigCalled);
+    assertTrue(master.enablePeerCalled);
+    assertTrue(master.disablePeerCalled);
+  }
+
+  @Test
+  public void testErrorBeforeUpdate() throws IOException, ReplicationException {
+    ((MockHMaster) UTIL.getHBaseCluster().getMaster()).reset(true);
+    doTest();
+  }
+
+  @Test
+  public void testErrorAfterUpdate() throws IOException, ReplicationException {
+    ((MockHMaster) UTIL.getHBaseCluster().getMaster()).reset(false);
+    doTest();
+  }
+
+  public static final class MockHMaster extends HMaster {
+
+    volatile boolean addPeerCalled;
+
+    volatile boolean removePeerCalled;
+
+    volatile boolean updatePeerConfigCalled;
+
+    volatile boolean enablePeerCalled;
+
+    volatile boolean disablePeerCalled;
+
+    private ReplicationPeerManager manager;
+
+    public MockHMaster(Configuration conf) throws IOException, KeeperException {
+      super(conf);
+    }
+
+    private Object invokeWithError(InvocationOnMock invocation, boolean errorBeforeUpdate)
+        throws Throwable {
+      if (errorBeforeUpdate) {
+        throw new ReplicationException("mock error before update");
+      }
+      invocation.callRealMethod();
+      throw new ReplicationException("mock error after update");
+    }
+
+    public void reset(boolean errorBeforeUpdate) throws ReplicationException {
+      addPeerCalled = false;
+      removePeerCalled = false;
+      updatePeerConfigCalled = false;
+      enablePeerCalled = false;
+      disablePeerCalled = false;
+      ReplicationPeerManager m = super.getReplicationPeerManager();
+      manager = spy(m);
+      doAnswer(invocation -> {
+        if (!addPeerCalled) {
+          addPeerCalled = true;
+          return invokeWithError(invocation, errorBeforeUpdate);
+        } else {
+          return invocation.callRealMethod();
+        }
+      }).when(manager).addPeer(anyString(), any(ReplicationPeerConfig.class), anyBoolean());
+      doAnswer(invocation -> {
+        if (!removePeerCalled) {
+          removePeerCalled = true;
+          return invokeWithError(invocation, errorBeforeUpdate);
+        } else {
+          return invocation.callRealMethod();
+        }
+      }).when(manager).removePeer(anyString());
+      doAnswer(invocation -> {
+        if (!updatePeerConfigCalled) {
+          updatePeerConfigCalled = true;
+          return invokeWithError(invocation, errorBeforeUpdate);
+        } else {
+          return invocation.callRealMethod();
+        }
+      }).when(manager).updatePeerConfig(anyString(), any(ReplicationPeerConfig.class));
+      doAnswer(invocation -> {
+        if (!enablePeerCalled) {
+          enablePeerCalled = true;
+          return invokeWithError(invocation, errorBeforeUpdate);
+        } else {
+          return invocation.callRealMethod();
+        }
+      }).when(manager).enablePeer(anyString());
+      doAnswer(invocation -> {
+        if (!disablePeerCalled) {
+          disablePeerCalled = true;
+          return invokeWithError(invocation, errorBeforeUpdate);
+        } else {
+          return invocation.callRealMethod();
+        }
+      }).when(manager).disablePeer(anyString());
+    }
+
+    @Override
+    public ReplicationPeerManager getReplicationPeerManager() {
+      return manager;
+    }
+  }
+}


[25/26] hbase git commit: HBASE-19617 Remove ReplicationQueues, use ReplicationQueueStorage directly

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/264890a9/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
index b6cf54d..4b9ed74 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.util.List;
 import java.util.UUID;
 
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -32,9 +31,10 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Interface that defines a replication source
@@ -47,15 +47,10 @@ public interface ReplicationSourceInterface {
    * @param conf the configuration to use
    * @param fs the file system to use
    * @param manager the manager to use
-   * @param replicationQueues
-   * @param replicationPeers
    * @param server the server for this region server
-   * @param peerClusterZnode
-   * @param clusterId
-   * @throws IOException
    */
   void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
-      ReplicationQueues replicationQueues, ReplicationPeers replicationPeers, Server server,
+      ReplicationQueueStorage queueStorage, ReplicationPeers replicationPeers, Server server,
       String peerClusterZnode, UUID clusterId, ReplicationEndpoint replicationEndpoint,
       WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/264890a9/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
index a263fc3..f5e0e1b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
@@ -34,12 +34,14 @@ import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -47,6 +49,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Connection;
@@ -60,7 +63,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeer;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -68,6 +71,7 @@ import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
@@ -95,7 +99,7 @@ public class ReplicationSourceManager implements ReplicationListener {
   private final List<ReplicationSourceInterface> sources;
   // List of all the sources we got from died RSs
   private final List<ReplicationSourceInterface> oldsources;
-  private final ReplicationQueues replicationQueues;
+  private final ReplicationQueueStorage queueStorage;
   private final ReplicationTracker replicationTracker;
   private final ReplicationPeers replicationPeers;
   // UUID for this cluster
@@ -130,7 +134,7 @@ public class ReplicationSourceManager implements ReplicationListener {
 
   /**
    * Creates a replication manager and sets the watch on all the other registered region servers
-   * @param replicationQueues the interface for manipulating replication queues
+   * @param queueStorage the interface for manipulating replication queues
    * @param replicationPeers
    * @param replicationTracker
    * @param conf the configuration to use
@@ -140,14 +144,14 @@ public class ReplicationSourceManager implements ReplicationListener {
    * @param oldLogDir the directory where old logs are archived
    * @param clusterId
    */
-  public ReplicationSourceManager(ReplicationQueues replicationQueues,
+  public ReplicationSourceManager(ReplicationQueueStorage queueStorage,
       ReplicationPeers replicationPeers, ReplicationTracker replicationTracker, Configuration conf,
       Server server, FileSystem fs, Path logDir, Path oldLogDir, UUID clusterId,
       WALFileLengthProvider walFileLengthProvider) throws IOException {
     //CopyOnWriteArrayList is thread-safe.
     //Generally, reading is more than modifying.
     this.sources = new CopyOnWriteArrayList<>();
-    this.replicationQueues = replicationQueues;
+    this.queueStorage = queueStorage;
     this.replicationPeers = replicationPeers;
     this.replicationTracker = replicationTracker;
     this.server = server;
@@ -184,6 +188,19 @@ public class ReplicationSourceManager implements ReplicationListener {
     connection = ConnectionFactory.createConnection(conf);
   }
 
+  @FunctionalInterface
+  private interface ReplicationQueueOperation {
+    void exec() throws ReplicationException;
+  }
+
+  private void abortWhenFail(ReplicationQueueOperation op) {
+    try {
+      op.exec();
+    } catch (ReplicationException e) {
+      server.abort("Failed to operate on replication queue", e);
+    }
+  }
+
   /**
    * Provide the id of the peer and a log key and this method will figure which
    * wal it belongs to and will log, for this region server, the current
@@ -195,12 +212,13 @@ public class ReplicationSourceManager implements ReplicationListener {
    * @param queueRecovered indicates if this queue comes from another region server
    * @param holdLogInZK if true then the log is retained in ZK
    */
-  public void logPositionAndCleanOldLogs(Path log, String id, long position,
-      boolean queueRecovered, boolean holdLogInZK) {
+  public void logPositionAndCleanOldLogs(Path log, String id, long position, boolean queueRecovered,
+      boolean holdLogInZK) {
     String fileName = log.getName();
-    this.replicationQueues.setLogPosition(id, fileName, position);
+    abortWhenFail(
+      () -> this.queueStorage.setWALPosition(server.getServerName(), id, fileName, position));
     if (holdLogInZK) {
-     return;
+      return;
     }
     cleanOldLogs(fileName, id, queueRecovered);
   }
@@ -227,36 +245,59 @@ public class ReplicationSourceManager implements ReplicationListener {
         }
       }
     }
- }
+  }
 
   private void cleanOldLogs(SortedSet<String> wals, String key, String id) {
     SortedSet<String> walSet = wals.headSet(key);
-    LOG.debug("Removing " + walSet.size() + " logs in the list: " + walSet);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Removing " + walSet.size() + " logs in the list: " + walSet);
+    }
     for (String wal : walSet) {
-      this.replicationQueues.removeLog(id, wal);
+      abortWhenFail(() -> this.queueStorage.removeWAL(server.getServerName(), id, wal));
     }
     walSet.clear();
   }
 
+  private void adoptAbandonedQueues() {
+    List<ServerName> currentReplicators = null;
+    try {
+      currentReplicators = queueStorage.getListOfReplicators();
+    } catch (ReplicationException e) {
+      server.abort("Failed to get all replicators", e);
+      return;
+    }
+    if (currentReplicators == null || currentReplicators.isEmpty()) {
+      return;
+    }
+    List<ServerName> otherRegionServers = replicationTracker.getListOfRegionServers().stream()
+        .map(ServerName::valueOf).collect(Collectors.toList());
+    LOG.info(
+      "Current list of replicators: " + currentReplicators + " other RSs: " + otherRegionServers);
+
+    // Look if there's anything to process after a restart
+    for (ServerName rs : currentReplicators) {
+      if (!otherRegionServers.contains(rs)) {
+        transferQueues(rs);
+      }
+    }
+  }
+
   /**
-   * Adds a normal source per registered peer cluster and tries to process all
-   * old region server wal queues
+   * Adds a normal source per registered peer cluster and tries to process all old region server wal
+   * queues
+   * <p>
+   * The returned future is for adoptAbandonedQueues task.
    */
-  void init() throws IOException, ReplicationException {
+  Future<?> init() throws IOException, ReplicationException {
     for (String id : this.replicationPeers.getConnectedPeerIds()) {
       addSource(id);
       if (replicationForBulkLoadDataEnabled) {
         // Check if peer exists in hfile-refs queue, if not add it. This can happen in the case
         // when a peer was added before replication for bulk loaded data was enabled.
-        this.replicationQueues.addPeerToHFileRefs(id);
+        this.queueStorage.addPeerToHFileRefs(id);
       }
     }
-    AdoptAbandonedQueuesWorker adoptionWorker = new AdoptAbandonedQueuesWorker();
-    try {
-      this.executor.execute(adoptionWorker);
-    } catch (RejectedExecutionException ex) {
-      LOG.info("Cancelling the adoption of abandoned queues because of " + ex.getMessage());
-    }
+    return this.executor.submit(this::adoptAbandonedQueues);
   }
 
   /**
@@ -264,15 +305,12 @@ public class ReplicationSourceManager implements ReplicationListener {
    * need to enqueue the latest log of each wal group and do replication
    * @param id the id of the peer cluster
    * @return the source that was created
-   * @throws IOException
    */
   @VisibleForTesting
   ReplicationSourceInterface addSource(String id) throws IOException, ReplicationException {
     ReplicationPeerConfig peerConfig = replicationPeers.getReplicationPeerConfig(id);
     ReplicationPeer peer = replicationPeers.getConnectedPeer(id);
-    ReplicationSourceInterface src = getReplicationSource(this.conf, this.fs, this,
-      this.replicationQueues, this.replicationPeers, server, id, this.clusterId, peerConfig, peer,
-      walFileLengthProvider);
+    ReplicationSourceInterface src = getReplicationSource(id, peerConfig, peer);
     synchronized (this.walsById) {
       this.sources.add(src);
       Map<String, SortedSet<String>> walsByGroup = new HashMap<>();
@@ -287,11 +325,10 @@ public class ReplicationSourceManager implements ReplicationListener {
             logs.add(name);
             walsByGroup.put(walPrefix, logs);
             try {
-              this.replicationQueues.addLog(id, name);
+              this.queueStorage.addWAL(server.getServerName(), id, name);
             } catch (ReplicationException e) {
-              String message =
-                  "Cannot add log to queue when creating a new source, queueId=" + id
-                      + ", filename=" + name;
+              String message = "Cannot add log to queue when creating a new source, queueId=" + id +
+                ", filename=" + name;
               server.stop(message);
               throw e;
             }
@@ -316,7 +353,7 @@ public class ReplicationSourceManager implements ReplicationListener {
    * @param peerId Id of the peer cluster queue of wals to delete
    */
   public void deleteSource(String peerId, boolean closeConnection) {
-    this.replicationQueues.removeQueue(peerId);
+    abortWhenFail(() -> this.queueStorage.removeQueue(server.getServerName(), peerId));
     if (closeConnection) {
       this.replicationPeers.peerDisconnected(peerId);
     }
@@ -376,8 +413,8 @@ public class ReplicationSourceManager implements ReplicationListener {
   }
 
   @VisibleForTesting
-  List<String> getAllQueues() {
-    return replicationQueues.getAllQueues();
+  List<String> getAllQueues() throws ReplicationException {
+    return queueStorage.getAllQueues(server.getServerName());
   }
 
   void preLogRoll(Path newLog) throws IOException {
@@ -411,10 +448,10 @@ public class ReplicationSourceManager implements ReplicationListener {
     synchronized (replicationPeers) {
       for (String id : replicationPeers.getConnectedPeerIds()) {
         try {
-          this.replicationQueues.addLog(id, logName);
+          this.queueStorage.addWAL(server.getServerName(), id, logName);
         } catch (ReplicationException e) {
-          throw new IOException("Cannot add log to replication queue"
-              + " when creating a new source, queueId=" + id + ", filename=" + logName, e);
+          throw new IOException("Cannot add log to replication queue" +
+            " when creating a new source, queueId=" + id + ", filename=" + logName, e);
         }
       }
     }
@@ -461,19 +498,11 @@ public class ReplicationSourceManager implements ReplicationListener {
 
   /**
    * Factory method to create a replication source
-   * @param conf the configuration to use
-   * @param fs the file system to use
-   * @param manager the manager to use
-   * @param server the server object for this region server
    * @param peerId the id of the peer cluster
    * @return the created source
-   * @throws IOException
    */
-  private ReplicationSourceInterface getReplicationSource(Configuration conf, FileSystem fs,
-      ReplicationSourceManager manager, ReplicationQueues replicationQueues,
-      ReplicationPeers replicationPeers, Server server, String peerId, UUID clusterId,
-      ReplicationPeerConfig peerConfig, ReplicationPeer replicationPeer,
-      WALFileLengthProvider walFileLengthProvider) throws IOException {
+  private ReplicationSourceInterface getReplicationSource(String peerId,
+      ReplicationPeerConfig peerConfig, ReplicationPeer replicationPeer) throws IOException {
     RegionServerCoprocessorHost rsServerHost = null;
     TableDescriptors tableDescriptors = null;
     if (server instanceof HRegionServer) {
@@ -490,9 +519,8 @@ public class ReplicationSourceManager implements ReplicationListener {
         // Default to HBase inter-cluster replication endpoint
         replicationEndpointImpl = HBaseInterClusterReplicationEndpoint.class.getName();
       }
-      @SuppressWarnings("rawtypes")
-      Class c = Class.forName(replicationEndpointImpl);
-      replicationEndpoint = (ReplicationEndpoint) c.newInstance();
+      replicationEndpoint = Class.forName(replicationEndpointImpl)
+          .asSubclass(ReplicationEndpoint.class).newInstance();
       if(rsServerHost != null) {
         ReplicationEndpoint newReplicationEndPoint = rsServerHost
             .postCreateReplicationEndPoint(replicationEndpoint);
@@ -509,7 +537,7 @@ public class ReplicationSourceManager implements ReplicationListener {
 
     MetricsSource metrics = new MetricsSource(peerId);
     // init replication source
-    src.init(conf, fs, manager, replicationQueues, replicationPeers, server, peerId, clusterId,
+    src.init(conf, fs, this, queueStorage, replicationPeers, server, peerId, clusterId,
       replicationEndpoint, walFileLengthProvider, metrics);
 
     // init replication endpoint
@@ -520,21 +548,21 @@ public class ReplicationSourceManager implements ReplicationListener {
   }
 
   /**
-   * Transfer all the queues of the specified to this region server.
-   * First it tries to grab a lock and if it works it will move the
-   * znodes and finally will delete the old znodes.
-   *
+   * Transfer all the queues of the specified to this region server. First it tries to grab a lock
+   * and if it works it will move the znodes and finally will delete the old znodes.
+   * <p>
    * It creates one old source for any type of source of the old rs.
-   * @param rsZnode
    */
-  private void transferQueues(String rsZnode) {
-    NodeFailoverWorker transfer =
-        new NodeFailoverWorker(rsZnode, this.replicationQueues, this.replicationPeers,
-            this.clusterId);
+  private void transferQueues(ServerName deadRS) {
+    if (server.getServerName().equals(deadRS)) {
+      // it's just us, give up
+      return;
+    }
+    NodeFailoverWorker transfer = new NodeFailoverWorker(deadRS);
     try {
       this.executor.execute(transfer);
     } catch (RejectedExecutionException ex) {
-      LOG.info("Cancelling the transfer of " + rsZnode + " because of " + ex.getMessage());
+      LOG.info("Cancelling the transfer of " + deadRS + " because of " + ex.getMessage());
     }
   }
 
@@ -571,7 +599,7 @@ public class ReplicationSourceManager implements ReplicationListener {
       LOG.info("Peer " + id + " connected success, trying to start the replication source thread.");
       addSource(id);
       if (replicationForBulkLoadDataEnabled) {
-        this.replicationQueues.addPeerToHFileRefs(id);
+        this.queueStorage.addPeerToHFileRefs(id);
       }
     }
   }
@@ -624,12 +652,12 @@ public class ReplicationSourceManager implements ReplicationListener {
       deleteSource(id, true);
     }
     // Remove HFile Refs znode from zookeeper
-    this.replicationQueues.removePeerFromHFileRefs(id);
+    abortWhenFail(() -> this.queueStorage.removePeerFromHFileRefs(id));
   }
 
   @Override
   public void regionServerRemoved(String regionserver) {
-    transferQueues(regionserver);
+    transferQueues(ServerName.valueOf(regionserver));
   }
 
   /**
@@ -638,37 +666,21 @@ public class ReplicationSourceManager implements ReplicationListener {
    */
   class NodeFailoverWorker extends Thread {
 
-    private String rsZnode;
-    private final ReplicationQueues rq;
-    private final ReplicationPeers rp;
-    private final UUID clusterId;
+    private final ServerName deadRS;
 
-    /**
-     * @param rsZnode
-     */
-    public NodeFailoverWorker(String rsZnode) {
-      this(rsZnode, replicationQueues, replicationPeers, ReplicationSourceManager.this.clusterId);
-    }
-
-    public NodeFailoverWorker(String rsZnode, final ReplicationQueues replicationQueues,
-        final ReplicationPeers replicationPeers, final UUID clusterId) {
-      super("Failover-for-"+rsZnode);
-      this.rsZnode = rsZnode;
-      this.rq = replicationQueues;
-      this.rp = replicationPeers;
-      this.clusterId = clusterId;
+    @VisibleForTesting
+    public NodeFailoverWorker(ServerName deadRS) {
+      super("Failover-for-" + deadRS);
+      this.deadRS = deadRS;
     }
 
     @Override
     public void run() {
-      if (this.rq.isThisOurRegionServer(rsZnode)) {
-        return;
-      }
       // Wait a bit before transferring the queues, we may be shutting down.
       // This sleep may not be enough in some cases.
       try {
         Thread.sleep(sleepBeforeFailover +
-            (long) (ThreadLocalRandom.current().nextFloat() * sleepBeforeFailover));
+          (long) (ThreadLocalRandom.current().nextFloat() * sleepBeforeFailover));
       } catch (InterruptedException e) {
         LOG.warn("Interrupted while waiting before transferring a queue.");
         Thread.currentThread().interrupt();
@@ -679,25 +691,30 @@ public class ReplicationSourceManager implements ReplicationListener {
         return;
       }
       Map<String, Set<String>> newQueues = new HashMap<>();
-      List<String> peers = rq.getUnClaimedQueueIds(rsZnode);
-      while (peers != null && !peers.isEmpty()) {
-        Pair<String, SortedSet<String>> peer = this.rq.claimQueue(rsZnode,
-          peers.get(ThreadLocalRandom.current().nextInt(peers.size())));
-        long sleep = sleepBeforeFailover/2;
-        if (peer != null) {
-          newQueues.put(peer.getFirst(), peer.getSecond());
-          sleep = sleepBeforeFailover;
+      try {
+        List<String> peers = queueStorage.getAllQueues(deadRS);
+        while (!peers.isEmpty()) {
+          Pair<String, SortedSet<String>> peer = queueStorage.claimQueue(deadRS,
+            peers.get(ThreadLocalRandom.current().nextInt(peers.size())), server.getServerName());
+          long sleep = sleepBeforeFailover / 2;
+          if (!peer.getSecond().isEmpty()) {
+            newQueues.put(peer.getFirst(), peer.getSecond());
+            sleep = sleepBeforeFailover;
+          }
+          try {
+            Thread.sleep(sleep);
+          } catch (InterruptedException e) {
+            LOG.warn("Interrupted while waiting before transferring a queue.");
+            Thread.currentThread().interrupt();
+          }
+          peers = queueStorage.getAllQueues(deadRS);
         }
-        try {
-          Thread.sleep(sleep);
-        } catch (InterruptedException e) {
-          LOG.warn("Interrupted while waiting before transferring a queue.");
-          Thread.currentThread().interrupt();
+        if (!peers.isEmpty()) {
+          queueStorage.removeReplicatorIfQueueIsEmpty(deadRS);
         }
-        peers = rq.getUnClaimedQueueIds(rsZnode);
-      }
-      if (peers != null) {
-        rq.removeReplicatorIfQueueIsEmpty(rsZnode);
+      } catch (ReplicationException e) {
+        server.abort("Failed to claim queue from dead regionserver", e);
+        return;
       }
       // Copying over the failed queue is completed.
       if (newQueues.isEmpty()) {
@@ -722,8 +739,8 @@ public class ReplicationSourceManager implements ReplicationListener {
                 + ex);
           }
           if (peer == null || peerConfig == null) {
-            LOG.warn("Skipping failover for peer:" + actualPeerId + " of node" + rsZnode);
-            replicationQueues.removeQueue(peerId);
+            LOG.warn("Skipping failover for peer:" + actualPeerId + " of node " + deadRS);
+            abortWhenFail(() -> queueStorage.removeQueue(server.getServerName(), peerId));
             continue;
           }
           // track sources in walsByIdRecoveredQueues
@@ -740,13 +757,11 @@ public class ReplicationSourceManager implements ReplicationListener {
           }
 
           // enqueue sources
-          ReplicationSourceInterface src =
-              getReplicationSource(conf, fs, ReplicationSourceManager.this, this.rq, this.rp,
-                server, peerId, this.clusterId, peerConfig, peer, walFileLengthProvider);
+          ReplicationSourceInterface src = getReplicationSource(peerId, peerConfig, peer);
           // synchronized on oldsources to avoid adding recovered source for the to-be-removed peer
           // see removePeer
           synchronized (oldsources) {
-            if (!this.rp.getConnectedPeerIds().contains(src.getPeerId())) {
+            if (!replicationPeers.getConnectedPeerIds().contains(src.getPeerId())) {
               src.terminate("Recovered queue doesn't belong to any current peer");
               closeRecoveredQueue(src);
               continue;
@@ -765,29 +780,6 @@ public class ReplicationSourceManager implements ReplicationListener {
     }
   }
 
-  class AdoptAbandonedQueuesWorker extends Thread{
-
-    public AdoptAbandonedQueuesWorker() {}
-
-    @Override
-    public void run() {
-      List<String> currentReplicators = replicationQueues.getListOfReplicators();
-      if (currentReplicators == null || currentReplicators.isEmpty()) {
-        return;
-      }
-      List<String> otherRegionServers = replicationTracker.getListOfRegionServers();
-      LOG.info("Current list of replicators: " + currentReplicators + " other RSs: "
-        + otherRegionServers);
-
-      // Look if there's anything to process after a restart
-      for (String rs : currentReplicators) {
-        if (!otherRegionServers.contains(rs)) {
-          transferQueues(rs);
-        }
-      }
-    }
-  }
-
   /**
    * Get the directory where wals are archived
    * @return the directory where wals are archived
@@ -846,7 +838,11 @@ public class ReplicationSourceManager implements ReplicationListener {
   }
 
   public void cleanUpHFileRefs(String peerId, List<String> files) {
-    this.replicationQueues.removeHFileRefs(peerId, files);
+    abortWhenFail(() -> this.queueStorage.removeHFileRefs(peerId, files));
+  }
+
+  int activeFailoverTaskCount() {
+    return executor.getActiveCount();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/264890a9/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
index 21b8ac5..9ec244a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java
@@ -1,4 +1,4 @@
-/*
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
@@ -37,22 +36,19 @@ import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
- * In a scenario of Replication based Disaster/Recovery, when hbase
- * Master-Cluster crashes, this tool is used to sync-up the delta from Master to
- * Slave using the info from ZooKeeper. The tool will run on Master-Cluser, and
- * assume ZK, Filesystem and NetWork still available after hbase crashes
+ * In a scenario of Replication based Disaster/Recovery, when hbase Master-Cluster crashes, this
+ * tool is used to sync-up the delta from Master to Slave using the info from ZooKeeper. The tool
+ * will run on Master-Cluser, and assume ZK, Filesystem and NetWork still available after hbase
+ * crashes
  *
+ * <pre>
  * hbase org.apache.hadoop.hbase.replication.regionserver.ReplicationSyncUp
+ * </pre>
  */
-
 public class ReplicationSyncUp extends Configured implements Tool {
 
-  private static final Logger LOG = LoggerFactory.getLogger(ReplicationSyncUp.class.getName());
-
   private static Configuration conf;
 
   private static final long SLEEP_TIME = 10000;
@@ -105,13 +101,14 @@ public class ReplicationSyncUp extends Configured implements Tool {
     System.out.println("Start Replication Server start");
     replication = new Replication(new DummyServer(zkw), fs, logDir, oldLogDir);
     manager = replication.getReplicationManager();
-    manager.init();
+    manager.init().get();
 
     try {
-      int numberOfOldSource = 1; // default wait once
-      while (numberOfOldSource > 0) {
+      while (manager.activeFailoverTaskCount() > 0) {
+        Thread.sleep(SLEEP_TIME);
+      }
+      while (manager.getOldSources().size() > 0) {
         Thread.sleep(SLEEP_TIME);
-        numberOfOldSource = manager.getOldSources().size();
       }
     } catch (InterruptedException e) {
       System.err.println("didn't wait long enough:" + e);
@@ -121,7 +118,7 @@ public class ReplicationSyncUp extends Configured implements Tool {
     manager.join();
     zkw.close();
 
-    return (0);
+    return 0;
   }
 
   static class DummyServer implements Server {

http://git-wip-us.apache.org/repos/asf/hbase/blob/264890a9/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
index e678e1b..8e7c83f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
@@ -45,9 +45,8 @@ import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.replication.ReplicationFactory;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -112,9 +111,8 @@ public class TestLogsCleaner {
 
     Replication.decorateMasterConfiguration(conf);
     Server server = new DummyServer();
-    ReplicationQueues repQueues = ReplicationFactory.getReplicationQueues(
-        new ReplicationQueuesArguments(conf, server, server.getZooKeeper()));
-    repQueues.init(server.getServerName().toString());
+    ReplicationQueueStorage queueStorage =
+        ReplicationStorageFactory.getReplicationQueueStorage(server.getZooKeeper(), conf);
     final Path oldLogDir = new Path(TEST_UTIL.getDataTestDir(), HConstants.HREGION_OLDLOGDIR_NAME);
     final Path oldProcedureWALDir = new Path(oldLogDir, "masterProcedureWALs");
     String fakeMachineName = URLEncoder.encode(server.getServerName().toString(), "UTF8");
@@ -145,7 +143,7 @@ public class TestLogsCleaner {
       // Case 4: put 3 WALs in ZK indicating that they are scheduled for replication so these
       // files would pass TimeToLiveLogCleaner but would be rejected by ReplicationLogCleaner
       if (i % (30 / 3) == 1) {
-        repQueues.addLog(fakeMachineName, fileName.getName());
+        queueStorage.addWAL(server.getServerName(), fakeMachineName, fileName.getName());
         LOG.info("Replication log file: " + fileName);
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/264890a9/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
index cfad645..6a34493 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java
@@ -47,9 +47,8 @@ import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -73,19 +72,16 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
 
 @Category({ MasterTests.class, SmallTests.class })
 public class TestReplicationHFileCleaner {
-  private static final Logger LOG = LoggerFactory.getLogger(ReplicationQueuesZKImpl.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TestReplicationHFileCleaner.class);
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static Server server;
-  private static ReplicationQueues rq;
+  private static ReplicationQueueStorage rq;
   private static ReplicationPeers rp;
   private static final String peerId = "TestReplicationHFileCleaner";
   private static Configuration conf = TEST_UTIL.getConfiguration();
   static FileSystem fs = null;
   Path root;
 
-  /**
-   * @throws java.lang.Exception
-   */
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL.startMiniZKCluster();
@@ -94,20 +90,10 @@ public class TestReplicationHFileCleaner {
     Replication.decorateMasterConfiguration(conf);
     rp = ReplicationFactory.getReplicationPeers(server.getZooKeeper(), conf, server);
     rp.init();
-    rq = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, server, server.getZooKeeper()));
-    rq.init(server.getServerName().toString());
-    try {
-      fs = FileSystem.get(conf);
-    } finally {
-      if (fs != null) {
-        fs.close();
-      }
-    }
+    rq = ReplicationStorageFactory.getReplicationQueueStorage(server.getZooKeeper(), conf);
+    fs = FileSystem.get(conf);
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     TEST_UTIL.shutdownMiniZKCluster();

http://git-wip-us.apache.org/repos/asf/hbase/blob/264890a9/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java
index 8178266..2ad8bd7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationZKNodeCleaner.java
@@ -26,10 +26,8 @@ import java.util.Map;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.replication.ReplicationFactory;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesZKImpl;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
@@ -49,14 +47,12 @@ public class TestReplicationZKNodeCleaner {
 
   private final Configuration conf;
   private final ZKWatcher zkw;
-  private final ReplicationQueues repQueues;
+  private final ReplicationQueueStorage repQueues;
 
   public TestReplicationZKNodeCleaner() throws Exception {
     conf = TEST_UTIL.getConfiguration();
     zkw = new ZKWatcher(conf, "TestReplicationZKNodeCleaner", null);
-    repQueues = ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, null,
-        zkw));
-    assertTrue(repQueues instanceof ReplicationQueuesZKImpl);
+    repQueues = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf);
   }
 
   @BeforeClass
@@ -72,9 +68,8 @@ public class TestReplicationZKNodeCleaner {
 
   @Test
   public void testReplicationZKNodeCleaner() throws Exception {
-    repQueues.init(SERVER_ONE.getServerName());
     // add queue for ID_ONE which isn't exist
-    repQueues.addLog(ID_ONE, "file1");
+    repQueues.addWAL(SERVER_ONE, ID_ONE, "file1");
 
     ReplicationZKNodeCleaner cleaner = new ReplicationZKNodeCleaner(conf, zkw, null);
     Map<ServerName, List<String>> undeletedQueues = cleaner.getUnDeletedQueues();
@@ -84,7 +79,7 @@ public class TestReplicationZKNodeCleaner {
     assertTrue(undeletedQueues.get(SERVER_ONE).contains(ID_ONE));
 
     // add a recovery queue for ID_TWO which isn't exist
-    repQueues.addLog(ID_TWO + "-" + SERVER_TWO, "file2");
+    repQueues.addWAL(SERVER_ONE, ID_TWO + "-" + SERVER_TWO, "file2");
 
     undeletedQueues = cleaner.getUnDeletedQueues();
     assertEquals(1, undeletedQueues.size());
@@ -100,11 +95,10 @@ public class TestReplicationZKNodeCleaner {
 
   @Test
   public void testReplicationZKNodeCleanerChore() throws Exception {
-    repQueues.init(SERVER_ONE.getServerName());
     // add queue for ID_ONE which isn't exist
-    repQueues.addLog(ID_ONE, "file1");
+    repQueues.addWAL(SERVER_ONE, ID_ONE, "file1");
     // add a recovery queue for ID_TWO which isn't exist
-    repQueues.addLog(ID_TWO + "-" + SERVER_TWO, "file2");
+    repQueues.addWAL(SERVER_ONE, ID_TWO + "-" + SERVER_TWO, "file2");
 
     // Wait the cleaner chore to run
     Thread.sleep(20000);

http://git-wip-us.apache.org/repos/asf/hbase/blob/264890a9/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
index 7ea79f9..14c5e56 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/ReplicationSourceDummy.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -46,9 +45,10 @@ public class ReplicationSourceDummy implements ReplicationSourceInterface {
   MetricsSource metrics;
   WALFileLengthProvider walFileLengthProvider;
   AtomicBoolean startup = new AtomicBoolean(false);
+
   @Override
   public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
-      ReplicationQueues rq, ReplicationPeers rp, Server server, String peerClusterId,
+      ReplicationQueueStorage rq, ReplicationPeers rp, Server server, String peerClusterId,
       UUID clusterId, ReplicationEndpoint replicationEndpoint,
       WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException {
     this.manager = manager;

http://git-wip-us.apache.org/repos/asf/hbase/blob/264890a9/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
index 0a602ad..0313b3b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationSyncUpTool.java
@@ -1,4 +1,4 @@
-/*
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -64,7 +64,6 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
 
   @Before
   public void setUp() throws Exception {
-
     HColumnDescriptor fam;
 
     t1_syncupSource = new HTableDescriptor(t1_su);
@@ -100,7 +99,7 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
    * check's gone Also check the puts and deletes are not replicated back to
    * the originating cluster.
    */
-  @Test(timeout = 300000)
+  @Test
   public void testSyncUpTool() throws Exception {
 
     /**
@@ -176,7 +175,6 @@ public class TestReplicationSyncUpTool extends TestReplicationBase {
      * verify correctly replicated to Slave
      */
     mimicSyncUpAfterPut();
-
   }
 
   protected void setupReplication() throws Exception {

http://git-wip-us.apache.org/repos/asf/hbase/blob/264890a9/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
index 9f234a8..ed0814d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java
@@ -67,10 +67,10 @@ import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationSourceDummy;
 import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.NodeFailoverWorker;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
@@ -95,6 +95,7 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@@ -328,18 +329,14 @@ public abstract class TestReplicationSourceManager {
 
   @Test
   public void testClaimQueues() throws Exception {
-    final Server server = new DummyServer("hostname0.example.org");
-
-
-    ReplicationQueues rq =
-        ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(server.getConfiguration(), server,
-          server.getZooKeeper()));
-    rq.init(server.getServerName().toString());
+    Server server = new DummyServer("hostname0.example.org");
+    ReplicationQueueStorage rq = ReplicationStorageFactory
+        .getReplicationQueueStorage(server.getZooKeeper(), server.getConfiguration());
     // populate some znodes in the peer znode
     files.add("log1");
     files.add("log2");
     for (String file : files) {
-      rq.addLog("1", file);
+      rq.addWAL(server.getServerName(), "1", file);
     }
     // create 3 DummyServers
     Server s1 = new DummyServer("dummyserver1.example.org");
@@ -347,12 +344,9 @@ public abstract class TestReplicationSourceManager {
     Server s3 = new DummyServer("dummyserver3.example.org");
 
     // create 3 DummyNodeFailoverWorkers
-    DummyNodeFailoverWorker w1 = new DummyNodeFailoverWorker(
-        server.getServerName().getServerName(), s1);
-    DummyNodeFailoverWorker w2 = new DummyNodeFailoverWorker(
-        server.getServerName().getServerName(), s2);
-    DummyNodeFailoverWorker w3 = new DummyNodeFailoverWorker(
-        server.getServerName().getServerName(), s3);
+    DummyNodeFailoverWorker w1 = new DummyNodeFailoverWorker(server.getServerName(), s1);
+    DummyNodeFailoverWorker w2 = new DummyNodeFailoverWorker(server.getServerName(), s2);
+    DummyNodeFailoverWorker w3 = new DummyNodeFailoverWorker(server.getServerName(), s3);
 
     latch = new CountDownLatch(3);
     // start the threads
@@ -371,11 +365,9 @@ public abstract class TestReplicationSourceManager {
 
   @Test
   public void testCleanupFailoverQueues() throws Exception {
-    final Server server = new DummyServer("hostname1.example.org");
-    ReplicationQueues rq =
-        ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(server.getConfiguration(), server,
-          server.getZooKeeper()));
-    rq.init(server.getServerName().toString());
+    Server server = new DummyServer("hostname1.example.org");
+    ReplicationQueueStorage rq = ReplicationStorageFactory
+        .getReplicationQueueStorage(server.getZooKeeper(), server.getConfiguration());
     // populate some znodes in the peer znode
     SortedSet<String> files = new TreeSet<>();
     String group = "testgroup";
@@ -384,19 +376,14 @@ public abstract class TestReplicationSourceManager {
     files.add(file1);
     files.add(file2);
     for (String file : files) {
-      rq.addLog("1", file);
+      rq.addWAL(server.getServerName(), "1", file);
     }
     Server s1 = new DummyServer("dummyserver1.example.org");
-    ReplicationQueues rq1 =
-        ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(s1.getConfiguration(), s1,
-            s1.getZooKeeper()));
-    rq1.init(s1.getServerName().toString());
     ReplicationPeers rp1 =
         ReplicationFactory.getReplicationPeers(s1.getZooKeeper(), s1.getConfiguration(), s1);
     rp1.init();
     NodeFailoverWorker w1 =
-        manager.new NodeFailoverWorker(server.getServerName().getServerName(), rq1, rp1, new UUID(
-            new Long(1), new Long(2)));
+        manager.new NodeFailoverWorker(server.getServerName());
     w1.run();
     assertEquals(1, manager.getWalsByIdRecoveredQueues().size());
     String id = "1-" + server.getServerName().getServerName();
@@ -408,17 +395,16 @@ public abstract class TestReplicationSourceManager {
 
   @Test
   public void testCleanupUnknownPeerZNode() throws Exception {
-    final Server server = new DummyServer("hostname2.example.org");
-    ReplicationQueues rq = ReplicationFactory.getReplicationQueues(
-      new ReplicationQueuesArguments(server.getConfiguration(), server, server.getZooKeeper()));
-    rq.init(server.getServerName().toString());
+    Server server = new DummyServer("hostname2.example.org");
+    ReplicationQueueStorage rq = ReplicationStorageFactory
+        .getReplicationQueueStorage(server.getZooKeeper(), server.getConfiguration());
     // populate some znodes in the peer znode
     // add log to an unknown peer
     String group = "testgroup";
-    rq.addLog("2", group + ".log1");
-    rq.addLog("2", group + ".log2");
+    rq.addWAL(server.getServerName(), "2", group + ".log1");
+    rq.addWAL(server.getServerName(), "2", group + ".log2");
 
-    NodeFailoverWorker w1 = manager.new NodeFailoverWorker(server.getServerName().getServerName());
+    NodeFailoverWorker w1 = manager.new NodeFailoverWorker(server.getServerName());
     w1.run();
 
     // The log of the unknown peer should be removed from zk
@@ -481,10 +467,8 @@ public abstract class TestReplicationSourceManager {
         .setClusterKey("localhost:" + utility.getZkCluster().getClientPort() + ":/hbase");
     try {
       DummyServer server = new DummyServer();
-      final ReplicationQueues rq =
-          ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(
-              server.getConfiguration(), server, server.getZooKeeper()));
-      rq.init(server.getServerName().toString());
+      ReplicationQueueStorage rq = ReplicationStorageFactory
+          .getReplicationQueueStorage(server.getZooKeeper(), server.getConfiguration());
       // Purposely fail ReplicationSourceManager.addSource() by causing ReplicationSourceInterface
       // initialization to throw an exception.
       conf.set("replication.replicationsource.implementation",
@@ -498,11 +482,11 @@ public abstract class TestReplicationSourceManager {
       assertNull(manager.getSource(peerId));
 
       // Create a replication queue for the fake peer
-      rq.addLog(peerId, "FakeFile");
+      rq.addWAL(server.getServerName(), peerId, "FakeFile");
       // Unregister peer, this should remove the peer and clear all queues associated with it
       // Need to wait for the ReplicationTracker to pick up the changes and notify listeners.
       removePeerAndWait(peerId);
-      assertFalse(rq.getAllQueues().contains(peerId));
+      assertFalse(rq.getAllQueues(server.getServerName()).contains(peerId));
     } finally {
       conf.set("replication.replicationsource.implementation", replicationSourceImplName);
       removePeerAndWait(peerId);
@@ -625,11 +609,12 @@ public abstract class TestReplicationSourceManager {
       }
     }
     Waiter.waitFor(conf, 20000, new Waiter.Predicate<Exception>() {
-      @Override public boolean evaluate() throws Exception {
+      @Override
+      public boolean evaluate() throws Exception {
         List<String> peers = rp.getAllPeerIds();
-        return (!manager.getAllQueues().contains(peerId)) && (rp.getConnectedPeer(peerId) == null)
-            && (!peers.contains(peerId))
-            && manager.getSource(peerId) == null;
+        return (!manager.getAllQueues().contains(peerId)) &&
+          (rp.getConnectedPeer(peerId) == null) && (!peers.contains(peerId)) &&
+          manager.getSource(peerId) == null;
       }
     });
   }
@@ -672,25 +657,24 @@ public abstract class TestReplicationSourceManager {
   static class DummyNodeFailoverWorker extends Thread {
     private Map<String, Set<String>> logZnodesMap;
     Server server;
-    private String deadRsZnode;
-    ReplicationQueues rq;
+    private ServerName deadRS;
+    ReplicationQueueStorage rq;
 
-    public DummyNodeFailoverWorker(String znode, Server s) throws Exception {
-      this.deadRsZnode = znode;
+    public DummyNodeFailoverWorker(ServerName deadRS, Server s) throws Exception {
+      this.deadRS = deadRS;
       this.server = s;
-      this.rq =
-          ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(server.getConfiguration(), server,
-            server.getZooKeeper()));
-      this.rq.init(this.server.getServerName().toString());
+      this.rq = ReplicationStorageFactory.getReplicationQueueStorage(server.getZooKeeper(),
+        server.getConfiguration());
     }
 
     @Override
     public void run() {
       try {
         logZnodesMap = new HashMap<>();
-        List<String> queues = rq.getUnClaimedQueueIds(deadRsZnode);
-        for(String queue:queues){
-          Pair<String, SortedSet<String>> pair = rq.claimQueue(deadRsZnode, queue);
+        List<String> queues = rq.getAllQueues(deadRS);
+        for (String queue : queues) {
+          Pair<String, SortedSet<String>> pair =
+              rq.claimQueue(deadRS, queue, server.getServerName());
           if (pair != null) {
             logZnodesMap.put(pair.getFirst(), pair.getSecond());
           }
@@ -729,7 +713,7 @@ public abstract class TestReplicationSourceManager {
 
     @Override
     public void init(Configuration conf, FileSystem fs, ReplicationSourceManager manager,
-        ReplicationQueues rq, ReplicationPeers rp, Server server, String peerClusterId,
+        ReplicationQueueStorage rq, ReplicationPeers rp, Server server, String peerClusterId,
         UUID clusterId, ReplicationEndpoint replicationEndpoint,
         WALFileLengthProvider walFileLengthProvider, MetricsSource metrics) throws IOException {
       throw new IOException("Failing deliberately");

http://git-wip-us.apache.org/repos/asf/hbase/blob/264890a9/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java
index aeab8b0..c6d9eef 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManagerZkImpl.java
@@ -25,11 +25,10 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
-import org.apache.hadoop.hbase.replication.ReplicationQueues;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationSourceDummy;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.junit.BeforeClass;
@@ -41,8 +40,9 @@ import org.junit.experimental.categories.Category;
  * ReplicationQueuesClientZkImpl. Also includes extra tests outside of those in
  * TestReplicationSourceManager that test ReplicationQueueZkImpl-specific behaviors.
  */
-@Category({ReplicationTests.class, MediumTests.class})
+@Category({ ReplicationTests.class, MediumTests.class })
 public class TestReplicationSourceManagerZkImpl extends TestReplicationSourceManager {
+
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     conf = HBaseConfiguration.create();
@@ -58,16 +58,14 @@ public class TestReplicationSourceManagerZkImpl extends TestReplicationSourceMan
   // Tests the naming convention of adopted queues for ReplicationQueuesZkImpl
   @Test
   public void testNodeFailoverDeadServerParsing() throws Exception {
-    final Server server = new DummyServer("ec2-54-234-230-108.compute-1.amazonaws.com");
-    ReplicationQueues repQueues =
-      ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(conf, server,
-        server.getZooKeeper()));
-    repQueues.init(server.getServerName().toString());
+    Server server = new DummyServer("ec2-54-234-230-108.compute-1.amazonaws.com");
+    ReplicationQueueStorage queueStorage =
+        ReplicationStorageFactory.getReplicationQueueStorage(server.getZooKeeper(), conf);
     // populate some znodes in the peer znode
     files.add("log1");
     files.add("log2");
     for (String file : files) {
-      repQueues.addLog("1", file);
+      queueStorage.addWAL(server.getServerName(), "1", file);
     }
 
     // create 3 DummyServers
@@ -76,30 +74,22 @@ public class TestReplicationSourceManagerZkImpl extends TestReplicationSourceMan
     Server s3 = new DummyServer("ec2-23-20-187-167.compute-1.amazonaws.com");
 
     // simulate three servers fail sequentially
-    ReplicationQueues rq1 =
-      ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(s1.getConfiguration(), s1,
-        s1.getZooKeeper()));
-    rq1.init(s1.getServerName().toString());
-    String serverName = server.getServerName().getServerName();
-    List<String> unclaimed = rq1.getUnClaimedQueueIds(serverName);
-    rq1.claimQueue(serverName, unclaimed.get(0)).getSecond();
-    rq1.removeReplicatorIfQueueIsEmpty(unclaimed.get(0));
-    ReplicationQueues rq2 =
-      ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(s2.getConfiguration(), s2,
-        s2.getZooKeeper()));
-    rq2.init(s2.getServerName().toString());
-    serverName = s1.getServerName().getServerName();
-    unclaimed = rq2.getUnClaimedQueueIds(serverName);
-    rq2.claimQueue(serverName, unclaimed.get(0)).getSecond();
-    rq2.removeReplicatorIfQueueIsEmpty(unclaimed.get(0));
-    ReplicationQueues rq3 =
-      ReplicationFactory.getReplicationQueues(new ReplicationQueuesArguments(s3.getConfiguration(), s3,
-        s3.getZooKeeper()));
-    rq3.init(s3.getServerName().toString());
-    serverName = s2.getServerName().getServerName();
-    unclaimed = rq3.getUnClaimedQueueIds(serverName);
-    String queue3 = rq3.claimQueue(serverName, unclaimed.get(0)).getFirst();
-    rq3.removeReplicatorIfQueueIsEmpty(unclaimed.get(0));
+    ServerName serverName = server.getServerName();
+    List<String> unclaimed = queueStorage.getAllQueues(serverName);
+    queueStorage.claimQueue(serverName, unclaimed.get(0), s1.getServerName());
+    queueStorage.removeReplicatorIfQueueIsEmpty(serverName);
+
+    serverName = s1.getServerName();
+    unclaimed = queueStorage.getAllQueues(serverName);
+    queueStorage.claimQueue(serverName, unclaimed.get(0), s2.getServerName());
+    queueStorage.removeReplicatorIfQueueIsEmpty(serverName);
+
+    serverName = s2.getServerName();
+    unclaimed = queueStorage.getAllQueues(serverName);
+    String queue3 =
+        queueStorage.claimQueue(serverName, unclaimed.get(0), s3.getServerName()).getFirst();
+    queueStorage.removeReplicatorIfQueueIsEmpty(serverName);
+
     ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(queue3);
     List<ServerName> result = replicationQueueInfo.getDeadRegionServers();
     // verify


[03/26] hbase git commit: HBASE-19628 ByteBufferCell should extend ExtendedCell

Posted by zh...@apache.org.
HBASE-19628 ByteBufferCell should extend ExtendedCell


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/6b39062e
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/6b39062e
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/6b39062e

Branch: refs/heads/HBASE-19397
Commit: 6b39062e8667e674fb9d5a7a3578fe7d21e7ef05
Parents: 467a466
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Wed Dec 27 22:09:42 2017 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Wed Dec 27 22:26:40 2017 +0800

----------------------------------------------------------------------
 .../mapreduce/MapReduceHFileSplitterJob.java    |   7 +-
 .../hadoop/hbase/filter/ColumnPrefixFilter.java |   8 +-
 .../hadoop/hbase/filter/KeyOnlyFilter.java      |  57 ++-
 .../hadoop/hbase/filter/PrefixFilter.java       |   8 +-
 .../hbase/shaded/protobuf/ProtobufUtil.java     |  20 +-
 .../hadoop/hbase/filter/TestKeyOnlyFilter.java  |   6 +-
 .../org/apache/hadoop/hbase/ByteBufferCell.java | 120 -----
 .../hadoop/hbase/ByteBufferExtendedCell.java    | 121 +++++
 .../hadoop/hbase/ByteBufferKeyOnlyKeyValue.java |  44 +-
 .../apache/hadoop/hbase/ByteBufferKeyValue.java |   6 +-
 .../apache/hadoop/hbase/CellComparatorImpl.java |  72 +--
 .../java/org/apache/hadoop/hbase/CellUtil.java  | 181 +++----
 .../hadoop/hbase/ExtendedCellBuilder.java       |   2 +-
 .../apache/hadoop/hbase/PrivateCellUtil.java    | 502 ++++++++++---------
 .../java/org/apache/hadoop/hbase/TagUtil.java   |   6 +-
 .../io/encoding/BufferedDataBlockEncoder.java   |  14 +-
 .../hbase/io/encoding/RowIndexSeekerV1.java     |   8 +-
 .../hadoop/hbase/TestByteBufferKeyValue.java    |   4 +-
 .../hbase/io/TestTagCompressionContext.java     |  10 +-
 .../hadoop/hbase/mapreduce/CellSortReducer.java |   4 +-
 .../hbase/mapreduce/HFileOutputFormat2.java     |   8 +-
 .../apache/hadoop/hbase/mapreduce/Import.java   |  10 +-
 .../hadoop/hbase/mapreduce/WALPlayer.java       |   6 +-
 .../apache/hadoop/hbase/util/MapReduceCell.java | 271 ----------
 .../hbase/util/MapReduceExtendedCell.java       | 270 ++++++++++
 .../hbase/mapreduce/TestImportExport.java       |   4 +-
 .../hadoop/hbase/mapreduce/TestWALPlayer.java   |   4 +-
 .../hadoop/hbase/io/hfile/HFileWriterImpl.java  |  29 +-
 .../hbase/regionserver/ByteBufferChunkCell.java |  48 --
 .../regionserver/ByteBufferChunkKeyValue.java   |  48 ++
 .../hadoop/hbase/regionserver/CellChunkMap.java |   2 +-
 .../hbase/regionserver/MemStoreLABImpl.java     |   4 +-
 .../regionserver/NoTagByteBufferChunkCell.java  |  48 --
 .../NoTagByteBufferChunkKeyValue.java           |  48 ++
 .../hbase/regionserver/RSRpcServices.java       |   6 +-
 .../encoding/TestBufferedDataBlockEncoder.java  |   4 +-
 36 files changed, 1059 insertions(+), 951 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java
----------------------------------------------------------------------
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java
index 31428d0..cb5df5e 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/mapreduce/MapReduceHFileSplitterJob.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.mapreduce.HFileInputFormat;
 import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2;
 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.MapReduceCell;
+import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Mapper;
@@ -78,7 +78,8 @@ public class MapReduceHFileSplitterJob extends Configured implements Tool {
     @Override
     public void map(NullWritable key, Cell value, Context context)
         throws IOException, InterruptedException {
-      context.write(new ImmutableBytesWritable(CellUtil.cloneRow(value)), new MapReduceCell(value));
+      context.write(new ImmutableBytesWritable(CellUtil.cloneRow(value)),
+          new MapReduceExtendedCell(value));
     }
 
     @Override
@@ -113,7 +114,7 @@ public class MapReduceHFileSplitterJob extends Configured implements Tool {
       job.setReducerClass(CellSortReducer.class);
       Path outputDir = new Path(hfileOutPath);
       FileOutputFormat.setOutputPath(job, outputDir);
-      job.setMapOutputValueClass(MapReduceCell.class);
+      job.setMapOutputValueClass(MapReduceExtendedCell.class);
       try (Connection conn = ConnectionFactory.createConnection(conf);
           Table table = conn.getTable(tableName);
           RegionLocator regionLocator = conn.getRegionLocator(tableName)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
index 4811691..b4e7a0f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
@@ -22,7 +22,7 @@ package org.apache.hadoop.hbase.filter;
 import java.io.IOException;
 import java.util.ArrayList;
 
-import org.apache.hadoop.hbase.ByteBufferCell;
+import org.apache.hadoop.hbase.ByteBufferExtendedCell;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -95,9 +95,9 @@ public class ColumnPrefixFilter extends FilterBase {
   }
 
   private static int compareQualifierPart(Cell cell, int length, byte[] prefix) {
-    if (cell instanceof ByteBufferCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferCell) cell).getQualifierByteBuffer(),
-          ((ByteBufferCell) cell).getQualifierPosition(), length, prefix, 0, length);
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getQualifierPosition(), length, prefix, 0, length);
     }
     return Bytes.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(), length, prefix, 0,
         length);

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
index 32286b6..4487292 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
@@ -18,21 +18,24 @@
  */
 package org.apache.hadoop.hbase.filter;
 
-
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
-
-import org.apache.hadoop.hbase.ByteBufferCell;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Optional;
+import org.apache.hadoop.hbase.ByteBufferExtendedCell;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ClassSize;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
 import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
 
 /**
  * A filter that will only return the key component of each KV (the value will
@@ -60,8 +63,8 @@ public class KeyOnlyFilter extends FilterBase {
   }
 
   private Cell createKeyOnlyCell(Cell c) {
-    if (c instanceof ByteBufferCell) {
-      return new KeyOnlyByteBufferCell((ByteBufferCell) c, lenAsVal);
+    if (c instanceof ByteBufferExtendedCell) {
+      return new KeyOnlyByteBufferExtendedCell((ByteBufferExtendedCell) c, lenAsVal);
     } else {
       return new KeyOnlyCell(c, lenAsVal);
     }
@@ -77,7 +80,7 @@ public class KeyOnlyFilter extends FilterBase {
   public ReturnCode filterCell(final Cell ignored) throws IOException {
     return ReturnCode.INCLUDE;
   }
-  
+
   public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
     Preconditions.checkArgument((filterArguments.isEmpty() || filterArguments.size() == 1),
                                 "Expected: 0 or 1 but got: %s", filterArguments.size());
@@ -244,11 +247,13 @@ public class KeyOnlyFilter extends FilterBase {
     }
   }
 
-  static class KeyOnlyByteBufferCell extends ByteBufferCell {
-    private ByteBufferCell cell;
+  static class KeyOnlyByteBufferExtendedCell extends ByteBufferExtendedCell {
+    public static final int FIXED_OVERHEAD = ClassSize.OBJECT + ClassSize.REFERENCE
+        + Bytes.SIZEOF_BOOLEAN;
+    private ByteBufferExtendedCell cell;
     private boolean lenAsVal;
 
-    public KeyOnlyByteBufferCell(ByteBufferCell c, boolean lenAsVal) {
+    public KeyOnlyByteBufferExtendedCell(ByteBufferExtendedCell c, boolean lenAsVal) {
       this.cell = c;
       this.lenAsVal = lenAsVal;
     }
@@ -309,6 +314,21 @@ public class KeyOnlyFilter extends FilterBase {
     }
 
     @Override
+    public void setSequenceId(long seqId) throws IOException {
+      cell.setSequenceId(seqId);
+    }
+
+    @Override
+    public void setTimestamp(long ts) throws IOException {
+      cell.setTimestamp(ts);
+    }
+
+    @Override
+    public void setTimestamp(byte[] ts) throws IOException {
+      cell.setTimestamp(ts);
+    }
+
+    @Override
     public long getSequenceId() {
       return 0;
     }
@@ -409,6 +429,21 @@ public class KeyOnlyFilter extends FilterBase {
     public int getTagsPosition() {
       return 0;
     }
+
+    @Override
+    public Iterator<Tag> getTags() {
+      return Collections.emptyIterator();
+    }
+
+    @Override
+    public Optional<Tag> getTag(byte type) {
+      return Optional.empty();
+    }
+
+    @Override
+    public long heapSize() {
+      return ClassSize.align(FIXED_OVERHEAD + cell.heapSize());
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
index 161c1a5..09a3304 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.hbase.filter;
 
 import java.util.ArrayList;
 
-import org.apache.hadoop.hbase.ByteBufferCell;
+import org.apache.hadoop.hbase.ByteBufferExtendedCell;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
@@ -61,9 +61,9 @@ public class PrefixFilter extends FilterBase {
     // else return true, filter row
     // if we are passed the prefix, set flag
     int cmp;
-    if (firstRowCell instanceof ByteBufferCell) {
-      cmp = ByteBufferUtils.compareTo(((ByteBufferCell) firstRowCell).getRowByteBuffer(),
-          ((ByteBufferCell) firstRowCell).getRowPosition(), this.prefix.length,
+    if (firstRowCell instanceof ByteBufferExtendedCell) {
+      cmp = ByteBufferUtils.compareTo(((ByteBufferExtendedCell) firstRowCell).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) firstRowCell).getRowPosition(), this.prefix.length,
           this.prefix, 0, this.prefix.length);
     } else {
       cmp = Bytes.compareTo(firstRowCell.getRowArray(), firstRowCell.getRowOffset(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index 17b1141..8f44135 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -40,7 +40,7 @@ import java.util.stream.Collectors;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.ByteBufferCell;
+import org.apache.hadoop.hbase.ByteBufferExtendedCell;
 import org.apache.hadoop.hbase.CacheEvictionStats;
 import org.apache.hadoop.hbase.CacheEvictionStatsBuilder;
 import org.apache.hadoop.hbase.Cell;
@@ -2039,17 +2039,17 @@ public final class ProtobufUtil {
     // Doing this is going to kill us if we do it for all data passed.
     // St.Ack 20121205
     CellProtos.Cell.Builder kvbuilder = CellProtos.Cell.newBuilder();
-    if (kv instanceof ByteBufferCell) {
-      kvbuilder.setRow(wrap(((ByteBufferCell) kv).getRowByteBuffer(),
-        ((ByteBufferCell) kv).getRowPosition(), kv.getRowLength()));
-      kvbuilder.setFamily(wrap(((ByteBufferCell) kv).getFamilyByteBuffer(),
-        ((ByteBufferCell) kv).getFamilyPosition(), kv.getFamilyLength()));
-      kvbuilder.setQualifier(wrap(((ByteBufferCell) kv).getQualifierByteBuffer(),
-        ((ByteBufferCell) kv).getQualifierPosition(), kv.getQualifierLength()));
+    if (kv instanceof ByteBufferExtendedCell) {
+      kvbuilder.setRow(wrap(((ByteBufferExtendedCell) kv).getRowByteBuffer(),
+        ((ByteBufferExtendedCell) kv).getRowPosition(), kv.getRowLength()));
+      kvbuilder.setFamily(wrap(((ByteBufferExtendedCell) kv).getFamilyByteBuffer(),
+        ((ByteBufferExtendedCell) kv).getFamilyPosition(), kv.getFamilyLength()));
+      kvbuilder.setQualifier(wrap(((ByteBufferExtendedCell) kv).getQualifierByteBuffer(),
+        ((ByteBufferExtendedCell) kv).getQualifierPosition(), kv.getQualifierLength()));
       kvbuilder.setCellType(CellProtos.CellType.valueOf(kv.getTypeByte()));
       kvbuilder.setTimestamp(kv.getTimestamp());
-      kvbuilder.setValue(wrap(((ByteBufferCell) kv).getValueByteBuffer(),
-        ((ByteBufferCell) kv).getValuePosition(), kv.getValueLength()));
+      kvbuilder.setValue(wrap(((ByteBufferExtendedCell) kv).getValueByteBuffer(),
+        ((ByteBufferExtendedCell) kv).getValuePosition(), kv.getValueLength()));
       // TODO : Once tags become first class then we may have to set tags to kvbuilder.
     } else {
       kvbuilder.setRow(

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java
index f957b59..28fe2bf 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/filter/TestKeyOnlyFilter.java
@@ -20,9 +20,7 @@ package org.apache.hadoop.hbase.filter;
 import static org.junit.Assert.assertTrue;
 
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
 import java.util.Collection;
-import java.util.List;
 import org.apache.hadoop.hbase.ByteBufferKeyValue;
 
 import org.apache.hadoop.hbase.CellUtil;
@@ -30,7 +28,7 @@ import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.KeyValueUtil;
-import org.apache.hadoop.hbase.filter.KeyOnlyFilter.KeyOnlyByteBufferCell;
+import org.apache.hadoop.hbase.filter.KeyOnlyFilter.KeyOnlyByteBufferExtendedCell;
 import org.apache.hadoop.hbase.filter.KeyOnlyFilter.KeyOnlyCell;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -82,7 +80,7 @@ public class TestKeyOnlyFilter {
     KeyValue KeyOnlyKeyValue = new KeyValue(newBuffer);
 
     KeyOnlyCell keyOnlyCell = new KeyOnlyCell(kv, lenAsVal);
-    KeyOnlyByteBufferCell keyOnlyByteBufferedCell = new KeyOnlyByteBufferCell(
+    KeyOnlyByteBufferExtendedCell keyOnlyByteBufferedCell = new KeyOnlyByteBufferExtendedCell(
         bbCell, lenAsVal);
 
     assertTrue(CellUtil.matchingRows(KeyOnlyKeyValue, keyOnlyCell));

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferCell.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferCell.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferCell.java
deleted file mode 100644
index b3be0ac..0000000
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferCell.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase;
-
-import java.nio.ByteBuffer;
-
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * This class is a server side extension to the {@link Cell} interface. It is used when the
- * Cell is backed by a {@link ByteBuffer}: i.e. <code>cell instanceof ByteBufferedCell</code>.
- *
- * <p>This class has getters for the row, column family, column qualifier, value and tags hosting
- * ByteBuffers. It also has getters of the *position* within a ByteBuffer where these
- * field bytes begin. These are needed because a single ByteBuffer may back one or many Cell
- * instances -- it depends on the implementation -- so the ByteBuffer position as returned by
- * {@link ByteBuffer#arrayOffset()} cannot be relied upon. Also, do not confuse these position
- * methods with the getXXXOffset methods from the super Interface, {@link Cell}; dependent up on
- * implementation, the Cell getXXXOffset methods can return the same value as a call to its
- * equivalent position method from below BUT they can also stray; if a ByteBufferedCell, use the
- * below position methods to find where a field begins.
- *
- * <p>Use the getXXXLength methods from Cell to find a fields length.
- *
- * <p>A Cell object can be of this type only on the server side.
- *
- * <p>WARNING: If a Cell is backed by an offheap ByteBuffer, any call to getXXXArray() will result
- * in a temporary byte array creation and a bytes copy. Avoid these allocations by using the
- * appropriate Cell access server-side: i.e. ByteBufferedCell when backed by a ByteBuffer and Cell
- * when it is not.
- */
-/*
- * Even though all the methods are abstract, ByteBufferCell is not made to be an interface with
- * intent. In CellComparator compare method, we have instance of check to decide whether to use
- * getXXXArray() or getXXXByteBuffer(). This is a very hot method in read and write paths.
- * if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
-      ....
-    }
-    if (left instanceof ByteBufferCell) {
-      ....
-    }
-    if (right instanceof ByteBufferCell) {
-      ....
-    }
-    return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
-        right.getRowArray(), right.getRowOffset(), right.getRowLength());
- * We did JMH micro benchmark tests with both left and right cells as ByteBufferCell, one only
- * ByteBufferCell and both as Cells. This is compared against JMH results on compare logic with out
- * any instance of checks. We noticed that if ByteBufferCell is an interface, the benchmark result
- * seems to be very bad for case of both right and left are Cell only (Not ByteBufferCell). When
- * ByteBufferCell is an abstract class all 4 possible cases giving almost similar performance number
- * compared with compare logic with no instance of checks.
- */
-@InterfaceAudience.Private
-public abstract class ByteBufferCell implements Cell {
-  /**
-   * @return The {@link ByteBuffer} containing the row bytes.
-   */
-  public abstract ByteBuffer getRowByteBuffer();
-
-  /**
-   * @return Position in the {@link ByteBuffer} where row bytes start
-   */
-  public abstract int getRowPosition();
-
-  /**
-   * @return The {@link ByteBuffer} containing the column family bytes.
-   */
-  public abstract ByteBuffer getFamilyByteBuffer();
-
-  /**
-   * @return Position in the {@link ByteBuffer} where column family bytes start
-   */
-  public abstract int getFamilyPosition();
-
-  /**
-   * @return The {@link ByteBuffer} containing the column qualifier bytes.
-   */
-  public abstract ByteBuffer getQualifierByteBuffer();
-
-  /**
-   * @return Position in the {@link ByteBuffer} where column qualifier bytes start
-   */
-  public abstract int getQualifierPosition();
-
-  /**
-   * @return The {@link ByteBuffer} containing the value bytes.
-   */
-  public abstract ByteBuffer getValueByteBuffer();
-
-  /**
-   * @return Position in the {@link ByteBuffer} where value bytes start
-   */
-  public abstract int getValuePosition();
-
-  /**
-   * @return The {@link ByteBuffer} containing the tag bytes.
-   */
-  public abstract ByteBuffer getTagsByteBuffer();
-
-  /**
-   * @return Position in the {@link ByteBuffer} where tag bytes start
-   */
-  public abstract int getTagsPosition();
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferExtendedCell.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferExtendedCell.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferExtendedCell.java
new file mode 100644
index 0000000..3e4cc95
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferExtendedCell.java
@@ -0,0 +1,121 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import java.nio.ByteBuffer;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * This class is a server side extension to the {@link Cell} interface. It is used when the
+ * Cell is backed by a {@link ByteBuffer}: i.e. <code>cell instanceof ByteBufferedCell</code>.
+ *
+ * <p>This class has getters for the row, column family, column qualifier, value and tags hosting
+ * ByteBuffers. It also has getters of the *position* within a ByteBuffer where these
+ * field bytes begin. These are needed because a single ByteBuffer may back one or many Cell
+ * instances -- it depends on the implementation -- so the ByteBuffer position as returned by
+ * {@link ByteBuffer#arrayOffset()} cannot be relied upon. Also, do not confuse these position
+ * methods with the getXXXOffset methods from the super Interface, {@link Cell}; dependent up on
+ * implementation, the Cell getXXXOffset methods can return the same value as a call to its
+ * equivalent position method from below BUT they can also stray; if a ByteBufferedCell, use the
+ * below position methods to find where a field begins.
+ *
+ * <p>Use the getXXXLength methods from Cell to find a fields length.
+ *
+ * <p>A Cell object can be of this type only on the server side.
+ *
+ * <p>WARNING: If a Cell is backed by an offheap ByteBuffer, any call to getXXXArray() will result
+ * in a temporary byte array creation and a bytes copy. Avoid these allocations by using the
+ * appropriate Cell access server-side: i.e. ByteBufferedCell when backed by a ByteBuffer and Cell
+ * when it is not.
+ */
+/*
+ * Even though all the methods are abstract, ByteBufferExtendedCell is not made to be an interface
+ * with intent. In CellComparator compare method, we have instance of check to decide whether to
+ * use getXXXArray() or getXXXByteBuffer(). This is a very hot method in read and write paths.
+ * if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
+      ....
+    }
+    if (left instanceof ByteBufferExtendedCell) {
+      ....
+    }
+    if (right instanceof ByteBufferExtendedCell) {
+      ....
+    }
+    return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
+        right.getRowArray(), right.getRowOffset(), right.getRowLength());
+ * We did JMH micro benchmark tests with both left and right cells as ByteBufferExtendedCell, one
+ * only ByteBufferExtendedCell and both as Cells. This is compared against JMH results on compare
+ * logic with out any instance of checks. We noticed that if ByteBufferExtendedCell is an
+ * interface, the benchmark result seems to be very bad for case of both right and left are Cell
+ * only (Not ByteBufferExtendedCell). When ByteBufferExtendedCell is an abstract class all 4
+ * possible cases giving almost similar performance number compared with compare logic with no
+ * instance of checks.
+ */
+@InterfaceAudience.Private
+public abstract class ByteBufferExtendedCell implements ExtendedCell {
+  /**
+   * @return The {@link ByteBuffer} containing the row bytes.
+   */
+  public abstract ByteBuffer getRowByteBuffer();
+
+  /**
+   * @return Position in the {@link ByteBuffer} where row bytes start
+   */
+  public abstract int getRowPosition();
+
+  /**
+   * @return The {@link ByteBuffer} containing the column family bytes.
+   */
+  public abstract ByteBuffer getFamilyByteBuffer();
+
+  /**
+   * @return Position in the {@link ByteBuffer} where column family bytes start
+   */
+  public abstract int getFamilyPosition();
+
+  /**
+   * @return The {@link ByteBuffer} containing the column qualifier bytes.
+   */
+  public abstract ByteBuffer getQualifierByteBuffer();
+
+  /**
+   * @return Position in the {@link ByteBuffer} where column qualifier bytes start
+   */
+  public abstract int getQualifierPosition();
+
+  /**
+   * @return The {@link ByteBuffer} containing the value bytes.
+   */
+  public abstract ByteBuffer getValueByteBuffer();
+
+  /**
+   * @return Position in the {@link ByteBuffer} where value bytes start
+   */
+  public abstract int getValuePosition();
+
+  /**
+   * @return The {@link ByteBuffer} containing the tag bytes.
+   */
+  public abstract ByteBuffer getTagsByteBuffer();
+
+  /**
+   * @return Position in the {@link ByteBuffer} where tag bytes start
+   */
+  public abstract int getTagsPosition();
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
index 713314e..7ecb946 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyOnlyKeyValue.java
@@ -18,10 +18,14 @@
  */
 package org.apache.hadoop.hbase;
 
+import java.io.IOException;
 import java.nio.ByteBuffer;
-
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Optional;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -30,8 +34,9 @@ import org.apache.yetus.audience.InterfaceAudience;
  * (onheap and offheap).
  */
 @InterfaceAudience.Private
-public class ByteBufferKeyOnlyKeyValue extends ByteBufferCell {
-
+public class ByteBufferKeyOnlyKeyValue extends ByteBufferExtendedCell {
+  public static final int FIXED_OVERHEAD = ClassSize.OBJECT + ClassSize.REFERENCE
+      + (2 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_SHORT;
   private ByteBuffer buf;
   private int offset = 0; // offset into buffer where key starts at
   private int length = 0; // length of this.
@@ -152,6 +157,21 @@ public class ByteBufferKeyOnlyKeyValue extends ByteBufferCell {
   }
 
   @Override
+  public void setSequenceId(long seqId) throws IOException {
+    throw new IllegalArgumentException("This is a key only Cell");
+  }
+
+  @Override
+  public void setTimestamp(long ts) throws IOException {
+    throw new IllegalArgumentException("This is a key only Cell");
+  }
+
+  @Override
+  public void setTimestamp(byte[] ts) throws IOException {
+    throw new IllegalArgumentException("This is a key only Cell");
+  }
+
+  @Override
   public long getSequenceId() {
     return 0;
   }
@@ -246,4 +266,22 @@ public class ByteBufferKeyOnlyKeyValue extends ByteBufferCell {
   public String toString() {
     return CellUtil.toString(this, false);
   }
+
+  @Override
+  public Iterator<Tag> getTags() {
+    return Collections.emptyIterator();
+  }
+
+  @Override
+  public Optional<Tag> getTag(byte type) {
+    return Optional.empty();
+  }
+
+  @Override
+  public long heapSize() {
+    if (this.buf.hasArray()) {
+      return ClassSize.align(FIXED_OVERHEAD + length);
+    }
+    return ClassSize.align(FIXED_OVERHEAD);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java
index 798180a..54af38f 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ByteBufferKeyValue.java
@@ -28,11 +28,11 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 
 /**
- * This Cell is an implementation of {@link ByteBufferCell} where the data resides in
+ * This Cell is an implementation of {@link ByteBufferExtendedCell} where the data resides in
  * off heap/ on heap ByteBuffer
  */
 @InterfaceAudience.Private
-public class ByteBufferKeyValue extends ByteBufferCell implements ExtendedCell {
+public class ByteBufferKeyValue extends ByteBufferExtendedCell {
 
   protected final ByteBuffer buf;
   protected final int offset;
@@ -329,7 +329,7 @@ public class ByteBufferKeyValue extends ByteBufferCell implements ExtendedCell {
     return calculateHashForKey(this);
   }
 
-  private int calculateHashForKey(ByteBufferCell cell) {
+  private int calculateHashForKey(ByteBufferExtendedCell cell) {
     int rowHash = ByteBufferUtils.hashCode(cell.getRowByteBuffer(), cell.getRowPosition(),
       cell.getRowLength());
     int familyHash = ByteBufferUtils.hashCode(cell.getFamilyByteBuffer(), cell.getFamilyPosition(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
index 771fdaa..f1abbef 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
@@ -108,26 +108,26 @@ public class CellComparatorImpl implements CellComparator {
    */
   @Override
   public final int compareFamilies(Cell left, Cell right) {
-    if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getFamilyByteBuffer(),
-          ((ByteBufferCell) left).getFamilyPosition(), left.getFamilyLength(),
-          ((ByteBufferCell) right).getFamilyByteBuffer(),
-          ((ByteBufferCell) right).getFamilyPosition(), right.getFamilyLength());
+    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
+          ((ByteBufferExtendedCell) left).getFamilyPosition(), left.getFamilyLength(),
+          ((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
+          ((ByteBufferExtendedCell) right).getFamilyPosition(), right.getFamilyLength());
     }
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getFamilyByteBuffer(),
-          ((ByteBufferCell) left).getFamilyPosition(), left.getFamilyLength(),
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
+          ((ByteBufferExtendedCell) left).getFamilyPosition(), left.getFamilyLength(),
           right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
     }
-    if (right instanceof ByteBufferCell) {
+    if (right instanceof ByteBufferExtendedCell) {
       // Notice how we flip the order of the compare here. We used to negate the return value but
       // see what FindBugs says
       // http://findbugs.sourceforge.net/bugDescriptions.html#RV_NEGATING_RESULT_OF_COMPARETO
       // It suggest flipping the order to get same effect and 'safer'.
       return ByteBufferUtils.compareTo(
           left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
-          ((ByteBufferCell)right).getFamilyByteBuffer(),
-          ((ByteBufferCell)right).getFamilyPosition(), right.getFamilyLength());
+          ((ByteBufferExtendedCell)right).getFamilyByteBuffer(),
+          ((ByteBufferExtendedCell)right).getFamilyPosition(), right.getFamilyLength());
     }
     return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
         right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
@@ -141,28 +141,28 @@ public class CellComparatorImpl implements CellComparator {
    */
   @Override
   public final int compareQualifiers(Cell left, Cell right) {
-    if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
+    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
       return ByteBufferUtils
-          .compareTo(((ByteBufferCell) left).getQualifierByteBuffer(),
-              ((ByteBufferCell) left).getQualifierPosition(),
-              left.getQualifierLength(), ((ByteBufferCell) right).getQualifierByteBuffer(),
-              ((ByteBufferCell) right).getQualifierPosition(),
+          .compareTo(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
+              ((ByteBufferExtendedCell) left).getQualifierPosition(),
+              left.getQualifierLength(), ((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
+              ((ByteBufferExtendedCell) right).getQualifierPosition(),
               right.getQualifierLength());
     }
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getQualifierByteBuffer(),
-          ((ByteBufferCell) left).getQualifierPosition(), left.getQualifierLength(),
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) left).getQualifierPosition(), left.getQualifierLength(),
           right.getQualifierArray(), right.getQualifierOffset(), right.getQualifierLength());
     }
-    if (right instanceof ByteBufferCell) {
+    if (right instanceof ByteBufferExtendedCell) {
       // Notice how we flip the order of the compare here. We used to negate the return value but
       // see what FindBugs says
       // http://findbugs.sourceforge.net/bugDescriptions.html#RV_NEGATING_RESULT_OF_COMPARETO
       // It suggest flipping the order to get same effect and 'safer'.
       return ByteBufferUtils.compareTo(left.getQualifierArray(),
           left.getQualifierOffset(), left.getQualifierLength(),
-          ((ByteBufferCell)right).getQualifierByteBuffer(),
-          ((ByteBufferCell)right).getQualifierPosition(), right.getQualifierLength());
+          ((ByteBufferExtendedCell)right).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell)right).getQualifierPosition(), right.getQualifierLength());
     }
     return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
         left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
@@ -183,25 +183,25 @@ public class CellComparatorImpl implements CellComparator {
     if (left == right) {
       return 0;
     }
-    if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getRowByteBuffer(),
-          ((ByteBufferCell) left).getRowPosition(), left.getRowLength(),
-          ((ByteBufferCell) right).getRowByteBuffer(),
-          ((ByteBufferCell) right).getRowPosition(), right.getRowLength());
+    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) left).getRowPosition(), left.getRowLength(),
+          ((ByteBufferExtendedCell) right).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) right).getRowPosition(), right.getRowLength());
     }
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getRowByteBuffer(),
-          ((ByteBufferCell) left).getRowPosition(), left.getRowLength(),
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) left).getRowPosition(), left.getRowLength(),
           right.getRowArray(), right.getRowOffset(), right.getRowLength());
     }
-    if (right instanceof ByteBufferCell) {
+    if (right instanceof ByteBufferExtendedCell) {
       // Notice how we flip the order of the compare here. We used to negate the return value but
       // see what FindBugs says
       // http://findbugs.sourceforge.net/bugDescriptions.html#RV_NEGATING_RESULT_OF_COMPARETO
       // It suggest flipping the order to get same effect and 'safer'.
       return ByteBufferUtils.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
-          ((ByteBufferCell)right).getRowByteBuffer(),
-          ((ByteBufferCell)right).getRowPosition(), right.getRowLength());
+          ((ByteBufferExtendedCell)right).getRowByteBuffer(),
+          ((ByteBufferExtendedCell)right).getRowPosition(), right.getRowLength());
     }
     return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
         right.getRowArray(), right.getRowOffset(), right.getRowLength());
@@ -225,9 +225,9 @@ public class CellComparatorImpl implements CellComparator {
    */
   @Override
   public int compareRows(Cell left, byte[] right, int roffset, int rlength) {
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getRowByteBuffer(),
-          ((ByteBufferCell) left).getRowPosition(), left.getRowLength(), right,
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) left).getRowPosition(), left.getRowLength(), right,
           roffset, rlength);
     }
     return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(), right,

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
index f320083..d6dadca 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
@@ -206,9 +206,10 @@ public final class CellUtil {
    */
   public static int copyRowTo(Cell cell, byte[] destination, int destinationOffset) {
     short rowLen = cell.getRowLength();
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyFromBufferToArray(destination, ((ByteBufferCell) cell).getRowByteBuffer(),
-        ((ByteBufferCell) cell).getRowPosition(), destinationOffset, rowLen);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils.copyFromBufferToArray(destination,
+          ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getRowPosition(), destinationOffset, rowLen);
     } else {
       System.arraycopy(cell.getRowArray(), cell.getRowOffset(), destination, destinationOffset,
         rowLen);
@@ -225,9 +226,9 @@ public final class CellUtil {
    */
   public static int copyRowTo(Cell cell, ByteBuffer destination, int destinationOffset) {
     short rowLen = cell.getRowLength();
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferCell) cell).getRowByteBuffer(),
-        destination, ((ByteBufferCell) cell).getRowPosition(), destinationOffset, rowLen);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+        destination, ((ByteBufferExtendedCell) cell).getRowPosition(), destinationOffset, rowLen);
     } else {
       ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getRowArray(),
         cell.getRowOffset(), rowLen);
@@ -241,10 +242,10 @@ public final class CellUtil {
    * @return the byte[] containing the row
    */
   public static byte[] copyRow(Cell cell) {
-    if (cell instanceof ByteBufferCell) {
-      return ByteBufferUtils.copyOfRange(((ByteBufferCell) cell).getRowByteBuffer(),
-        ((ByteBufferCell) cell).getRowPosition(),
-        ((ByteBufferCell) cell).getRowPosition() + cell.getRowLength());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.copyOfRange(((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getRowPosition(),
+        ((ByteBufferExtendedCell) cell).getRowPosition() + cell.getRowLength());
     } else {
       return Arrays.copyOfRange(cell.getRowArray(), cell.getRowOffset(),
         cell.getRowOffset() + cell.getRowLength());
@@ -260,10 +261,10 @@ public final class CellUtil {
    */
   public static int copyFamilyTo(Cell cell, byte[] destination, int destinationOffset) {
     byte fLen = cell.getFamilyLength();
-    if (cell instanceof ByteBufferCell) {
+    if (cell instanceof ByteBufferExtendedCell) {
       ByteBufferUtils.copyFromBufferToArray(destination,
-        ((ByteBufferCell) cell).getFamilyByteBuffer(), ((ByteBufferCell) cell).getFamilyPosition(),
-        destinationOffset, fLen);
+          ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getFamilyPosition(), destinationOffset, fLen);
     } else {
       System.arraycopy(cell.getFamilyArray(), cell.getFamilyOffset(), destination,
         destinationOffset, fLen);
@@ -280,9 +281,9 @@ public final class CellUtil {
    */
   public static int copyFamilyTo(Cell cell, ByteBuffer destination, int destinationOffset) {
     byte fLen = cell.getFamilyLength();
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferCell) cell).getFamilyByteBuffer(),
-        destination, ((ByteBufferCell) cell).getFamilyPosition(), destinationOffset, fLen);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
+        destination, ((ByteBufferExtendedCell) cell).getFamilyPosition(), destinationOffset, fLen);
     } else {
       ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getFamilyArray(),
         cell.getFamilyOffset(), fLen);
@@ -299,10 +300,10 @@ public final class CellUtil {
    */
   public static int copyQualifierTo(Cell cell, byte[] destination, int destinationOffset) {
     int qlen = cell.getQualifierLength();
-    if (cell instanceof ByteBufferCell) {
+    if (cell instanceof ByteBufferExtendedCell) {
       ByteBufferUtils.copyFromBufferToArray(destination,
-        ((ByteBufferCell) cell).getQualifierByteBuffer(),
-        ((ByteBufferCell) cell).getQualifierPosition(), destinationOffset, qlen);
+        ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getQualifierPosition(), destinationOffset, qlen);
     } else {
       System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(), destination,
         destinationOffset, qlen);
@@ -319,9 +320,11 @@ public final class CellUtil {
    */
   public static int copyQualifierTo(Cell cell, ByteBuffer destination, int destinationOffset) {
     int qlen = cell.getQualifierLength();
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferCell) cell).getQualifierByteBuffer(),
-        destination, ((ByteBufferCell) cell).getQualifierPosition(), destinationOffset, qlen);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils.copyFromBufferToBuffer(
+          ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
+          destination, ((ByteBufferExtendedCell) cell).getQualifierPosition(),
+          destinationOffset, qlen);
     } else {
       ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset,
         cell.getQualifierArray(), cell.getQualifierOffset(), qlen);
@@ -338,10 +341,10 @@ public final class CellUtil {
    */
   public static int copyValueTo(Cell cell, byte[] destination, int destinationOffset) {
     int vlen = cell.getValueLength();
-    if (cell instanceof ByteBufferCell) {
+    if (cell instanceof ByteBufferExtendedCell) {
       ByteBufferUtils.copyFromBufferToArray(destination,
-        ((ByteBufferCell) cell).getValueByteBuffer(), ((ByteBufferCell) cell).getValuePosition(),
-        destinationOffset, vlen);
+          ((ByteBufferExtendedCell) cell).getValueByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getValuePosition(), destinationOffset, vlen);
     } else {
       System.arraycopy(cell.getValueArray(), cell.getValueOffset(), destination, destinationOffset,
         vlen);
@@ -358,9 +361,9 @@ public final class CellUtil {
    */
   public static int copyValueTo(Cell cell, ByteBuffer destination, int destinationOffset) {
     int vlen = cell.getValueLength();
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferCell) cell).getValueByteBuffer(),
-        destination, ((ByteBufferCell) cell).getValuePosition(), destinationOffset, vlen);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
+        destination, ((ByteBufferExtendedCell) cell).getValuePosition(), destinationOffset, vlen);
     } else {
       ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getValueArray(),
         cell.getValueOffset(), vlen);
@@ -747,20 +750,20 @@ public final class CellUtil {
   public static boolean matchingFamily(final Cell left, final Cell right) {
     byte lfamlength = left.getFamilyLength();
     byte rfamlength = right.getFamilyLength();
-    if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) left).getFamilyByteBuffer(),
-          ((ByteBufferCell) left).getFamilyPosition(), lfamlength,
-          ((ByteBufferCell) right).getFamilyByteBuffer(),
-          ((ByteBufferCell) right).getFamilyPosition(), rfamlength);
+    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
+          ((ByteBufferExtendedCell) left).getFamilyPosition(), lfamlength,
+          ((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
+          ((ByteBufferExtendedCell) right).getFamilyPosition(), rfamlength);
     }
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) left).getFamilyByteBuffer(),
-          ((ByteBufferCell) left).getFamilyPosition(), lfamlength,
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
+          ((ByteBufferExtendedCell) left).getFamilyPosition(), lfamlength,
           right.getFamilyArray(), right.getFamilyOffset(), rfamlength);
     }
-    if (right instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) right).getFamilyByteBuffer(),
-          ((ByteBufferCell) right).getFamilyPosition(), rfamlength,
+    if (right instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getFamilyByteBuffer(),
+          ((ByteBufferExtendedCell) right).getFamilyPosition(), rfamlength,
           left.getFamilyArray(), left.getFamilyOffset(), lfamlength);
     }
     return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), lfamlength,
@@ -786,20 +789,20 @@ public final class CellUtil {
   public static boolean matchingQualifier(final Cell left, final Cell right) {
     int lqlength = left.getQualifierLength();
     int rqlength = right.getQualifierLength();
-    if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) left).getQualifierByteBuffer(),
-          ((ByteBufferCell) left).getQualifierPosition(), lqlength,
-          ((ByteBufferCell) right).getQualifierByteBuffer(),
-          ((ByteBufferCell) right).getQualifierPosition(), rqlength);
+    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) left).getQualifierPosition(), lqlength,
+          ((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) right).getQualifierPosition(), rqlength);
     }
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) left).getQualifierByteBuffer(),
-          ((ByteBufferCell) left).getQualifierPosition(), lqlength,
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) left).getQualifierPosition(), lqlength,
           right.getQualifierArray(), right.getQualifierOffset(), rqlength);
     }
-    if (right instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) right).getQualifierByteBuffer(),
-          ((ByteBufferCell) right).getQualifierPosition(), rqlength,
+    if (right instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) right).getQualifierPosition(), rqlength,
           left.getQualifierArray(), left.getQualifierOffset(), lqlength);
     }
     return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
@@ -864,20 +867,20 @@ public final class CellUtil {
 
   public static boolean matchingValue(final Cell left, final Cell right, int lvlength,
       int rvlength) {
-    if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) left).getValueByteBuffer(),
-        ((ByteBufferCell) left).getValuePosition(), lvlength,
-        ((ByteBufferCell) right).getValueByteBuffer(),
-        ((ByteBufferCell) right).getValuePosition(), rvlength);
+    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getValueByteBuffer(),
+        ((ByteBufferExtendedCell) left).getValuePosition(), lvlength,
+        ((ByteBufferExtendedCell) right).getValueByteBuffer(),
+        ((ByteBufferExtendedCell) right).getValuePosition(), rvlength);
     }
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) left).getValueByteBuffer(),
-        ((ByteBufferCell) left).getValuePosition(), lvlength, right.getValueArray(),
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getValueByteBuffer(),
+        ((ByteBufferExtendedCell) left).getValuePosition(), lvlength, right.getValueArray(),
         right.getValueOffset(), rvlength);
     }
-    if (right instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) right).getValueByteBuffer(),
-        ((ByteBufferCell) right).getValuePosition(), rvlength, left.getValueArray(),
+    if (right instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getValueByteBuffer(),
+        ((ByteBufferExtendedCell) right).getValuePosition(), rvlength, left.getValueArray(),
         left.getValueOffset(), lvlength);
     }
     return Bytes.equals(left.getValueArray(), left.getValueOffset(), lvlength,
@@ -885,9 +888,9 @@ public final class CellUtil {
   }
 
   public static boolean matchingValue(final Cell left, final byte[] buf) {
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getValueByteBuffer(),
-          ((ByteBufferCell) left).getValuePosition(), left.getValueLength(), buf, 0,
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getValueByteBuffer(),
+          ((ByteBufferExtendedCell) left).getValuePosition(), left.getValueLength(), buf, 0,
           buf.length) == 0;
     }
     return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(), buf, 0,
@@ -1199,9 +1202,11 @@ public final class CellUtil {
   @Deprecated
   public static void writeQualifierSkippingBytes(DataOutputStream out, Cell cell,
       int qlength, int commonPrefix) throws IOException {
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyBufferToStream((DataOutput)out, ((ByteBufferCell) cell).getQualifierByteBuffer(),
-        ((ByteBufferCell) cell).getQualifierPosition() + commonPrefix, qlength - commonPrefix);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils.copyBufferToStream((DataOutput)out,
+          ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getQualifierPosition() + commonPrefix,
+          qlength - commonPrefix);
     } else {
       out.write(cell.getQualifierArray(), cell.getQualifierOffset() + commonPrefix,
         qlength - commonPrefix);
@@ -1355,24 +1360,24 @@ public final class CellUtil {
     short lrowlength = left.getRowLength();
     short rrowlength = right.getRowLength();
     if (lrowlength != rrowlength) return false;
-    if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) left).getRowByteBuffer(),
-        ((ByteBufferCell) left).getRowPosition(), lrowlength,
-        ((ByteBufferCell) right).getRowByteBuffer(), ((ByteBufferCell) right).getRowPosition(),
-        rrowlength);
+    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) left).getRowPosition(), lrowlength,
+          ((ByteBufferExtendedCell) right).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) right).getRowPosition(), rrowlength);
     }
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) left).getRowByteBuffer(),
-        ((ByteBufferCell) left).getRowPosition(), lrowlength, right.getRowArray(),
-        right.getRowOffset(), rrowlength);
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) left).getRowPosition(), lrowlength, right.getRowArray(),
+          right.getRowOffset(), rrowlength);
     }
-    if (right instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) right).getRowByteBuffer(),
-        ((ByteBufferCell) right).getRowPosition(), rrowlength, left.getRowArray(),
-        left.getRowOffset(), lrowlength);
+    if (right instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) right).getRowPosition(), rrowlength, left.getRowArray(),
+          left.getRowOffset(), lrowlength);
     }
     return Bytes.equals(left.getRowArray(), left.getRowOffset(), lrowlength, right.getRowArray(),
-      right.getRowOffset(), rrowlength);
+        right.getRowOffset(), rrowlength);
   }
 
   /**
@@ -1433,10 +1438,10 @@ public final class CellUtil {
    *         cell's qualifier is lesser than byte[] and 0 otherwise
    */
   public final static int compareQualifiers(Cell left, byte[] right, int rOffset, int rLength) {
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getQualifierByteBuffer(),
-        ((ByteBufferCell) left).getQualifierPosition(), left.getQualifierLength(), right, rOffset,
-        rLength);
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) left).getQualifierPosition(),
+          left.getQualifierLength(), right, rOffset, rLength);
     }
     return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
       left.getQualifierLength(), right, rOffset, rLength);
@@ -1481,9 +1486,9 @@ public final class CellUtil {
    *         cell's family is lesser than byte[] and 0 otherwise
    */
   public final static int compareFamilies(Cell left, byte[] right, int roffset, int rlength) {
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.compareTo(((ByteBufferCell) left).getFamilyByteBuffer(),
-        ((ByteBufferCell) left).getFamilyPosition(), left.getFamilyLength(), right, roffset,
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
+        ((ByteBufferExtendedCell) left).getFamilyPosition(), left.getFamilyLength(), right, roffset,
         rlength);
     }
     return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilder.java
index b964d67..0afe273 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilder.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilder.java
@@ -26,7 +26,7 @@ import org.apache.yetus.audience.InterfaceAudience;
  * {@link Tag} and memstoreTS/mvcc are internal implementation detail
  *  that should not be exposed publicly.
  * Use {@link ExtendedCellBuilderFactory} to get ExtendedCellBuilder instance.
- * TODO: ditto for ByteBufferCell?
+ * TODO: ditto for ByteBufferExtendedCell?
  */
 @InterfaceAudience.Private
 public interface ExtendedCellBuilder extends RawCellBuilder {


[20/26] hbase git commit: HBASE-19642 Fix locking for peer modification procedure

Posted by zh...@apache.org.
HBASE-19642 Fix locking for peer modification procedure


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/28b93f41
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/28b93f41
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/28b93f41

Branch: refs/heads/HBASE-19397
Commit: 28b93f41c95d082b2734f8e4a303dfdcf609e8b8
Parents: 4850dbd
Author: zhangduo <zh...@apache.org>
Authored: Wed Dec 27 18:27:13 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Dec 28 08:51:59 2017 +0800

----------------------------------------------------------------------
 .../procedure/MasterProcedureScheduler.java     | 14 +++++++++++++
 .../master/replication/ModifyPeerProcedure.java | 21 +++++++++++++++++---
 2 files changed, 32 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/28b93f41/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
index 5f4665c..05c8439 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java
@@ -610,6 +610,20 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler {
     public boolean requireExclusiveLock(Procedure proc) {
       return requirePeerExclusiveLock((PeerProcedureInterface) proc);
     }
+
+    @Override
+    public boolean isAvailable() {
+      if (isEmpty()) {
+        return false;
+      }
+      if (getLockStatus().hasExclusiveLock()) {
+        // if we have an exclusive lock already taken
+        // only child of the lock owner can be executed
+        Procedure nextProc = peek();
+        return nextProc != null && getLockStatus().hasLockAccess(nextProc);
+      }
+      return true;
+    }
   }
 
   // ============================================================================

http://git-wip-us.apache.org/repos/asf/hbase/blob/28b93f41/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
index 279fbc7..a682606 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
@@ -46,6 +46,8 @@ public abstract class ModifyPeerProcedure
 
   protected String peerId;
 
+  private volatile boolean locked;
+
   // used to keep compatible with old client where we can only returns after updateStorage.
   protected ProcedurePrepareLatch latch;
 
@@ -145,17 +147,30 @@ public abstract class ModifyPeerProcedure
 
   @Override
   protected LockState acquireLock(MasterProcedureEnv env) {
-    return env.getProcedureScheduler().waitPeerExclusiveLock(this, peerId)
-      ? LockState.LOCK_EVENT_WAIT
-      : LockState.LOCK_ACQUIRED;
+    if (env.getProcedureScheduler().waitPeerExclusiveLock(this, peerId)) {
+      return  LockState.LOCK_EVENT_WAIT;
+    }
+    locked = true;
+    return LockState.LOCK_ACQUIRED;
   }
 
   @Override
   protected void releaseLock(MasterProcedureEnv env) {
+    locked = false;
     env.getProcedureScheduler().wakePeerExclusiveLock(this, peerId);
   }
 
   @Override
+  protected boolean holdLock(MasterProcedureEnv env) {
+    return true;
+  }
+
+  @Override
+  protected boolean hasLock(MasterProcedureEnv env) {
+    return locked;
+  }
+
+  @Override
   protected void rollbackState(MasterProcedureEnv env, PeerModificationState state)
       throws IOException, InterruptedException {
     if (state == PeerModificationState.PRE_PEER_MODIFICATION) {


[08/26] hbase git commit: HBASE-19647 Logging cleanups; emit regionname when RegionTooBusyException inside RetriesExhausted... make netty connect/disconnect TRACE-level; ADDENDUM

Posted by zh...@apache.org.
HBASE-19647 Logging cleanups; emit regionname when RegionTooBusyException inside RetriesExhausted... make netty connect/disconnect TRACE-level; ADDENDUM


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/2c65f039
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/2c65f039
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/2c65f039

Branch: refs/heads/HBASE-19397
Commit: 2c65f039eddb6733ad2ac2fb89a8f5427495e243
Parents: 01b1f48
Author: Michael Stack <st...@apache.org>
Authored: Wed Dec 27 14:14:07 2017 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Wed Dec 27 14:14:58 2017 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/RegionTooBusyException.java     | 11 +++++------
 .../RetriesExhaustedWithDetailsException.java    |  2 ++
 .../hadoop/hbase/regionserver/HRegion.java       | 19 ++++++++++---------
 .../hadoop/hbase/client/TestMetaCache.java       |  2 +-
 .../apache/hadoop/hbase/ipc/TestRpcMetrics.java  |  2 +-
 5 files changed, 19 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2c65f039/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionTooBusyException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionTooBusyException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionTooBusyException.java
index 7b03b45..49431b6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionTooBusyException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/RegionTooBusyException.java
@@ -24,21 +24,20 @@ import org.apache.yetus.audience.InterfaceAudience;
 /**
  * Thrown by a region server if it will block and wait to serve a request.
  * For example, the client wants to insert something to a region while the
- * region is compacting.
+ * region is compacting. Keep variance in the passed 'msg' low because its msg is used as a key
+ * over in {@link org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException}
+ * grouping failure types.
  */
 @InterfaceAudience.Public
 public class RegionTooBusyException extends IOException {
   private static final long serialVersionUID = 1728345723728342L;
 
-  /** default constructor */
-  public RegionTooBusyException() {
-    super();
-  }
-
   /**
    * Constructor
    * @param msg message
    */
+  // Be careful. Keep variance in the passed 'msg' low because its msg is used as a key over in
+  // RetriesExhaustedWithDetailsException grouping failure types.
   public RegionTooBusyException(final String msg) {
     super(msg);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2c65f039/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java
index e7eda2a..0cce728 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RetriesExhaustedWithDetailsException.java
@@ -157,6 +157,8 @@ extends RetriesExhaustedException {
       if (t instanceof DoNotRetryIOException ||
           t instanceof RegionTooBusyException) {
         // If RegionTooBusyException, print message since it has Region name in it.
+        // RegionTooBusyException message was edited to remove variance. Has regionname, server,
+        // and why the exception; no longer has duration it waited on lock nor current memsize.
         name = t.getMessage();
       } else {
         name = t.getClass().getSimpleName();

http://git-wip-us.apache.org/repos/asf/hbase/blob/2c65f039/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index cde6874..ca1bfd3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -4170,12 +4170,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     if (this.memstoreDataSize.get() > this.blockingMemStoreSize) {
       blockedRequestsCount.increment();
       requestFlush();
-      throw new RegionTooBusyException("Over memstore limit, " +
-          "regionName=" + (this.getRegionInfo() == null ? "unknown" :
-          this.getRegionInfo().getRegionNameAsString()) +
-          ", server=" + (this.getRegionServerServices() == null ? "unknown" :
+      // Don't print current limit because it will vary too much. The message is used as a key
+      // over in RetriesExhaustedWithDetailsException processing.
+      throw new RegionTooBusyException("Over memstore limit; regionName=" +
+          (this.getRegionInfo() == null? "unknown": this.getRegionInfo().getRegionNameAsString()) +
+          ", server=" + (this.getRegionServerServices() == null ? "unknown":
           this.getRegionServerServices().getServerName()) +
-          ", memstoreSize=" + memstoreDataSize.get() +
           ", blockingMemStoreSize=" + blockingMemStoreSize);
     }
   }
@@ -8183,11 +8183,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       final long waitTime = Math.min(maxBusyWaitDuration,
           busyWaitDuration * Math.min(multiplier, maxBusyWaitMultiplier));
       if (!lock.tryLock(waitTime, TimeUnit.MILLISECONDS)) {
-        throw new RegionTooBusyException(
-            "failed to get a lock in " + waitTime + " ms. " +
-                "regionName=" + (this.getRegionInfo() == null ? "unknown" :
+        // Don't print millis. Message is used as a key over in
+        // RetriesExhaustedWithDetailsException processing.
+        throw new RegionTooBusyException("Failed to obtain lock; regionName=" +
+            (this.getRegionInfo() == null? "unknown":
                 this.getRegionInfo().getRegionNameAsString()) +
-                ", server=" + (this.getRegionServerServices() == null ? "unknown" :
+            ", server=" + (this.getRegionServerServices() == null? "unknown":
                 this.getRegionServerServices().getServerName()));
       }
     } catch (InterruptedException ie) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/2c65f039/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java
index b1fda41..ecdd685 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java
@@ -187,7 +187,7 @@ public class TestMetaCache {
   public static List<Throwable> metaCachePreservingExceptions() {
     return new ArrayList<Throwable>() {{
       add(new RegionOpeningException(" "));
-      add(new RegionTooBusyException());
+      add(new RegionTooBusyException("Some old message"));
       add(new ThrottlingException(" "));
       add(new MultiActionResultTooLarge(" "));
       add(new RetryImmediatelyException(" "));

http://git-wip-us.apache.org/repos/asf/hbase/blob/2c65f039/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcMetrics.java
index a0769da..cfcfb21 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcMetrics.java
@@ -133,7 +133,7 @@ public class TestRpcMetrics {
     HELPER.assertCounter("exceptions", 1, serverSource);
 
     mrpc.exception(new RegionMovedException(ServerName.parseServerName("localhost:60020"), 100));
-    mrpc.exception(new RegionTooBusyException());
+    mrpc.exception(new RegionTooBusyException("Some region"));
     mrpc.exception(new OutOfOrderScannerNextException());
     mrpc.exception(new NotServingRegionException());
     HELPER.assertCounter("exceptions.RegionMovedException", 1, serverSource);


[11/26] hbase git commit: HBASE-19599 Remove ReplicationQueuesClient, use ReplicationQueueStorage directly

Posted by zh...@apache.org.
HBASE-19599 Remove ReplicationQueuesClient, use ReplicationQueueStorage directly


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/8af1ce3e
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/8af1ce3e
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/8af1ce3e

Branch: refs/heads/HBASE-19397
Commit: 8af1ce3e79754b89bc59c7b6d0139a53e7948f7a
Parents: 19b20af
Author: zhangduo <zh...@apache.org>
Authored: Mon Dec 25 18:49:56 2017 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Thu Dec 28 08:51:59 2017 +0800

----------------------------------------------------------------------
 .../hbase/replication/ReplicationFactory.java   |  19 +-
 .../replication/ReplicationPeersZKImpl.java     |  24 +-
 .../replication/ReplicationQueueStorage.java    |  26 +-
 .../replication/ReplicationQueuesClient.java    |  93 -----
 .../ReplicationQueuesClientArguments.java       |  40 --
 .../ReplicationQueuesClientZKImpl.java          | 176 ---------
 .../replication/ZKReplicationQueueStorage.java  |  90 ++++-
 .../replication/TestReplicationStateBasic.java  | 378 +++++++++++++++++++
 .../replication/TestReplicationStateZKImpl.java | 148 ++++++++
 .../TestZKReplicationQueueStorage.java          |  74 ++++
 .../cleaner/ReplicationZKNodeCleaner.java       |  71 ++--
 .../cleaner/ReplicationZKNodeCleanerChore.java  |   5 +-
 .../replication/ReplicationPeerManager.java     |  31 +-
 .../master/ReplicationHFileCleaner.java         | 108 ++----
 .../master/ReplicationLogCleaner.java           |  35 +-
 .../regionserver/DumpReplicationQueues.java     |  77 ++--
 .../hbase/util/hbck/ReplicationChecker.java     |  14 +-
 .../client/TestAsyncReplicationAdminApi.java    |  31 +-
 .../replication/TestReplicationAdmin.java       |   2 +
 .../hbase/master/cleaner/TestLogsCleaner.java   |  29 +-
 .../cleaner/TestReplicationHFileCleaner.java    |  58 +--
 .../cleaner/TestReplicationZKNodeCleaner.java   |  12 +-
 .../replication/TestReplicationStateBasic.java  | 378 -------------------
 .../replication/TestReplicationStateZKImpl.java | 227 -----------
 .../TestReplicationSourceManagerZkImpl.java     |  84 ++---
 25 files changed, 907 insertions(+), 1323 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java
----------------------------------------------------------------------
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 9f4ad18..6c1c213 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
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -37,20 +36,14 @@ public class ReplicationFactory {
       args);
   }
 
-  public static ReplicationQueuesClient
-      getReplicationQueuesClient(ReplicationQueuesClientArguments args) throws Exception {
-    return (ReplicationQueuesClient) ConstructorUtils
-        .invokeConstructor(ReplicationQueuesClientZKImpl.class, args);
-  }
-
-  public static ReplicationPeers getReplicationPeers(final ZKWatcher zk, Configuration conf,
-                                                     Abortable abortable) {
+  public static ReplicationPeers getReplicationPeers(ZKWatcher zk, Configuration conf,
+      Abortable abortable) {
     return getReplicationPeers(zk, conf, null, abortable);
   }
 
-  public static ReplicationPeers getReplicationPeers(final ZKWatcher zk, Configuration conf,
-                                                     final ReplicationQueuesClient queuesClient, Abortable abortable) {
-    return new ReplicationPeersZKImpl(zk, conf, queuesClient, abortable);
+  public static ReplicationPeers getReplicationPeers(ZKWatcher zk, Configuration conf,
+      ReplicationQueueStorage queueStorage, Abortable abortable) {
+    return new ReplicationPeersZKImpl(zk, conf, queueStorage, abortable);
   }
 
   public static ReplicationTracker getReplicationTracker(ZKWatcher zookeeper,

http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
----------------------------------------------------------------------
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 8e2c5f4..f2e5647 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
@@ -32,10 +32,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.CompoundConfiguration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.ServerName;
 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 +48,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
@@ -79,17 +81,17 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
 
   // Map of peer clusters keyed by their id
   private Map<String, ReplicationPeerZKImpl> peerClusters;
-  private final ReplicationQueuesClient queuesClient;
+  private final ReplicationQueueStorage queueStorage;
   private Abortable abortable;
 
   private static final Logger LOG = LoggerFactory.getLogger(ReplicationPeersZKImpl.class);
 
-  public ReplicationPeersZKImpl(final ZKWatcher zk, final Configuration conf,
-                                final ReplicationQueuesClient queuesClient, Abortable abortable) {
+  public ReplicationPeersZKImpl(ZKWatcher zk, Configuration conf,
+      ReplicationQueueStorage queueStorage, Abortable abortable) {
     super(zk, conf, abortable);
     this.abortable = abortable;
     this.peerClusters = new ConcurrentHashMap<>();
-    this.queuesClient = queuesClient;
+    this.queueStorage = queueStorage;
   }
 
   @Override
@@ -511,14 +513,16 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
   }
 
   private void checkQueuesDeleted(String peerId) throws ReplicationException {
-    if (queuesClient == null) return;
+    if (queueStorage == null) {
+      return;
+    }
     try {
-      List<String> replicators = queuesClient.getListOfReplicators();
+      List<ServerName> replicators = queueStorage.getListOfReplicators();
       if (replicators == null || replicators.isEmpty()) {
         return;
       }
-      for (String replicator : replicators) {
-        List<String> queueIds = queuesClient.getAllQueues(replicator);
+      for (ServerName replicator : replicators) {
+        List<String> queueIds = queueStorage.getAllQueues(replicator);
         for (String queueId : queueIds) {
           ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
           if (queueInfo.getPeerId().equals(peerId)) {
@@ -529,7 +533,7 @@ public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
       }
       // Check for hfile-refs queue
       if (-1 != ZKUtil.checkExists(zookeeper, hfileRefsZNode)
-          && queuesClient.getAllPeersFromHFileRefsQueue().contains(peerId)) {
+          && queueStorage.getAllPeersFromHFileRefsQueue().contains(peerId)) {
         throw new IllegalArgumentException("Undeleted queue for peerId: " + peerId
             + ", found in hfile-refs node path " + hfileRefsZNode);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
index 7210d9a..e774148 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
@@ -78,6 +78,14 @@ public interface ReplicationQueueStorage {
       throws ReplicationException;
 
   /**
+   * Get a list of all WALs in the given queue on the given region server.
+   * @param serverName the server name of the region server that owns the queue
+   * @param queueId a String that identifies the queue
+   * @return a list of WALs
+   */
+  List<String> getWALsInQueue(ServerName serverName, String queueId) throws ReplicationException;
+
+  /**
    * Get a list of all queues for the specified region server.
    * @param serverName the server name of the region server that owns the set of queues
    * @return a list of queueIds
@@ -108,8 +116,8 @@ public interface ReplicationQueueStorage {
 
   /**
    * Load all wals in all replication queues. This method guarantees to return a snapshot which
-   * contains all WALs in the zookeeper at the start of this call even there is concurrent queue
-   * failover. However, some newly created WALs during the call may not be included.
+   * contains all WALs at the start of this call even there is concurrent queue failover. However,
+   * some newly created WALs during the call may not be included.
    */
   Set<String> getAllWALs() throws ReplicationException;
 
@@ -143,13 +151,6 @@ public interface ReplicationQueueStorage {
   void removeHFileRefs(String peerId, List<String> files) throws ReplicationException;
 
   /**
-   * Get the change version number of replication hfile references node. This can be used as
-   * optimistic locking to get a consistent snapshot of the replication queues of hfile references.
-   * @return change version number of hfile references node
-   */
-  int getHFileRefsNodeChangeVersion() throws ReplicationException;
-
-  /**
    * Get list of all peers from hfile reference queue.
    * @return a list of peer ids
    */
@@ -161,4 +162,11 @@ public interface ReplicationQueueStorage {
    * @return a list of hfile references
    */
   List<String> getReplicableHFiles(String peerId) throws ReplicationException;
+
+  /**
+   * Load all hfile references in all replication queues. This method guarantees to return a
+   * snapshot which contains all hfile references at the start of this call. However, some newly
+   * created hfile references during the call may not be included.
+   */
+  Set<String> getAllHFileRefs() throws ReplicationException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java
----------------------------------------------------------------------
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
deleted file mode 100644
index 2c513fa..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClient.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.replication;
-
-import java.util.List;
-import java.util.Set;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-
-/**
- * This provides an interface for clients of replication to view replication queues. These queues
- * keep track of the sources(WALs/HFile references) that still need to be replicated to remote
- * clusters.
- */
-@InterfaceAudience.Private
-public interface ReplicationQueuesClient {
-
-  /**
-   * Initialize the replication queue client interface.
-   */
-  public void init() throws ReplicationException;
-
-  /**
-   * Get a list of all region servers that have outstanding replication queues. These servers could
-   * be alive, dead or from a previous run of the cluster.
-   * @return a list of server names
-   * @throws KeeperException zookeeper exception
-   */
-  List<String> getListOfReplicators() throws KeeperException;
-
-  /**
-   * Get a list of all WALs in the given queue on the given region server.
-   * @param serverName the server name of the region server that owns the queue
-   * @param queueId a String that identifies the queue
-   * @return a list of WALs, null if this region server is dead and has no outstanding queues
-   * @throws KeeperException zookeeper exception
-   */
-  List<String> getLogsInQueue(String serverName, String queueId) throws KeeperException;
-
-  /**
-   * Get a list of all queues for the specified region server.
-   * @param serverName the server name of the region server that owns the set of queues
-   * @return a list of queueIds, null if this region server is not a replicator.
-   */
-  List<String> getAllQueues(String serverName) throws KeeperException;
-
-  /**
-   * Load all wals in all replication queues from ZK. This method guarantees to return a
-   * snapshot which contains all WALs in the zookeeper at the start of this call even there
-   * is concurrent queue failover. However, some newly created WALs during the call may
-   * not be included.
-   */
-   Set<String> getAllWALs() throws KeeperException;
-
-  /**
-   * Get the change version number of replication hfile references node. This can be used as
-   * optimistic locking to get a consistent snapshot of the replication queues of hfile references.
-   * @return change version number of hfile references node
-   */
-  int getHFileRefsNodeChangeVersion() throws KeeperException;
-
-  /**
-   * Get list of all peers from hfile reference queue.
-   * @return a list of peer ids
-   * @throws KeeperException zookeeper exception
-   */
-  List<String> getAllPeersFromHFileRefsQueue() throws KeeperException;
-
-  /**
-   * Get a list of all hfile references in the given peer.
-   * @param peerId a String that identifies the peer
-   * @return a list of hfile references, null if not found any
-   * @throws KeeperException zookeeper exception
-   */
-  List<String> getReplicableHFiles(String peerId) throws KeeperException;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientArguments.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientArguments.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientArguments.java
deleted file mode 100644
index 9b79294..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientArguments.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.replication;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * Wrapper around common arguments used to construct ReplicationQueuesClient. Used to construct
- * various ReplicationQueuesClient Implementations with different constructor arguments by
- * reflection.
- */
-@InterfaceAudience.Private
-public class ReplicationQueuesClientArguments extends ReplicationQueuesArguments {
-  public ReplicationQueuesClientArguments(Configuration conf, Abortable abort,
-     ZKWatcher zk) {
-    super(conf, abort, zk);
-  }
-  public ReplicationQueuesClientArguments(Configuration conf, Abortable abort) {
-    super(conf, abort);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
----------------------------------------------------------------------
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
deleted file mode 100644
index e85b42a..0000000
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueuesClientZKImpl.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/*
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.replication;
-
-import java.util.List;
-import java.util.Set;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableSet;
-import org.apache.hadoop.hbase.shaded.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;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@InterfaceAudience.Private
-public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implements
-    ReplicationQueuesClient {
-
-  Logger LOG = LoggerFactory.getLogger(ReplicationQueuesClientZKImpl.class);
-
-  public ReplicationQueuesClientZKImpl(ReplicationQueuesClientArguments args) {
-    this(args.getZk(), args.getConf(), args.getAbortable());
-  }
-
-  public ReplicationQueuesClientZKImpl(final ZKWatcher zk, Configuration conf,
-                                       Abortable abortable) {
-    super(zk, conf, abortable);
-  }
-
-  @Override
-  public void init() throws ReplicationException {
-    try {
-      if (ZKUtil.checkExists(this.zookeeper, this.queuesZNode) < 0) {
-        ZKUtil.createWithParents(this.zookeeper, this.queuesZNode);
-      }
-    } catch (KeeperException e) {
-      throw new ReplicationException("Internal error while initializing a queues client", e);
-    }
-  }
-
-  @Override
-  public List<String> getLogsInQueue(String serverName, String queueId) throws KeeperException {
-    String znode = ZNodePaths.joinZNode(this.queuesZNode, serverName);
-    znode = ZNodePaths.joinZNode(znode, queueId);
-    List<String> result = null;
-    try {
-      result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to get list of wals for queueId=" + queueId
-          + " and serverName=" + serverName, e);
-      throw e;
-    }
-    return result;
-  }
-
-  @Override
-  public List<String> getAllQueues(String serverName) throws KeeperException {
-    String znode = ZNodePaths.joinZNode(this.queuesZNode, serverName);
-    List<String> result = null;
-    try {
-      result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to get list of queues for serverName=" + serverName, e);
-      throw e;
-    }
-    return result;
-  }
-
-  @Override
-  public Set<String> getAllWALs() throws KeeperException {
-    /**
-     * Load all wals in all replication queues from ZK. This method guarantees to return a
-     * snapshot which contains all WALs in the zookeeper at the start of this call even there
-     * is concurrent queue failover. However, some newly created WALs during the call may
-     * not be included.
-     */
-    for (int retry = 0; ; retry++) {
-      int v0 = getQueuesZNodeCversion();
-      List<String> rss = getListOfReplicators();
-      if (rss == null || rss.isEmpty()) {
-        LOG.debug("Didn't find any region server that replicates, won't prevent any deletions.");
-        return ImmutableSet.of();
-      }
-      Set<String> wals = Sets.newHashSet();
-      for (String rs : rss) {
-        List<String> listOfPeers = getAllQueues(rs);
-        // if rs just died, this will be null
-        if (listOfPeers == null) {
-          continue;
-        }
-        for (String id : listOfPeers) {
-          List<String> peersWals = getLogsInQueue(rs, id);
-          if (peersWals != null) {
-            wals.addAll(peersWals);
-          }
-        }
-      }
-      int v1 = getQueuesZNodeCversion();
-      if (v0 == v1) {
-        return wals;
-      }
-      LOG.info(String.format("Replication queue node cversion changed from %d to %d, retry = %d",
-        v0, v1, retry));
-    }
-  }
-
-  public int getQueuesZNodeCversion() throws KeeperException {
-    try {
-      Stat stat = new Stat();
-      ZKUtil.getDataNoWatch(this.zookeeper, this.queuesZNode, stat);
-      return stat.getCversion();
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to get stat of replication rs node", e);
-      throw e;
-    }
-  }
-
-  @Override
-  public int getHFileRefsNodeChangeVersion() throws KeeperException {
-    Stat stat = new Stat();
-    try {
-      ZKUtil.getDataNoWatch(this.zookeeper, this.hfileRefsZNode, stat);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to get stat of replication hfile references node.", e);
-      throw e;
-    }
-    return stat.getCversion();
-  }
-
-  @Override
-  public List<String> getAllPeersFromHFileRefsQueue() throws KeeperException {
-    List<String> result = null;
-    try {
-      result = ZKUtil.listChildrenNoWatch(this.zookeeper, this.hfileRefsZNode);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to get list of all peers in hfile references node.", e);
-      throw e;
-    }
-    return result;
-  }
-
-  @Override
-  public List<String> getReplicableHFiles(String peerId) throws KeeperException {
-    String znode = ZNodePaths.joinZNode(this.hfileRefsZNode, peerId);
-    List<String> result = null;
-    try {
-      result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
-    } catch (KeeperException e) {
-      this.abortable.abort("Failed to get list of hfile references for peerId=" + peerId, e);
-      throw e;
-    }
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
index 7015d7f..0275d52 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
@@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.util.CollectionUtils.nullToEmpty;
 
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.SortedSet;
@@ -49,7 +50,7 @@ import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 
 /**
  * ZK based replication queue storage.
@@ -61,7 +62,7 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
   private static final Logger LOG = LoggerFactory.getLogger(ZKReplicationQueueStorage.class);
 
   public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY =
-    "zookeeper.znode.replication.hfile.refs";
+      "zookeeper.znode.replication.hfile.refs";
   public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT = "hfile-refs";
 
   /**
@@ -256,11 +257,23 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
     }
   }
 
-  private List<String> getLogsInQueue0(ServerName serverName, String queueId)
+  private List<String> getWALsInQueue0(ServerName serverName, String queueId)
       throws KeeperException {
     return nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, getQueueNode(serverName, queueId)));
   }
 
+  @Override
+  public List<String> getWALsInQueue(ServerName serverName, String queueId)
+      throws ReplicationException {
+    try {
+      return getWALsInQueue0(serverName, queueId);
+    } catch (KeeperException e) {
+      throw new ReplicationException(
+          "Failed to get wals in queue (serverName=" + serverName + ", queueId=" + queueId + ")",
+          e);
+    }
+  }
+
   private List<String> getAllQueues0(ServerName serverName) throws KeeperException {
     return nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, getRsNode(serverName)));
   }
@@ -274,7 +287,9 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
     }
   }
 
-  private int getQueuesZNodeCversion() throws KeeperException {
+  // will be overridden in UTs
+  @VisibleForTesting
+  protected int getQueuesZNodeCversion() throws KeeperException {
     Stat stat = new Stat();
     ZKUtil.getDataNoWatch(this.zookeeper, this.queuesZNode, stat);
     return stat.getCversion();
@@ -290,10 +305,10 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
           LOG.debug("Didn't find any region server that replicates, won't prevent any deletions.");
           return Collections.emptySet();
         }
-        Set<String> wals = Sets.newHashSet();
+        Set<String> wals = new HashSet<>();
         for (ServerName rs : rss) {
           for (String queueId : getAllQueues0(rs)) {
-            wals.addAll(getLogsInQueue0(rs, queueId));
+            wals.addAll(getWALsInQueue0(rs, queueId));
           }
         }
         int v1 = getQueuesZNodeCversion();
@@ -356,9 +371,9 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
     if (debugEnabled) {
       LOG.debug("Adding hfile references " + pairs + " in queue " + peerNode);
     }
-    List<ZKUtilOp> listOfOps =
-      pairs.stream().map(p -> p.getSecond().getName()).map(n -> getHFileNode(peerNode, n))
-          .map(f -> ZKUtilOp.createAndFailSilent(f, HConstants.EMPTY_BYTE_ARRAY)).collect(toList());
+    List<ZKUtilOp> listOfOps = pairs.stream().map(p -> p.getSecond().getName())
+        .map(n -> getHFileNode(peerNode, n))
+        .map(f -> ZKUtilOp.createAndFailSilent(f, HConstants.EMPTY_BYTE_ARRAY)).collect(toList());
     if (debugEnabled) {
       LOG.debug("The multi list size for adding hfile references in zk for node " + peerNode +
         " is " + listOfOps.size());
@@ -391,35 +406,70 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
     }
   }
 
-  @Override
-  public int getHFileRefsNodeChangeVersion() throws ReplicationException {
-    Stat stat = new Stat();
-    try {
-      ZKUtil.getDataNoWatch(zookeeper, hfileRefsZNode, stat);
-    } catch (KeeperException e) {
-      throw new ReplicationException("Failed to get stat of replication hfile references node.", e);
-    }
-    return stat.getCversion();
+  private List<String> getAllPeersFromHFileRefsQueue0() throws KeeperException {
+    return nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, hfileRefsZNode));
   }
 
   @Override
   public List<String> getAllPeersFromHFileRefsQueue() throws ReplicationException {
     try {
-      return nullToEmpty(ZKUtil.listChildrenNoWatch(zookeeper, hfileRefsZNode));
+      return getAllPeersFromHFileRefsQueue0();
     } catch (KeeperException e) {
       throw new ReplicationException("Failed to get list of all peers in hfile references node.",
           e);
     }
   }
 
+  private List<String> getReplicableHFiles0(String peerId) throws KeeperException {
+    return nullToEmpty(ZKUtil.listChildrenNoWatch(this.zookeeper, getHFileRefsPeerNode(peerId)));
+  }
+
   @Override
   public List<String> getReplicableHFiles(String peerId) throws ReplicationException {
     try {
-      return nullToEmpty(ZKUtil.listChildrenNoWatch(this.zookeeper, getHFileRefsPeerNode(peerId)));
+      return getReplicableHFiles0(peerId);
     } catch (KeeperException e) {
       throw new ReplicationException("Failed to get list of hfile references for peer " + peerId,
           e);
     }
   }
 
+  // will be overridden in UTs
+  @VisibleForTesting
+  protected int getHFileRefsZNodeCversion() throws ReplicationException {
+    Stat stat = new Stat();
+    try {
+      ZKUtil.getDataNoWatch(zookeeper, hfileRefsZNode, stat);
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to get stat of replication hfile references node.", e);
+    }
+    return stat.getCversion();
+  }
+
+  @Override
+  public Set<String> getAllHFileRefs() throws ReplicationException {
+    try {
+      for (int retry = 0;; retry++) {
+        int v0 = getHFileRefsZNodeCversion();
+        List<String> peers = getAllPeersFromHFileRefsQueue();
+        if (peers.isEmpty()) {
+          LOG.debug("Didn't find any peers with hfile references, won't prevent any deletions.");
+          return Collections.emptySet();
+        }
+        Set<String> hfileRefs = new HashSet<>();
+        for (String peer : peers) {
+          hfileRefs.addAll(getReplicableHFiles0(peer));
+        }
+        int v1 = getHFileRefsZNodeCversion();
+        if (v0 == v1) {
+          return hfileRefs;
+        }
+        LOG.debug(String.format(
+          "Replication hfile references node cversion changed from " + "%d to %d, retry = %d", v0,
+          v1, retry));
+      }
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to get all hfile refs", e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
new file mode 100644
index 0000000..6fe869c
--- /dev/null
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateBasic.java
@@ -0,0 +1,378 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.zookeeper.ZKConfig;
+import org.apache.zookeeper.KeeperException;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * White box testing for replication state interfaces. Implementations should extend this class, and
+ * initialize the interfaces properly.
+ */
+public abstract class TestReplicationStateBasic {
+
+  protected ReplicationQueues rq1;
+  protected ReplicationQueues rq2;
+  protected ReplicationQueues rq3;
+  protected ReplicationQueueStorage rqs;
+  protected ServerName server1 = ServerName.valueOf("hostname1.example.org", 1234, 12345);
+  protected ServerName server2 = ServerName.valueOf("hostname2.example.org", 1234, 12345);
+  protected ServerName server3 = ServerName.valueOf("hostname3.example.org", 1234, 12345);
+  protected ReplicationPeers rp;
+  protected static final String ID_ONE = "1";
+  protected static final String ID_TWO = "2";
+  protected static String KEY_ONE;
+  protected static String KEY_TWO;
+
+  // For testing when we try to replicate to ourself
+  protected String OUR_ID = "3";
+  protected String OUR_KEY;
+
+  protected static int zkTimeoutCount;
+  protected static final int ZK_MAX_COUNT = 300;
+  protected static final int ZK_SLEEP_INTERVAL = 100; // millis
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestReplicationStateBasic.class);
+
+  @Test
+  public void testReplicationQueueStorage() throws ReplicationException {
+    // Test methods with empty state
+    assertEquals(0, rqs.getListOfReplicators().size());
+    assertTrue(rqs.getWALsInQueue(server1, "qId1").isEmpty());
+    assertTrue(rqs.getAllQueues(server1).isEmpty());
+
+    /*
+     * Set up data Two replicators: -- server1: three queues with 0, 1 and 2 log files each --
+     * server2: zero queues
+     */
+    rq1.init(server1.getServerName());
+    rq2.init(server2.getServerName());
+    rq1.addLog("qId1", "trash");
+    rq1.removeLog("qId1", "trash");
+    rq1.addLog("qId2", "filename1");
+    rq1.addLog("qId3", "filename2");
+    rq1.addLog("qId3", "filename3");
+    rq2.addLog("trash", "trash");
+    rq2.removeQueue("trash");
+
+    List<ServerName> reps = rqs.getListOfReplicators();
+    assertEquals(2, reps.size());
+    assertTrue(server1.getServerName(), reps.contains(server1));
+    assertTrue(server2.getServerName(), reps.contains(server2));
+
+    assertTrue(rqs.getWALsInQueue(ServerName.valueOf("bogus", 12345, 12345), "bogus").isEmpty());
+    assertTrue(rqs.getWALsInQueue(server1, "bogus").isEmpty());
+    assertEquals(0, rqs.getWALsInQueue(server1, "qId1").size());
+    assertEquals(1, rqs.getWALsInQueue(server1, "qId2").size());
+    assertEquals("filename1", rqs.getWALsInQueue(server1, "qId2").get(0));
+
+    assertTrue(rqs.getAllQueues(ServerName.valueOf("bogus", 12345, -1L)).isEmpty());
+    assertEquals(0, rqs.getAllQueues(server2).size());
+    List<String> list = rqs.getAllQueues(server1);
+    assertEquals(3, list.size());
+    assertTrue(list.contains("qId2"));
+    assertTrue(list.contains("qId3"));
+  }
+
+  @Test
+  public void testReplicationQueues() throws ReplicationException {
+    rq1.init(server1.getServerName());
+    rq2.init(server2.getServerName());
+    rq3.init(server3.getServerName());
+    // Initialize ReplicationPeer so we can add peers (we don't transfer lone queues)
+    rp.init();
+
+    // 3 replicators should exist
+    assertEquals(3, rq1.getListOfReplicators().size());
+    rq1.removeQueue("bogus");
+    rq1.removeLog("bogus", "bogus");
+    rq1.removeAllQueues();
+    assertEquals(0, rq1.getAllQueues().size());
+    assertEquals(0, rq1.getLogPosition("bogus", "bogus"));
+    assertNull(rq1.getLogsInQueue("bogus"));
+    assertNull(rq1.getUnClaimedQueueIds(ServerName.valueOf("bogus", 1234, -1L).toString()));
+
+    rq1.setLogPosition("bogus", "bogus", 5L);
+
+    populateQueues();
+
+    assertEquals(3, rq1.getListOfReplicators().size());
+    assertEquals(0, rq2.getLogsInQueue("qId1").size());
+    assertEquals(5, rq3.getLogsInQueue("qId5").size());
+    assertEquals(0, rq3.getLogPosition("qId1", "filename0"));
+    rq3.setLogPosition("qId5", "filename4", 354L);
+    assertEquals(354L, rq3.getLogPosition("qId5", "filename4"));
+
+    assertEquals(5, rq3.getLogsInQueue("qId5").size());
+    assertEquals(0, rq2.getLogsInQueue("qId1").size());
+    assertEquals(0, rq1.getAllQueues().size());
+    assertEquals(1, rq2.getAllQueues().size());
+    assertEquals(5, rq3.getAllQueues().size());
+
+    assertEquals(0, rq3.getUnClaimedQueueIds(server1.getServerName()).size());
+    rq3.removeReplicatorIfQueueIsEmpty(server1.getServerName());
+    assertEquals(2, rq3.getListOfReplicators().size());
+
+    List<String> queues = rq2.getUnClaimedQueueIds(server3.getServerName());
+    assertEquals(5, queues.size());
+    for (String queue : queues) {
+      rq2.claimQueue(server3.getServerName(), queue);
+    }
+    rq2.removeReplicatorIfQueueIsEmpty(server3.getServerName());
+    assertEquals(1, rq2.getListOfReplicators().size());
+
+    // Try to claim our own queues
+    assertNull(rq2.getUnClaimedQueueIds(server2.getServerName()));
+    rq2.removeReplicatorIfQueueIsEmpty(server2.getServerName());
+
+    assertEquals(6, rq2.getAllQueues().size());
+
+    rq2.removeAllQueues();
+
+    assertEquals(0, rq2.getListOfReplicators().size());
+  }
+
+  @Test
+  public void testInvalidClusterKeys() throws ReplicationException, KeeperException {
+    rp.init();
+
+    try {
+      rp.registerPeer(ID_ONE,
+        new ReplicationPeerConfig().setClusterKey("hostname1.example.org:1234:hbase"));
+      fail("Should throw an IllegalArgumentException because " +
+        "zookeeper.znode.parent is missing leading '/'.");
+    } catch (IllegalArgumentException e) {
+      // Expected.
+    }
+
+    try {
+      rp.registerPeer(ID_ONE,
+        new ReplicationPeerConfig().setClusterKey("hostname1.example.org:1234:/"));
+      fail("Should throw an IllegalArgumentException because zookeeper.znode.parent is missing.");
+    } catch (IllegalArgumentException e) {
+      // Expected.
+    }
+
+    try {
+      rp.registerPeer(ID_ONE,
+        new ReplicationPeerConfig().setClusterKey("hostname1.example.org::/hbase"));
+      fail("Should throw an IllegalArgumentException because " +
+        "hbase.zookeeper.property.clientPort is missing.");
+    } catch (IllegalArgumentException e) {
+      // Expected.
+    }
+  }
+
+  @Test
+  public void testHfileRefsReplicationQueues() throws ReplicationException, KeeperException {
+    rp.init();
+    rq1.init(server1.getServerName());
+
+    List<Pair<Path, Path>> files1 = new ArrayList<>(3);
+    files1.add(new Pair<>(null, new Path("file_1")));
+    files1.add(new Pair<>(null, new Path("file_2")));
+    files1.add(new Pair<>(null, new Path("file_3")));
+    assertTrue(rqs.getReplicableHFiles(ID_ONE).isEmpty());
+    assertEquals(0, rqs.getAllPeersFromHFileRefsQueue().size());
+    rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
+    rq1.addPeerToHFileRefs(ID_ONE);
+    rq1.addHFileRefs(ID_ONE, files1);
+    assertEquals(1, rqs.getAllPeersFromHFileRefsQueue().size());
+    assertEquals(3, rqs.getReplicableHFiles(ID_ONE).size());
+    List<String> hfiles2 = new ArrayList<>(files1.size());
+    for (Pair<Path, Path> p : files1) {
+      hfiles2.add(p.getSecond().getName());
+    }
+    String removedString = hfiles2.remove(0);
+    rq1.removeHFileRefs(ID_ONE, hfiles2);
+    assertEquals(1, rqs.getReplicableHFiles(ID_ONE).size());
+    hfiles2 = new ArrayList<>(1);
+    hfiles2.add(removedString);
+    rq1.removeHFileRefs(ID_ONE, hfiles2);
+    assertEquals(0, rqs.getReplicableHFiles(ID_ONE).size());
+    rp.unregisterPeer(ID_ONE);
+  }
+
+  @Test
+  public void testRemovePeerForHFileRefs() throws ReplicationException, KeeperException {
+    rq1.init(server1.getServerName());
+
+    rp.init();
+    rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
+    rq1.addPeerToHFileRefs(ID_ONE);
+    rp.registerPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO));
+    rq1.addPeerToHFileRefs(ID_TWO);
+
+    List<Pair<Path, Path>> files1 = new ArrayList<>(3);
+    files1.add(new Pair<>(null, new Path("file_1")));
+    files1.add(new Pair<>(null, new Path("file_2")));
+    files1.add(new Pair<>(null, new Path("file_3")));
+    rq1.addHFileRefs(ID_ONE, files1);
+    rq1.addHFileRefs(ID_TWO, files1);
+    assertEquals(2, rqs.getAllPeersFromHFileRefsQueue().size());
+    assertEquals(3, rqs.getReplicableHFiles(ID_ONE).size());
+    assertEquals(3, rqs.getReplicableHFiles(ID_TWO).size());
+
+    rp.unregisterPeer(ID_ONE);
+    rq1.removePeerFromHFileRefs(ID_ONE);
+    assertEquals(1, rqs.getAllPeersFromHFileRefsQueue().size());
+    assertTrue(rqs.getReplicableHFiles(ID_ONE).isEmpty());
+    assertEquals(3, rqs.getReplicableHFiles(ID_TWO).size());
+
+    rp.unregisterPeer(ID_TWO);
+    rq1.removePeerFromHFileRefs(ID_TWO);
+    assertEquals(0, rqs.getAllPeersFromHFileRefsQueue().size());
+    assertTrue(rqs.getReplicableHFiles(ID_TWO).isEmpty());
+  }
+
+  @Test
+  public void testReplicationPeers() throws Exception {
+    rp.init();
+
+    // Test methods with non-existent peer ids
+    try {
+      rp.unregisterPeer("bogus");
+      fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
+    } catch (IllegalArgumentException e) {
+    }
+    try {
+      rp.enablePeer("bogus");
+      fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
+    } catch (IllegalArgumentException e) {
+    }
+    try {
+      rp.disablePeer("bogus");
+      fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
+    } catch (IllegalArgumentException e) {
+    }
+    try {
+      rp.getStatusOfPeer("bogus");
+      fail("Should have thrown an IllegalArgumentException when passed a bogus peerId");
+    } catch (IllegalArgumentException e) {
+    }
+    assertFalse(rp.peerConnected("bogus"));
+    rp.peerDisconnected("bogus");
+
+    assertNull(rp.getPeerConf("bogus"));
+    assertNumberOfPeers(0);
+
+    // Add some peers
+    rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
+    assertNumberOfPeers(1);
+    rp.registerPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO));
+    assertNumberOfPeers(2);
+
+    // Test methods with a peer that is added but not connected
+    try {
+      rp.getStatusOfPeer(ID_ONE);
+      fail("There are no connected peers, should have thrown an IllegalArgumentException");
+    } catch (IllegalArgumentException e) {
+    }
+    assertEquals(KEY_ONE, ZKConfig.getZooKeeperClusterKey(rp.getPeerConf(ID_ONE).getSecond()));
+    rp.unregisterPeer(ID_ONE);
+    rp.peerDisconnected(ID_ONE);
+    assertNumberOfPeers(1);
+
+    // Add one peer
+    rp.registerPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE));
+    rp.peerConnected(ID_ONE);
+    assertNumberOfPeers(2);
+    assertTrue(rp.getStatusOfPeer(ID_ONE));
+    rp.disablePeer(ID_ONE);
+    // now we do not rely on zk watcher to trigger the state change so we need to trigger it
+    // manually...
+    assertEquals(PeerState.DISABLED, rp.getConnectedPeer(ID_ONE).getPeerState(true));
+    assertConnectedPeerStatus(false, ID_ONE);
+    rp.enablePeer(ID_ONE);
+    // now we do not rely on zk watcher to trigger the state change so we need to trigger it
+    // manually...
+    assertEquals(PeerState.ENABLED, rp.getConnectedPeer(ID_ONE).getPeerState(true));
+    assertConnectedPeerStatus(true, ID_ONE);
+
+    // Disconnect peer
+    rp.peerDisconnected(ID_ONE);
+    assertNumberOfPeers(2);
+    try {
+      rp.getStatusOfPeer(ID_ONE);
+      fail("There are no connected peers, should have thrown an IllegalArgumentException");
+    } catch (IllegalArgumentException e) {
+    }
+  }
+
+  protected void assertConnectedPeerStatus(boolean status, String peerId) throws Exception {
+    // we can first check if the value was changed in the store, if it wasn't then fail right away
+    if (status != rp.getStatusOfPeerFromBackingStore(peerId)) {
+      fail("ConnectedPeerStatus was " + !status + " but expected " + status + " in ZK");
+    }
+    while (true) {
+      if (status == rp.getStatusOfPeer(peerId)) {
+        return;
+      }
+      if (zkTimeoutCount < ZK_MAX_COUNT) {
+        LOG.debug("ConnectedPeerStatus was " + !status + " but expected " + status +
+          ", sleeping and trying again.");
+        Thread.sleep(ZK_SLEEP_INTERVAL);
+      } else {
+        fail("Timed out waiting for ConnectedPeerStatus to be " + status);
+      }
+    }
+  }
+
+  protected void assertNumberOfPeers(int total) {
+    assertEquals(total, rp.getAllPeerConfigs().size());
+    assertEquals(total, rp.getAllPeerIds().size());
+    assertEquals(total, rp.getAllPeerIds().size());
+  }
+
+  /*
+   * three replicators: rq1 has 0 queues, rq2 has 1 queue with no logs, rq3 has 5 queues with 1, 2,
+   * 3, 4, 5 log files respectively
+   */
+  protected void populateQueues() throws ReplicationException {
+    rq1.addLog("trash", "trash");
+    rq1.removeQueue("trash");
+
+    rq2.addLog("qId1", "trash");
+    rq2.removeLog("qId1", "trash");
+
+    for (int i = 1; i < 6; i++) {
+      for (int j = 0; j < i; j++) {
+        rq3.addLog("qId" + i, "filename" + j);
+      }
+      // Add peers for the corresponding queues so they are not orphans
+      rp.registerPeer("qId" + i,
+        new ReplicationPeerConfig().setClusterKey("localhost:2818:/bogus" + i));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
new file mode 100644
index 0000000..5fe7c55
--- /dev/null
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java
@@ -0,0 +1,148 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.ClusterId;
+import org.apache.hadoop.hbase.HBaseZKTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
+import org.apache.hadoop.hbase.zookeeper.ZKConfig;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
+import org.apache.zookeeper.KeeperException;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Category({ ReplicationTests.class, MediumTests.class })
+public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestReplicationStateZKImpl.class);
+
+  private static Configuration conf;
+  private static HBaseZKTestingUtility utility;
+  private static ZKWatcher zkw;
+  private static String replicationZNode;
+  private ReplicationQueuesZKImpl rqZK;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    utility = new HBaseZKTestingUtility();
+    utility.startMiniZKCluster();
+    conf = utility.getConfiguration();
+    conf.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
+    zkw = utility.getZooKeeperWatcher();
+    String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
+    replicationZNode = ZNodePaths.joinZNode(zkw.znodePaths.baseZNode, replicationZNodeName);
+    KEY_ONE = initPeerClusterState("/hbase1");
+    KEY_TWO = initPeerClusterState("/hbase2");
+  }
+
+  private static String initPeerClusterState(String baseZKNode)
+      throws IOException, KeeperException {
+    // Add a dummy region server and set up the cluster id
+    Configuration testConf = new Configuration(conf);
+    testConf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, baseZKNode);
+    ZKWatcher zkw1 = new ZKWatcher(testConf, "test1", null);
+    String fakeRs = ZNodePaths.joinZNode(zkw1.znodePaths.rsZNode, "hostname1.example.org:1234");
+    ZKUtil.createWithParents(zkw1, fakeRs);
+    ZKClusterId.setClusterId(zkw1, new ClusterId());
+    return ZKConfig.getZooKeeperClusterKey(testConf);
+  }
+
+  @Before
+  public void setUp() {
+    zkTimeoutCount = 0;
+    WarnOnlyAbortable abortable = new WarnOnlyAbortable();
+    try {
+      rq1 = ReplicationFactory
+          .getReplicationQueues(new ReplicationQueuesArguments(conf, abortable, zkw));
+      rq2 = ReplicationFactory
+          .getReplicationQueues(new ReplicationQueuesArguments(conf, abortable, zkw));
+      rq3 = ReplicationFactory
+          .getReplicationQueues(new ReplicationQueuesArguments(conf, abortable, zkw));
+      rqs = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf);
+    } catch (Exception e) {
+      // This should not occur, because getReplicationQueues() only throws for
+      // TableBasedReplicationQueuesImpl
+      fail("ReplicationFactory.getReplicationQueues() threw an IO Exception");
+    }
+    rp = ReplicationFactory.getReplicationPeers(zkw, conf, zkw);
+    OUR_KEY = ZKConfig.getZooKeeperClusterKey(conf);
+    rqZK = new ReplicationQueuesZKImpl(zkw, conf, abortable);
+  }
+
+  @After
+  public void tearDown() throws KeeperException, IOException {
+    ZKUtil.deleteNodeRecursively(zkw, replicationZNode);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    utility.shutdownMiniZKCluster();
+  }
+
+  @Test
+  public void testIsPeerPath_PathToParentOfPeerNode() {
+    assertFalse(rqZK.isPeerPath(rqZK.peersZNode));
+  }
+
+  @Test
+  public void testIsPeerPath_PathToChildOfPeerNode() {
+    String peerChild = ZNodePaths.joinZNode(ZNodePaths.joinZNode(rqZK.peersZNode, "1"), "child");
+    assertFalse(rqZK.isPeerPath(peerChild));
+  }
+
+  @Test
+  public void testIsPeerPath_ActualPeerPath() {
+    String peerPath = ZNodePaths.joinZNode(rqZK.peersZNode, "1");
+    assertTrue(rqZK.isPeerPath(peerPath));
+  }
+
+  private static class WarnOnlyAbortable implements Abortable {
+
+    @Override
+    public void abort(String why, Throwable e) {
+      LOG.warn("TestReplicationStateZKImpl received abort, ignoring.  Reason: " + why);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(e.toString(), e);
+      }
+    }
+
+    @Override
+    public boolean isAborted() {
+      return false;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
index d5bba0d..786730f 100644
--- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
@@ -23,15 +23,18 @@ import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Set;
 import java.util.SortedSet;
 
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseZKTestingUtility;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -115,6 +118,15 @@ public class TestZKReplicationQueueStorage {
     assertEquals(2, queueIds.size());
     assertThat(queueIds, hasItems("1", "2"));
 
+    List<String> wals1 = STORAGE.getWALsInQueue(serverName1, queue1);
+    List<String> wals2 = STORAGE.getWALsInQueue(serverName1, queue2);
+    assertEquals(10, wals1.size());
+    assertEquals(10, wals1.size());
+    for (int i = 0; i < 10; i++) {
+      assertThat(wals1, hasItems(getFileName("file1", i)));
+      assertThat(wals2, hasItems(getFileName("file2", i)));
+    }
+
     for (int i = 0; i < 10; i++) {
       assertEquals(0, STORAGE.getWALPosition(serverName1, queue1, getFileName("file1", i)));
       assertEquals(0, STORAGE.getWALPosition(serverName1, queue2, getFileName("file2", i)));
@@ -157,10 +169,20 @@ public class TestZKReplicationQueueStorage {
     queueIds = STORAGE.getAllQueues(serverName1);
     assertEquals(1, queueIds.size());
     assertThat(queueIds, hasItems("2"));
+    wals2 = STORAGE.getWALsInQueue(serverName1, queue2);
+    assertEquals(5, wals2.size());
+    for (i = 0; i < 10; i += 2) {
+      assertThat(wals2, hasItems(getFileName("file2", i)));
+    }
 
     queueIds = STORAGE.getAllQueues(serverName2);
     assertEquals(1, queueIds.size());
     assertThat(queueIds, hasItems(peer1.getFirst()));
+    wals1 = STORAGE.getWALsInQueue(serverName2, peer1.getFirst());
+    assertEquals(5, wals1.size());
+    for (i = 1; i < 10; i += 2) {
+      assertThat(wals1, hasItems(getFileName("file1", i)));
+    }
 
     Set<String> allWals = STORAGE.getAllWALs();
     assertEquals(10, allWals.size());
@@ -168,4 +190,56 @@ public class TestZKReplicationQueueStorage {
       assertThat(allWals, hasItems(i % 2 == 0 ? getFileName("file2", i) : getFileName("file1", i)));
     }
   }
+
+  // For HBASE-12865
+  @Test
+  public void testClaimQueueChangeCversion() throws ReplicationException, KeeperException {
+    ServerName serverName1 = ServerName.valueOf("127.0.0.1", 8000, 10000);
+    STORAGE.addWAL(serverName1, "1", "file");
+
+    int v0 = STORAGE.getQueuesZNodeCversion();
+    ServerName serverName2 = ServerName.valueOf("127.0.0.1", 8001, 10001);
+    STORAGE.claimQueue(serverName1, "1", serverName2);
+    int v1 = STORAGE.getQueuesZNodeCversion();
+    // cversion should increase by 1 since a child node is deleted
+    assertEquals(1, v1 - v0);
+  }
+
+  private ZKReplicationQueueStorage createWithUnstableCversion() throws IOException {
+    return new ZKReplicationQueueStorage(UTIL.getZooKeeperWatcher(), UTIL.getConfiguration()) {
+
+      private int called = 0;
+
+      @Override
+      protected int getQueuesZNodeCversion() throws KeeperException {
+        if (called < 4) {
+          called++;
+        }
+        return called;
+      }
+    };
+  }
+
+  @Test
+  public void testGetAllWALsCversionChange() throws IOException, ReplicationException {
+    ZKReplicationQueueStorage storage = createWithUnstableCversion();
+    storage.addWAL(getServerName(0), "1", "file");
+    // This should return eventually when cversion stabilizes
+    Set<String> allWals = storage.getAllWALs();
+    assertEquals(1, allWals.size());
+    assertThat(allWals, hasItems("file"));
+  }
+
+  // For HBASE-14621
+  @Test
+  public void testGetAllHFileRefsCversionChange() throws IOException, ReplicationException {
+    ZKReplicationQueueStorage storage = createWithUnstableCversion();
+    storage.addPeerToHFileRefs("1");
+    Path p = new Path("/test");
+    storage.addHFileRefs("1", Arrays.asList(Pair.newPair(p, p)));
+    // This should return eventually when cversion stabilizes
+    Set<String> allHFileRefs = storage.getAllHFileRefs();
+    assertEquals(1, allHFileRefs.size());
+    assertThat(allHFileRefs, hasItems("test"));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java
index 97deab5..af41399 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleaner.java
@@ -23,21 +23,23 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 import java.util.Map.Entry;
+import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
-import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
+import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -48,23 +50,19 @@ import org.slf4j.LoggerFactory;
 @InterfaceAudience.Private
 public class ReplicationZKNodeCleaner {
   private static final Logger LOG = LoggerFactory.getLogger(ReplicationZKNodeCleaner.class);
-  private final ZKWatcher zkw;
-  private final ReplicationQueuesClient queuesClient;
+  private final ReplicationQueueStorage queueStorage;
   private final ReplicationPeers replicationPeers;
   private final ReplicationQueueDeletor queueDeletor;
 
   public ReplicationZKNodeCleaner(Configuration conf, ZKWatcher zkw, Abortable abortable)
       throws IOException {
     try {
-      this.zkw = zkw;
-      this.queuesClient = ReplicationFactory
-          .getReplicationQueuesClient(new ReplicationQueuesClientArguments(conf, abortable, zkw));
-      this.queuesClient.init();
-      this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, this.queuesClient,
-        abortable);
+      this.queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf);
+      this.replicationPeers =
+          ReplicationFactory.getReplicationPeers(zkw, conf, this.queueStorage, abortable);
       this.replicationPeers.init();
       this.queueDeletor = new ReplicationQueueDeletor(zkw, conf, abortable);
-    } catch (Exception e) {
+    } catch (ReplicationException e) {
       throw new IOException("failed to construct ReplicationZKNodeCleaner", e);
     }
   }
@@ -73,16 +71,16 @@ public class ReplicationZKNodeCleaner {
    * @return undeletedQueues replicator with its queueIds for removed peers
    * @throws IOException
    */
-  public Map<String, List<String>> getUnDeletedQueues() throws IOException {
-    Map<String, List<String>> undeletedQueues = new HashMap<>();
+  public Map<ServerName, List<String>> getUnDeletedQueues() throws IOException {
+    Map<ServerName, List<String>> undeletedQueues = new HashMap<>();
     Set<String> peerIds = new HashSet<>(this.replicationPeers.getAllPeerIds());
     try {
-      List<String> replicators = this.queuesClient.getListOfReplicators();
+      List<ServerName> replicators = this.queueStorage.getListOfReplicators();
       if (replicators == null || replicators.isEmpty()) {
         return undeletedQueues;
       }
-      for (String replicator : replicators) {
-        List<String> queueIds = this.queuesClient.getAllQueues(replicator);
+      for (ServerName replicator : replicators) {
+        List<String> queueIds = this.queueStorage.getAllQueues(replicator);
         for (String queueId : queueIds) {
           ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
           if (!peerIds.contains(queueInfo.getPeerId())) {
@@ -96,7 +94,7 @@ public class ReplicationZKNodeCleaner {
           }
         }
       }
-    } catch (KeeperException ke) {
+    } catch (ReplicationException ke) {
       throw new IOException("Failed to get the replication queues of all replicators", ke);
     }
     return undeletedQueues;
@@ -105,25 +103,21 @@ public class ReplicationZKNodeCleaner {
   /**
    * @return undeletedHFileRefsQueue replicator with its undeleted queueIds for removed peers in
    *         hfile-refs queue
-   * @throws IOException
    */
   public Set<String> getUnDeletedHFileRefsQueues() throws IOException {
     Set<String> undeletedHFileRefsQueue = new HashSet<>();
     Set<String> peerIds = new HashSet<>(this.replicationPeers.getAllPeerIds());
     String hfileRefsZNode = queueDeletor.getHfileRefsZNode();
     try {
-      if (-1 == ZKUtil.checkExists(zkw, hfileRefsZNode)) {
-        return null;
-      }
-      List<String> listOfPeers = this.queuesClient.getAllPeersFromHFileRefsQueue();
+      List<String> listOfPeers = this.queueStorage.getAllPeersFromHFileRefsQueue();
       Set<String> peers = new HashSet<>(listOfPeers);
       peers.removeAll(peerIds);
       if (!peers.isEmpty()) {
         undeletedHFileRefsQueue.addAll(peers);
       }
-    } catch (KeeperException e) {
-      throw new IOException("Failed to get list of all peers from hfile-refs znode "
-          + hfileRefsZNode, e);
+    } catch (ReplicationException e) {
+      throw new IOException(
+          "Failed to get list of all peers from hfile-refs znode " + hfileRefsZNode, e);
     }
     return undeletedHFileRefsQueue;
   }
@@ -137,21 +131,20 @@ public class ReplicationZKNodeCleaner {
     /**
      * @param replicator The regionserver which has undeleted queue
      * @param queueId The undeleted queue id
-     * @throws IOException
      */
-    public void removeQueue(final String replicator, final String queueId) throws IOException {
-      String queueZnodePath =
-        ZNodePaths.joinZNode(ZNodePaths.joinZNode(this.queuesZNode, replicator), queueId);
+    public void removeQueue(final ServerName replicator, final String queueId) throws IOException {
+      String queueZnodePath = ZNodePaths
+          .joinZNode(ZNodePaths.joinZNode(this.queuesZNode, replicator.getServerName()), queueId);
       try {
         ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId);
         if (!replicationPeers.getAllPeerIds().contains(queueInfo.getPeerId())) {
           ZKUtil.deleteNodeRecursively(this.zookeeper, queueZnodePath);
-          LOG.info("Successfully removed replication queue, replicator: " + replicator
-              + ", queueId: " + queueId);
+          LOG.info("Successfully removed replication queue, replicator: " + replicator +
+            ", queueId: " + queueId);
         }
       } catch (KeeperException e) {
-        throw new IOException("Failed to delete queue, replicator: " + replicator + ", queueId: "
-            + queueId);
+        throw new IOException(
+            "Failed to delete queue, replicator: " + replicator + ", queueId: " + queueId);
       }
     }
 
@@ -183,9 +176,9 @@ public class ReplicationZKNodeCleaner {
    * @param undeletedQueues replicator with its queueIds for removed peers
    * @throws IOException
    */
-  public void removeQueues(final Map<String, List<String>> undeletedQueues) throws IOException {
-    for (Entry<String, List<String>> replicatorAndQueueIds : undeletedQueues.entrySet()) {
-      String replicator = replicatorAndQueueIds.getKey();
+  public void removeQueues(final Map<ServerName, List<String>> undeletedQueues) throws IOException {
+    for (Entry<ServerName, List<String>> replicatorAndQueueIds : undeletedQueues.entrySet()) {
+      ServerName replicator = replicatorAndQueueIds.getKey();
       for (String queueId : replicatorAndQueueIds.getValue()) {
         queueDeletor.removeQueue(replicator, queueId);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleanerChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleanerChore.java
index 8d5df9b..19ca804 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleanerChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationZKNodeCleanerChore.java
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.master.cleaner;
 
 import java.io.IOException;
@@ -23,6 +22,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -45,11 +45,10 @@ public class ReplicationZKNodeCleanerChore extends ScheduledChore {
   @Override
   protected void chore() {
     try {
-      Map<String, List<String>> undeletedQueues = cleaner.getUnDeletedQueues();
+      Map<ServerName, List<String>> undeletedQueues = cleaner.getUnDeletedQueues();
       cleaner.removeQueues(undeletedQueues);
     } catch (IOException e) {
       LOG.warn("Failed to clean replication zk node", e);
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
index 5abd874..84abfeb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.replication.BaseReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfigBuilder;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
 import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
@@ -147,28 +148,13 @@ public final class ReplicationPeerManager {
     }
   }
 
-  private ReplicationPeerConfig copy(ReplicationPeerConfig peerConfig) {
-    ReplicationPeerConfig copiedPeerConfig = new ReplicationPeerConfig();
-    copiedPeerConfig.getConfiguration().putAll(peerConfig.getConfiguration());
-    copiedPeerConfig.getPeerData().putAll(peerConfig.getPeerData());
-    copiedPeerConfig.setTableCFsMap(peerConfig.getTableCFsMap());
-    copiedPeerConfig.setNamespaces(peerConfig.getNamespaces());
-    copiedPeerConfig.setExcludeTableCFsMap(peerConfig.getExcludeTableCFsMap());
-    copiedPeerConfig.setExcludeNamespaces(peerConfig.getExcludeNamespaces());
-    copiedPeerConfig.setBandwidth(peerConfig.getBandwidth());
-    copiedPeerConfig.setReplicateAllUserTables(peerConfig.replicateAllUserTables());
-    copiedPeerConfig.setClusterKey(peerConfig.getClusterKey());
-    copiedPeerConfig.setReplicationEndpointImpl(peerConfig.getReplicationEndpointImpl());
-    return copiedPeerConfig;
-  }
-
   public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
       throws ReplicationException {
     if (peers.containsKey(peerId)) {
       // this should be a retry, just return
       return;
     }
-    ReplicationPeerConfig copiedPeerConfig = copy(peerConfig);
+    ReplicationPeerConfig copiedPeerConfig = ReplicationPeerConfig.newBuilder(peerConfig).build();
     peerStorage.addPeer(peerId, copiedPeerConfig, enabled);
     peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, copiedPeerConfig));
   }
@@ -205,13 +191,14 @@ public final class ReplicationPeerManager {
     // the checking rules are too complicated here so we give up checking whether this is a retry.
     ReplicationPeerDescription desc = peers.get(peerId);
     ReplicationPeerConfig oldPeerConfig = desc.getPeerConfig();
-    ReplicationPeerConfig newPeerConfig = copy(peerConfig);
+    ReplicationPeerConfigBuilder newPeerConfigBuilder =
+        ReplicationPeerConfig.newBuilder(peerConfig);
     // we need to use the new conf to overwrite the old one.
-    newPeerConfig.getConfiguration().putAll(oldPeerConfig.getConfiguration());
-    newPeerConfig.getConfiguration().putAll(peerConfig.getConfiguration());
-    newPeerConfig.getPeerData().putAll(oldPeerConfig.getPeerData());
-    newPeerConfig.getPeerData().putAll(peerConfig.getPeerData());
-
+    newPeerConfigBuilder.putAllConfiguration(oldPeerConfig.getConfiguration());
+    newPeerConfigBuilder.putAllConfiguration(peerConfig.getConfiguration());
+    newPeerConfigBuilder.putAllConfiguration(oldPeerConfig.getConfiguration());
+    newPeerConfigBuilder.putAllConfiguration(peerConfig.getConfiguration());
+    ReplicationPeerConfig newPeerConfig = newPeerConfigBuilder.build();
     peerStorage.updatePeerConfig(peerId, newPeerConfig);
     peers.put(peerId, new ReplicationPeerDescription(peerId, desc.isEnabled(), newPeerConfig));
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
index 4e9d67a..39a6ba6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationHFileCleaner.java
@@ -1,42 +1,44 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable
- * law or agreed to in writing, software distributed under the License is distributed on an "AS IS"
- * BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License
- * for the specific language governing permissions and limitations under the License.
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
  */
 package org.apache.hadoop.hbase.replication.master;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Predicate;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableSet;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets;
-
 import java.io.IOException;
 import java.util.Collections;
-import java.util.List;
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
-import org.apache.hadoop.hbase.replication.ReplicationFactory;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
-import org.apache.zookeeper.KeeperException;
+import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Predicate;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
+
 /**
  * Implementation of a file cleaner that checks if a hfile is still scheduled for replication before
  * deleting it from hfile archive directory.
@@ -45,7 +47,7 @@ import org.slf4j.LoggerFactory;
 public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
   private static final Logger LOG = LoggerFactory.getLogger(ReplicationHFileCleaner.class);
   private ZKWatcher zkw;
-  private ReplicationQueuesClient rqc;
+  private ReplicationQueueStorage rqs;
   private boolean stopped = false;
 
   @Override
@@ -60,8 +62,8 @@ public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
     try {
       // The concurrently created new hfile entries in ZK may not be included in the return list,
       // but they won't be deleted because they're not in the checking set.
-      hfileRefs = loadHFileRefsFromPeers();
-    } catch (KeeperException e) {
+      hfileRefs = rqs.getAllHFileRefs();
+    } catch (ReplicationException e) {
       LOG.warn("Failed to read hfile references from zookeeper, skipping checking deletable files");
       return Collections.emptyList();
     }
@@ -82,37 +84,6 @@ public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
     });
   }
 
-  /**
-   * Load all hfile references in all replication queues from ZK. This method guarantees to return a
-   * snapshot which contains all hfile references in the zookeeper at the start of this call.
-   * However, some newly created hfile references during the call may not be included.
-   */
-  private Set<String> loadHFileRefsFromPeers() throws KeeperException {
-    Set<String> hfileRefs = Sets.newHashSet();
-    List<String> listOfPeers;
-    for (int retry = 0;; retry++) {
-      int v0 = rqc.getHFileRefsNodeChangeVersion();
-      hfileRefs.clear();
-      listOfPeers = rqc.getAllPeersFromHFileRefsQueue();
-      if (listOfPeers == null) {
-        LOG.debug("Didn't find any peers with hfile references, won't prevent any deletions.");
-        return ImmutableSet.of();
-      }
-      for (String id : listOfPeers) {
-        List<String> peerHFileRefs = rqc.getReplicableHFiles(id);
-        if (peerHFileRefs != null) {
-          hfileRefs.addAll(peerHFileRefs);
-        }
-      }
-      int v1 = rqc.getHFileRefsNodeChangeVersion();
-      if (v0 == v1) {
-        return hfileRefs;
-      }
-      LOG.debug(String.format("Replication hfile references node cversion changed from "
-          + "%d to %d, retry = %d", v0, v1, retry));
-    }
-  }
-
   @Override
   public void setConf(Configuration config) {
     // If either replication or replication of bulk load hfiles is disabled, keep all members null
@@ -139,17 +110,15 @@ public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
   public void setConf(Configuration conf, ZKWatcher zk) {
     super.setConf(conf);
     try {
-      initReplicationQueuesClient(conf, zk);
+      initReplicationQueueStorage(conf, zk);
     } catch (Exception e) {
       LOG.error("Error while configuring " + this.getClass().getName(), e);
     }
   }
 
-  private void initReplicationQueuesClient(Configuration conf, ZKWatcher zk)
-      throws Exception {
+  private void initReplicationQueueStorage(Configuration conf, ZKWatcher zk) {
     this.zkw = zk;
-    this.rqc = ReplicationFactory.getReplicationQueuesClient(new ReplicationQueuesClientArguments(
-        conf, new WarnOnlyAbortable(), zkw));
+    this.rqs = ReplicationStorageFactory.getReplicationQueueStorage(zk, conf);
   }
 
   @Override
@@ -179,25 +148,12 @@ public class ReplicationHFileCleaner extends BaseHFileCleanerDelegate {
     }
 
     try {
-      hfileRefsFromQueue = loadHFileRefsFromPeers();
-    } catch (KeeperException e) {
+      hfileRefsFromQueue = rqs.getAllHFileRefs();
+    } catch (ReplicationException e) {
       LOG.warn("Failed to read hfile references from zookeeper, skipping checking deletable "
           + "file for " + fStat.getPath());
       return false;
     }
     return !hfileRefsFromQueue.contains(fStat.getPath().getName());
   }
-
-  private static class WarnOnlyAbortable implements Abortable {
-    @Override
-    public void abort(String why, Throwable e) {
-      LOG.warn("ReplicationHFileCleaner received abort, ignoring.  Reason: " + why);
-      LOG.debug(e.toString(), e);
-    }
-
-    @Override
-    public boolean isAborted() {
-      return false;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8af1ce3e/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
index 773e10e..e8d6fa3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationLogCleaner.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -24,16 +23,14 @@ import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.master.cleaner.BaseLogCleanerDelegate;
-import org.apache.hadoop.hbase.replication.ReplicationFactory;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
-import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments;
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 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;
 
@@ -49,7 +46,7 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
 public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
   private static final Logger LOG = LoggerFactory.getLogger(ReplicationLogCleaner.class);
   private ZKWatcher zkw;
-  private ReplicationQueuesClient replicationQueues;
+  private ReplicationQueueStorage queueStorage;
   private boolean stopped = false;
   private Set<String> wals;
   private long readZKTimestamp = 0;
@@ -60,8 +57,8 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
     try {
       // The concurrently created new WALs may not be included in the return list,
       // but they won't be deleted because they're not in the checking set.
-      wals = replicationQueues.getAllWALs();
-    } catch (KeeperException e) {
+      wals = queueStorage.getAllWALs();
+    } catch (ReplicationException e) {
       LOG.warn("Failed to read zookeeper, skipping checking deletable files");
       wals = null;
     }
@@ -112,9 +109,7 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
     super.setConf(conf);
     try {
       this.zkw = zk;
-      this.replicationQueues = ReplicationFactory.getReplicationQueuesClient(
-          new ReplicationQueuesClientArguments(conf, new WarnOnlyAbortable(), zkw));
-      this.replicationQueues.init();
+      this.queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zk, conf);
     } catch (Exception e) {
       LOG.error("Error while configuring " + this.getClass().getName(), e);
     }
@@ -134,18 +129,4 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
   public boolean isStopped() {
     return this.stopped;
   }
-
-  private static class WarnOnlyAbortable implements Abortable {
-
-    @Override
-    public void abort(String why, Throwable e) {
-      LOG.warn("ReplicationLogCleaner received abort, ignoring.  Reason: " + why);
-      LOG.debug(e.toString(), e);
-    }
-
-    @Override
-    public boolean isAborted() {
-      return false;
-    }
-  }
 }


[02/26] hbase git commit: HBASE-19628 ByteBufferCell should extend ExtendedCell

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java
index 363f0bd..69eaab0 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java
@@ -83,17 +83,17 @@ public final class PrivateCellUtil {
   /********************* misc *************************************/
 
   public static byte getRowByte(Cell cell, int index) {
-    if (cell instanceof ByteBufferCell) {
-      return ((ByteBufferCell) cell).getRowByteBuffer()
-          .get(((ByteBufferCell) cell).getRowPosition() + index);
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ((ByteBufferExtendedCell) cell).getRowByteBuffer()
+          .get(((ByteBufferExtendedCell) cell).getRowPosition() + index);
     }
     return cell.getRowArray()[cell.getRowOffset() + index];
   }
 
   public static byte getQualifierByte(Cell cell, int index) {
-    if (cell instanceof ByteBufferCell) {
-      return ((ByteBufferCell) cell).getQualifierByteBuffer()
-          .get(((ByteBufferCell) cell).getQualifierPosition() + index);
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ((ByteBufferExtendedCell) cell).getQualifierByteBuffer()
+          .get(((ByteBufferExtendedCell) cell).getQualifierPosition() + index);
     }
     return cell.getQualifierArray()[cell.getQualifierOffset() + index];
   }
@@ -115,15 +115,16 @@ public final class PrivateCellUtil {
    * @return A new cell which is having the extra tags also added to it.
    */
   public static Cell createCell(Cell cell, byte[] tags) {
-    if (cell instanceof ByteBufferCell) {
-      return new TagRewriteByteBufferCell((ByteBufferCell) cell, tags);
+    if (cell instanceof ByteBufferExtendedCell) {
+      return new TagRewriteByteBufferExtendedCell((ByteBufferExtendedCell) cell, tags);
     }
     return new TagRewriteCell(cell, tags);
   }
 
   public static Cell createCell(Cell cell, byte[] value, byte[] tags) {
-    if (cell instanceof ByteBufferCell) {
-      return new ValueAndTagRewriteByteBufferCell((ByteBufferCell) cell, value, tags);
+    if (cell instanceof ByteBufferExtendedCell) {
+      return new ValueAndTagRewriteByteBufferExtendedCell((ByteBufferExtendedCell) cell,
+          value, tags);
     }
     return new ValueAndTagRewriteCell(cell, value, tags);
   }
@@ -314,25 +315,24 @@ public final class PrivateCellUtil {
     }
   }
 
-  static class TagRewriteByteBufferCell extends ByteBufferCell implements ExtendedCell {
+  static class TagRewriteByteBufferExtendedCell extends ByteBufferExtendedCell {
 
-    protected ByteBufferCell cell;
+    protected ByteBufferExtendedCell cell;
     protected byte[] tags;
     private static final int HEAP_SIZE_OVERHEAD = ClassSize.OBJECT + 2 * ClassSize.REFERENCE;
 
     /**
-     * @param cell The original ByteBufferCell which it rewrites
+     * @param cell The original ByteBufferExtendedCell which it rewrites
      * @param tags the tags bytes. The array suppose to contain the tags bytes alone.
      */
-    public TagRewriteByteBufferCell(ByteBufferCell cell, byte[] tags) {
-      assert cell instanceof ExtendedCell;
+    public TagRewriteByteBufferExtendedCell(ByteBufferExtendedCell cell, byte[] tags) {
       assert tags != null;
       this.cell = cell;
       this.tags = tags;
       // tag offset will be treated as 0 and length this.tags.length
-      if (this.cell instanceof TagRewriteByteBufferCell) {
+      if (this.cell instanceof TagRewriteByteBufferExtendedCell) {
         // Cleaning the ref so that the byte[] can be GCed
-        ((TagRewriteByteBufferCell) this.cell).tags = null;
+        ((TagRewriteByteBufferExtendedCell) this.cell).tags = null;
       }
     }
 
@@ -490,8 +490,9 @@ public final class PrivateCellUtil {
     @Override
     public ExtendedCell deepClone() {
       Cell clonedBaseCell = ((ExtendedCell) this.cell).deepClone();
-      if (clonedBaseCell instanceof ByteBufferCell) {
-        return new TagRewriteByteBufferCell((ByteBufferCell) clonedBaseCell, this.tags);
+      if (clonedBaseCell instanceof ByteBufferExtendedCell) {
+        return new TagRewriteByteBufferExtendedCell((ByteBufferExtendedCell) clonedBaseCell,
+            this.tags);
       }
       return new TagRewriteCell(clonedBaseCell, this.tags);
     }
@@ -585,7 +586,10 @@ public final class PrivateCellUtil {
       return write(out, withTags, this.cell, this.value, this.tags);
     }
 
-    // Made into a static method so as to reuse the logic within ValueAndTagRewriteByteBufferCell
+    /**
+     * Made into a static method so as to reuse the logic within
+     * ValueAndTagRewriteByteBufferExtendedCell
+     */
     static int write(OutputStream out, boolean withTags, Cell cell, byte[] value, byte[] tags)
         throws IOException {
       int valLen = value == null ? 0 : value.length;
@@ -617,7 +621,10 @@ public final class PrivateCellUtil {
       write(buf, offset, this.cell, this.value, this.tags);
     }
 
-    // Made into a static method so as to reuse the logic within ValueAndTagRewriteByteBufferCell
+    /**
+     * Made into a static method so as to reuse the logic
+     * within ValueAndTagRewriteByteBufferExtendedCell
+     */
     static void write(ByteBuffer buf, int offset, Cell cell, byte[] value, byte[] tags) {
       offset = ByteBufferUtils.putInt(buf, offset, KeyValueUtil.keyLength(cell));// Key length
       offset = ByteBufferUtils.putInt(buf, offset, value.length);// Value length
@@ -638,11 +645,12 @@ public final class PrivateCellUtil {
     }
   }
 
-  static class ValueAndTagRewriteByteBufferCell extends TagRewriteByteBufferCell {
+  static class ValueAndTagRewriteByteBufferExtendedCell extends TagRewriteByteBufferExtendedCell {
 
     protected byte[] value;
 
-    public ValueAndTagRewriteByteBufferCell(ByteBufferCell cell, byte[] value, byte[] tags) {
+    public ValueAndTagRewriteByteBufferExtendedCell(ByteBufferExtendedCell cell,
+        byte[] value, byte[] tags) {
       super(cell, tags);
       this.value = value;
     }
@@ -699,9 +707,9 @@ public final class PrivateCellUtil {
     @Override
     public ExtendedCell deepClone() {
       Cell clonedBaseCell = ((ExtendedCell) this.cell).deepClone();
-      if (clonedBaseCell instanceof ByteBufferCell) {
-        return new ValueAndTagRewriteByteBufferCell((ByteBufferCell) clonedBaseCell, this.value,
-            this.tags);
+      if (clonedBaseCell instanceof ByteBufferExtendedCell) {
+        return new ValueAndTagRewriteByteBufferExtendedCell(
+            (ByteBufferExtendedCell) clonedBaseCell, this.value, this.tags);
       }
       return new ValueAndTagRewriteCell(clonedBaseCell, this.value, this.tags);
     }
@@ -709,19 +717,21 @@ public final class PrivateCellUtil {
 
   public static boolean matchingRows(final Cell left, final byte[] buf, final int offset,
       final int length) {
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) left).getRowByteBuffer(),
-        ((ByteBufferCell) left).getRowPosition(), left.getRowLength(), buf, offset, length);
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) left).getRowPosition(), left.getRowLength(),
+          buf, offset, length);
     }
     return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(), buf, offset,
-      length);
+        length);
   }
 
   public static boolean matchingFamily(final Cell left, final byte[] buf, final int offset,
       final int length) {
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) left).getFamilyByteBuffer(),
-        ((ByteBufferCell) left).getFamilyPosition(), left.getFamilyLength(), buf, offset, length);
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getFamilyByteBuffer(),
+          ((ByteBufferExtendedCell) left).getFamilyPosition(), left.getFamilyLength(),
+          buf, offset, length);
     }
     return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(), buf,
       offset, length);
@@ -740,10 +750,10 @@ public final class PrivateCellUtil {
     if (buf == null) {
       return left.getQualifierLength() == 0;
     }
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) left).getQualifierByteBuffer(),
-        ((ByteBufferCell) left).getQualifierPosition(), left.getQualifierLength(), buf, offset,
-        length);
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) left).getQualifierPosition(), left.getQualifierLength(),
+          buf, offset, length);
     }
     return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
       left.getQualifierLength(), buf, offset, length);
@@ -759,24 +769,25 @@ public final class PrivateCellUtil {
 
   public static boolean matchingValue(final Cell left, final Cell right, int lvlength,
       int rvlength) {
-    if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) left).getValueByteBuffer(),
-        ((ByteBufferCell) left).getValuePosition(), lvlength,
-        ((ByteBufferCell) right).getValueByteBuffer(), ((ByteBufferCell) right).getValuePosition(),
-        rvlength);
+    if (left instanceof ByteBufferExtendedCell && right instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getValueByteBuffer(),
+          ((ByteBufferExtendedCell) left).getValuePosition(), lvlength,
+          ((ByteBufferExtendedCell) right).getValueByteBuffer(),
+          ((ByteBufferExtendedCell) right).getValuePosition(), rvlength);
     }
-    if (left instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) left).getValueByteBuffer(),
-        ((ByteBufferCell) left).getValuePosition(), lvlength, right.getValueArray(),
-        right.getValueOffset(), rvlength);
+    if (left instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) left).getValueByteBuffer(),
+          ((ByteBufferExtendedCell) left).getValuePosition(), lvlength, right.getValueArray(),
+          right.getValueOffset(), rvlength);
     }
-    if (right instanceof ByteBufferCell) {
-      return ByteBufferUtils.equals(((ByteBufferCell) right).getValueByteBuffer(),
-        ((ByteBufferCell) right).getValuePosition(), rvlength, left.getValueArray(),
-        left.getValueOffset(), lvlength);
+    if (right instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.equals(((ByteBufferExtendedCell) right).getValueByteBuffer(),
+          ((ByteBufferExtendedCell) right).getValuePosition(), rvlength, left.getValueArray(),
+          left.getValueOffset(), lvlength);
     }
-    return Bytes.equals(left.getValueArray(), left.getValueOffset(), lvlength,
-      right.getValueArray(), right.getValueOffset(), rvlength);
+    return Bytes
+        .equals(left.getValueArray(), left.getValueOffset(), lvlength, right.getValueArray(),
+            right.getValueOffset(), rvlength);
   }
 
   public static boolean matchingType(Cell a, Cell b) {
@@ -837,13 +848,13 @@ public final class PrivateCellUtil {
    */
   public static int copyTagsTo(Cell cell, byte[] destination, int destinationOffset) {
     int tlen = cell.getTagsLength();
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyFromBufferToArray(destination,
-        ((ByteBufferCell) cell).getTagsByteBuffer(), ((ByteBufferCell) cell).getTagsPosition(),
-        destinationOffset, tlen);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils
+        .copyFromBufferToArray(destination, ((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getTagsPosition(), destinationOffset, tlen);
     } else {
-      System.arraycopy(cell.getTagsArray(), cell.getTagsOffset(), destination, destinationOffset,
-        tlen);
+      System
+        .arraycopy(cell.getTagsArray(), cell.getTagsOffset(), destination, destinationOffset, tlen);
     }
     return destinationOffset + tlen;
   }
@@ -857,9 +868,9 @@ public final class PrivateCellUtil {
    */
   public static int copyTagsTo(Cell cell, ByteBuffer destination, int destinationOffset) {
     int tlen = cell.getTagsLength();
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferCell) cell).getTagsByteBuffer(),
-        destination, ((ByteBufferCell) cell).getTagsPosition(), destinationOffset, tlen);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
+        destination, ((ByteBufferExtendedCell) cell).getTagsPosition(), destinationOffset, tlen);
     } else {
       ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getTagsArray(),
         cell.getTagsOffset(), tlen);
@@ -887,14 +898,15 @@ public final class PrivateCellUtil {
    * @return null if there is no tag of the passed in tag type
    */
   public static Optional<Tag> getTag(Cell cell, byte type) {
-    boolean bufferBacked = cell instanceof ByteBufferCell;
+    boolean bufferBacked = cell instanceof ByteBufferExtendedCell;
     int length = cell.getTagsLength();
-    int offset = bufferBacked ? ((ByteBufferCell) cell).getTagsPosition() : cell.getTagsOffset();
+    int offset =
+      bufferBacked ? ((ByteBufferExtendedCell) cell).getTagsPosition() : cell.getTagsOffset();
     int pos = offset;
     while (pos < offset + length) {
       int tagLen;
       if (bufferBacked) {
-        ByteBuffer tagsBuffer = ((ByteBufferCell) cell).getTagsByteBuffer();
+        ByteBuffer tagsBuffer = ((ByteBufferExtendedCell) cell).getTagsByteBuffer();
         tagLen = ByteBufferUtils.readAsInt(tagsBuffer, pos, TAG_LENGTH_SIZE);
         if (ByteBufferUtils.toByte(tagsBuffer, pos + TAG_LENGTH_SIZE) == type) {
           return Optional.of(new ByteBufferTag(tagsBuffer, pos, tagLen + TAG_LENGTH_SIZE));
@@ -922,9 +934,9 @@ public final class PrivateCellUtil {
     if (tagsLength == 0) {
       return TagUtil.EMPTY_TAGS_ITR;
     }
-    if (cell instanceof ByteBufferCell) {
-      return tagsIterator(((ByteBufferCell) cell).getTagsByteBuffer(),
-        ((ByteBufferCell) cell).getTagsPosition(), tagsLength);
+    if (cell instanceof ByteBufferExtendedCell) {
+      return tagsIterator(((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getTagsPosition(), tagsLength);
     }
     return CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
   }
@@ -1000,10 +1012,11 @@ public final class PrivateCellUtil {
    */
   public static void writeRowSkippingBytes(DataOutputStream out, Cell cell, short rlength,
       int commonPrefix) throws IOException {
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyBufferToStream((DataOutput) out,
-        ((ByteBufferCell) cell).getRowByteBuffer(),
-        ((ByteBufferCell) cell).getRowPosition() + commonPrefix, rlength - commonPrefix);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils
+          .copyBufferToStream((DataOutput) out, ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+              ((ByteBufferExtendedCell) cell).getRowPosition() + commonPrefix,
+              rlength - commonPrefix);
     } else {
       out.write(cell.getRowArray(), cell.getRowOffset() + commonPrefix, rlength - commonPrefix);
     }
@@ -1028,23 +1041,27 @@ public final class PrivateCellUtil {
     int commonPrefix = KeyValue.ROW_LENGTH_SIZE;
     if (rLen1 != rLen2) {
       // early out when the RK length itself is not matching
-      return ByteBufferUtils.findCommonPrefix(Bytes.toBytes(rLen1), 0, KeyValue.ROW_LENGTH_SIZE,
-        Bytes.toBytes(rLen2), 0, KeyValue.ROW_LENGTH_SIZE);
+      return ByteBufferUtils
+          .findCommonPrefix(Bytes.toBytes(rLen1), 0, KeyValue.ROW_LENGTH_SIZE, Bytes.toBytes(rLen2),
+              0, KeyValue.ROW_LENGTH_SIZE);
     }
     // Compare the RKs
     int rkCommonPrefix = 0;
-    if (c1 instanceof ByteBufferCell && c2 instanceof ByteBufferCell) {
-      rkCommonPrefix = ByteBufferUtils.findCommonPrefix(((ByteBufferCell) c1).getRowByteBuffer(),
-        ((ByteBufferCell) c1).getRowPosition(), rLen1, ((ByteBufferCell) c2).getRowByteBuffer(),
-        ((ByteBufferCell) c2).getRowPosition(), rLen2);
+    if (c1 instanceof ByteBufferExtendedCell && c2 instanceof ByteBufferExtendedCell) {
+      rkCommonPrefix = ByteBufferUtils
+          .findCommonPrefix(((ByteBufferExtendedCell) c1).getRowByteBuffer(),
+              ((ByteBufferExtendedCell) c1).getRowPosition(), rLen1,
+              ((ByteBufferExtendedCell) c2).getRowByteBuffer(),
+              ((ByteBufferExtendedCell) c2).getRowPosition(), rLen2);
     } else {
       // There cannot be a case where one cell is BBCell and other is KeyValue. This flow comes
       // either
       // in flush or compactions. In flushes both cells are KV and in case of compaction it will be
       // either
       // KV or BBCell
-      rkCommonPrefix = ByteBufferUtils.findCommonPrefix(c1.getRowArray(), c1.getRowOffset(), rLen1,
-        c2.getRowArray(), c2.getRowOffset(), rLen2);
+      rkCommonPrefix = ByteBufferUtils
+          .findCommonPrefix(c1.getRowArray(), c1.getRowOffset(), rLen1, c2.getRowArray(),
+              c2.getRowOffset(), rLen2);
     }
     commonPrefix += rkCommonPrefix;
     if (rkCommonPrefix != rLen1) {
@@ -1067,14 +1084,16 @@ public final class PrivateCellUtil {
       commonPrefix += KeyValue.FAMILY_LENGTH_SIZE;
       // Compare the CF names
       int fCommonPrefix;
-      if (c1 instanceof ByteBufferCell && c2 instanceof ByteBufferCell) {
-        fCommonPrefix = ByteBufferUtils.findCommonPrefix(
-          ((ByteBufferCell) c1).getFamilyByteBuffer(), ((ByteBufferCell) c1).getFamilyPosition(),
-          fLen1, ((ByteBufferCell) c2).getFamilyByteBuffer(),
-          ((ByteBufferCell) c2).getFamilyPosition(), fLen2);
+      if (c1 instanceof ByteBufferExtendedCell && c2 instanceof ByteBufferExtendedCell) {
+        fCommonPrefix = ByteBufferUtils
+            .findCommonPrefix(((ByteBufferExtendedCell) c1).getFamilyByteBuffer(),
+                ((ByteBufferExtendedCell) c1).getFamilyPosition(), fLen1,
+                ((ByteBufferExtendedCell) c2).getFamilyByteBuffer(),
+                ((ByteBufferExtendedCell) c2).getFamilyPosition(), fLen2);
       } else {
-        fCommonPrefix = ByteBufferUtils.findCommonPrefix(c1.getFamilyArray(), c1.getFamilyOffset(),
-          fLen1, c2.getFamilyArray(), c2.getFamilyOffset(), fLen2);
+        fCommonPrefix = ByteBufferUtils
+            .findCommonPrefix(c1.getFamilyArray(), c1.getFamilyOffset(), fLen1, c2.getFamilyArray(),
+                c2.getFamilyOffset(), fLen2);
       }
       commonPrefix += fCommonPrefix;
       if (fCommonPrefix != fLen1) {
@@ -1085,22 +1104,25 @@ public final class PrivateCellUtil {
     int qLen1 = c1.getQualifierLength();
     int qLen2 = c2.getQualifierLength();
     int qCommon;
-    if (c1 instanceof ByteBufferCell && c2 instanceof ByteBufferCell) {
-      qCommon = ByteBufferUtils.findCommonPrefix(((ByteBufferCell) c1).getQualifierByteBuffer(),
-        ((ByteBufferCell) c1).getQualifierPosition(), qLen1,
-        ((ByteBufferCell) c2).getQualifierByteBuffer(),
-        ((ByteBufferCell) c2).getQualifierPosition(), qLen2);
+    if (c1 instanceof ByteBufferExtendedCell && c2 instanceof ByteBufferExtendedCell) {
+      qCommon = ByteBufferUtils
+          .findCommonPrefix(((ByteBufferExtendedCell) c1).getQualifierByteBuffer(),
+              ((ByteBufferExtendedCell) c1).getQualifierPosition(), qLen1,
+              ((ByteBufferExtendedCell) c2).getQualifierByteBuffer(),
+              ((ByteBufferExtendedCell) c2).getQualifierPosition(), qLen2);
     } else {
-      qCommon = ByteBufferUtils.findCommonPrefix(c1.getQualifierArray(), c1.getQualifierOffset(),
-        qLen1, c2.getQualifierArray(), c2.getQualifierOffset(), qLen2);
+      qCommon = ByteBufferUtils
+          .findCommonPrefix(c1.getQualifierArray(), c1.getQualifierOffset(), qLen1,
+              c2.getQualifierArray(), c2.getQualifierOffset(), qLen2);
     }
     commonPrefix += qCommon;
     if (!withTsType || Math.max(qLen1, qLen2) != qCommon) {
       return commonPrefix;
     }
     // Compare the timestamp parts
-    int tsCommonPrefix = ByteBufferUtils.findCommonPrefix(Bytes.toBytes(c1.getTimestamp()), 0,
-      KeyValue.TIMESTAMP_SIZE, Bytes.toBytes(c2.getTimestamp()), 0, KeyValue.TIMESTAMP_SIZE);
+    int tsCommonPrefix = ByteBufferUtils
+        .findCommonPrefix(Bytes.toBytes(c1.getTimestamp()), 0, KeyValue.TIMESTAMP_SIZE,
+            Bytes.toBytes(c2.getTimestamp()), 0, KeyValue.TIMESTAMP_SIZE);
     commonPrefix += tsCommonPrefix;
     if (tsCommonPrefix != KeyValue.TIMESTAMP_SIZE) {
       return commonPrefix;
@@ -1195,9 +1217,9 @@ public final class PrivateCellUtil {
    * @return result comparing cell's row
    */
   public static int compareRow(Cell cell, ByteArrayComparable comparator) {
-    if (cell instanceof ByteBufferCell) {
-      return comparator.compareTo(((ByteBufferCell) cell).getRowByteBuffer(),
-        ((ByteBufferCell) cell).getRowPosition(), cell.getRowLength());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return comparator.compareTo(((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength());
     }
     return comparator.compareTo(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
   }
@@ -1209,9 +1231,9 @@ public final class PrivateCellUtil {
    * @return result comparing cell's column family
    */
   public static int compareFamily(Cell cell, ByteArrayComparable comparator) {
-    if (cell instanceof ByteBufferCell) {
-      return comparator.compareTo(((ByteBufferCell) cell).getFamilyByteBuffer(),
-        ((ByteBufferCell) cell).getFamilyPosition(), cell.getFamilyLength());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return comparator.compareTo(((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getFamilyPosition(), cell.getFamilyLength());
     }
     return comparator.compareTo(cell.getFamilyArray(), cell.getFamilyOffset(),
       cell.getFamilyLength());
@@ -1224,9 +1246,9 @@ public final class PrivateCellUtil {
    * @return result comparing cell's qualifier
    */
   public static int compareQualifier(Cell cell, ByteArrayComparable comparator) {
-    if (cell instanceof ByteBufferCell) {
-      return comparator.compareTo(((ByteBufferCell) cell).getQualifierByteBuffer(),
-        ((ByteBufferCell) cell).getQualifierPosition(), cell.getQualifierLength());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return comparator.compareTo(((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getQualifierPosition(), cell.getQualifierLength());
     }
     return comparator.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(),
       cell.getQualifierLength());
@@ -1262,9 +1284,9 @@ public final class PrivateCellUtil {
    * @return result comparing cell's value
    */
   public static int compareValue(Cell cell, ByteArrayComparable comparator) {
-    if (cell instanceof ByteBufferCell) {
-      return comparator.compareTo(((ByteBufferCell) cell).getValueByteBuffer(),
-        ((ByteBufferCell) cell).getValuePosition(), cell.getValueLength());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return comparator.compareTo(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getValuePosition(), cell.getValueLength());
     }
     return comparator.compareTo(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
   }
@@ -1375,8 +1397,7 @@ public final class PrivateCellUtil {
    * These cells are used in reseeks/seeks to improve the read performance. They are not real cells
    * that are returned back to the clients
    */
-  private static abstract class EmptyByteBufferCell extends ByteBufferCell
-      implements ExtendedCell {
+  private static abstract class EmptyByteBufferExtendedCell extends ByteBufferExtendedCell {
 
     @Override
     public void setSequenceId(long seqId) {
@@ -1578,7 +1599,7 @@ public final class PrivateCellUtil {
     }
   }
 
-  private static class FirstOnRowByteBufferCell extends EmptyByteBufferCell {
+  private static class FirstOnRowByteBufferExtendedCell extends EmptyByteBufferExtendedCell {
     private static final int FIXED_OVERHEAD =
         ClassSize.OBJECT // object
         + ClassSize.REFERENCE // row buffer
@@ -1588,7 +1609,7 @@ public final class PrivateCellUtil {
     private final int roffset;
     private final short rlength;
 
-    public FirstOnRowByteBufferCell(final ByteBuffer row, int roffset, short rlength) {
+    public FirstOnRowByteBufferExtendedCell(final ByteBuffer row, int roffset, short rlength) {
       this.rowBuff = row;
       this.roffset = roffset;
       this.rlength = rlength;
@@ -1633,7 +1654,7 @@ public final class PrivateCellUtil {
     }
   }
 
-  private static class LastOnRowByteBufferCell extends EmptyByteBufferCell {
+  private static class LastOnRowByteBufferExtendedCell extends EmptyByteBufferExtendedCell {
     private static final int FIXED_OVERHEAD =
         ClassSize.OBJECT // object
       + ClassSize.REFERENCE // rowBuff
@@ -1643,7 +1664,7 @@ public final class PrivateCellUtil {
     private final int roffset;
     private final short rlength;
 
-    public LastOnRowByteBufferCell(final ByteBuffer row, int roffset, short rlength) {
+    public LastOnRowByteBufferExtendedCell(final ByteBuffer row, int roffset, short rlength) {
       this.rowBuff = row;
       this.roffset = roffset;
       this.rlength = rlength;
@@ -1688,9 +1709,10 @@ public final class PrivateCellUtil {
     }
   }
 
-  private static class FirstOnRowColByteBufferCell extends FirstOnRowByteBufferCell {
+  private static class FirstOnRowColByteBufferExtendedCell
+      extends FirstOnRowByteBufferExtendedCell {
     private static final int FIXED_OVERHEAD =
-        FirstOnRowByteBufferCell.FIXED_OVERHEAD
+        FirstOnRowByteBufferExtendedCell.FIXED_OVERHEAD
         + ClassSize.REFERENCE * 2 // family buffer and column buffer
         + Bytes.SIZEOF_INT * 3 // famOffset, colOffset, colLength
         + Bytes.SIZEOF_BYTE; // famLength
@@ -1701,7 +1723,7 @@ public final class PrivateCellUtil {
     private final int colOffset;
     private final int colLength;
 
-    public FirstOnRowColByteBufferCell(final ByteBuffer row, int roffset, short rlength,
+    public FirstOnRowColByteBufferExtendedCell(final ByteBuffer row, int roffset, short rlength,
         final ByteBuffer famBuff, final int famOffset, final byte famLength, final ByteBuffer col,
         final int colOffset, final int colLength) {
       super(row, roffset, rlength);
@@ -1843,13 +1865,14 @@ public final class PrivateCellUtil {
     }
   }
 
-  private static class FirstOnRowColTSByteBufferCell extends FirstOnRowColByteBufferCell {
+  private static class FirstOnRowColTSByteBufferExtendedCell
+      extends FirstOnRowColByteBufferExtendedCell {
     private static final int FIXED_OVERHEAD =
-        FirstOnRowColByteBufferCell.FIXED_OVERHEAD
+        FirstOnRowColByteBufferExtendedCell.FIXED_OVERHEAD
             + Bytes.SIZEOF_LONG; // ts
     private long ts;
 
-    public FirstOnRowColTSByteBufferCell(ByteBuffer rBuffer, int roffset, short rlength,
+    public FirstOnRowColTSByteBufferExtendedCell(ByteBuffer rBuffer, int roffset, short rlength,
         ByteBuffer fBuffer, int foffset, byte flength, ByteBuffer qBuffer, int qoffset, int qlength,
         long ts) {
       super(rBuffer, roffset, rlength, fBuffer, foffset, flength, qBuffer, qoffset, qlength);
@@ -1983,9 +2006,9 @@ public final class PrivateCellUtil {
     }
   }
 
-  private static class LastOnRowColByteBufferCell extends LastOnRowByteBufferCell {
+  private static class LastOnRowColByteBufferExtendedCell extends LastOnRowByteBufferExtendedCell {
     private static final int FIXED_OVERHEAD =
-        LastOnRowByteBufferCell.FIXED_OVERHEAD
+        LastOnRowByteBufferExtendedCell.FIXED_OVERHEAD
             + ClassSize.REFERENCE * 2 // fBuffer and qBuffer
             + Bytes.SIZEOF_INT * 3 // foffset, qoffset, qlength
             + Bytes.SIZEOF_BYTE; // flength
@@ -1996,7 +2019,7 @@ public final class PrivateCellUtil {
     private final int qoffset;
     private final int qlength;
 
-    public LastOnRowColByteBufferCell(ByteBuffer rBuffer, int roffset, short rlength,
+    public LastOnRowColByteBufferExtendedCell(ByteBuffer rBuffer, int roffset, short rlength,
         ByteBuffer fBuffer, int foffset, byte flength, ByteBuffer qBuffer, int qoffset,
         int qlength) {
       super(rBuffer, roffset, rlength);
@@ -2125,15 +2148,16 @@ public final class PrivateCellUtil {
     int qLen = cell.getQualifierLength();
     // Using just one if/else loop instead of every time checking before writing every
     // component of cell
-    if (cell instanceof ByteBufferCell) {
+    if (cell instanceof ByteBufferExtendedCell) {
       out.writeShort(rowLen);
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getRowByteBuffer(),
-        ((ByteBufferCell) cell).getRowPosition(), rowLen);
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getRowPosition(), rowLen);
       out.writeByte(fLen);
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getFamilyByteBuffer(),
-        ((ByteBufferCell) cell).getFamilyPosition(), fLen);
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getQualifierByteBuffer(),
-        ((ByteBufferCell) cell).getQualifierPosition(), qLen);
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getFamilyPosition(), fLen);
+      ByteBufferUtils
+        .copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getQualifierPosition(), qLen);
     } else {
       out.writeShort(rowLen);
       out.write(cell.getRowArray(), cell.getRowOffset(), rowLen);
@@ -2214,15 +2238,16 @@ public final class PrivateCellUtil {
     int qLen = cell.getQualifierLength();
     // Using just one if/else loop instead of every time checking before writing every
     // component of cell
-    if (cell instanceof ByteBufferCell) {
+    if (cell instanceof ByteBufferExtendedCell) {
       StreamUtils.writeShort(out, rowLen);
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getRowByteBuffer(),
-        ((ByteBufferCell) cell).getRowPosition(), rowLen);
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getRowPosition(), rowLen);
       out.write(fLen);
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getFamilyByteBuffer(),
-        ((ByteBufferCell) cell).getFamilyPosition(), fLen);
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getQualifierByteBuffer(),
-        ((ByteBufferCell) cell).getQualifierPosition(), qLen);
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getFamilyPosition(), fLen);
+      ByteBufferUtils
+        .copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getQualifierPosition(), qLen);
     } else {
       StreamUtils.writeShort(out, rowLen);
       out.write(cell.getRowArray(), cell.getRowOffset(), rowLen);
@@ -2233,7 +2258,7 @@ public final class PrivateCellUtil {
     StreamUtils.writeLong(out, cell.getTimestamp());
     out.write(cell.getTypeByte());
     return Bytes.SIZEOF_SHORT + rowLen + Bytes.SIZEOF_BYTE + fLen + qLen + Bytes.SIZEOF_LONG
-        + Bytes.SIZEOF_BYTE;
+      + Bytes.SIZEOF_BYTE;
   }
 
   /**
@@ -2322,9 +2347,9 @@ public final class PrivateCellUtil {
    * @throws IOException
    */
   public static void writeRow(OutputStream out, Cell cell, short rlength) throws IOException {
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getRowByteBuffer(),
-        ((ByteBufferCell) cell).getRowPosition(), rlength);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getRowPosition(), rlength);
     } else {
       out.write(cell.getRowArray(), cell.getRowOffset(), rlength);
     }
@@ -2338,9 +2363,9 @@ public final class PrivateCellUtil {
    * @throws IOException
    */
   public static void writeFamily(OutputStream out, Cell cell, byte flength) throws IOException {
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getFamilyByteBuffer(),
-        ((ByteBufferCell) cell).getFamilyPosition(), flength);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getFamilyPosition(), flength);
     } else {
       out.write(cell.getFamilyArray(), cell.getFamilyOffset(), flength);
     }
@@ -2354,9 +2379,10 @@ public final class PrivateCellUtil {
    * @throws IOException
    */
   public static void writeQualifier(OutputStream out, Cell cell, int qlength) throws IOException {
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getQualifierByteBuffer(),
-        ((ByteBufferCell) cell).getQualifierPosition(), qlength);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils
+        .copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getQualifierPosition(), qlength);
     } else {
       out.write(cell.getQualifierArray(), cell.getQualifierOffset(), qlength);
     }
@@ -2371,13 +2397,14 @@ public final class PrivateCellUtil {
    */
   public static void writeQualifierSkippingBytes(DataOutputStream out, Cell cell, int qlength,
       int commonPrefix) throws IOException {
-    if (cell instanceof ByteBufferCell) {
+    if (cell instanceof ByteBufferExtendedCell) {
       ByteBufferUtils.copyBufferToStream((DataOutput) out,
-        ((ByteBufferCell) cell).getQualifierByteBuffer(),
-        ((ByteBufferCell) cell).getQualifierPosition() + commonPrefix, qlength - commonPrefix);
+          ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getQualifierPosition() + commonPrefix,
+          qlength - commonPrefix);
     } else {
       out.write(cell.getQualifierArray(), cell.getQualifierOffset() + commonPrefix,
-        qlength - commonPrefix);
+          qlength - commonPrefix);
     }
   }
 
@@ -2389,9 +2416,9 @@ public final class PrivateCellUtil {
    * @throws IOException
    */
   public static void writeValue(OutputStream out, Cell cell, int vlength) throws IOException {
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getValueByteBuffer(),
-        ((ByteBufferCell) cell).getValuePosition(), vlength);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getValueByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getValuePosition(), vlength);
     } else {
       out.write(cell.getValueArray(), cell.getValueOffset(), vlength);
     }
@@ -2405,9 +2432,9 @@ public final class PrivateCellUtil {
    * @throws IOException
    */
   public static void writeTags(OutputStream out, Cell cell, int tagsLength) throws IOException {
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getTagsByteBuffer(),
-        ((ByteBufferCell) cell).getTagsPosition(), tagsLength);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils.copyBufferToStream(out, ((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getTagsPosition(), tagsLength);
     } else {
       out.write(cell.getTagsArray(), cell.getTagsOffset(), tagsLength);
     }
@@ -2440,9 +2467,9 @@ public final class PrivateCellUtil {
    * @return rowkey as int
    */
   public static int getRowAsInt(Cell cell) {
-    if (cell instanceof ByteBufferCell) {
-      return ByteBufferUtils.toInt(((ByteBufferCell) cell).getRowByteBuffer(),
-        ((ByteBufferCell) cell).getRowPosition());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.toInt(((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getRowPosition());
     }
     return Bytes.toInt(cell.getRowArray(), cell.getRowOffset());
   }
@@ -2453,9 +2480,9 @@ public final class PrivateCellUtil {
    * @return value as long
    */
   public static long getValueAsLong(Cell cell) {
-    if (cell instanceof ByteBufferCell) {
-      return ByteBufferUtils.toLong(((ByteBufferCell) cell).getValueByteBuffer(),
-        ((ByteBufferCell) cell).getValuePosition());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.toLong(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getValuePosition());
     }
     return Bytes.toLong(cell.getValueArray(), cell.getValueOffset());
   }
@@ -2466,9 +2493,9 @@ public final class PrivateCellUtil {
    * @return value as int
    */
   public static int getValueAsInt(Cell cell) {
-    if (cell instanceof ByteBufferCell) {
-      return ByteBufferUtils.toInt(((ByteBufferCell) cell).getValueByteBuffer(),
-        ((ByteBufferCell) cell).getValuePosition());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.toInt(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getValuePosition());
     }
     return Bytes.toInt(cell.getValueArray(), cell.getValueOffset());
   }
@@ -2479,9 +2506,9 @@ public final class PrivateCellUtil {
    * @return value as double
    */
   public static double getValueAsDouble(Cell cell) {
-    if (cell instanceof ByteBufferCell) {
-      return ByteBufferUtils.toDouble(((ByteBufferCell) cell).getValueByteBuffer(),
-        ((ByteBufferCell) cell).getValuePosition());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.toDouble(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getValuePosition());
     }
     return Bytes.toDouble(cell.getValueArray(), cell.getValueOffset());
   }
@@ -2492,9 +2519,9 @@ public final class PrivateCellUtil {
    * @return value as BigDecimal
    */
   public static BigDecimal getValueAsBigDecimal(Cell cell) {
-    if (cell instanceof ByteBufferCell) {
-      return ByteBufferUtils.toBigDecimal(((ByteBufferCell) cell).getValueByteBuffer(),
-        ((ByteBufferCell) cell).getValuePosition(), cell.getValueLength());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ByteBufferUtils.toBigDecimal(((ByteBufferExtendedCell) cell).getValueByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getValuePosition(), cell.getValueLength());
     }
     return Bytes.toBigDecimal(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
   }
@@ -2508,9 +2535,9 @@ public final class PrivateCellUtil {
    */
   public static void compressTags(OutputStream out, Cell cell,
       TagCompressionContext tagCompressionContext) throws IOException {
-    if (cell instanceof ByteBufferCell) {
-      tagCompressionContext.compressTags(out, ((ByteBufferCell) cell).getTagsByteBuffer(),
-        ((ByteBufferCell) cell).getTagsPosition(), cell.getTagsLength());
+    if (cell instanceof ByteBufferExtendedCell) {
+      tagCompressionContext.compressTags(out, ((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getTagsPosition(), cell.getTagsLength());
     } else {
       tagCompressionContext.compressTags(out, cell.getTagsArray(), cell.getTagsOffset(),
         cell.getTagsLength());
@@ -2518,9 +2545,9 @@ public final class PrivateCellUtil {
   }
 
   public static void compressRow(OutputStream out, Cell cell, Dictionary dict) throws IOException {
-    if (cell instanceof ByteBufferCell) {
-      Dictionary.write(out, ((ByteBufferCell) cell).getRowByteBuffer(),
-        ((ByteBufferCell) cell).getRowPosition(), cell.getRowLength(), dict);
+    if (cell instanceof ByteBufferExtendedCell) {
+      Dictionary.write(out, ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(), dict);
     } else {
       Dictionary.write(out, cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), dict);
     }
@@ -2528,9 +2555,9 @@ public final class PrivateCellUtil {
 
   public static void compressFamily(OutputStream out, Cell cell, Dictionary dict)
       throws IOException {
-    if (cell instanceof ByteBufferCell) {
-      Dictionary.write(out, ((ByteBufferCell) cell).getFamilyByteBuffer(),
-        ((ByteBufferCell) cell).getFamilyPosition(), cell.getFamilyLength(), dict);
+    if (cell instanceof ByteBufferExtendedCell) {
+      Dictionary.write(out, ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getFamilyPosition(), cell.getFamilyLength(), dict);
     } else {
       Dictionary.write(out, cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
         dict);
@@ -2539,9 +2566,9 @@ public final class PrivateCellUtil {
 
   public static void compressQualifier(OutputStream out, Cell cell, Dictionary dict)
       throws IOException {
-    if (cell instanceof ByteBufferCell) {
-      Dictionary.write(out, ((ByteBufferCell) cell).getQualifierByteBuffer(),
-        ((ByteBufferCell) cell).getQualifierPosition(), cell.getQualifierLength(), dict);
+    if (cell instanceof ByteBufferExtendedCell) {
+      Dictionary.write(out, ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
+        ((ByteBufferExtendedCell) cell).getQualifierPosition(), cell.getQualifierLength(), dict);
     } else {
       Dictionary.write(out, cell.getQualifierArray(), cell.getQualifierOffset(),
         cell.getQualifierLength(), dict);
@@ -2668,13 +2695,14 @@ public final class PrivateCellUtil {
   }
 
   static Cell createNextOnRowCol(Cell cell, long ts, byte type) {
-    if (cell instanceof ByteBufferCell) {
-      return new LastOnRowColByteBufferCell(
-          ((ByteBufferCell) cell).getRowByteBuffer(), ((ByteBufferCell) cell).getRowPosition(),
-          cell.getRowLength(), ((ByteBufferCell) cell).getFamilyByteBuffer(),
-          ((ByteBufferCell) cell).getFamilyPosition(), cell.getFamilyLength(),
-          ((ByteBufferCell) cell).getQualifierByteBuffer(),
-          ((ByteBufferCell) cell).getQualifierPosition(), cell.getQualifierLength()) {
+    if (cell instanceof ByteBufferExtendedCell) {
+      return new LastOnRowColByteBufferExtendedCell(
+          ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(),
+          ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getFamilyPosition(), cell.getFamilyLength(),
+          ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getQualifierPosition(), cell.getQualifierLength()) {
         @Override
         public long getTimestamp() {
           return ts;
@@ -2686,8 +2714,8 @@ public final class PrivateCellUtil {
         }
       };
     }
-    return new LastOnRowColCell(cell.getRowArray(), cell.getRowOffset(),
-        cell.getRowLength(), cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
+    return new LastOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
+        cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
         cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()) {
       @Override
       public long getTimestamp() {
@@ -2788,13 +2816,12 @@ public final class PrivateCellUtil {
    * @return First possible Cell on passed Cell's row.
    */
   public static Cell createFirstOnRow(final Cell cell) {
-    if (cell instanceof ByteBufferCell) {
-      return new FirstOnRowByteBufferCell(
-          ((ByteBufferCell) cell).getRowByteBuffer(), ((ByteBufferCell) cell).getRowPosition(),
-          cell.getRowLength());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return new FirstOnRowByteBufferExtendedCell(
+          ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength());
     }
-    return new FirstOnRowCell(cell.getRowArray(), cell.getRowOffset(),
-        cell.getRowLength());
+    return new FirstOnRowCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
   }
 
   public static Cell createFirstOnRow(final byte[] row, int roffset, short rlength) {
@@ -2803,7 +2830,7 @@ public final class PrivateCellUtil {
 
   public static Cell createFirstOnRow(final byte[] row, final byte[] family, final byte[] col) {
     return createFirstOnRow(row, 0, (short) row.length, family, 0, (byte) family.length, col, 0,
-      col.length);
+        col.length);
   }
 
   public static Cell createFirstOnRow(final byte[] row, int roffset, short rlength,
@@ -2817,23 +2844,24 @@ public final class PrivateCellUtil {
   }
 
   public static Cell createFirstOnRowFamily(Cell cell, byte[] fArray, int foff, int flen) {
-    if (cell instanceof ByteBufferCell) {
-      return new FirstOnRowColByteBufferCell(
-          ((ByteBufferCell) cell).getRowByteBuffer(), ((ByteBufferCell) cell).getRowPosition(),
-          cell.getRowLength(), ByteBuffer.wrap(fArray), foff, (byte) flen,
-          HConstants.EMPTY_BYTE_BUFFER, 0, 0);
+    if (cell instanceof ByteBufferExtendedCell) {
+      return new FirstOnRowColByteBufferExtendedCell(
+          ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(),
+          ByteBuffer.wrap(fArray), foff, (byte) flen, HConstants.EMPTY_BYTE_BUFFER, 0, 0);
     }
     return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
         fArray, foff, (byte) flen, HConstants.EMPTY_BYTE_ARRAY, 0, 0);
   }
 
   public static Cell createFirstOnRowCol(final Cell cell) {
-    if (cell instanceof ByteBufferCell) {
-      return new FirstOnRowColByteBufferCell(
-          ((ByteBufferCell) cell).getRowByteBuffer(), ((ByteBufferCell) cell).getRowPosition(),
-          cell.getRowLength(), HConstants.EMPTY_BYTE_BUFFER, 0, (byte) 0,
-          ((ByteBufferCell) cell).getQualifierByteBuffer(),
-          ((ByteBufferCell) cell).getQualifierPosition(), cell.getQualifierLength());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return new FirstOnRowColByteBufferExtendedCell(
+          ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(),
+          HConstants.EMPTY_BYTE_BUFFER, 0, (byte) 0,
+          ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getQualifierPosition(), cell.getQualifierLength());
     }
     return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
         HConstants.EMPTY_BYTE_ARRAY, 0, (byte) 0, cell.getQualifierArray(),
@@ -2857,16 +2885,17 @@ public final class PrivateCellUtil {
    * @return Last possible Cell on passed Cell's rk:cf and passed qualifier.
    */
   public static Cell createFirstOnRowCol(final Cell cell, byte[] qArray, int qoffest, int qlength) {
-    if (cell instanceof ByteBufferCell) {
-      return new FirstOnRowColByteBufferCell(
-          ((ByteBufferCell) cell).getRowByteBuffer(), ((ByteBufferCell) cell).getRowPosition(),
-          cell.getRowLength(), ((ByteBufferCell) cell).getFamilyByteBuffer(),
-          ((ByteBufferCell) cell).getFamilyPosition(), cell.getFamilyLength(),
+    if (cell instanceof ByteBufferExtendedCell) {
+      return new FirstOnRowColByteBufferExtendedCell(
+          ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(),
+          ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getFamilyPosition(), cell.getFamilyLength(),
           ByteBuffer.wrap(qArray), qoffest, qlength);
     }
-    return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(),
-        cell.getRowLength(), cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
-        qArray, qoffest, qlength);
+    return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
+        cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(), qArray, qoffest,
+        qlength);
   }
 
   /**
@@ -2877,13 +2906,14 @@ public final class PrivateCellUtil {
    * @param ts
    */
   public static Cell createFirstOnRowColTS(Cell cell, long ts) {
-    if (cell instanceof ByteBufferCell) {
-      return new FirstOnRowColTSByteBufferCell(((ByteBufferCell) cell).getRowByteBuffer(),
-          ((ByteBufferCell) cell).getRowPosition(), cell.getRowLength(),
-          ((ByteBufferCell) cell).getFamilyByteBuffer(),
-          ((ByteBufferCell) cell).getFamilyPosition(), cell.getFamilyLength(),
-          ((ByteBufferCell) cell).getQualifierByteBuffer(),
-          ((ByteBufferCell) cell).getQualifierPosition(), cell.getQualifierLength(), ts);
+    if (cell instanceof ByteBufferExtendedCell) {
+      return new FirstOnRowColTSByteBufferExtendedCell(
+          ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(),
+          ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getFamilyPosition(), cell.getFamilyLength(),
+          ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getQualifierPosition(), cell.getQualifierLength(), ts);
     }
     return new FirstOnRowColTSCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
         cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
@@ -2896,10 +2926,9 @@ public final class PrivateCellUtil {
    * @return Last possible Cell on passed Cell's row.
    */
   public static Cell createLastOnRow(final Cell cell) {
-    if (cell instanceof ByteBufferCell) {
-      return new LastOnRowByteBufferCell(
-          ((ByteBufferCell) cell).getRowByteBuffer(), ((ByteBufferCell) cell).getRowPosition(),
-          cell.getRowLength());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return new LastOnRowByteBufferExtendedCell(((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength());
     }
     return new LastOnRowCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
   }
@@ -2916,13 +2945,14 @@ public final class PrivateCellUtil {
    * @return Last possible Cell on passed Cell's rk:cf:q.
    */
   public static Cell createLastOnRowCol(final Cell cell) {
-    if (cell instanceof ByteBufferCell) {
-      return new LastOnRowColByteBufferCell(((ByteBufferCell) cell).getRowByteBuffer(),
-          ((ByteBufferCell) cell).getRowPosition(), cell.getRowLength(),
-          ((ByteBufferCell) cell).getFamilyByteBuffer(),
-          ((ByteBufferCell) cell).getFamilyPosition(), cell.getFamilyLength(),
-          ((ByteBufferCell) cell).getQualifierByteBuffer(),
-          ((ByteBufferCell) cell).getQualifierPosition(), cell.getQualifierLength());
+    if (cell instanceof ByteBufferExtendedCell) {
+      return new LastOnRowColByteBufferExtendedCell(
+          ((ByteBufferExtendedCell) cell).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getRowPosition(), cell.getRowLength(),
+          ((ByteBufferExtendedCell) cell).getFamilyByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getFamilyPosition(), cell.getFamilyLength(),
+          ((ByteBufferExtendedCell) cell).getQualifierByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getQualifierPosition(), cell.getQualifierLength());
     }
     return new LastOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
         cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java
index 34c78a5..9527173 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java
@@ -99,9 +99,9 @@ public final class TagUtil {
     }
     byte[] b = new byte[tags.length + cellTagsLen];
     int pos = Bytes.putBytes(b, 0, tags, 0, tags.length);
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyFromBufferToArray(b, ((ByteBufferCell) cell).getTagsByteBuffer(),
-          ((ByteBufferCell) cell).getTagsPosition(), pos, cellTagsLen);
+    if (cell instanceof ByteBufferExtendedCell) {
+      ByteBufferUtils.copyFromBufferToArray(b, ((ByteBufferExtendedCell) cell).getTagsByteBuffer(),
+          ((ByteBufferExtendedCell) cell).getTagsPosition(), pos, cellTagsLen);
     } else {
       Bytes.putBytes(b, pos, cell.getTagsArray(), cell.getTagsOffset(), cellTagsLen);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
index 9bcda01..7fe2b71 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/BufferedDataBlockEncoder.java
@@ -21,8 +21,7 @@ import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
-
-import org.apache.hadoop.hbase.ByteBufferCell;
+import org.apache.hadoop.hbase.ByteBufferExtendedCell;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
@@ -252,7 +251,7 @@ abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder {
 
     private Cell toOffheapCell(ByteBuffer valAndTagsBuffer, int vOffset,
         int tagsLenSerializationSize) {
-      ByteBuffer tagsBuf =  HConstants.EMPTY_BYTE_BUFFER;
+      ByteBuffer tagsBuf = HConstants.EMPTY_BYTE_BUFFER;
       int tOffset = 0;
       if (this.includeTags) {
         if (this.tagCompressionContext == null) {
@@ -263,8 +262,9 @@ abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder {
           tOffset = 0;
         }
       }
-      return new OffheapDecodedCell(ByteBuffer.wrap(Bytes.copy(keyBuffer, 0, this.keyLength)),
-          currentKey.getRowLength(), currentKey.getFamilyOffset(), currentKey.getFamilyLength(),
+      return new OffheapDecodedExtendedCell(
+          ByteBuffer.wrap(Bytes.copy(keyBuffer, 0, this.keyLength)), currentKey.getRowLength(),
+          currentKey.getFamilyOffset(), currentKey.getFamilyLength(),
           currentKey.getQualifierOffset(), currentKey.getQualifierLength(),
           currentKey.getTimestamp(), currentKey.getTypeByte(), valAndTagsBuffer, vOffset,
           this.valueLength, memstoreTS, tagsBuf, tOffset, this.tagsLength);
@@ -477,7 +477,7 @@ abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder {
     }
   }
 
-  protected static class OffheapDecodedCell extends ByteBufferCell implements ExtendedCell {
+  protected static class OffheapDecodedExtendedCell extends ByteBufferExtendedCell {
     private static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT
         + (3 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_LONG) + (7 * Bytes.SIZEOF_INT)
         + (Bytes.SIZEOF_SHORT) + (2 * Bytes.SIZEOF_BYTE) + (3 * ClassSize.BYTE_BUFFER));
@@ -497,7 +497,7 @@ abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder {
     private int tagsLength;
     private long seqId;
 
-    protected OffheapDecodedCell(ByteBuffer keyBuffer, short rowLength, int familyOffset,
+    protected OffheapDecodedExtendedCell(ByteBuffer keyBuffer, short rowLength, int familyOffset,
         byte familyLength, int qualOffset, int qualLength, long timeStamp, byte typeByte,
         ByteBuffer valueBuffer, int valueOffset, int valueLen, long seqId, ByteBuffer tagsBuffer,
         int tagsOffset, int tagsLength) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
index 02ed721..1dd4e19 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/RowIndexSeekerV1.java
@@ -18,7 +18,7 @@ package org.apache.hadoop.hbase.io.encoding;
 
 import java.nio.ByteBuffer;
 
-import org.apache.hadoop.hbase.ByteBufferCell;
+import org.apache.hadoop.hbase.ByteBufferExtendedCell;
 import org.apache.hadoop.hbase.ByteBufferKeyOnlyKeyValue;
 import org.apache.hadoop.hbase.ByteBufferKeyValue;
 import org.apache.hadoop.hbase.Cell;
@@ -154,10 +154,10 @@ public class RowIndexSeekerV1 extends AbstractEncodedSeeker {
   }
 
   private int compareRows(ByteBuffer row, Cell seekCell) {
-    if (seekCell instanceof ByteBufferCell) {
+    if (seekCell instanceof ByteBufferExtendedCell) {
       return ByteBufferUtils.compareTo(row, row.position(), row.remaining(),
-          ((ByteBufferCell) seekCell).getRowByteBuffer(),
-          ((ByteBufferCell) seekCell).getRowPosition(),
+          ((ByteBufferExtendedCell) seekCell).getRowByteBuffer(),
+          ((ByteBufferExtendedCell) seekCell).getRowPosition(),
           seekCell.getRowLength());
     } else {
       return ByteBufferUtils.compareTo(row, row.position(), row.remaining(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/test/java/org/apache/hadoop/hbase/TestByteBufferKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestByteBufferKeyValue.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestByteBufferKeyValue.java
index 030bc8f..97a4276 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestByteBufferKeyValue.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestByteBufferKeyValue.java
@@ -56,7 +56,7 @@ public class TestByteBufferKeyValue {
     KeyValue kvCell = new KeyValue(row1, fam1, qual1, 0L, Type.Put, row1);
     ByteBuffer buf = ByteBuffer.allocateDirect(kvCell.getBuffer().length);
     ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), 0, kvCell.getBuffer().length);
-    ByteBufferCell offheapKV = new ByteBufferKeyValue(buf, 0, buf.capacity(), 0L);
+    ByteBufferExtendedCell offheapKV = new ByteBufferKeyValue(buf, 0, buf.capacity(), 0L);
     assertEquals(
       ROW1,
       ByteBufferUtils.toStringBinary(offheapKV.getRowByteBuffer(),
@@ -179,7 +179,7 @@ public class TestByteBufferKeyValue {
     ByteBuffer buf = ByteBuffer.allocateDirect(kvCell.getKeyLength());
     ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), kvCell.getKeyOffset(),
       kvCell.getKeyLength());
-    ByteBufferCell offheapKeyOnlyKV = new ByteBufferKeyOnlyKeyValue(buf, 0, buf.capacity());
+    ByteBufferExtendedCell offheapKeyOnlyKV = new ByteBufferKeyOnlyKeyValue(buf, 0, buf.capacity());
     assertEquals(
       ROW1,
       ByteBufferUtils.toStringBinary(offheapKeyOnlyKV.getRowByteBuffer(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java
index 5f4115e..96181e1 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/io/TestTagCompressionContext.java
@@ -27,7 +27,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.hbase.ArrayBackedTag;
-import org.apache.hadoop.hbase.ByteBufferCell;
+import org.apache.hadoop.hbase.ByteBufferExtendedCell;
 import org.apache.hadoop.hbase.ByteBufferKeyValue;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
@@ -80,10 +80,10 @@ public class TestTagCompressionContext {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream daos = new ByteBufferWriterDataOutputStream(baos);
     TagCompressionContext context = new TagCompressionContext(LRUDictionary.class, Byte.MAX_VALUE);
-    ByteBufferCell kv1 = (ByteBufferCell)createOffheapKVWithTags(2);
+    ByteBufferExtendedCell kv1 = (ByteBufferExtendedCell)createOffheapKVWithTags(2);
     int tagsLength1 = kv1.getTagsLength();
     context.compressTags(daos, kv1.getTagsByteBuffer(), kv1.getTagsPosition(), tagsLength1);
-    ByteBufferCell kv2 = (ByteBufferCell)createOffheapKVWithTags(3);
+    ByteBufferExtendedCell kv2 = (ByteBufferExtendedCell)createOffheapKVWithTags(3);
     int tagsLength2 = kv2.getTagsLength();
     context.compressTags(daos, kv2.getTagsByteBuffer(), kv2.getTagsPosition(), tagsLength2);
 
@@ -129,10 +129,10 @@ public class TestTagCompressionContext {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream daos = new ByteBufferWriterDataOutputStream(baos);
     TagCompressionContext context = new TagCompressionContext(LRUDictionary.class, Byte.MAX_VALUE);
-    ByteBufferCell kv1 = (ByteBufferCell)createOffheapKVWithTags(1);
+    ByteBufferExtendedCell kv1 = (ByteBufferExtendedCell)createOffheapKVWithTags(1);
     int tagsLength1 = kv1.getTagsLength();
     context.compressTags(daos, kv1.getTagsByteBuffer(), kv1.getTagsPosition(), tagsLength1);
-    ByteBufferCell kv2 = (ByteBufferCell)createOffheapKVWithTags(3);
+    ByteBufferExtendedCell kv2 = (ByteBufferExtendedCell)createOffheapKVWithTags(3);
     int tagsLength2 = kv2.getTagsLength();
     context.compressTags(daos, kv2.getTagsByteBuffer(), kv2.getTagsPosition(), tagsLength2);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CellSortReducer.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CellSortReducer.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CellSortReducer.java
index 499accb..de961cf 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CellSortReducer.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/CellSortReducer.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.util.MapReduceCell;
+import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -53,7 +53,7 @@ public class CellSortReducer
     context.setStatus("Read " + map.getClass());
     int index = 0;
     for (Cell kv: map) {
-      context.write(row, new MapReduceCell(kv));
+      context.write(row, new MapReduceExtendedCell(kv));
       if (++index % 100 == 0) context.setStatus("Wrote " + index);
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
index 4a67933..312b315 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java
@@ -75,7 +75,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.MapReduceCell;
+import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.io.Text;
@@ -586,7 +586,7 @@ public class HFileOutputFormat2
       Class<? extends OutputFormat<?, ?>> cls) throws IOException {
     Configuration conf = job.getConfiguration();
     job.setOutputKeyClass(ImmutableBytesWritable.class);
-    job.setOutputValueClass(MapReduceCell.class);
+    job.setOutputValueClass(MapReduceExtendedCell.class);
     job.setOutputFormatClass(cls);
 
     if (multiTableInfo.stream().distinct().count() != multiTableInfo.size()) {
@@ -601,7 +601,7 @@ public class HFileOutputFormat2
     // sort the incoming values.
     // TODO it would be nice to pick one or the other of these formats.
     if (KeyValue.class.equals(job.getMapOutputValueClass())
-        || MapReduceCell.class.equals(job.getMapOutputValueClass())) {
+        || MapReduceExtendedCell.class.equals(job.getMapOutputValueClass())) {
       job.setReducerClass(CellSortReducer.class);
     } else if (Put.class.equals(job.getMapOutputValueClass())) {
       job.setReducerClass(PutSortReducer.class);
@@ -661,7 +661,7 @@ public class HFileOutputFormat2
     Configuration conf = job.getConfiguration();
 
     job.setOutputKeyClass(ImmutableBytesWritable.class);
-    job.setOutputValueClass(MapReduceCell.class);
+    job.setOutputValueClass(MapReduceExtendedCell.class);
     job.setOutputFormatClass(HFileOutputFormat2.class);
 
     ArrayList<TableDescriptor> singleTableDescriptor = new ArrayList<>(1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
index e8e1de0..e2b22d7 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/Import.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
@@ -61,7 +62,6 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.MapReduceCell;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.io.WritableComparable;
@@ -180,7 +180,7 @@ public class Import extends Configured implements Tool {
       int index = 0;
       for (Cell kv : kvs) {
         context.write(new ImmutableBytesWritable(CellUtil.cloneRow(kv)),
-          new MapReduceCell(kv));
+          new MapReduceExtendedCell(kv));
         if (++index % 100 == 0)
           context.setStatus("Wrote " + index + " KeyValues, "
               + "and the rowkey whose is being wrote is " + Bytes.toString(kv.getRowArray()));
@@ -280,7 +280,7 @@ public class Import extends Configured implements Tool {
             kv = filterKv(filter, kv);
             // skip if we filtered it out
             if (kv == null) continue;
-            context.write(row, new MapReduceCell(convertKv(kv, cfRenameMap)));
+            context.write(row, new MapReduceExtendedCell(convertKv(kv, cfRenameMap)));
           }
         }
       } catch (InterruptedException e) {
@@ -631,7 +631,7 @@ public class Import extends Configured implements Tool {
         Path outputDir = new Path(hfileOutPath);
         FileOutputFormat.setOutputPath(job, outputDir);
         job.setMapOutputKeyClass(CellWritableComparable.class);
-        job.setMapOutputValueClass(MapReduceCell.class);
+        job.setMapOutputValueClass(MapReduceExtendedCell.class);
         job.getConfiguration().setClass("mapreduce.job.output.key.comparator.class",
             CellWritableComparable.CellWritableComparator.class,
             RawComparator.class);
@@ -654,7 +654,7 @@ public class Import extends Configured implements Tool {
         Path outputDir = new Path(hfileOutPath);
         FileOutputFormat.setOutputPath(job, outputDir);
         job.setMapOutputKeyClass(ImmutableBytesWritable.class);
-        job.setMapOutputValueClass(MapReduceCell.class);
+        job.setMapOutputValueClass(MapReduceExtendedCell.class);
         HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator);
         TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
             org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
index 7212d4b..3c3a1c0 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.MapReduceCell;
+import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.mapreduce.Job;
@@ -111,7 +111,7 @@ public class WALPlayer extends Configured implements Tool {
               continue;
             }
             context.write(new ImmutableBytesWritable(CellUtil.cloneRow(cell)),
-              new MapReduceCell(cell));
+              new MapReduceExtendedCell(cell));
           }
         }
       } catch (InterruptedException e) {
@@ -302,7 +302,7 @@ public class WALPlayer extends Configured implements Tool {
       job.setReducerClass(CellSortReducer.class);
       Path outputDir = new Path(hfileOutPath);
       FileOutputFormat.setOutputPath(job, outputDir);
-      job.setMapOutputValueClass(MapReduceCell.class);
+      job.setMapOutputValueClass(MapReduceExtendedCell.class);
       try (Connection conn = ConnectionFactory.createConnection(conf);
           Table table = conn.getTable(tableName);
           RegionLocator regionLocator = conn.getRegionLocator(tableName)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceCell.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceCell.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceCell.java
deleted file mode 100644
index 38ff59b..0000000
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceCell.java
+++ /dev/null
@@ -1,271 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.util;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-
-import org.apache.hadoop.hbase.ByteBufferCell;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
-import org.apache.hadoop.hbase.ExtendedCell;
-import org.apache.hadoop.hbase.PrivateCellUtil;
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * A wrapper for a cell to be used with mapreduce, as the output value class for mappers/reducers.
- */
-@InterfaceAudience.Private
-public class MapReduceCell extends ByteBufferCell implements ExtendedCell {
-
-  private final Cell cell;
-
-  public MapReduceCell(Cell cell) {
-    this.cell = cell;
-  }
-
-  @Override
-  public byte[] getRowArray() {
-    return this.cell.getRowArray();
-  }
-
-  @Override
-  public int getRowOffset() {
-    return this.cell.getRowOffset();
-  }
-
-  @Override
-  public short getRowLength() {
-    return this.cell.getRowLength();
-  }
-
-  @Override
-  public byte[] getFamilyArray() {
-    return this.cell.getFamilyArray();
-  }
-
-  @Override
-  public int getFamilyOffset() {
-    return this.cell.getFamilyOffset();
-  }
-
-  @Override
-  public byte getFamilyLength() {
-    return this.cell.getFamilyLength();
-  }
-
-  @Override
-  public byte[] getQualifierArray() {
-    return this.cell.getQualifierArray();
-  }
-
-  @Override
-  public int getQualifierOffset() {
-    return this.cell.getQualifierOffset();
-  }
-
-  @Override
-  public int getQualifierLength() {
-    return this.cell.getQualifierLength();
-  }
-
-  @Override
-  public long getTimestamp() {
-    return this.cell.getTimestamp();
-  }
-
-  @Override
-  public byte getTypeByte() {
-    return this.cell.getTypeByte();
-  }
-
-  @Override
-  public long getSequenceId() {
-    return this.cell.getSequenceId();
-  }
-
-  @Override
-  public byte[] getValueArray() {
-    return this.cell.getValueArray();
-  }
-
-  @Override
-  public int getValueOffset() {
-    return this.cell.getValueOffset();
-  }
-
-  @Override
-  public int getValueLength() {
-    return this.cell.getValueLength();
-  }
-
-  @Override
-  public byte[] getTagsArray() {
-    return this.cell.getTagsArray();
-  }
-
-  @Override
-  public int getTagsOffset() {
-    return this.cell.getTagsOffset();
-  }
-
-  @Override
-  public int getTagsLength() {
-    return this.cell.getTagsLength();
-  }
-
-  @Override
-  public ByteBuffer getRowByteBuffer() {
-    if (cell instanceof ByteBufferCell) {
-      return ((ByteBufferCell) this.cell).getRowByteBuffer();
-    } else {
-      return ByteBuffer.wrap(CellUtil.cloneRow(this.cell));
-    }
-  }
-
-  @Override
-  public int getRowPosition() {
-    if (cell instanceof ByteBufferCell) {
-      return ((ByteBufferCell) this.cell).getRowPosition();
-    } else {
-      return 0;
-    }
-  }
-
-  @Override
-  public ByteBuffer getFamilyByteBuffer() {
-    if (cell instanceof ByteBufferCell) {
-      return ((ByteBufferCell) this.cell).getFamilyByteBuffer();
-    } else {
-      return ByteBuffer.wrap(CellUtil.cloneFamily(this.cell));
-    }
-  }
-
-  @Override
-  public int getFamilyPosition() {
-    if (cell instanceof ByteBufferCell) {
-      return ((ByteBufferCell) this.cell).getFamilyPosition();
-    } else {
-      return 0;
-    }
-  }
-
-  @Override
-  public ByteBuffer getQualifierByteBuffer() {
-    if (cell instanceof ByteBufferCell) {
-      return ((ByteBufferCell) this.cell).getQualifierByteBuffer();
-    } else {
-      return ByteBuffer.wrap(CellUtil.cloneQualifier(this.cell));
-    }
-  }
-
-  @Override
-  public int getQualifierPosition() {
-    if (cell instanceof ByteBufferCell) {
-      return ((ByteBufferCell) this.cell).getQualifierPosition();
-    } else {
-      return 0;
-    }
-  }
-
-  @Override
-  public ByteBuffer getValueByteBuffer() {
-    if (cell instanceof ByteBufferCell) {
-      return ((ByteBufferCell) this.cell).getValueByteBuffer();
-    } else {
-      return ByteBuffer.wrap(CellUtil.cloneValue(this.cell));
-    }
-  }
-
-  @Override
-  public int getValuePosition() {
-    if (cell instanceof ByteBufferCell) {
-      return ((ByteBufferCell) this.cell).getValuePosition();
-    } else {
-      return 0;
-    }
-  }
-
-  @Override
-  public ByteBuffer getTagsByteBuffer() {
-    if (cell instanceof ByteBufferCell) {
-      return ((ByteBufferCell) this.cell).getTagsByteBuffer();
-    } else {
-      return ByteBuffer.wrap(CellUtil.cloneTags(this.cell));
-    }
-  }
-
-  @Override
-  public int getTagsPosition() {
-    if (cell instanceof ByteBufferCell) {
-      return ((ByteBufferCell) this.cell).getTagsPosition();
-    } else {
-      return 0;
-    }
-  }
-
-  @Override
-  public String toString() {
-    return this.cell.toString();
-  }
-
-  @Override
-  public void setSequenceId(long seqId) throws IOException {
-    PrivateCellUtil.setSequenceId(cell, seqId);
-  }
-
-  @Override
-  public void setTimestamp(long ts) throws IOException {
-    PrivateCellUtil.setTimestamp(cell, ts);
-  }
-
-  @Override
-  public void setTimestamp(byte[] ts) throws IOException {
-    PrivateCellUtil.setTimestamp(cell, ts);
-  }
-
-  @Override
-  public long heapSize() {
-    return PrivateCellUtil.estimatedHeapSizeOf(cell);
-  }
-
-  @Override
-  public int write(OutputStream out, boolean withTags) throws IOException {
-    return PrivateCellUtil.writeCell(cell, out, withTags);
-  }
-
-  @Override
-  public int getSerializedSize(boolean withTags) {
-    return PrivateCellUtil.estimatedSerializedSizeOf(cell) - Bytes.SIZEOF_INT;
-  }
-
-  @Override
-  public void write(ByteBuffer buf, int offset) {
-    PrivateCellUtil.writeCellToBuffer(cell, buf, offset);
-  }
-
-  @Override
-  public ExtendedCell deepClone() {
-    try {
-      return (ExtendedCell) PrivateCellUtil.deepClone(cell);
-    } catch (CloneNotSupportedException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b39062e/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceExtendedCell.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceExtendedCell.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceExtendedCell.java
new file mode 100644
index 0000000..73eb7d8
--- /dev/null
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/util/MapReduceExtendedCell.java
@@ -0,0 +1,270 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import org.apache.hadoop.hbase.ByteBufferExtendedCell;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.ExtendedCell;
+import org.apache.hadoop.hbase.PrivateCellUtil;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A wrapper for a cell to be used with mapreduce, as the output value class for mappers/reducers.
+ */
+@InterfaceAudience.Private
+public class MapReduceExtendedCell extends ByteBufferExtendedCell {
+
+  private final Cell cell;
+
+  public MapReduceExtendedCell(Cell cell) {
+    this.cell = cell;
+  }
+
+  @Override
+  public byte[] getRowArray() {
+    return this.cell.getRowArray();
+  }
+
+  @Override
+  public int getRowOffset() {
+    return this.cell.getRowOffset();
+  }
+
+  @Override
+  public short getRowLength() {
+    return this.cell.getRowLength();
+  }
+
+  @Override
+  public byte[] getFamilyArray() {
+    return this.cell.getFamilyArray();
+  }
+
+  @Override
+  public int getFamilyOffset() {
+    return this.cell.getFamilyOffset();
+  }
+
+  @Override
+  public byte getFamilyLength() {
+    return this.cell.getFamilyLength();
+  }
+
+  @Override
+  public byte[] getQualifierArray() {
+    return this.cell.getQualifierArray();
+  }
+
+  @Override
+  public int getQualifierOffset() {
+    return this.cell.getQualifierOffset();
+  }
+
+  @Override
+  public int getQualifierLength() {
+    return this.cell.getQualifierLength();
+  }
+
+  @Override
+  public long getTimestamp() {
+    return this.cell.getTimestamp();
+  }
+
+  @Override
+  public byte getTypeByte() {
+    return this.cell.getTypeByte();
+  }
+
+  @Override
+  public long getSequenceId() {
+    return this.cell.getSequenceId();
+  }
+
+  @Override
+  public byte[] getValueArray() {
+    return this.cell.getValueArray();
+  }
+
+  @Override
+  public int getValueOffset() {
+    return this.cell.getValueOffset();
+  }
+
+  @Override
+  public int getValueLength() {
+    return this.cell.getValueLength();
+  }
+
+  @Override
+  public byte[] getTagsArray() {
+    return this.cell.getTagsArray();
+  }
+
+  @Override
+  public int getTagsOffset() {
+    return this.cell.getTagsOffset();
+  }
+
+  @Override
+  public int getTagsLength() {
+    return this.cell.getTagsLength();
+  }
+
+  @Override
+  public ByteBuffer getRowByteBuffer() {
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ((ByteBufferExtendedCell) this.cell).getRowByteBuffer();
+    } else {
+      return ByteBuffer.wrap(CellUtil.cloneRow(this.cell));
+    }
+  }
+
+  @Override
+  public int getRowPosition() {
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ((ByteBufferExtendedCell) this.cell).getRowPosition();
+    } else {
+      return 0;
+    }
+  }
+
+  @Override
+  public ByteBuffer getFamilyByteBuffer() {
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ((ByteBufferExtendedCell) this.cell).getFamilyByteBuffer();
+    } else {
+      return ByteBuffer.wrap(CellUtil.cloneFamily(this.cell));
+    }
+  }
+
+  @Override
+  public int getFamilyPosition() {
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ((ByteBufferExtendedCell) this.cell).getFamilyPosition();
+    } else {
+      return 0;
+    }
+  }
+
+  @Override
+  public ByteBuffer getQualifierByteBuffer() {
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ((ByteBufferExtendedCell) this.cell).getQualifierByteBuffer();
+    } else {
+      return ByteBuffer.wrap(CellUtil.cloneQualifier(this.cell));
+    }
+  }
+
+  @Override
+  public int getQualifierPosition() {
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ((ByteBufferExtendedCell) this.cell).getQualifierPosition();
+    } else {
+      return 0;
+    }
+  }
+
+  @Override
+  public ByteBuffer getValueByteBuffer() {
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ((ByteBufferExtendedCell) this.cell).getValueByteBuffer();
+    } else {
+      return ByteBuffer.wrap(CellUtil.cloneValue(this.cell));
+    }
+  }
+
+  @Override
+  public int getValuePosition() {
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ((ByteBufferExtendedCell) this.cell).getValuePosition();
+    } else {
+      return 0;
+    }
+  }
+
+  @Override
+  public ByteBuffer getTagsByteBuffer() {
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ((ByteBufferExtendedCell) this.cell).getTagsByteBuffer();
+    } else {
+      return ByteBuffer.wrap(CellUtil.cloneTags(this.cell));
+    }
+  }
+
+  @Override
+  public int getTagsPosition() {
+    if (cell instanceof ByteBufferExtendedCell) {
+      return ((ByteBufferExtendedCell) this.cell).getTagsPosition();
+    } else {
+      return 0;
+    }
+  }
+
+  @Override
+  public String toString() {
+    return this.cell.toString();
+  }
+
+  @Override
+  public void setSequenceId(long seqId) throws IOException {
+    PrivateCellUtil.setSequenceId(cell, seqId);
+  }
+
+  @Override
+  public void setTimestamp(long ts) throws IOException {
+    PrivateCellUtil.setTimestamp(cell, ts);
+  }
+
+  @Override
+  public void setTimestamp(byte[] ts) throws IOException {
+    PrivateCellUtil.setTimestamp(cell, ts);
+  }
+
+  @Override
+  public long heapSize() {
+    return PrivateCellUtil.estimatedHeapSizeOf(cell);
+  }
+
+  @Override
+  public int write(OutputStream out, boolean withTags) throws IOException {
+    return PrivateCellUtil.writeCell(cell, out, withTags);
+  }
+
+  @Override
+  public int getSerializedSize(boolean withTags) {
+    return PrivateCellUtil.estimatedSerializedSizeOf(cell) - Bytes.SIZEOF_INT;
+  }
+
+  @Override
+  public void write(ByteBuffer buf, int offset) {
+    PrivateCellUtil.writeCellToBuffer(cell, buf, offset);
+  }
+
+  @Override
+  public ExtendedCell deepClone() {
+    try {
+      return (ExtendedCell) PrivateCellUtil.deepClone(cell);
+    } catch (CloneNotSupportedException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}