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 2018/02/05 04:14:35 UTC

[01/28] hbase git commit: HBASE-19720 Rename WALKey#getTabnename to WALKey#getTableName [Forced Update!]

Repository: hbase
Updated Branches:
  refs/heads/HBASE-19064 ab73dd502 -> 8a79f26c0 (forced update)


HBASE-19720 Rename WALKey#getTabnename to WALKey#getTableName


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

Branch: refs/heads/HBASE-19064
Commit: 2f4d0b94bc61b00f1d7c549e8dafb4cc420fab18
Parents: fc6d140
Author: Chia-Ping Tsai <ch...@gmail.com>
Authored: Sun Jan 7 16:26:37 2018 +0800
Committer: Chia-Ping Tsai <ch...@gmail.com>
Committed: Fri Feb 2 19:43:26 2018 +0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/mapreduce/WALPlayer.java |  8 ++++----
 .../apache/hadoop/hbase/mapreduce/TestImportExport.java   |  2 +-
 .../org/apache/hadoop/hbase/mapreduce/TestWALPlayer.java  |  2 +-
 .../hadoop/hbase/protobuf/ReplicationProtbufUtil.java     |  3 +--
 .../apache/hadoop/hbase/regionserver/wal/ReaderBase.java  |  2 +-
 .../hbase/replication/NamespaceTableCfWALEntryFilter.java | 10 +++++-----
 .../hbase/replication/SystemTableWALEntryFilter.java      |  2 +-
 .../HBaseInterClusterReplicationEndpoint.java             |  2 +-
 .../src/main/java/org/apache/hadoop/hbase/wal/WALKey.java |  5 ++---
 .../main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java |  2 +-
 .../java/org/apache/hadoop/hbase/wal/WALSplitter.java     |  4 ++--
 .../hadoop/hbase/coprocessor/SimpleRegionObserver.java    |  2 +-
 .../apache/hadoop/hbase/regionserver/TestWALLockup.java   |  4 ++--
 .../hbase/regionserver/wal/AbstractTestProtobufLog.java   |  2 +-
 .../TestRegionReplicaReplicationEndpointNoMaster.java     |  2 +-
 .../java/org/apache/hadoop/hbase/wal/TestWALFactory.java  |  4 ++--
 16 files changed, 27 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2f4d0b94/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 40f03f1..97ae81c 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
@@ -105,7 +105,7 @@ public class WALPlayer extends Configured implements Tool {
     throws IOException {
       try {
         // skip all other tables
-        if (Bytes.equals(table, key.getTablename().getName())) {
+        if (Bytes.equals(table, key.getTableName().getName())) {
           for (Cell cell : value.getCells()) {
             if (WALEdit.isMetaEditFamily(cell)) {
               continue;
@@ -145,10 +145,10 @@ public class WALPlayer extends Configured implements Tool {
     public void map(WALKey key, WALEdit value, Context context)
     throws IOException {
       try {
-        if (tables.isEmpty() || tables.containsKey(key.getTablename())) {
+        if (tables.isEmpty() || tables.containsKey(key.getTableName())) {
           TableName targetTable = tables.isEmpty() ?
-                key.getTablename() :
-                tables.get(key.getTablename());
+                key.getTableName() :
+                tables.get(key.getTableName());
           ImmutableBytesWritable tableOut = new ImmutableBytesWritable(targetTable.getName());
           Put put = null;
           Delete del = null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/2f4d0b94/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 2c35254..9349816 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
@@ -791,7 +791,7 @@ public class TestImportExport {
 
     @Override
     public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit) {
-      if (logKey.getTablename().getNameAsString().equalsIgnoreCase(
+      if (logKey.getTableName().getNameAsString().equalsIgnoreCase(
           this.regionInfo.getTable().getNameAsString()) && (!logEdit.isMetaEdit())) {
         isVisited = true;
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2f4d0b94/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 993a077..0dfe863 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
@@ -173,7 +173,7 @@ public class TestWALPlayer {
     configuration.set(tableConfigKey, "table");
     WALKeyValueMapper mapper = new WALKeyValueMapper();
     WALKey key = mock(WALKey.class);
-    when(key.getTablename()).thenReturn(TableName.valueOf("table"));
+    when(key.getTableName()).thenReturn(TableName.valueOf("table"));
     @SuppressWarnings("unchecked")
     Mapper<WALKey, WALEdit, ImmutableBytesWritable, Cell>.Context context = mock(Context.class);
     when(context.getConfiguration()).thenReturn(configuration);

http://git-wip-us.apache.org/repos/asf/hbase/blob/2f4d0b94/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
index 79fd000..81dd59e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
@@ -46,7 +46,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
-import org.apache.hadoop.hbase.wal.WALKey;
 
 @InterfaceAudience.Private
 public class ReplicationProtbufUtil {
@@ -116,7 +115,7 @@ public class ReplicationProtbufUtil {
           UnsafeByteOperations.unsafeWrap(encodedRegionName == null
             ? key.getEncodedRegionName()
             : encodedRegionName));
-      keyBuilder.setTableName(UnsafeByteOperations.unsafeWrap(key.getTablename().getName()));
+      keyBuilder.setTableName(UnsafeByteOperations.unsafeWrap(key.getTableName().getName()));
       long sequenceId = key.getSequenceId();
       keyBuilder.setLogSequenceNumber(sequenceId);
       keyBuilder.setWriteTime(key.getWriteTime());

http://git-wip-us.apache.org/repos/asf/hbase/blob/2f4d0b94/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java
index f057358..f242cef 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ReaderBase.java
@@ -100,7 +100,7 @@ public abstract class ReaderBase implements AbstractFSWALProvider.Reader {
     try {
       hasEntry = readNext(e);
     } catch (IllegalArgumentException iae) {
-      TableName tableName = e.getKey().getTablename();
+      TableName tableName = e.getKey().getTableName();
       if (tableName != null && tableName.equals(TableName.OLD_ROOT_TABLE_NAME)) {
         // It is old ROOT table edit, ignore it
         LOG.info("Got an old ROOT edit, ignoring ");

http://git-wip-us.apache.org/repos/asf/hbase/blob/2f4d0b94/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/NamespaceTableCfWALEntryFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/NamespaceTableCfWALEntryFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/NamespaceTableCfWALEntryFilter.java
index b540416..ad6e5a6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/NamespaceTableCfWALEntryFilter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/NamespaceTableCfWALEntryFilter.java
@@ -57,7 +57,7 @@ public class NamespaceTableCfWALEntryFilter implements WALEntryFilter, WALCellFi
 
   @Override
   public Entry filter(Entry entry) {
-    TableName tabName = entry.getKey().getTablename();
+    TableName tabName = entry.getKey().getTableName();
     String namespace = tabName.getNamespaceAsString();
     ReplicationPeerConfig peerConfig = this.peer.getPeerConfig();
 
@@ -110,10 +110,10 @@ public class NamespaceTableCfWALEntryFilter implements WALEntryFilter, WALCellFi
 
       if (CellUtil.matchingColumn(cell, WALEdit.METAFAMILY, WALEdit.BULK_LOAD)) {
         cell = bulkLoadFilter.filterCell(cell,
-          fam -> filterByExcludeTableCfs(entry.getKey().getTablename(), Bytes.toString(fam),
+          fam -> filterByExcludeTableCfs(entry.getKey().getTableName(), Bytes.toString(fam),
             excludeTableCfs));
       } else {
-        if (filterByExcludeTableCfs(entry.getKey().getTablename(),
+        if (filterByExcludeTableCfs(entry.getKey().getTableName(),
           Bytes.toString(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()),
           excludeTableCfs)) {
           return null;
@@ -130,9 +130,9 @@ public class NamespaceTableCfWALEntryFilter implements WALEntryFilter, WALCellFi
 
       if (CellUtil.matchingColumn(cell, WALEdit.METAFAMILY, WALEdit.BULK_LOAD)) {
         cell = bulkLoadFilter.filterCell(cell,
-          fam -> filterByTableCfs(entry.getKey().getTablename(), Bytes.toString(fam), tableCfs));
+          fam -> filterByTableCfs(entry.getKey().getTableName(), Bytes.toString(fam), tableCfs));
       } else {
-        if (filterByTableCfs(entry.getKey().getTablename(),
+        if (filterByTableCfs(entry.getKey().getTableName(),
           Bytes.toString(cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength()),
           tableCfs)) {
           return null;

http://git-wip-us.apache.org/repos/asf/hbase/blob/2f4d0b94/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/SystemTableWALEntryFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/SystemTableWALEntryFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/SystemTableWALEntryFilter.java
index dc9aaf4..1984436 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/SystemTableWALEntryFilter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/SystemTableWALEntryFilter.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.wal.WAL.Entry;
 public class SystemTableWALEntryFilter implements WALEntryFilter {
   @Override
   public Entry filter(Entry entry) {
-    if (entry.getKey().getTablename().isSystemTable()) {
+    if (entry.getKey().getTableName().isSystemTable()) {
       return null;
     }
     return entry;

http://git-wip-us.apache.org/repos/asf/hbase/blob/2f4d0b94/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
index 49ea1f7..fd3c671 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/HBaseInterClusterReplicationEndpoint.java
@@ -257,7 +257,7 @@ public class HBaseInterClusterReplicationEndpoint extends HBaseReplicationEndpoi
       ArrayList<Entry> thisList = new ArrayList<Entry>(entries.size());
       entryLists.add(thisList);
       for (Entry e : entries) {
-        if (!e.getKey().getTablename().equals(table)) {
+        if (!e.getKey().getTableName().equals(table)) {
           thisList.add(e);
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2f4d0b94/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
index 0c818db..af3cf71 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKey.java
@@ -29,7 +29,6 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.NavigableMap;
 import java.util.UUID;
 
 
@@ -57,7 +56,7 @@ public interface WALKey extends SequenceId, Comparable<WALKey> {
   /**
    * @return table name
    */
-  TableName getTablename();
+  TableName getTableName();
 
   /**
    * @return the write time
@@ -96,7 +95,7 @@ public interface WALKey extends SequenceId, Comparable<WALKey> {
    */
   default Map<String, Object> toStringMap() {
     Map<String, Object> stringMap = new HashMap<>();
-    stringMap.put("table", getTablename());
+    stringMap.put("table", getTableName());
     stringMap.put("region", Bytes.toStringBinary(getEncodedRegionName()));
     stringMap.put("sequence", getSequenceId());
     return stringMap;

http://git-wip-us.apache.org/repos/asf/hbase/blob/2f4d0b94/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java
index 0a5acda..7abd4a5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java
@@ -352,7 +352,7 @@ public class WALKeyImpl implements WALKey {
 
   /** @return table name */
   @Override
-  public TableName getTablename() {
+  public TableName getTableName() {
     return tablename;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/2f4d0b94/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
index 0d73045..3f64d75 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
@@ -470,7 +470,7 @@ public class WALSplitter {
   static Path getRegionSplitEditsPath(final FileSystem fs,
       final Entry logEntry, final Path rootDir, String fileNameBeingSplit)
   throws IOException {
-    Path tableDir = FSUtils.getTableDir(rootDir, logEntry.getKey().getTablename());
+    Path tableDir = FSUtils.getTableDir(rootDir, logEntry.getKey().getTableName());
     String encodedRegionName = Bytes.toString(logEntry.getKey().getEncodedRegionName());
     Path regiondir = HRegion.getRegionDir(tableDir, encodedRegionName);
     Path dir = getRegionDirRecoveredEditsDir(regiondir);
@@ -877,7 +877,7 @@ public class WALSplitter {
       synchronized (this) {
         buffer = buffers.get(key.getEncodedRegionName());
         if (buffer == null) {
-          buffer = new RegionEntryBuffer(key.getTablename(), key.getEncodedRegionName());
+          buffer = new RegionEntryBuffer(key.getTableName(), key.getEncodedRegionName());
           buffers.put(key.getEncodedRegionName(), buffer);
         }
         incrHeap= buffer.appendEntry(entry);

http://git-wip-us.apache.org/repos/asf/hbase/blob/2f4d0b94/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
index 0d864b6..62623b0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java
@@ -600,7 +600,7 @@ public class SimpleRegionObserver implements RegionCoprocessor, RegionObserver {
   @Override
   public void preWALRestore(ObserverContext<? extends RegionCoprocessorEnvironment> env,
       RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
-    String tableName = logKey.getTablename().getNameAsString();
+    String tableName = logKey.getTableName().getNameAsString();
     if (tableName.equals(TABLE_SKIPPED)) {
       // skip recovery of TABLE_SKIPPED for testing purpose
       env.bypass();

http://git-wip-us.apache.org/repos/asf/hbase/blob/2f4d0b94/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
index 7a7eb5e..1e59248 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
@@ -590,14 +590,14 @@ public class TestWALLockup {
     @Override
     public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit)
         throws IOException {
-      if (logKey.getTablename().getNameAsString().equalsIgnoreCase("sleep")) {
+      if (logKey.getTableName().getNameAsString().equalsIgnoreCase("sleep")) {
         try {
           Thread.sleep(1000);
         } catch (InterruptedException e) {
           e.printStackTrace();
         }
       }
-      if (logKey.getTablename().getNameAsString()
+      if (logKey.getTableName().getNameAsString()
           .equalsIgnoreCase("DamagedWALException")) {
         throw new DamagedWALException("Failed appending");
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2f4d0b94/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java
index e49cda0..c3f3277 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java
@@ -177,7 +177,7 @@ public abstract class AbstractTestProtobufLog<W extends Closeable> {
         assertNotNull(entry);
         assertEquals(columnCount, entry.getEdit().size());
         assertArrayEquals(hri.getEncodedNameAsBytes(), entry.getKey().getEncodedRegionName());
-        assertEquals(tableName, entry.getKey().getTablename());
+        assertEquals(tableName, entry.getKey().getTableName());
         int idx = 0;
         for (Cell val : entry.getEdit().getCells()) {
           assertTrue(Bytes.equals(row, 0, row.length, val.getRowArray(), val.getRowOffset(),

http://git-wip-us.apache.org/repos/asf/hbase/blob/2f4d0b94/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
index 33ea8d7..15066dd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestRegionReplicaReplicationEndpointNoMaster.java
@@ -167,7 +167,7 @@ public class TestRegionReplicaReplicationEndpointNoMaster {
     public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
                              RegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
       // only keep primary region's edits
-      if (logKey.getTablename().equals(tableName) && info.getReplicaId() == 0) {
+      if (logKey.getTableName().equals(tableName) && info.getReplicaId() == 0) {
         // Presume type is a WALKeyImpl
         entries.add(new Entry((WALKeyImpl)logKey, logEdit));
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2f4d0b94/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
index a65d97c..c391601 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALFactory.java
@@ -536,7 +536,7 @@ public class TestWALFactory {
         WALKey key = entry.getKey();
         WALEdit val = entry.getEdit();
         assertTrue(Bytes.equals(info.getEncodedNameAsBytes(), key.getEncodedRegionName()));
-        assertTrue(htd.getTableName().equals(key.getTablename()));
+        assertTrue(htd.getTableName().equals(key.getTableName()));
         Cell cell = val.getCells().get(0);
         assertTrue(Bytes.equals(row, 0, row.length, cell.getRowArray(), cell.getRowOffset(),
           cell.getRowLength()));
@@ -592,7 +592,7 @@ public class TestWALFactory {
       for (Cell val : entry.getEdit().getCells()) {
         assertTrue(Bytes.equals(hri.getEncodedNameAsBytes(),
           entry.getKey().getEncodedRegionName()));
-        assertTrue(htd.getTableName().equals(entry.getKey().getTablename()));
+        assertTrue(htd.getTableName().equals(entry.getKey().getTableName()));
         assertTrue(Bytes.equals(row, 0, row.length, val.getRowArray(), val.getRowOffset(),
           val.getRowLength()));
         assertEquals((byte) (idx + '0'), CellUtil.cloneValue(val)[0]);


[24/28] hbase git commit: HBASE-19083 Introduce a new log writer which can write to two HDFSes

Posted by zh...@apache.org.
HBASE-19083 Introduce a new log writer which can write to two HDFSes


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

Branch: refs/heads/HBASE-19064
Commit: 9eb748fce0a56eef1ca459e4a627e923f2acac0a
Parents: 7f7f2b2
Author: zhangduo <zh...@apache.org>
Authored: Thu Jan 11 21:08:02 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Feb 5 12:10:54 2018 +0800

----------------------------------------------------------------------
 .../hbase/regionserver/wal/AsyncFSWAL.java      |  26 ++--
 .../regionserver/wal/CombinedAsyncWriter.java   | 134 ++++++++++++++++++
 .../hbase/regionserver/wal/DualAsyncFSWAL.java  |  67 +++++++++
 .../wal/AbstractTestProtobufLog.java            | 110 +++------------
 .../regionserver/wal/ProtobufLogTestHelper.java |  99 ++++++++++++++
 .../regionserver/wal/TestAsyncProtobufLog.java  |  32 +----
 .../wal/TestCombinedAsyncWriter.java            | 136 +++++++++++++++++++
 .../hbase/regionserver/wal/TestProtobufLog.java |  14 +-
 .../regionserver/wal/WriterOverAsyncWriter.java |  63 +++++++++
 9 files changed, 536 insertions(+), 145 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9eb748fc/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
index faf3b77..24094e0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
@@ -606,8 +606,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     }
   }
 
-  @Override
-  protected AsyncWriter createWriterInstance(Path path) throws IOException {
+  protected final AsyncWriter createAsyncWriter(FileSystem fs, Path path) throws IOException {
     try {
       return AsyncFSWALProvider.createAsyncWriter(conf, fs, path, false, eventLoopGroup,
         channelClass);
@@ -623,6 +622,11 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     }
   }
 
+  @Override
+  protected AsyncWriter createWriterInstance(Path path) throws IOException {
+    return createAsyncWriter(fs, path);
+  }
+
   private void waitForSafePoint() {
     consumeLock.lock();
     try {
@@ -665,21 +669,21 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     } finally {
       consumeLock.unlock();
     }
-    return executeClose(closeExecutor, oldWriter);
+    return executeClose(oldWriter);
   }
 
   @Override
   protected void doShutdown() throws IOException {
     waitForSafePoint();
-    executeClose(closeExecutor, writer);
+    executeClose(writer);
     closeExecutor.shutdown();
     try {
       if (!closeExecutor.awaitTermination(waitOnShutdownInSeconds, TimeUnit.SECONDS)) {
-        LOG.error("We have waited " + waitOnShutdownInSeconds + " seconds but"
-          + " the close of async writer doesn't complete."
-          + "Please check the status of underlying filesystem"
-          + " or increase the wait time by the config \""
-          + ASYNC_WAL_WAIT_ON_SHUTDOWN_IN_SECONDS + "\"");
+        LOG.error("We have waited " + waitOnShutdownInSeconds + " seconds but" +
+          " the close of async writer doesn't complete." +
+          "Please check the status of underlying filesystem" +
+          " or increase the wait time by the config \"" + ASYNC_WAL_WAIT_ON_SHUTDOWN_IN_SECONDS +
+          "\"");
       }
     } catch (InterruptedException e) {
       LOG.error("The wait for close of async writer is interrupted");
@@ -692,7 +696,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     }
   }
 
-  private static long executeClose(ExecutorService closeExecutor, AsyncWriter writer) {
+  protected final long executeClose(AsyncWriter writer) {
     long fileLength;
     if (writer != null) {
       fileLength = writer.getLength();
@@ -700,7 +704,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
         try {
           writer.close();
         } catch (IOException e) {
-          LOG.warn("close old writer failed", e);
+          LOG.warn("close writer failed", e);
         }
       });
     } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/9eb748fc/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CombinedAsyncWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CombinedAsyncWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CombinedAsyncWriter.java
new file mode 100644
index 0000000..8ecfede
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CombinedAsyncWriter.java
@@ -0,0 +1,134 @@
+/**
+ * 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.wal;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
+
+/**
+ * An {@link AsyncWriter} wrapper which writes data to a set of {@link AsyncWriter} instances.
+ */
+@InterfaceAudience.Private
+public abstract class CombinedAsyncWriter implements AsyncWriter {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CombinedAsyncWriter.class);
+
+  protected final ImmutableList<AsyncWriter> writers;
+
+  protected CombinedAsyncWriter(ImmutableList<AsyncWriter> writers) {
+    this.writers = writers;
+  }
+
+  @Override
+  public long getLength() {
+    return writers.get(0).getLength();
+  }
+
+  @Override
+  public void close() throws IOException {
+    Exception error = null;
+    for (AsyncWriter writer : writers) {
+      try {
+        writer.close();
+      } catch (Exception e) {
+        LOG.warn("close writer failed", e);
+        if (error == null) {
+          error = e;
+        }
+      }
+    }
+    if (error != null) {
+      throw new IOException("Failed to close at least one writer, please see the warn log above. " +
+        "The cause is the first exception occured", error);
+    }
+  }
+
+  protected abstract void doSync(CompletableFuture<Long> future);
+
+  @Override
+  public CompletableFuture<Long> sync() {
+    CompletableFuture<Long> future = new CompletableFuture<>();
+    doSync(future);
+    return future;
+  }
+
+  @Override
+  public void append(Entry entry) {
+    writers.forEach(w -> w.append(entry));
+  }
+
+  public enum Mode {
+    SEQUENTIAL, PARALLEL
+  }
+
+  public static CombinedAsyncWriter create(Mode mode, AsyncWriter writer, AsyncWriter... writers) {
+    ImmutableList<AsyncWriter> ws =
+        ImmutableList.<AsyncWriter> builder().add(writer).add(writers).build();
+    switch (mode) {
+      case SEQUENTIAL:
+        return new CombinedAsyncWriter(ws) {
+
+          private void doSync(CompletableFuture<Long> future, Long length, int index) {
+            if (index == writers.size()) {
+              future.complete(length);
+              return;
+            }
+            writers.get(index).sync().whenComplete((len, error) -> {
+              if (error != null) {
+                future.completeExceptionally(error);
+                return;
+              }
+              doSync(future, len, index + 1);
+            });
+          }
+
+          @Override
+          protected void doSync(CompletableFuture<Long> future) {
+            doSync(future, null, 0);
+          }
+        };
+      case PARALLEL:
+        return new CombinedAsyncWriter(ws) {
+
+          @Override
+          protected void doSync(CompletableFuture<Long> future) {
+            AtomicInteger remaining = new AtomicInteger(writers.size());
+            writers.forEach(w -> w.sync().whenComplete((length, error) -> {
+              if (error != null) {
+                future.completeExceptionally(error);
+                return;
+              }
+              if (remaining.decrementAndGet() == 0) {
+                future.complete(length);
+              }
+            }));
+          }
+        };
+      default:
+        throw new IllegalArgumentException("Unknown mode: " + mode);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/9eb748fc/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java
new file mode 100644
index 0000000..6bf9e02
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.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.regionserver.wal;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.io.netty.channel.Channel;
+import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
+
+/**
+ * An AsyncFSWAL which writes data to two filesystems.
+ */
+@InterfaceAudience.Private
+public class DualAsyncFSWAL extends AsyncFSWAL {
+
+  private final FileSystem remoteFs;
+
+  private final Path remoteWalDir;
+
+  public DualAsyncFSWAL(FileSystem fs, FileSystem remoteFs, Path rootDir, Path remoteRootDir,
+      String logDir, String archiveDir, Configuration conf, List<WALActionsListener> listeners,
+      boolean failIfWALExists, String prefix, String suffix, EventLoopGroup eventLoopGroup,
+      Class<? extends Channel> channelClass) throws FailedLogCloseException, IOException {
+    super(fs, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix, suffix,
+        eventLoopGroup, channelClass);
+    this.remoteFs = remoteFs;
+    this.remoteWalDir = new Path(remoteRootDir, logDir);
+  }
+
+  @Override
+  protected AsyncWriter createWriterInstance(Path path) throws IOException {
+    AsyncWriter localWriter = super.createWriterInstance(path);
+    AsyncWriter remoteWriter;
+    boolean succ = false;
+    try {
+      remoteWriter = createAsyncWriter(remoteFs, new Path(remoteWalDir, path.getName()));
+      succ = true;
+    } finally {
+      if (!succ) {
+        executeClose(localWriter);
+      }
+    }
+    return CombinedAsyncWriter.create(CombinedAsyncWriter.Mode.SEQUENTIAL, remoteWriter,
+      localWriter);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/9eb748fc/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java
index c3f3277..5098609 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestProtobufLog.java
@@ -18,33 +18,15 @@
  */
 package org.apache.hadoop.hbase.regionserver.wal;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import java.io.Closeable;
 import java.io.IOException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
-import org.apache.hadoop.hbase.coprocessor.SampleRegionWALCoprocessor;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.wal.WAL;
-import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALFactory;
-import org.apache.hadoop.hbase.wal.WALKeyImpl;
+import org.apache.hadoop.hbase.wal.WALProvider;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -56,8 +38,8 @@ import org.junit.rules.TestName;
 /**
  * WAL tests that can be reused across providers.
  */
-public abstract class AbstractTestProtobufLog<W extends Closeable> {
-  protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+public abstract class AbstractTestProtobufLog {
+  protected static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   protected FileSystem fs;
   protected Path dir;
@@ -93,14 +75,7 @@ public abstract class AbstractTestProtobufLog<W extends Closeable> {
     TEST_UTIL.getConfiguration().setInt("dfs.client.socket-timeout", 5000);
 
     // faster failover with cluster.shutdown();fs.close() idiom
-    TEST_UTIL.getConfiguration()
-        .setInt("hbase.ipc.client.connect.max.retries", 1);
-    TEST_UTIL.getConfiguration().setInt(
-        "dfs.client.block.recovery.retries", 1);
-    TEST_UTIL.getConfiguration().setInt(
-      "hbase.ipc.client.connection.maxidletime", 500);
-    TEST_UTIL.getConfiguration().set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY,
-        SampleRegionWALCoprocessor.class.getName());
+    TEST_UTIL.getConfiguration().setInt("dfs.client.block.recovery.retries", 1);
     TEST_UTIL.startMiniDFSCluster(3);
   }
 
@@ -131,77 +106,24 @@ public abstract class AbstractTestProtobufLog<W extends Closeable> {
    * @throws IOException
    */
   private void doRead(boolean withTrailer) throws IOException {
-    final int columnCount = 5;
-    final int recordCount = 5;
-    final TableName tableName =
-        TableName.valueOf("tablename");
-    final byte[] row = Bytes.toBytes("row");
+    int columnCount = 5;
+    int recordCount = 5;
+    TableName tableName = TableName.valueOf("tablename");
+    byte[] row = Bytes.toBytes("row");
     long timestamp = System.currentTimeMillis();
     Path path = new Path(dir, "tempwal");
     // delete the log if already exists, for test only
     fs.delete(path, true);
-    W writer = null;
-    ProtobufLogReader reader = null;
-    try {
-      HRegionInfo hri = new HRegionInfo(tableName,
-          HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW);
-      HTableDescriptor htd = new HTableDescriptor(tableName);
-      fs.mkdirs(dir);
-      // Write log in pb format.
-      writer = createWriter(path);
-      for (int i = 0; i < recordCount; ++i) {
-        WALKeyImpl key = new WALKeyImpl(
-            hri.getEncodedNameAsBytes(), tableName, i, timestamp, HConstants.DEFAULT_CLUSTER_ID);
-        WALEdit edit = new WALEdit();
-        for (int j = 0; j < columnCount; ++j) {
-          if (i == 0) {
-            htd.addFamily(new HColumnDescriptor("column" + j));
-          }
-          String value = i + "" + j;
-          edit.add(new KeyValue(row, row, row, timestamp, Bytes.toBytes(value)));
-        }
-        append(writer, new WAL.Entry(key, edit));
-      }
-      sync(writer);
-      if (withTrailer) writer.close();
-
-      // Now read the log using standard means.
-      reader = (ProtobufLogReader) wals.createReader(fs, path);
-      if (withTrailer) {
-        assertNotNull(reader.trailer);
-      } else {
-        assertNull(reader.trailer);
-      }
-      for (int i = 0; i < recordCount; ++i) {
-        WAL.Entry entry = reader.next();
-        assertNotNull(entry);
-        assertEquals(columnCount, entry.getEdit().size());
-        assertArrayEquals(hri.getEncodedNameAsBytes(), entry.getKey().getEncodedRegionName());
-        assertEquals(tableName, entry.getKey().getTableName());
-        int idx = 0;
-        for (Cell val : entry.getEdit().getCells()) {
-          assertTrue(Bytes.equals(row, 0, row.length, val.getRowArray(), val.getRowOffset(),
-            val.getRowLength()));
-          String value = i + "" + idx;
-          assertArrayEquals(Bytes.toBytes(value), CellUtil.cloneValue(val));
-          idx++;
-        }
-      }
-      WAL.Entry entry = reader.next();
-      assertNull(entry);
-    } finally {
-      if (writer != null) {
-        writer.close();
-      }
-      if (reader != null) {
-        reader.close();
+    fs.mkdirs(dir);
+    try (WALProvider.Writer writer = createWriter(path)) {
+      ProtobufLogTestHelper.doWrite(writer, withTrailer, tableName, columnCount, recordCount, row,
+        timestamp);
+      try (ProtobufLogReader reader = (ProtobufLogReader) wals.createReader(fs, path)) {
+        ProtobufLogTestHelper.doRead(reader, withTrailer, tableName, columnCount, recordCount, row,
+          timestamp);
       }
     }
   }
 
-  protected abstract W createWriter(Path path) throws IOException;
-
-  protected abstract void append(W writer, WAL.Entry entry) throws IOException;
-
-  protected abstract void sync(W writer) throws IOException;
+  protected abstract WALProvider.Writer createWriter(Path path) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/9eb748fc/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java
new file mode 100644
index 0000000..ecd8e6c
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java
@@ -0,0 +1,99 @@
+/**
+ * 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.wal;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.stream.IntStream;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
+import org.apache.hadoop.hbase.wal.WALProvider;
+
+/**
+ * Helper class for testing protobuf log.
+ */
+final class ProtobufLogTestHelper {
+
+  private ProtobufLogTestHelper() {
+  }
+
+  private static byte[] toValue(int prefix, int suffix) {
+    return Bytes.toBytes(prefix + "-" + suffix);
+  }
+
+  private static RegionInfo toRegionInfo(TableName tableName) {
+    return RegionInfoBuilder.newBuilder(tableName).setRegionId(1024).build();
+  }
+
+  public static void doWrite(WALProvider.Writer writer, boolean withTrailer, TableName tableName,
+      int columnCount, int recordCount, byte[] row, long timestamp) throws IOException {
+    RegionInfo hri = toRegionInfo(tableName);
+    for (int i = 0; i < recordCount; i++) {
+      WALKeyImpl key = new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, i, timestamp,
+          HConstants.DEFAULT_CLUSTER_ID);
+      WALEdit edit = new WALEdit();
+      int prefix = i;
+      IntStream.range(0, columnCount).mapToObj(j -> toValue(prefix, j))
+          .map(value -> new KeyValue(row, row, row, timestamp, value)).forEachOrdered(edit::add);
+      writer.append(new WAL.Entry(key, edit));
+    }
+    writer.sync();
+    if (withTrailer) {
+      writer.close();
+    }
+  }
+
+  public static void doRead(ProtobufLogReader reader, boolean withTrailer, TableName tableName,
+      int columnCount, int recordCount, byte[] row, long timestamp) throws IOException {
+    if (withTrailer) {
+      assertNotNull(reader.trailer);
+    } else {
+      assertNull(reader.trailer);
+    }
+    RegionInfo hri = toRegionInfo(tableName);
+    for (int i = 0; i < recordCount; ++i) {
+      WAL.Entry entry = reader.next();
+      assertNotNull(entry);
+      assertEquals(columnCount, entry.getEdit().size());
+      assertArrayEquals(hri.getEncodedNameAsBytes(), entry.getKey().getEncodedRegionName());
+      assertEquals(tableName, entry.getKey().getTableName());
+      int idx = 0;
+      for (Cell val : entry.getEdit().getCells()) {
+        assertTrue(Bytes.equals(row, 0, row.length, val.getRowArray(), val.getRowOffset(),
+          val.getRowLength()));
+        assertArrayEquals(toValue(i, idx), CellUtil.cloneValue(val));
+        idx++;
+      }
+    }
+    assertNull(reader.next());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/9eb748fc/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java
index 0ea75b6..7626dcf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncProtobufLog.java
@@ -18,29 +18,24 @@
 package org.apache.hadoop.hbase.regionserver.wal;
 
 import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.util.concurrent.ExecutionException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
-import org.apache.hadoop.hbase.wal.WAL.Entry;
-import org.apache.hadoop.hbase.wal.WALProvider;
-import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter;
+import org.apache.hadoop.hbase.wal.WALProvider.Writer;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.experimental.categories.Category;
 
-import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
 import org.apache.hbase.thirdparty.io.netty.channel.Channel;
 import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
 import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
 import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel;
 
 @Category({ RegionServerTests.class, MediumTests.class })
-public class TestAsyncProtobufLog extends AbstractTestProtobufLog<WALProvider.AsyncWriter> {
+public class TestAsyncProtobufLog extends AbstractTestProtobufLog {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
@@ -64,25 +59,8 @@ public class TestAsyncProtobufLog extends AbstractTestProtobufLog<WALProvider.As
   }
 
   @Override
-  protected AsyncWriter createWriter(Path path) throws IOException {
-    return AsyncFSWALProvider.createAsyncWriter(TEST_UTIL.getConfiguration(), fs, path, false,
-      EVENT_LOOP_GROUP.next(), CHANNEL_CLASS);
-  }
-
-  @Override
-  protected void append(AsyncWriter writer, Entry entry) throws IOException {
-    writer.append(entry);
-  }
-
-  @Override
-  protected void sync(AsyncWriter writer) throws IOException {
-    try {
-      writer.sync().get();
-    } catch (InterruptedException e) {
-      throw new InterruptedIOException();
-    } catch (ExecutionException e) {
-      Throwables.propagateIfPossible(e.getCause());
-      throw new IOException(e.getCause());
-    }
+  protected Writer createWriter(Path path) throws IOException {
+    return new WriterOverAsyncWriter(AsyncFSWALProvider.createAsyncWriter(
+      TEST_UTIL.getConfiguration(), fs, path, false, EVENT_LOOP_GROUP.next(), CHANNEL_CLASS));
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/9eb748fc/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCombinedAsyncWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCombinedAsyncWriter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCombinedAsyncWriter.java
new file mode 100644
index 0000000..d74f9d8
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCombinedAsyncWriter.java
@@ -0,0 +1,136 @@
+/**
+ * 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.wal;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
+import org.apache.hadoop.hbase.wal.AsyncFSWALProvider.AsyncWriter;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+import org.junit.runners.Parameterized.Parameters;
+
+import org.apache.hbase.thirdparty.io.netty.channel.Channel;
+import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
+import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel;
+
+@RunWith(Parameterized.class)
+@Category({ RegionServerTests.class, MediumTests.class })
+public class TestCombinedAsyncWriter {
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static EventLoopGroup EVENT_LOOP_GROUP;
+
+  private static Class<? extends Channel> CHANNEL_CLASS;
+
+  private static WALFactory WALS;
+
+  @Rule
+  public final TestName name = new TestName();
+
+  @Parameter
+  public CombinedAsyncWriter.Mode mode;
+
+  @Parameters(name = "{index}: mode={0}")
+  public static List<Object[]> params() {
+    return Arrays.asList(new Object[] { CombinedAsyncWriter.Mode.SEQUENTIAL },
+      new Object[] { CombinedAsyncWriter.Mode.PARALLEL });
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    EVENT_LOOP_GROUP = new NioEventLoopGroup();
+    CHANNEL_CLASS = NioSocketChannel.class;
+    UTIL.startMiniDFSCluster(3);
+    UTIL.getTestFileSystem().mkdirs(UTIL.getDataTestDirOnTestFS());
+    WALS =
+      new WALFactory(UTIL.getConfiguration(), null, TestCombinedAsyncWriter.class.getSimpleName());
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    if (WALS != null) {
+      WALS.close();
+    }
+    EVENT_LOOP_GROUP.shutdownGracefully().syncUninterruptibly();
+    UTIL.shutdownMiniDFSCluster();
+  }
+
+  @Test
+  public void testWithTrailer() throws IOException {
+    doTest(true);
+  }
+
+  @Test
+  public void testWithoutTrailer() throws IOException {
+    doTest(false);
+  }
+
+  private Path getPath(int index) throws IOException {
+    String methodName = name.getMethodName().replaceAll("[^A-Za-z0-9_-]", "_");
+    return new Path(UTIL.getDataTestDirOnTestFS(), methodName + "-" + index);
+  }
+
+  private void doTest(boolean withTrailer) throws IOException {
+    int columnCount = 5;
+    int recordCount = 5;
+    TableName tableName = TableName.valueOf("tablename");
+    byte[] row = Bytes.toBytes("row");
+    long timestamp = System.currentTimeMillis();
+    Path path1 = getPath(1);
+    Path path2 = getPath(2);
+    FileSystem fs = UTIL.getTestFileSystem();
+    Configuration conf = UTIL.getConfiguration();
+    try (
+      AsyncWriter writer1 = AsyncFSWALProvider.createAsyncWriter(conf, fs, path1, false,
+        EVENT_LOOP_GROUP.next(), CHANNEL_CLASS);
+      AsyncWriter writer2 = AsyncFSWALProvider.createAsyncWriter(conf, fs, path2, false,
+        EVENT_LOOP_GROUP.next(), CHANNEL_CLASS);
+      CombinedAsyncWriter writer = CombinedAsyncWriter.create(mode, writer1, writer2)) {
+      ProtobufLogTestHelper.doWrite(new WriterOverAsyncWriter(writer), withTrailer, tableName,
+        columnCount, recordCount, row, timestamp);
+      try (ProtobufLogReader reader = (ProtobufLogReader) WALS.createReader(fs, path1)) {
+        ProtobufLogTestHelper.doRead(reader, withTrailer, tableName, columnCount, recordCount, row,
+          timestamp);
+      }
+      try (ProtobufLogReader reader = (ProtobufLogReader) WALS.createReader(fs, path2)) {
+        ProtobufLogTestHelper.doRead(reader, withTrailer, tableName, columnCount, recordCount, row,
+          timestamp);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/9eb748fc/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java
index 080b5be..d429a01 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestProtobufLog.java
@@ -23,14 +23,12 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.wal.FSHLogProvider;
-import org.apache.hadoop.hbase.wal.WAL.Entry;
-import org.apache.hadoop.hbase.wal.WALProvider;
 import org.apache.hadoop.hbase.wal.WALProvider.Writer;
 import org.junit.ClassRule;
 import org.junit.experimental.categories.Category;
 
 @Category({ RegionServerTests.class, MediumTests.class })
-public class TestProtobufLog extends AbstractTestProtobufLog<WALProvider.Writer> {
+public class TestProtobufLog extends AbstractTestProtobufLog {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
@@ -40,14 +38,4 @@ public class TestProtobufLog extends AbstractTestProtobufLog<WALProvider.Writer>
   protected Writer createWriter(Path path) throws IOException {
     return FSHLogProvider.createWriter(TEST_UTIL.getConfiguration(), fs, path, false);
   }
-
-  @Override
-  protected void append(Writer writer, Entry entry) throws IOException {
-    writer.append(entry);
-  }
-
-  @Override
-  protected void sync(Writer writer) throws IOException {
-    writer.sync();
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/9eb748fc/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/WriterOverAsyncWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/WriterOverAsyncWriter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/WriterOverAsyncWriter.java
new file mode 100644
index 0000000..da70ee5
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/WriterOverAsyncWriter.java
@@ -0,0 +1,63 @@
+/**
+ * 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.wal;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.concurrent.ExecutionException;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WALProvider;
+import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter;
+
+import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
+
+class WriterOverAsyncWriter implements WALProvider.Writer {
+
+  private final WALProvider.AsyncWriter asyncWriter;
+
+  public WriterOverAsyncWriter(AsyncWriter asyncWriter) {
+    this.asyncWriter = asyncWriter;
+  }
+
+  @Override
+  public void close() throws IOException {
+    asyncWriter.close();
+  }
+
+  @Override
+  public long getLength() {
+    return asyncWriter.getLength();
+  }
+
+  @Override
+  public void sync() throws IOException {
+    try {
+      asyncWriter.sync().get();
+    } catch (InterruptedException e) {
+      throw new InterruptedIOException();
+    } catch (ExecutionException e) {
+      Throwables.propagateIfPossible(e.getCause(), IOException.class);
+      throw new IOException(e.getCause());
+    }
+  }
+
+  @Override
+  public void append(Entry entry) throws IOException {
+    asyncWriter.append(entry);
+  }
+}


[12/28] hbase git commit: HBASE-19914 Refactor TestVisibilityLabelsOnNewVersionBehaviorTable

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/2e1ec3d3/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java
index 2e3d625..7a5232a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDeletes.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hbase.security.visibility;
 
-import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_NAME;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -27,17 +26,13 @@ import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Delete;
@@ -48,251 +43,64 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
 import org.junit.ClassRule;
-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;
 
-/**
- * Tests visibility labels with deletes
- */
-@Category({SecurityTests.class, MediumTests.class})
-public class TestVisibilityLabelsWithDeletes {
+@Category({ SecurityTests.class, LargeTests.class })
+public class TestVisibilityLabelsWithDeletes extends VisibilityLabelsWithDeletesTestBase {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestVisibilityLabelsWithDeletes.class);
-
-  private static final Logger LOG = LoggerFactory.getLogger(TestVisibilityLabelsWithDeletes.class);
-  private static final String TOPSECRET = "TOPSECRET";
-  private static final String PUBLIC = "PUBLIC";
-  private static final String PRIVATE = "PRIVATE";
-  private static final String CONFIDENTIAL = "CONFIDENTIAL";
-  private static final String SECRET = "SECRET";
-  public static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private static final byte[] row1 = Bytes.toBytes("row1");
-  private static final byte[] row2 = Bytes.toBytes("row2");
-  protected final static byte[] fam = Bytes.toBytes("info");
-  protected final static byte[] qual = Bytes.toBytes("qual");
-  private final static byte[] qual1 = Bytes.toBytes("qual1");
-  private final static byte[] qual2 = Bytes.toBytes("qual2");
-  protected final static byte[] value = Bytes.toBytes("value");
-  private final static byte[] value1 = Bytes.toBytes("value1");
-  public static Configuration conf;
-
-  @Rule
-  public final TestName TEST_NAME = new TestName();
-  public static User SUPERUSER;
-
-  @BeforeClass
-  public static void setupBeforeClass() throws Exception {
-    // setup configuration
-    conf = TEST_UTIL.getConfiguration();
-    VisibilityTestUtil.enableVisiblityLabels(conf);
-    conf.setClass(VisibilityUtils.VISIBILITY_LABEL_GENERATOR_CLASS, SimpleScanLabelGenerator.class,
-        ScanLabelGenerator.class);
-    conf.set("hbase.superuser", "admin");
-    TEST_UTIL.startMiniCluster(2);
-    SUPERUSER = User.createUserForTesting(conf, "admin", new String[] { "supergroup" });
-
-    // Wait for the labels table to become available
-    TEST_UTIL.waitTableEnabled(LABELS_TABLE_NAME.getName(), 50000);
-    addLabels();
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
-  }
+    HBaseClassTestRule.forClass(TestVisibilityLabelsWithDeletes.class);
 
-  @After
-  public void tearDown() throws Exception {
-  }
-
-  protected Table createTable(HColumnDescriptor fam) throws IOException {
-    TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    HTableDescriptor table = new HTableDescriptor(tableName);
-    table.addFamily(fam);
-    TEST_UTIL.getHBaseAdmin().createTable(table);
+  @Override
+  protected Table createTable(byte[] fam) throws IOException {
+    TableName tableName = TableName.valueOf(testName.getMethodName());
+    TEST_UTIL.getAdmin().createTable(TableDescriptorBuilder.newBuilder(tableName)
+        .addColumnFamily(ColumnFamilyDescriptorBuilder.of(fam)).build());
     return TEST_UTIL.getConnection().getTable(tableName);
   }
 
-  @Test
-  public void testVisibilityLabelsWithDeleteColumns() throws Throwable {
-    setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-
-    try (Table table = createTableAndWriteDataWithLabels(SECRET + "&" + TOPSECRET, SECRET)) {
-      PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
-        @Override
-        public Void run() throws Exception {
-          try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
-            Delete d = new Delete(row1);
-            d.setCellVisibility(new CellVisibility(TOPSECRET + "&" + SECRET));
-            d.addColumns(fam, qual);
-            table.delete(d);
-          } catch (Throwable t) {
-            throw new IOException(t);
-          }
-          return null;
-        }
-      };
-      SUPERUSER.runAs(actiona);
-
-      TEST_UTIL.getAdmin().flush(tableName);
-      Scan s = new Scan();
-      s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL));
-      ResultScanner scanner = table.getScanner(s);
-      Result[] next = scanner.next(3);
-      assertTrue(next.length == 1);
-      CellScanner cellScanner = next[0].cellScanner();
-      cellScanner.advance();
-      Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
-
-    }
+  private TableName createTable() throws IOException {
+    return createTable(-1);
   }
 
-  @Test
-  public void testVisibilityLabelsWithDeleteFamily() throws Exception {
-    setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    try (Table table = createTableAndWriteDataWithLabels(SECRET, CONFIDENTIAL + "|" + TOPSECRET)) {
-      PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
-        @Override
-        public Void run() throws Exception {
-          try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
-            Delete d = new Delete(row2);
-            d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL));
-            d.addFamily(fam);
-            table.delete(d);
-          } catch (Throwable t) {
-            throw new IOException(t);
-          }
-          return null;
-        }
-      };
-      SUPERUSER.runAs(actiona);
-
-      TEST_UTIL.getAdmin().flush(tableName);
-      Scan s = new Scan();
-      s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL));
-      ResultScanner scanner = table.getScanner(s);
-      Result[] next = scanner.next(3);
-      assertTrue(next.length == 1);
-      CellScanner cellScanner = next[0].cellScanner();
-      cellScanner.advance();
-      Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
-    }
+  private TableName createTable(int maxVersions) throws IOException {
+    TableName tableName = TableName.valueOf(testName.getMethodName());
+    createTable(tableName, maxVersions);
+    return tableName;
   }
 
-  @Test
-  public void testVisibilityLabelsWithDeleteFamilyVersion() throws Exception {
-    setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    long[] ts = new long[] { 123L, 125L };
-    try (Table table = createTableAndWriteDataWithLabels(ts,
-        CONFIDENTIAL + "|" + TOPSECRET, SECRET)) {
-      PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
-        @Override
-        public Void run() throws Exception {
-          try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
-            Delete d = new Delete(row1);
-            d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL));
-            d.addFamilyVersion(fam, 123L);
-            table.delete(d);
-          } catch (Throwable t) {
-            throw new IOException(t);
-          }
-          return null;
-        }
-      };
-      SUPERUSER.runAs(actiona);
-
-      TEST_UTIL.getAdmin().flush(tableName);
-      Scan s = new Scan();
-      s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL));
-      ResultScanner scanner = table.getScanner(s);
-      Result[] next = scanner.next(3);
-      assertTrue(next.length == 1);
-      CellScanner cellScanner = next[0].cellScanner();
-      cellScanner.advance();
-      Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
-    }
-  }
-
-  @Test
-  public void testVisibilityLabelsWithDeleteColumnExactVersion() throws Exception {
-    setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    long[] ts = new long[] { 123L, 125L };
-    try (Table table = createTableAndWriteDataWithLabels(ts,
-        CONFIDENTIAL + "|" + TOPSECRET, SECRET)) {
-      PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
-        @Override
-        public Void run() throws Exception {
-          try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
-            Delete d = new Delete(row1);
-            d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL));
-            d.addColumn(fam, qual, 123L);
-            table.delete(d);
-          } catch (Throwable t) {
-            throw new IOException(t);
-          }
-          return null;
-        }
-      };
-      SUPERUSER.runAs(actiona);
-
-      TEST_UTIL.getAdmin().flush(tableName);
-      Scan s = new Scan();
-      s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL));
-      ResultScanner scanner = table.getScanner(s);
-      Result[] next = scanner.next(3);
-      assertTrue(next.length == 1);
-      CellScanner cellScanner = next[0].cellScanner();
-      cellScanner.advance();
-      Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+  private void createTable(TableName tableName, int maxVersions) throws IOException {
+    ColumnFamilyDescriptorBuilder builder = ColumnFamilyDescriptorBuilder.newBuilder(fam);
+    if (maxVersions > 0) {
+      builder.setMaxVersions(maxVersions);
     }
+    TEST_UTIL.getAdmin().createTable(
+      TableDescriptorBuilder.newBuilder(tableName).addColumnFamily(builder.build()).build());
   }
 
   @Test
   public void testVisibilityLabelsWithDeleteColumnsWithMultipleVersions() throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
-            d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" +
-                SECRET + "&" + TOPSECRET+")"));
+            d.setCellVisibility(new CellVisibility(
+                "(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET + ")"));
             d.addColumns(fam, qual, 125L);
             table.delete(d);
           } catch (Throwable t) {
@@ -305,7 +113,7 @@ public class TestVisibilityLabelsWithDeletes {
 
       TEST_UTIL.getAdmin().flush(tableName);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -313,24 +121,24 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(127L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(126L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(125L, current.getTimestamp());
       cellScanner = next[1].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
     }
   }
 
@@ -338,14 +146,14 @@ public class TestVisibilityLabelsWithDeletes {
   public void testVisibilityLabelsWithDeleteColumnsWithMultipleVersionsNoTimestamp()
       throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d1 = new Delete(row1);
             d1.setCellVisibility(new CellVisibility(CONFIDENTIAL));
             d1.addColumns(fam, qual);
@@ -358,8 +166,8 @@ public class TestVisibilityLabelsWithDeletes {
             table.delete(d2);
 
             Delete d3 = new Delete(row1);
-            d3.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|("
-                + SECRET + "&" + TOPSECRET + ")"));
+            d3.setCellVisibility(new CellVisibility(
+                "(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET + ")"));
             d3.addColumns(fam, qual);
             table.delete(d3);
           } catch (Throwable t) {
@@ -370,7 +178,7 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -378,24 +186,23 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
     }
   }
 
   @Test
-  public void
-    testVisibilityLabelsWithDeleteColumnsWithNoMatchVisExpWithMultipleVersionsNoTimestamp()
+  public void testVisibilityLabelsWithDeleteColumnsNoMatchVisExpWithMultipleVersionsNoTimestamp()
       throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(CONFIDENTIAL));
             d.addColumns(fam, qual);
@@ -407,8 +214,8 @@ public class TestVisibilityLabelsWithDeletes {
             table.delete(d);
 
             d = new Delete(row1);
-            d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|("
-                + SECRET + "&" + TOPSECRET + ")"));
+            d.setCellVisibility(new CellVisibility(
+                "(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET + ")"));
             d.addColumns(fam, qual);
             table.delete(d);
           } catch (Throwable t) {
@@ -419,7 +226,7 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -427,13 +234,13 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       cellScanner = next[1].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
     }
   }
 
@@ -441,14 +248,14 @@ public class TestVisibilityLabelsWithDeletes {
   public void testVisibilityLabelsWithDeleteFamilyWithMultipleVersionsNoTimestamp()
       throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d1 = new Delete(row1);
             d1.setCellVisibility(new CellVisibility(CONFIDENTIAL));
             d1.addFamily(fam);
@@ -460,8 +267,8 @@ public class TestVisibilityLabelsWithDeletes {
             table.delete(d2);
 
             Delete d3 = new Delete(row1);
-            d3.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|("
-                + SECRET + "&" + TOPSECRET + ")"));
+            d3.setCellVisibility(new CellVisibility(
+                "(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET + ")"));
             d3.addFamily(fam);
             table.delete(d3);
           } catch (Throwable t) {
@@ -472,7 +279,7 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -480,19 +287,14 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
     }
   }
 
   @Test
   public void testDeleteColumnsWithoutAndWithVisibilityLabels() throws Exception {
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+    TableName tableName = createTable();
     try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
       Put put = new Put(row1);
       put.addColumn(fam, qual, value);
@@ -506,7 +308,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-              Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Scan s = new Scan();
             ResultScanner scanner = table.getScanner(s);
             Result[] next = scanner.next(3);
@@ -527,7 +329,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-              Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Scan s = new Scan();
             ResultScanner scanner = table.getScanner(s);
             Result[] next = scanner.next(3);
@@ -544,12 +346,7 @@ public class TestVisibilityLabelsWithDeletes {
 
   @Test
   public void testDeleteColumnsWithAndWithoutVisibilityLabels() throws Exception {
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+    TableName tableName = createTable();
     try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
       Put put = new Put(row1);
       put.addColumn(fam, qual, value);
@@ -564,7 +361,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-              Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Scan s = new Scan();
             ResultScanner scanner = table.getScanner(s);
             Result[] next = scanner.next(3);
@@ -584,7 +381,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-              Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Scan s = new Scan();
             ResultScanner scanner = table.getScanner(s);
             Result[] next = scanner.next(3);
@@ -601,12 +398,7 @@ public class TestVisibilityLabelsWithDeletes {
 
   @Test
   public void testDeleteFamiliesWithoutAndWithVisibilityLabels() throws Exception {
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+    TableName tableName = createTable();
     try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
       Put put = new Put(row1);
       put.addColumn(fam, qual, value);
@@ -620,7 +412,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-              Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Scan s = new Scan();
             ResultScanner scanner = table.getScanner(s);
             Result[] next = scanner.next(3);
@@ -641,7 +433,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-              Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Scan s = new Scan();
             ResultScanner scanner = table.getScanner(s);
             Result[] next = scanner.next(3);
@@ -658,12 +450,7 @@ public class TestVisibilityLabelsWithDeletes {
 
   @Test
   public void testDeleteFamiliesWithAndWithoutVisibilityLabels() throws Exception {
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+    TableName tableName = createTable();
     try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
       Put put = new Put(row1);
       put.addColumn(fam, qual, value);
@@ -678,7 +465,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-              Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Scan s = new Scan();
             ResultScanner scanner = table.getScanner(s);
             Result[] next = scanner.next(3);
@@ -698,7 +485,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-              Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Scan s = new Scan();
             ResultScanner scanner = table.getScanner(s);
             Result[] next = scanner.next(3);
@@ -715,12 +502,7 @@ public class TestVisibilityLabelsWithDeletes {
 
   @Test
   public void testDeletesWithoutAndWithVisibilityLabels() throws Exception {
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+    TableName tableName = createTable();
     try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
       Put put = new Put(row1);
       put.addColumn(fam, qual, value);
@@ -734,7 +516,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-              Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Scan s = new Scan();
             ResultScanner scanner = table.getScanner(s);
             // The delete would not be able to apply it because of visibility mismatch
@@ -756,7 +538,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-              Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Scan s = new Scan();
             ResultScanner scanner = table.getScanner(s);
             Result[] next = scanner.next(3);
@@ -774,13 +556,7 @@ public class TestVisibilityLabelsWithDeletes {
 
   @Test
   public void testVisibilityLabelsWithDeleteFamilyWithPutsReAppearing() throws Exception {
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    colDesc.setMaxVersions(5);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+    TableName tableName = createTable(5);
     try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
       Put put = new Put(Bytes.toBytes("row1"));
       put.addColumn(fam, qual, value);
@@ -795,7 +571,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(CONFIDENTIAL));
             d.addFamily(fam);
@@ -808,7 +584,7 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -821,7 +597,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(SECRET));
             d.addFamily(fam);
@@ -834,13 +610,13 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(CONFIDENTIAL));
       scanner = table.getScanner(s);
       next = scanner.next(3);
       assertEquals(1, next.length);
       s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET));
       scanner = table.getScanner(s);
       Result[] next1 = scanner.next(3);
@@ -850,13 +626,7 @@ public class TestVisibilityLabelsWithDeletes {
 
   @Test
   public void testVisibilityLabelsWithDeleteColumnsWithPutsReAppearing() throws Exception {
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    colDesc.setMaxVersions(5);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+    TableName tableName = createTable(5);
     try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
       Put put = new Put(Bytes.toBytes("row1"));
       put.addColumn(fam, qual, value);
@@ -871,7 +641,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(CONFIDENTIAL));
             d.addColumns(fam, qual);
@@ -884,7 +654,7 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -897,7 +667,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(SECRET));
             d.addColumns(fam, qual);
@@ -910,13 +680,13 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(CONFIDENTIAL));
       scanner = table.getScanner(s);
       next = scanner.next(3);
       assertEquals(1, next.length);
       s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET));
       scanner = table.getScanner(s);
       Result[] next1 = scanner.next(3);
@@ -926,13 +696,7 @@ public class TestVisibilityLabelsWithDeletes {
 
   @Test
   public void testVisibilityCombinations() throws Exception {
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    colDesc.setMaxVersions(5);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+    TableName tableName = createTable(5);
     try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
       Put put = new Put(Bytes.toBytes("row1"));
       put.addColumn(fam, qual, 123L, value);
@@ -946,7 +710,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(SECRET));
             d.addColumns(fam, qual, 126L);
@@ -956,7 +720,7 @@ public class TestVisibilityLabelsWithDeletes {
           }
 
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(CONFIDENTIAL));
             d.addColumn(fam, qual, 123L);
@@ -969,24 +733,18 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(CONFIDENTIAL, SECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
       assertEquals(0, next.length);
     }
   }
+
   @Test
   public void testVisibilityLabelsWithDeleteColumnWithSpecificVersionWithPutsReAppearing()
       throws Exception {
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    colDesc.setMaxVersions(5);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
-
+    TableName tableName = createTable(5);
     try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
       Put put1 = new Put(Bytes.toBytes("row1"));
       put1.addColumn(fam, qual, 123L, value);
@@ -998,7 +756,7 @@ public class TestVisibilityLabelsWithDeletes {
       table.put(createList(put1, put2));
 
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(CONFIDENTIAL, SECRET));
 
       ResultScanner scanner = table.getScanner(s);
@@ -1009,7 +767,7 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(CONFIDENTIAL));
             d.addColumn(fam, qual, 123L);
@@ -1017,7 +775,7 @@ public class TestVisibilityLabelsWithDeletes {
           }
 
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(SECRET));
             d.addColumn(fam, qual, 123L);
@@ -1030,7 +788,7 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(CONFIDENTIAL));
       scanner = table.getScanner(s);
       assertEquals(0, scanner.next(3).length);
@@ -1039,11 +797,10 @@ public class TestVisibilityLabelsWithDeletes {
   }
 
   @Test
-  public void
-    testVisibilityLabelsWithDeleteFamilyWithNoMatchingVisExpWithMultipleVersionsNoTimestamp()
+  public void testVisibilityLabelsWithDeleteFamilyNoMatchingVisExpWithMultipleVersionsNoTimestamp()
       throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
@@ -1058,12 +815,12 @@ public class TestVisibilityLabelsWithDeletes {
           d2.addFamily(fam);
 
           Delete d3 = new Delete(row1);
-          d3.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|("
-              + SECRET + "&" + TOPSECRET + ")"));
+          d3.setCellVisibility(new CellVisibility(
+              "(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET + ")"));
           d3.addFamily(fam);
 
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             table.delete(createList(d1, d2, d3));
           } catch (Throwable t) {
             throw new IOException(t);
@@ -1073,7 +830,7 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -1081,13 +838,13 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       cellScanner = next[1].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
       scanner.close();
     }
   }
@@ -1095,7 +852,7 @@ public class TestVisibilityLabelsWithDeletes {
   @Test
   public void testDeleteFamilyAndDeleteColumnsWithAndWithoutVisibilityExp() throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
@@ -1108,7 +865,7 @@ public class TestVisibilityLabelsWithDeletes {
           d2.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET));
           d2.addColumns(fam, qual);
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             table.delete(createList(d1, d2));
           } catch (Throwable t) {
             throw new IOException(t);
@@ -1118,7 +875,7 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -1126,41 +883,36 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(127L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(126L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(124L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(123L, current.getTimestamp());
       cellScanner = next[1].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
       scanner.close();
     }
   }
 
   private Table doPuts(TableName tableName) throws IOException, InterruptedIOException,
       RetriesExhaustedWithDetailsException, InterruptedException {
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    colDesc.setMaxVersions(5);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+    createTable(tableName, 5);
 
     List<Put> puts = new ArrayList<>(5);
     Put put = new Put(Bytes.toBytes("row1"));
@@ -1170,8 +922,8 @@ public class TestVisibilityLabelsWithDeletes {
 
     put = new Put(Bytes.toBytes("row1"));
     put.addColumn(fam, qual, 124L, value);
-    put.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|("
-    + TOPSECRET + "&" + SECRET+")"));
+    put.setCellVisibility(new CellVisibility(
+        "(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET + ")"));
     puts.add(put);
 
     put = new Put(Bytes.toBytes("row1"));
@@ -1181,21 +933,21 @@ public class TestVisibilityLabelsWithDeletes {
 
     put = new Put(Bytes.toBytes("row1"));
     put.addColumn(fam, qual, 126L, value);
-    put.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|("
-        + TOPSECRET + "&" + SECRET+")"));
+    put.setCellVisibility(new CellVisibility(
+        "(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET + ")"));
     puts.add(put);
 
     put = new Put(Bytes.toBytes("row1"));
     put.addColumn(fam, qual, 127L, value);
-    put.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|("
-        + TOPSECRET + "&" + SECRET+")"));
+    put.setCellVisibility(new CellVisibility(
+        "(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET + ")"));
     puts.add(put);
 
     TEST_UTIL.getAdmin().flush(tableName);
     put = new Put(Bytes.toBytes("row2"));
     put.addColumn(fam, qual, 127L, value);
-    put.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET
-        + "&" + SECRET + ")"));
+    put.setCellVisibility(new CellVisibility(
+        "(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET + ")"));
     puts.add(put);
 
     Table table = TEST_UTIL.getConnection().getTable(tableName);
@@ -1203,14 +955,9 @@ public class TestVisibilityLabelsWithDeletes {
     return table;
   }
 
-  private Table doPutsWithDiffCols(TableName tableName) throws IOException,
-      InterruptedIOException, RetriesExhaustedWithDetailsException, InterruptedException {
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    colDesc.setMaxVersions(5);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+  private Table doPutsWithDiffCols(TableName tableName) throws IOException, InterruptedIOException,
+      RetriesExhaustedWithDetailsException, InterruptedException {
+    createTable(tableName, 5);
 
     List<Put> puts = new ArrayList<>(5);
     Put put = new Put(Bytes.toBytes("row1"));
@@ -1220,8 +967,8 @@ public class TestVisibilityLabelsWithDeletes {
 
     put = new Put(Bytes.toBytes("row1"));
     put.addColumn(fam, qual, 124L, value);
-    put.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|("
-    + TOPSECRET + "&" + SECRET+")"));
+    put.setCellVisibility(new CellVisibility(
+        "(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET + ")"));
     puts.add(put);
 
     put = new Put(Bytes.toBytes("row1"));
@@ -1236,8 +983,8 @@ public class TestVisibilityLabelsWithDeletes {
 
     put = new Put(Bytes.toBytes("row1"));
     put.addColumn(fam, qual2, 127L, value);
-    put.setCellVisibility(new CellVisibility("(" + CONFIDENTIAL + "&" + PRIVATE + ")|("
-        + TOPSECRET + "&" + SECRET+")"));
+    put.setCellVisibility(new CellVisibility(
+        "(" + CONFIDENTIAL + "&" + PRIVATE + ")|(" + TOPSECRET + "&" + SECRET + ")"));
     puts.add(put);
 
     Table table = TEST_UTIL.getConnection().getTable(tableName);
@@ -1247,12 +994,7 @@ public class TestVisibilityLabelsWithDeletes {
 
   private Table doPutsWithoutVisibility(TableName tableName) throws IOException,
       InterruptedIOException, RetriesExhaustedWithDetailsException, InterruptedException {
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    colDesc.setMaxVersions(5);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+    createTable(tableName, 5);
     List<Put> puts = new ArrayList<>(5);
     Put put = new Put(Bytes.toBytes("row1"));
     put.addColumn(fam, qual, 123L, value);
@@ -1286,22 +1028,21 @@ public class TestVisibilityLabelsWithDeletes {
     return table;
   }
 
-
   @Test
   public void testDeleteColumnWithSpecificTimeStampUsingMultipleVersionsUnMatchingVisExpression()
       throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
-            d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" +
-                SECRET + "&" + TOPSECRET+")"));
+            d.setCellVisibility(new CellVisibility(
+                "(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET + ")"));
             d.addColumn(fam, qual, 125L);
             table.delete(d);
           } catch (Throwable t) {
@@ -1314,7 +1055,7 @@ public class TestVisibilityLabelsWithDeletes {
 
       TEST_UTIL.getAdmin().flush(tableName);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -1322,48 +1063,48 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(127L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(126L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(125L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(124L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(123L, current.getTimestamp());
       cellScanner = next[1].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
     }
   }
 
   @Test
   public void testDeleteColumnWithLatestTimeStampUsingMultipleVersions() throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET));
             d.addColumn(fam, qual);
@@ -1378,7 +1119,7 @@ public class TestVisibilityLabelsWithDeletes {
 
       TEST_UTIL.getAdmin().flush(tableName);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -1386,36 +1127,36 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(127L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(126L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(124L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(123L, current.getTimestamp());
       cellScanner = next[1].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
     }
   }
 
-  @Test (timeout=180000)
+  @Test(timeout = 180000)
   public void testDeleteColumnWithLatestTimeStampWhenNoVersionMatches() throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       Put put = new Put(Bytes.toBytes("row1"));
@@ -1426,9 +1167,9 @@ public class TestVisibilityLabelsWithDeletes {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
-            d.setCellVisibility(new CellVisibility(SECRET ));
+            d.setCellVisibility(new CellVisibility(SECRET));
             d.addColumn(fam, qual);
             table.delete(d);
           } catch (Throwable t) {
@@ -1441,7 +1182,7 @@ public class TestVisibilityLabelsWithDeletes {
 
       TEST_UTIL.getAdmin().flush(tableName);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -1449,34 +1190,34 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(128L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(127L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(126L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(125L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(124L, current.getTimestamp());
       cellScanner = next[1].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
 
       put = new Put(Bytes.toBytes("row1"));
       put.addColumn(fam, qual, 129L, value);
@@ -1485,7 +1226,7 @@ public class TestVisibilityLabelsWithDeletes {
 
       TEST_UTIL.getAdmin().flush(tableName);
       s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       scanner = table.getScanner(s);
       next = scanner.next(3);
@@ -1493,23 +1234,24 @@ public class TestVisibilityLabelsWithDeletes {
       cellScanner = next[0].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(129L, current.getTimestamp());
     }
   }
+
   @Test
   public void testDeleteColumnWithLatestTimeStampUsingMultipleVersionsAfterCompaction()
       throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET));
             d.addColumn(fam, qual);
@@ -1531,7 +1273,7 @@ public class TestVisibilityLabelsWithDeletes {
       // Sleep to ensure compaction happens. Need to do it in a better way
       Thread.sleep(5000);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -1539,43 +1281,43 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(127L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(126L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(124L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(123L, current.getTimestamp());
       cellScanner = next[1].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
     }
   }
 
   @Test
   public void testDeleteFamilyLatestTimeStampWithMulipleVersions() throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET));
             d.addFamily(fam);
@@ -1590,7 +1332,7 @@ public class TestVisibilityLabelsWithDeletes {
 
       TEST_UTIL.getAdmin().flush(tableName);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -1598,26 +1340,26 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(127L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(126L, current.getTimestamp());
       cellScanner = next[1].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
     }
   }
 
   @Test
   public void testDeleteColumnswithMultipleColumnsWithMultipleVersions() throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPutsWithDiffCols(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
@@ -1627,7 +1369,7 @@ public class TestVisibilityLabelsWithDeletes {
           d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET));
           d.addColumns(fam, qual, 125L);
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             table.delete(d);
           } catch (Throwable t) {
             throw new IOException(t);
@@ -1639,7 +1381,7 @@ public class TestVisibilityLabelsWithDeletes {
 
       TEST_UTIL.getAdmin().flush(tableName);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -1647,40 +1389,34 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(124L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(123L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertTrue(Bytes.equals(current.getQualifierArray(), current.getQualifierOffset(),
-          current.getQualifierLength(), qual1, 0, qual1.length));
+        current.getQualifierLength(), qual1, 0, qual1.length));
       assertEquals(126L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(127L, current.getTimestamp());
       assertTrue(Bytes.equals(current.getQualifierArray(), current.getQualifierOffset(),
-          current.getQualifierLength(), qual2, 0, qual2.length));
+        current.getQualifierLength(), qual2, 0, qual2.length));
     }
   }
 
   @Test
   public void testDeleteColumnsWithDiffColsAndTags() throws Exception {
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    colDesc.setMaxVersions(5);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+    TableName tableName = createTable(5);
     try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
       Put put = new Put(Bytes.toBytes("row1"));
       put.addColumn(fam, qual1, 125L, value);
@@ -1703,7 +1439,7 @@ public class TestVisibilityLabelsWithDeletes {
           d2.addColumns(fam, qual1, 125L);
 
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             table.delete(createList(d1, d2));
           } catch (Throwable t) {
             throw new IOException(t);
@@ -1713,22 +1449,17 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, CONFIDENTIAL));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
       assertEquals(1, next.length);
     }
   }
+
   @Test
   public void testDeleteColumnsWithDiffColsAndTags1() throws Exception {
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    Admin hBaseAdmin = TEST_UTIL.getAdmin();
-    HColumnDescriptor colDesc = new HColumnDescriptor(fam);
-    colDesc.setMaxVersions(5);
-    HTableDescriptor desc = new HTableDescriptor(tableName);
-    desc.addFamily(colDesc);
-    hBaseAdmin.createTable(desc);
+    TableName tableName = createTable(5);
     try (Table table = TEST_UTIL.getConnection().getTable(tableName)) {
       Put put = new Put(Bytes.toBytes("row1"));
       put.addColumn(fam, qual1, 125L, value);
@@ -1751,7 +1482,7 @@ public class TestVisibilityLabelsWithDeletes {
           d2.addColumns(fam, qual1, 126L);
 
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             table.delete(createList(d1, d2));
           } catch (Throwable t) {
             throw new IOException(t);
@@ -1761,24 +1492,25 @@ public class TestVisibilityLabelsWithDeletes {
       };
       SUPERUSER.runAs(actiona);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, CONFIDENTIAL));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
       assertEquals(1, next.length);
     }
   }
+
   @Test
   public void testDeleteFamilyWithoutCellVisibilityWithMulipleVersions() throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPutsWithoutVisibility(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.addFamily(fam);
             table.delete(d);
@@ -1792,7 +1524,7 @@ public class TestVisibilityLabelsWithDeletes {
 
       TEST_UTIL.getAdmin().flush(tableName);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -1801,8 +1533,8 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
     }
   }
 
@@ -1810,13 +1542,13 @@ public class TestVisibilityLabelsWithDeletes {
   public void testDeleteFamilyLatestTimeStampWithMulipleVersionsWithoutCellVisibilityInPuts()
       throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPutsWithoutVisibility(tableName)) {
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
             d.setCellVisibility(new CellVisibility(SECRET + "&" + TOPSECRET));
             d.addFamily(fam);
@@ -1830,7 +1562,7 @@ public class TestVisibilityLabelsWithDeletes {
       SUPERUSER.runAs(actiona);
       TEST_UTIL.getAdmin().flush(tableName);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -1838,51 +1570,51 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(127L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(126L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(125L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(124L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(123L, current.getTimestamp());
       cellScanner = next[1].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
     }
   }
 
   @Test
   public void testDeleteFamilySpecificTimeStampWithMulipleVersions() throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
-            d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|("
-                + SECRET + "&" + TOPSECRET + ")"));
+            d.setCellVisibility(new CellVisibility(
+                "(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET + ")"));
             d.addFamily(fam, 126L);
             table.delete(d);
           } catch (Throwable t) {
@@ -1895,7 +1627,7 @@ public class TestVisibilityLabelsWithDeletes {
 
       TEST_UTIL.getAdmin().flush(tableName);
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(6);
@@ -1903,41 +1635,41 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(127L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(125L, current.getTimestamp());
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(123L, current.getTimestamp());
       cellScanner = next[1].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
     }
   }
 
   @Test
   public void testScanAfterCompaction() throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     try (Table table = doPuts(tableName)) {
       TEST_UTIL.getAdmin().flush(tableName);
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
-            d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" +
-                SECRET + "&" + TOPSECRET+")"));
+            d.setCellVisibility(new CellVisibility(
+                "(" + PRIVATE + "&" + CONFIDENTIAL + ")|(" + SECRET + "&" + TOPSECRET + ")"));
             d.addFamily(fam, 126L);
             table.delete(d);
           } catch (Throwable t) {
@@ -1958,7 +1690,7 @@ public class TestVisibilityLabelsWithDeletes {
       Thread.sleep(5000);
       // Sleep to ensure compaction happens. Need to do it in a better way
       Scan s = new Scan();
-      s.setMaxVersions(5);
+      s.readVersions(5);
       s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL, TOPSECRET));
       ResultScanner scanner = table.getScanner(s);
       Result[] next = scanner.next(3);
@@ -1966,31 +1698,31 @@ public class TestVisibilityLabelsWithDeletes {
       CellScanner cellScanner = next[0].cellScanner();
       cellScanner.advance();
       Cell current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row1, 0, row1.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
       assertEquals(127L, current.getTimestamp());
       cellScanner = next[1].cellScanner();
       cellScanner.advance();
       current = cellScanner.current();
-      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(),
-          current.getRowLength(), row2, 0, row2.length));
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
     }
   }
 
   @Test
   public void testDeleteFamilySpecificTimeStampWithMulipleVersionsDoneTwice() throws Exception {
     setAuths();
-    final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
     // Do not flush here.
     try (Table table = doPuts(tableName)) {
       PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
         @Override
         public Void run() throws Exception {
           try (Connection connection = ConnectionFactory.createConnection(conf);
-               Table table = connection.getTable(tableName)) {
+            Table table = connection.getTable(tableName)) {
             Delete d = new Delete(row1);
-            d.setCellVisibility(new CellVisibility("(" + PRIVATE + "&" + CONFIDENTIAL + ")|("
-                + TOPSECRET + "&" 

<TRUNCATED>

[11/28] hbase git commit: HBASE-19914 Refactor TestVisibilityLabelsOnNewVersionBehaviorTable

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/2e1ec3d3/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsWithDeletesTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsWithDeletesTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsWithDeletesTestBase.java
new file mode 100644
index 0000000..ee5d3cc
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelsWithDeletesTestBase.java
@@ -0,0 +1,313 @@
+/**
+ * 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.security.visibility;
+
+import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_NAME;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Delete;
+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.client.Table;
+import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+/**
+ * Tests visibility labels with deletes
+ */
+public abstract class VisibilityLabelsWithDeletesTestBase {
+
+  protected static final String TOPSECRET = "TOPSECRET";
+  protected static final String PUBLIC = "PUBLIC";
+  protected static final String PRIVATE = "PRIVATE";
+  protected static final String CONFIDENTIAL = "CONFIDENTIAL";
+  protected static final String SECRET = "SECRET";
+  protected static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  protected static final byte[] row1 = Bytes.toBytes("row1");
+  protected static final byte[] row2 = Bytes.toBytes("row2");
+  protected final static byte[] fam = Bytes.toBytes("info");
+  protected final static byte[] qual = Bytes.toBytes("qual");
+  protected final static byte[] qual1 = Bytes.toBytes("qual1");
+  protected final static byte[] qual2 = Bytes.toBytes("qual2");
+  protected final static byte[] value = Bytes.toBytes("value");
+  protected final static byte[] value1 = Bytes.toBytes("value1");
+  protected static Configuration conf;
+
+  @Rule
+  public final TestName testName = new TestName();
+  protected static User SUPERUSER;
+
+  @BeforeClass
+  public static void setupBeforeClass() throws Exception {
+    // setup configuration
+    conf = TEST_UTIL.getConfiguration();
+    VisibilityTestUtil.enableVisiblityLabels(conf);
+    conf.setClass(VisibilityUtils.VISIBILITY_LABEL_GENERATOR_CLASS, SimpleScanLabelGenerator.class,
+      ScanLabelGenerator.class);
+    conf.set("hbase.superuser", "admin");
+    TEST_UTIL.startMiniCluster(2);
+    SUPERUSER = User.createUserForTesting(conf, "admin", new String[] { "supergroup" });
+
+    // Wait for the labels table to become available
+    TEST_UTIL.waitTableEnabled(LABELS_TABLE_NAME.getName(), 50000);
+    addLabels();
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  public static void addLabels() throws Exception {
+    PrivilegedExceptionAction<VisibilityLabelsResponse> action =
+      new PrivilegedExceptionAction<VisibilityLabelsResponse>() {
+        @Override
+        public VisibilityLabelsResponse run() throws Exception {
+          String[] labels = { SECRET, TOPSECRET, CONFIDENTIAL, PUBLIC, PRIVATE };
+          try (Connection conn = ConnectionFactory.createConnection(conf)) {
+            VisibilityClient.addLabels(conn, labels);
+          } catch (Throwable t) {
+            throw new IOException(t);
+          }
+          return null;
+        }
+      };
+    SUPERUSER.runAs(action);
+  }
+
+  protected abstract Table createTable(byte[] fam) throws IOException;
+
+  protected final void setAuths() throws IOException, InterruptedException {
+    PrivilegedExceptionAction<VisibilityLabelsResponse> action =
+      new PrivilegedExceptionAction<VisibilityLabelsResponse>() {
+        @Override
+        public VisibilityLabelsResponse run() throws Exception {
+          try (Connection conn = ConnectionFactory.createConnection(conf)) {
+            return VisibilityClient.setAuths(conn,
+              new String[] { CONFIDENTIAL, PRIVATE, SECRET, TOPSECRET }, SUPERUSER.getShortName());
+          } catch (Throwable e) {
+          }
+          return null;
+        }
+      };
+    SUPERUSER.runAs(action);
+  }
+
+  private Table createTableAndWriteDataWithLabels(String... labelExps) throws Exception {
+    Table table = createTable(fam);
+    int i = 1;
+    List<Put> puts = new ArrayList<>(labelExps.length);
+    for (String labelExp : labelExps) {
+      Put put = new Put(Bytes.toBytes("row" + i));
+      put.addColumn(fam, qual, HConstants.LATEST_TIMESTAMP, value);
+      put.setCellVisibility(new CellVisibility(labelExp));
+      puts.add(put);
+      table.put(put);
+      i++;
+    }
+    // table.put(puts);
+    return table;
+  }
+
+  private Table createTableAndWriteDataWithLabels(long[] timestamp, String... labelExps)
+      throws Exception {
+    Table table = createTable(fam);
+    int i = 1;
+    List<Put> puts = new ArrayList<>(labelExps.length);
+    for (String labelExp : labelExps) {
+      Put put = new Put(Bytes.toBytes("row" + i));
+      put.addColumn(fam, qual, timestamp[i - 1], value);
+      put.setCellVisibility(new CellVisibility(labelExp));
+      puts.add(put);
+      table.put(put);
+      TEST_UTIL.getAdmin().flush(table.getName());
+      i++;
+    }
+    return table;
+  }
+
+  @Test
+  public void testVisibilityLabelsWithDeleteColumns() throws Throwable {
+    setAuths();
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
+
+    try (Table table = createTableAndWriteDataWithLabels(SECRET + "&" + TOPSECRET, SECRET)) {
+      PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          try (Connection connection = ConnectionFactory.createConnection(conf);
+            Table table = connection.getTable(tableName)) {
+            Delete d = new Delete(row1);
+            d.setCellVisibility(new CellVisibility(TOPSECRET + "&" + SECRET));
+            d.addColumns(fam, qual);
+            table.delete(d);
+          } catch (Throwable t) {
+            throw new IOException(t);
+          }
+          return null;
+        }
+      };
+      SUPERUSER.runAs(actiona);
+
+      TEST_UTIL.getAdmin().flush(tableName);
+      Scan s = new Scan();
+      s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL));
+      ResultScanner scanner = table.getScanner(s);
+      Result[] next = scanner.next(3);
+      assertTrue(next.length == 1);
+      CellScanner cellScanner = next[0].cellScanner();
+      cellScanner.advance();
+      Cell current = cellScanner.current();
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
+
+    }
+  }
+
+  @Test
+  public void testVisibilityLabelsWithDeleteFamily() throws Exception {
+    setAuths();
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
+    try (Table table = createTableAndWriteDataWithLabels(SECRET, CONFIDENTIAL + "|" + TOPSECRET)) {
+      PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          try (Connection connection = ConnectionFactory.createConnection(conf);
+            Table table = connection.getTable(tableName)) {
+            Delete d = new Delete(row2);
+            d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL));
+            d.addFamily(fam);
+            table.delete(d);
+          } catch (Throwable t) {
+            throw new IOException(t);
+          }
+          return null;
+        }
+      };
+      SUPERUSER.runAs(actiona);
+
+      TEST_UTIL.getAdmin().flush(tableName);
+      Scan s = new Scan();
+      s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL));
+      ResultScanner scanner = table.getScanner(s);
+      Result[] next = scanner.next(3);
+      assertTrue(next.length == 1);
+      CellScanner cellScanner = next[0].cellScanner();
+      cellScanner.advance();
+      Cell current = cellScanner.current();
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row1, 0, row1.length));
+    }
+  }
+
+  @Test
+  public void testVisibilityLabelsWithDeleteFamilyVersion() throws Exception {
+    setAuths();
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
+    long[] ts = new long[] { 123L, 125L };
+    try (
+      Table table = createTableAndWriteDataWithLabels(ts, CONFIDENTIAL + "|" + TOPSECRET, SECRET)) {
+      PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          try (Connection connection = ConnectionFactory.createConnection(conf);
+            Table table = connection.getTable(tableName)) {
+            Delete d = new Delete(row1);
+            d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL));
+            d.addFamilyVersion(fam, 123L);
+            table.delete(d);
+          } catch (Throwable t) {
+            throw new IOException(t);
+          }
+          return null;
+        }
+      };
+      SUPERUSER.runAs(actiona);
+
+      TEST_UTIL.getAdmin().flush(tableName);
+      Scan s = new Scan();
+      s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL));
+      ResultScanner scanner = table.getScanner(s);
+      Result[] next = scanner.next(3);
+      assertTrue(next.length == 1);
+      CellScanner cellScanner = next[0].cellScanner();
+      cellScanner.advance();
+      Cell current = cellScanner.current();
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
+    }
+  }
+
+  @Test
+  public void testVisibilityLabelsWithDeleteColumnExactVersion() throws Exception {
+    setAuths();
+    final TableName tableName = TableName.valueOf(testName.getMethodName());
+    long[] ts = new long[] { 123L, 125L };
+    try (
+      Table table = createTableAndWriteDataWithLabels(ts, CONFIDENTIAL + "|" + TOPSECRET, SECRET)) {
+      PrivilegedExceptionAction<Void> actiona = new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws Exception {
+          try (Connection connection = ConnectionFactory.createConnection(conf);
+            Table table = connection.getTable(tableName)) {
+            Delete d = new Delete(row1);
+            d.setCellVisibility(new CellVisibility(TOPSECRET + "|" + CONFIDENTIAL));
+            d.addColumn(fam, qual, 123L);
+            table.delete(d);
+          } catch (Throwable t) {
+            throw new IOException(t);
+          }
+          return null;
+        }
+      };
+      SUPERUSER.runAs(actiona);
+
+      TEST_UTIL.getAdmin().flush(tableName);
+      Scan s = new Scan();
+      s.setAuthorizations(new Authorizations(SECRET, PRIVATE, CONFIDENTIAL));
+      ResultScanner scanner = table.getScanner(s);
+      Result[] next = scanner.next(3);
+      assertTrue(next.length == 1);
+      CellScanner cellScanner = next[0].cellScanner();
+      cellScanner.advance();
+      Cell current = cellScanner.current();
+      assertTrue(Bytes.equals(current.getRowArray(), current.getRowOffset(), current.getRowLength(),
+        row2, 0, row2.length));
+    }
+  }
+}


[14/28] hbase git commit: HBASE-19726 Failed to start HMaster due to infinite retrying on meta assign; ADDENDUM Fix failing TestMetaWithReplicas#testShutdownHandling; it was reading meta TableState

Posted by zh...@apache.org.
HBASE-19726 Failed to start HMaster due to infinite retrying on meta assign; ADDENDUM Fix failing TestMetaWithReplicas#testShutdownHandling; it was reading meta TableState


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

Branch: refs/heads/HBASE-19064
Commit: b0e998f2a50a50a8d84daa35baff1d4ac99d1c6a
Parents: 2e1ec3d
Author: Michael Stack <st...@apache.org>
Authored: Sat Feb 3 21:49:42 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Sat Feb 3 21:51:50 2018 -0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java  | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b0e998f2/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index f80bbc0..5dc0565 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -1109,6 +1109,9 @@ public class MetaTableAccessor {
   @Nullable
   public static TableState getTableState(Connection conn, TableName tableName)
       throws IOException {
+    if (tableName.equals(TableName.META_TABLE_NAME)) {
+      return new TableState(tableName, TableState.State.ENABLED);
+    }
     Table metaHTable = getMetaHTable(conn);
     Get get = new Get(tableName.getName()).addColumn(getTableFamily(), getTableStateColumn());
     long time = EnvironmentEdgeManager.currentTime();


[25/28] hbase git commit: HBASE-19864 Use protobuf instead of enum.ordinal to store SyncReplicationState

Posted by zh...@apache.org.
HBASE-19864 Use protobuf instead of enum.ordinal to store SyncReplicationState

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/8a79f26c
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/8a79f26c
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/8a79f26c

Branch: refs/heads/HBASE-19064
Commit: 8a79f26c014c37a6dcd82b9c3674acd51cef77b7
Parents: 0b0ecbe
Author: Guanghao Zhang <zg...@apache.org>
Authored: Fri Jan 26 16:50:48 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Feb 5 12:10:54 2018 +0800

----------------------------------------------------------------------
 .../replication/ReplicationPeerConfigUtil.java  | 22 ++++++++++++++---
 .../hbase/replication/SyncReplicationState.java | 17 +++++++++++++
 .../hbase/shaded/protobuf/RequestConverter.java |  7 +++---
 .../src/main/protobuf/Replication.proto         | 13 ++++++----
 .../replication/ZKReplicationPeerStorage.java   | 25 +++++++++-----------
 .../hadoop/hbase/master/MasterRpcServices.java  |  9 ++++---
 ...ransitPeerSyncReplicationStateProcedure.java |  9 ++++---
 .../TestReplicationSourceManager.java           |  2 +-
 8 files changed, 67 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8a79f26c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
index 86b49ea..5096824 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
@@ -398,7 +398,7 @@ public final class ReplicationPeerConfigUtil {
         ReplicationProtos.ReplicationState.State.ENABLED == desc.getState().getState();
     ReplicationPeerConfig config = convert(desc.getConfig());
     return new ReplicationPeerDescription(desc.getId(), enabled, config,
-        SyncReplicationState.valueOf(desc.getSyncReplicationState().getNumber()));
+      toSyncReplicationState(desc.getSyncReplicationState()));
   }
 
   public static ReplicationProtos.ReplicationPeerDescription
@@ -406,17 +406,33 @@ public final class ReplicationPeerConfigUtil {
     ReplicationProtos.ReplicationPeerDescription.Builder builder =
         ReplicationProtos.ReplicationPeerDescription.newBuilder();
     builder.setId(desc.getPeerId());
+
     ReplicationProtos.ReplicationState.Builder stateBuilder =
         ReplicationProtos.ReplicationState.newBuilder();
     stateBuilder.setState(desc.isEnabled() ? ReplicationProtos.ReplicationState.State.ENABLED :
         ReplicationProtos.ReplicationState.State.DISABLED);
     builder.setState(stateBuilder.build());
+
     builder.setConfig(convert(desc.getPeerConfig()));
-    builder.setSyncReplicationState(
-      ReplicationProtos.SyncReplicationState.forNumber(desc.getSyncReplicationState().ordinal()));
+    builder.setSyncReplicationState(toSyncReplicationState(desc.getSyncReplicationState()));
+
     return builder.build();
   }
 
+  public static ReplicationProtos.SyncReplicationState
+      toSyncReplicationState(SyncReplicationState state) {
+    ReplicationProtos.SyncReplicationState.Builder syncReplicationStateBuilder =
+        ReplicationProtos.SyncReplicationState.newBuilder();
+    syncReplicationStateBuilder
+        .setState(ReplicationProtos.SyncReplicationState.State.forNumber(state.ordinal()));
+    return syncReplicationStateBuilder.build();
+  }
+
+  public static SyncReplicationState
+      toSyncReplicationState(ReplicationProtos.SyncReplicationState state) {
+    return SyncReplicationState.valueOf(state.getState().getNumber());
+  }
+
   public static ReplicationPeerConfig appendTableCFsToReplicationPeerConfig(
       Map<TableName, List<String>> tableCfs, ReplicationPeerConfig peerConfig) {
     ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder(peerConfig);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8a79f26c/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java
index bd144e9..a65b144 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java
@@ -17,8 +17,15 @@
  */
 package org.apache.hadoop.hbase.replication;
 
+import java.util.Arrays;
+import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.yetus.audience.InterfaceAudience;
 
+import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+
 /**
  * Used by synchronous replication. Indicate the state of the current cluster in a synchronous
  * replication peer. The state may be one of {@link SyncReplicationState#ACTIVE},
@@ -45,4 +52,14 @@ public enum SyncReplicationState {
         throw new IllegalArgumentException("Unknown synchronous replication state " + value);
     }
   }
+
+  public static byte[] toByteArray(SyncReplicationState state) {
+    return ProtobufUtil
+        .prependPBMagic(ReplicationPeerConfigUtil.toSyncReplicationState(state).toByteArray());
+  }
+
+  public static SyncReplicationState parseFrom(byte[] bytes) throws InvalidProtocolBufferException {
+    return ReplicationPeerConfigUtil.toSyncReplicationState(ReplicationProtos.SyncReplicationState
+        .parseFrom(Arrays.copyOfRange(bytes, ProtobufUtil.lengthOfPBMagic(), bytes.length)));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8a79f26c/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index 659be2a..6ebcc89 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -1878,10 +1878,9 @@ public final class RequestConverter {
   }
 
   public static TransitReplicationPeerSyncReplicationStateRequest
-    buildTransitReplicationPeerSyncReplicationStateRequest(String peerId,
-      SyncReplicationState state) {
+      buildTransitReplicationPeerSyncReplicationStateRequest(String peerId,
+          SyncReplicationState state) {
     return TransitReplicationPeerSyncReplicationStateRequest.newBuilder().setPeerId(peerId)
-      .setSyncReplicationState(ReplicationProtos.SyncReplicationState.forNumber(state.ordinal()))
-      .build();
+        .setSyncReplicationState(ReplicationPeerConfigUtil.toSyncReplicationState(state)).build();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8a79f26c/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 de7b742..82a242d 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Replication.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Replication.proto
@@ -65,11 +65,14 @@ message ReplicationState {
 /**
  * Indicate the state of the current cluster in a synchronous replication peer.
  */
-enum SyncReplicationState {
-  NONE = 0;
-  ACTIVE = 1;
-  DOWNGRADE_ACTIVE = 2;
-  STANDBY = 3;
+message SyncReplicationState {
+  enum State {
+    NONE = 0;
+    ACTIVE = 1;
+    DOWNGRADE_ACTIVE = 2;
+    STANDBY = 3;
+  }
+  required State state = 1;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/8a79f26c/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
index 338ce3f..909daa0 100644
--- 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
@@ -17,12 +17,12 @@
  */
 package org.apache.hadoop.hbase.replication;
 
+import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
 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.Bytes;
 import org.apache.hadoop.hbase.util.CollectionUtils;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
@@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 
 /**
@@ -96,7 +95,7 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase
       ZKUtilOp.createAndFailSilent(getPeerStateNode(peerId),
         enabled ? ENABLED_ZNODE_BYTES : DISABLED_ZNODE_BYTES),
       ZKUtilOp.createAndFailSilent(getSyncReplicationStateNode(peerId),
-        Bytes.toBytes(syncReplicationState.ordinal())));
+        SyncReplicationState.toByteArray(syncReplicationState)));
     try {
       ZKUtil.createWithParents(zookeeper, peersZNode);
       ZKUtil.multiOrSequential(zookeeper, multiOps, false);
@@ -179,29 +178,27 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase
   }
 
   @Override
-  public void setPeerSyncReplicationState(String peerId, SyncReplicationState clusterState)
+  public void setPeerSyncReplicationState(String peerId, SyncReplicationState state)
       throws ReplicationException {
-    byte[] clusterStateBytes = Bytes.toBytes(clusterState.ordinal());
     try {
-      ZKUtil.setData(zookeeper, getSyncReplicationStateNode(peerId), clusterStateBytes);
+      ZKUtil.setData(zookeeper, getSyncReplicationStateNode(peerId),
+        SyncReplicationState.toByteArray(state));
     } catch (KeeperException e) {
       throw new ReplicationException(
-          "Unable to change the cluster state for the synchronous replication peer with id=" +
-              peerId,
-          e);
+        "Unable to change the cluster state for the synchronous replication peer with id=" + peerId,
+        e);
     }
   }
 
   @Override
   public SyncReplicationState getPeerSyncReplicationState(String peerId)
       throws ReplicationException {
-    byte[] data;
     try {
-      data = ZKUtil.getData(zookeeper, getSyncReplicationStateNode(peerId));
-    } catch (KeeperException | InterruptedException e) {
+      byte[] data = ZKUtil.getData(zookeeper, getSyncReplicationStateNode(peerId));
+      return SyncReplicationState.parseFrom(data);
+    } catch (KeeperException | InterruptedException | IOException e) {
       throw new ReplicationException(
-          "Error getting cluster state for the synchronous replication peer with id=" + peerId, e);
+        "Error getting cluster state for the synchronous replication peer with id=" + peerId, e);
     }
-    return SyncReplicationState.valueOf(Bytes.toInt(data));
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8a79f26c/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 f5bf117..e19d6ee 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
@@ -83,7 +83,6 @@ import org.apache.hadoop.hbase.regionserver.RpcSchedulerFactory;
 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.SyncReplicationState;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.AccessChecker;
 import org.apache.hadoop.hbase.security.access.AccessController;
@@ -1962,13 +1961,13 @@ public class MasterRpcServices extends RSRpcServices
 
   @Override
   public TransitReplicationPeerSyncReplicationStateResponse
-    transitReplicationPeerSyncReplicationState(RpcController controller,
-      TransitReplicationPeerSyncReplicationStateRequest request) throws ServiceException {
+      transitReplicationPeerSyncReplicationState(RpcController controller,
+          TransitReplicationPeerSyncReplicationStateRequest request) throws ServiceException {
     try {
       long procId = master.transitReplicationPeerSyncReplicationState(request.getPeerId(),
-        SyncReplicationState.valueOf(request.getSyncReplicationState().getNumber()));
+        ReplicationPeerConfigUtil.toSyncReplicationState(request.getSyncReplicationState()));
       return TransitReplicationPeerSyncReplicationStateResponse.newBuilder().setProcId(procId)
-        .build();
+          .build();
     } catch (ReplicationException | IOException e) {
       throw new ServiceException(e);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8a79f26c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java
index d26eecc..aad3b06 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
 
+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.master.procedure.ProcedurePrepareLatch;
@@ -33,7 +34,6 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationState;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TransitPeerSyncReplicationStateStateData;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
 
 /**
  * The procedure for transit current cluster state for a synchronous replication peer.
@@ -89,16 +89,15 @@ public class TransitPeerSyncReplicationStateProcedure extends ModifyPeerProcedur
   protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
     super.serializeStateData(serializer);
     serializer.serialize(TransitPeerSyncReplicationStateStateData.newBuilder()
-      .setSyncReplicationState(ReplicationProtos.SyncReplicationState.forNumber(state.ordinal()))
-      .build());
+        .setSyncReplicationState(ReplicationPeerConfigUtil.toSyncReplicationState(state)).build());
   }
 
   @Override
   protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
     super.deserializeStateData(serializer);
     TransitPeerSyncReplicationStateStateData data =
-      serializer.deserialize(TransitPeerSyncReplicationStateStateData.class);
-    state = SyncReplicationState.valueOf(data.getSyncReplicationState().getNumber());
+        serializer.deserialize(TransitPeerSyncReplicationStateStateData.class);
+    state = ReplicationPeerConfigUtil.toSyncReplicationState(data.getSyncReplicationState());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/8a79f26c/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 1204c9b..3d7cdaf 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
@@ -174,7 +174,7 @@ public abstract class TestReplicationSourceManager {
       ZKReplicationPeerStorage.ENABLED_ZNODE_BYTES);
     ZKUtil.createWithParents(zkw, "/hbase/replication/peers/1/sync-rep-state");
     ZKUtil.setData(zkw, "/hbase/replication/peers/1/sync-rep-state",
-      Bytes.toBytes(SyncReplicationState.NONE.ordinal()));
+      SyncReplicationState.toByteArray(SyncReplicationState.NONE));
     ZKUtil.createWithParents(zkw, "/hbase/replication/state");
     ZKUtil.setData(zkw, "/hbase/replication/state", ZKReplicationPeerStorage.ENABLED_ZNODE_BYTES);
 


[05/28] hbase git commit: HBASE-19726 Failed to start HMaster due to infinite retrying on meta assign

Posted by zh...@apache.org.
HBASE-19726 Failed to start HMaster due to infinite retrying on meta assign


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

Branch: refs/heads/HBASE-19064
Commit: 41974efa85a73f1967fc57c8e7df7b1344d56714
Parents: 811afad
Author: Michael Stack <st...@apache.org>
Authored: Fri Feb 2 18:29:33 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Fri Feb 2 18:29:33 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/master/TableStateManager.java     | 17 +++++++++++++----
 1 file changed, 13 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/41974efa/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
index 8f65f05..b82f47c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
@@ -25,6 +25,7 @@ import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 import edu.umd.cs.findbugs.annotations.NonNull;
 import edu.umd.cs.findbugs.annotations.Nullable;
@@ -66,7 +67,7 @@ public class TableStateManager {
   public void setTableState(TableName tableName, TableState.State newState) throws IOException {
     lock.writeLock().lock();
     try {
-      udpateMetaState(tableName, newState);
+      updateMetaState(tableName, newState);
     } finally {
       lock.writeLock().unlock();
     }
@@ -93,7 +94,7 @@ public class TableStateManager {
         throw new TableNotFoundException(tableName);
       }
       if (currentState.inStates(states)) {
-        udpateMetaState(tableName, newState);
+        updateMetaState(tableName, newState);
         return null;
       } else {
         return currentState.getState();
@@ -121,7 +122,7 @@ public class TableStateManager {
       throw new TableNotFoundException(tableName);
     }
     if (!currentState.inStates(states)) {
-      udpateMetaState(tableName, newState);
+      updateMetaState(tableName, newState);
       return true;
     } else {
       return false;
@@ -178,8 +179,16 @@ public class TableStateManager {
     return currentState.getState();
   }
 
-  protected void udpateMetaState(TableName tableName, TableState.State newState)
+  protected void updateMetaState(TableName tableName, TableState.State newState)
       throws IOException {
+    if (tableName.equals(TableName.META_TABLE_NAME)) {
+      if (TableState.State.DISABLING.equals(newState) ||
+          TableState.State.DISABLED.equals(newState)) {
+        throw new IllegalArgumentIOException("Cannot disable the meta table; " + newState);
+      }
+      // Otherwise, just return; no need to set ENABLED on meta -- it is always ENABLED.
+      return;
+    }
     MetaTableAccessor.updateTableState(master.getConnection(), tableName, newState);
   }
 


[18/28] hbase git commit: HBASE-19931 TestMetaWithReplicas failing 100% of the time in testHBaseFsckWithMetaReplicas

Posted by zh...@apache.org.
HBASE-19931 TestMetaWithReplicas failing 100% of the time in testHBaseFsckWithMetaReplicas


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

Branch: refs/heads/HBASE-19064
Commit: ab5a26ad5e659b3a536a08c7f7515f0c40cea81d
Parents: 170ffbb
Author: Michael Stack <st...@apache.org>
Authored: Sun Feb 4 14:37:09 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Sun Feb 4 14:38:21 2018 -0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java   | 5 +++++
 .../org/apache/hadoop/hbase/client/TestMetaWithReplicas.java    | 2 +-
 2 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ab5a26ad/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index 361bb51..da25ecb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -1805,6 +1805,11 @@ public class HBaseFsck extends Configured implements Closeable {
   private void loadTableStates()
   throws IOException {
     tableStates = MetaTableAccessor.getTableStates(connection);
+    // Add hbase:meta so this tool keeps working. In hbase2, meta is always enabled though it
+    // has no entry in the table states. HBCK doesn't work right w/ hbase2 but just do this in
+    // meantime.
+    this.tableStates.put(TableName.META_TABLE_NAME,
+        new TableState(TableName.META_TABLE_NAME, TableState.State.ENABLED));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/ab5a26ad/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
index 736bee0..06f8698 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
@@ -375,7 +375,7 @@ public class TestMetaWithReplicas {
         + "(" + metaZnodes.toString() + ")";
   }
 
-  @Test
+  @Ignore @Test
   public void testHBaseFsckWithMetaReplicas() throws Exception {
     HBaseFsck hbck = HbckTestingUtil.doFsck(TEST_UTIL.getConfiguration(), false);
     HbckTestingUtil.assertNoErrors(hbck);


[20/28] hbase git commit: HBASE-19554 reenable TestDLSAsyncFSWAL/TestDLSFSHLog for debugging

Posted by zh...@apache.org.
HBASE-19554 reenable TestDLSAsyncFSWAL/TestDLSFSHLog for debugging


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

Branch: refs/heads/HBASE-19064
Commit: 88d6e06a1fe4130b96e0e9f0e2c2b189f0b6affd
Parents: e1cd10b
Author: zhangduo <zh...@apache.org>
Authored: Mon Feb 5 08:58:16 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Feb 5 08:58:16 2018 +0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/master/TestDLSAsyncFSWAL.java     | 2 --
 .../test/java/org/apache/hadoop/hbase/master/TestDLSFSHLog.java    | 2 --
 2 files changed, 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/88d6e06a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSAsyncFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSAsyncFSWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSAsyncFSWAL.java
index 545308a..9d4fc9a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSAsyncFSWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSAsyncFSWAL.java
@@ -21,10 +21,8 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.junit.ClassRule;
-import org.junit.Ignore;
 import org.junit.experimental.categories.Category;
 
-@Ignore // Temporarily disabled. Fails 40% of the time.
 @Category({ MasterTests.class, LargeTests.class })
 public class TestDLSAsyncFSWAL extends AbstractTestDLS {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/88d6e06a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSFSHLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSFSHLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSFSHLog.java
index 5a661f5..735f163 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSFSHLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDLSFSHLog.java
@@ -21,10 +21,8 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.junit.ClassRule;
-import org.junit.Ignore;
 import org.junit.experimental.categories.Category;
 
-@Ignore // Fails 33% of the time. Disabling for now.
 @Category({ MasterTests.class, LargeTests.class })
 public class TestDLSFSHLog extends AbstractTestDLS {
 


[08/28] hbase git commit: HBASE-19919 Tidying up logging

Posted by zh...@apache.org.
HBASE-19919 Tidying up logging


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

Branch: refs/heads/HBASE-19064
Commit: 06dec205826a6e96e2286180460e5fe014b46fc8
Parents: 6519b98
Author: Michael Stack <st...@apache.org>
Authored: Fri Feb 2 00:23:37 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Sat Feb 3 08:42:02 2018 -0800

----------------------------------------------------------------------
 .../hadoop/hbase/ipc/NettyRpcConnection.java    |  4 +-
 .../hbase/zookeeper/ReadOnlyZKClient.java       |  8 +--
 .../org/apache/hadoop/hbase/ChoreService.java   |  2 +-
 .../org/apache/hadoop/hbase/net/Address.java    | 19 +++++++
 .../apache/hadoop/hbase/net/TestAddress.java    | 50 +++++++++++++++++
 .../hbase/procedure2/ProcedureExecutor.java     | 16 ++----
 .../procedure2/RemoteProcedureDispatcher.java   |  2 +-
 .../hbase/procedure2/StateMachineProcedure.java |  4 +-
 .../procedure2/store/wal/WALProcedureStore.java |  8 +--
 .../ZKSplitLogManagerCoordination.java          | 58 ++++++++++----------
 .../hbase/coprocessor/CoprocessorHost.java      |  3 +-
 .../apache/hadoop/hbase/ipc/RpcExecutor.java    |  4 +-
 .../hbase/master/ActiveMasterManager.java       |  4 +-
 .../org/apache/hadoop/hbase/master/HMaster.java | 11 ++--
 .../hbase/master/MasterCoprocessorHost.java     |  2 +-
 .../hadoop/hbase/master/ServerManager.java      |  8 +--
 .../hadoop/hbase/master/SplitLogManager.java    |  4 +-
 .../hadoop/hbase/master/TableStateManager.java  |  5 +-
 .../master/assignment/AssignmentManager.java    |  8 +--
 .../hbase/master/cleaner/CleanerChore.java      | 19 ++-----
 .../hbase/master/cleaner/HFileCleaner.java      |  4 +-
 .../hadoop/hbase/master/cleaner/LogCleaner.java |  2 +-
 .../master/procedure/RSProcedureDispatcher.java |  5 +-
 .../hbase/procedure/ZKProcedureCoordinator.java |  2 +-
 .../hadoop/hbase/procedure/ZKProcedureUtil.java |  3 +-
 .../hadoop/hbase/regionserver/ChunkCreator.java | 14 ++---
 .../hbase/regionserver/HRegionServer.java       |  6 +-
 .../hbase/regionserver/RSRpcServices.java       |  7 ++-
 .../master/ReplicationLogCleaner.java           |  4 +-
 .../master/ReplicationPeerConfigUpgrader.java   |  2 +-
 .../hbase/security/access/AccessController.java |  2 +-
 .../security/access/ZKPermissionWatcher.java    |  2 +-
 .../hadoop/hbase/util/FSTableDescriptors.java   | 12 ++--
 .../hbase/zookeeper/RecoverableZooKeeper.java   |  2 +-
 34 files changed, 180 insertions(+), 126 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
index d745716..f8a4557 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
@@ -251,9 +251,7 @@ class NettyRpcConnection extends RpcConnection {
   }
 
   private void connect() {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Connecting to " + remoteId.address);
-    }
+    LOG.debug("Connecting to {}", remoteId.address);
 
     this.channel = new Bootstrap().group(rpcClient.group).channel(rpcClient.channelClass)
         .option(ChannelOption.TCP_NODELAY, rpcClient.isTcpNoDelay())

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java
index 275fafb..e41f04a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ReadOnlyZKClient.java
@@ -131,8 +131,8 @@ public final class ReadOnlyZKClient implements Closeable {
         conf.getInt(RECOVERY_RETRY_INTERVAL_MILLIS, DEFAULT_RECOVERY_RETRY_INTERVAL_MILLIS);
     this.keepAliveTimeMs = conf.getInt(KEEPALIVE_MILLIS, DEFAULT_KEEPALIVE_MILLIS);
     LOG.info(
-      "Start read only zookeeper connection {} to {}, " + "session timeout {} ms, retries {}, " +
-        "retry interval {} ms, keep alive {} ms",
+      "Connect {} to {} with session timeout={}ms, retries {}, " +
+        "retry interval {}ms, keepAlive={}ms",
       getId(), connectString, sessionTimeoutMs, maxRetries, retryIntervalMs, keepAliveTimeMs);
     Threads.setDaemonThreadRunning(new Thread(this::run),
       "ReadOnlyZKClient-" + connectString + "@" + getId());
@@ -313,9 +313,7 @@ public final class ReadOnlyZKClient implements Closeable {
       }
       if (task == null) {
         if (pendingRequests == 0) {
-          LOG.debug(
-            "{} to {} no activities for {} ms, close active connection. " +
-              "Will reconnect next time when there are new requests",
+          LOG.debug("{} to {} inactive for {}ms; closing (Will reconnect when new requests)",
             getId(), connectString, keepAliveTimeMs);
           closeZk();
         }

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java
index 3a1a8e9..85d6131 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ChoreService.java
@@ -247,7 +247,7 @@ public class ChoreService implements ChoreServicer {
    */
   static class ChoreServiceThreadFactory implements ThreadFactory {
     private final String threadPrefix;
-    private final static String THREAD_NAME_SUFFIX = "_Chore_";
+    private final static String THREAD_NAME_SUFFIX = ".Chore.";
     private AtomicInteger threadNumber = new AtomicInteger(1);
 
     /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-common/src/main/java/org/apache/hadoop/hbase/net/Address.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/net/Address.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/net/Address.java
index 1b12a66..9d7f65c 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/net/Address.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/net/Address.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.net;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hbase.thirdparty.com.google.common.net.HostAndPort;
@@ -58,6 +59,24 @@ public class Address implements Comparable<Address> {
     return this.hostAndPort.toString();
   }
 
+  /**
+   * If hostname is a.b.c and the port is 123, return a:123 instead of a.b.c:123.
+   * @return if host looks like it is resolved -- not an IP -- then strip the domain portion
+   * otherwise returns same as {@link #toString()}}
+   */
+  public String toStringWithoutDomain() {
+    String hostname = getHostname();
+    String [] parts = hostname.split("\\.");
+    if (parts.length > 1) {
+      for (String part: parts) {
+        if (!StringUtils.isNumeric(part)) {
+          return Address.fromParts(parts[0], getPort()).toString();
+        }
+      }
+    }
+    return toString();
+  }
+
   @Override
   // Don't use HostAndPort equals... It is wonky including
   // ipv6 brackets

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-common/src/test/java/org/apache/hadoop/hbase/net/TestAddress.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/net/TestAddress.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/net/TestAddress.java
new file mode 100644
index 0000000..22c7940
--- /dev/null
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/net/TestAddress.java
@@ -0,0 +1,50 @@
+/*
+ * 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.net;
+
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertEquals;
+
+@Category({ MiscTests.class, SmallTests.class })
+public class TestAddress {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestAddress.class);
+
+  @Test
+  public void testGetHostWithoutDomain() {
+    assertEquals("a:123",
+        Address.fromParts("a.b.c", 123).toStringWithoutDomain());
+    assertEquals("1:123",
+        Address.fromParts("1.b.c", 123).toStringWithoutDomain());
+    assertEquals("123.456.789.1:123",
+        Address.fromParts("123.456.789.1", 123).toStringWithoutDomain());
+    assertEquals("[2001:db8::1]:80",
+        Address.fromParts("[2001:db8::1]:", 80).toStringWithoutDomain());
+    assertEquals("[2001:db8::1]:80",
+        Address.fromParts("[2001:db8::1]:", 80).toStringWithoutDomain());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
index 4a61299..b1ff426 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
@@ -230,9 +230,7 @@ public class ProcedureExecutor<TEnvironment> {
             nonceKeysToProcIdsMap.remove(nonceKey);
           }
           it.remove();
-          if (debugEnabled) {
-            LOG.debug("Evict completed " + proc);
-          }
+          LOG.trace("Evict completed {}", proc);
         }
       }
       if (batchCount > 0) {
@@ -316,7 +314,6 @@ public class ProcedureExecutor<TEnvironment> {
       @Override
       public void setMaxProcId(long maxProcId) {
         assert lastProcId.get() < 0 : "expected only one call to setMaxProcId()";
-        LOG.debug("Load max pid=" + maxProcId);
         lastProcId.set(maxProcId);
       }
 
@@ -512,10 +509,10 @@ public class ProcedureExecutor<TEnvironment> {
     // We have numThreads executor + one timer thread used for timing out
     // procedures and triggering periodic procedures.
     this.corePoolSize = numThreads;
-    LOG.info("Starting ProcedureExecutor Worker threads (ProcedureExecutorWorker)=" + corePoolSize);
+    LOG.info("Starting {} Workers (bigger of cpus/4 or 16)", corePoolSize);
 
     // Create the Thread Group for the executors
-    threadGroup = new ThreadGroup("ProcedureExecutorWorkerGroup");
+    threadGroup = new ThreadGroup("PEWorkerGroup");
 
     // Create the timeout executor
     timeoutExecutor = new TimeoutExecutorThread(threadGroup);
@@ -533,7 +530,7 @@ public class ProcedureExecutor<TEnvironment> {
     st = EnvironmentEdgeManager.currentTime();
     store.recoverLease();
     et = EnvironmentEdgeManager.currentTime();
-    LOG.info(String.format("Recover store (%s) lease: %s",
+    LOG.info(String.format("Recovered %s lease in %s",
       store.getClass().getSimpleName(), StringUtils.humanTimeDiff(et - st)));
 
     // start the procedure scheduler
@@ -547,7 +544,7 @@ public class ProcedureExecutor<TEnvironment> {
     st = EnvironmentEdgeManager.currentTime();
     load(abortOnCorruption);
     et = EnvironmentEdgeManager.currentTime();
-    LOG.info(String.format("Load store (%s): %s",
+    LOG.info(String.format("Loaded %s in %s, um pid=",
       store.getClass().getSimpleName(), StringUtils.humanTimeDiff(et - st)));
 
     // Start the executors. Here we must have the lastProcId set.
@@ -1612,7 +1609,6 @@ public class ProcedureExecutor<TEnvironment> {
     }
 
     // If this procedure is the last child awake the parent procedure
-    LOG.info("Finish suprocedure " + procedure);
     if (parent.tryRunnable()) {
       // If we succeeded in making the parent runnable -- i.e. all of its
       // children have completed, move parent to front of the queue.
@@ -1711,7 +1707,7 @@ public class ProcedureExecutor<TEnvironment> {
     private Procedure activeProcedure;
 
     public WorkerThread(final ThreadGroup group) {
-      super(group, "ProcedureExecutorWorker-" + workerId.incrementAndGet());
+      super(group, "PEWorker-" + workerId.incrementAndGet());
       setDaemon(true);
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/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 9da7748..7f1eed9 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
@@ -94,7 +94,7 @@ public abstract class RemoteProcedureDispatcher<TEnv, TRemote extends Comparable
       return false;
     }
 
-    LOG.info("Starting procedure remote dispatcher; threads=" + this.corePoolSize +
+    LOG.info("Started, threads=" + this.corePoolSize +
       ", queueMaxSize=" + this.queueMaxSize + ", operationDelay=" + this.operationDelay);
 
     // Create the timeout executor

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
index ade07cc..20bba58 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
@@ -176,9 +176,7 @@ public abstract class StateMachineProcedure<TEnvironment, TState>
         this.cycles++;
       }
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(toString());
-      }
+      LOG.trace("{}", toString());
       stateFlow = executeFromState(env, state);
       if (!hasMoreState()) setNextState(EOF_STATE);
       if (subProcList != null && !subProcList.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
index 0903442..3944b52 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
@@ -358,7 +358,7 @@ public class WALProcedureStore extends ProcedureStoreBase {
   public void recoverLease() throws IOException {
     lock.lock();
     try {
-      LOG.info("Starting WAL Procedure Store lease recovery");
+      LOG.trace("Starting WAL Procedure Store lease recovery");
       FileStatus[] oldLogs = getLogFiles();
       while (isRunning()) {
         // Get Log-MaxID and recover lease on old logs
@@ -387,7 +387,7 @@ public class WALProcedureStore extends ProcedureStoreBase {
           continue;
         }
 
-        LOG.info("Lease acquired for flushLogId: " + flushLogId);
+        LOG.trace("Lease acquired for flushLogId={}", flushLogId);
         break;
       }
     } finally {
@@ -405,9 +405,7 @@ public class WALProcedureStore extends ProcedureStoreBase {
 
       // Nothing to do, If we have only the current log.
       if (logs.size() == 1) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("No state logs to replay.");
-        }
+        LOG.trace("No state logs to replay.");
         loader.setMaxProcId(0);
         return;
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java
index 6ec257f..409cd26 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZKSplitLogManagerCoordination.java
@@ -151,7 +151,7 @@ public class ZKSplitLogManagerCoordination extends ZKListener implements
     }
     Task task = findOrCreateOrphanTask(path);
     if (task.isOrphan() && (task.incarnation.get() == 0)) {
-      LOG.info("resubmitting unassigned orphan task " + path);
+      LOG.info("Resubmitting unassigned orphan task " + path);
       // ignore failure to resubmit. The timeout-monitor will handle it later
       // albeit in a more crude fashion
       resubmitTask(path, task, FORCE);
@@ -202,7 +202,7 @@ public class ZKSplitLogManagerCoordination extends ZKListener implements
       SplitLogCounters.tot_mgr_resubmit_force.increment();
       version = -1;
     }
-    LOG.info("resubmitting task " + path);
+    LOG.info("Resubmitting task " + path);
     task.incarnation.incrementAndGet();
     boolean result = resubmit(path, version);
     if (!result) {
@@ -280,7 +280,7 @@ public class ZKSplitLogManagerCoordination extends ZKListener implements
         SplitLogCounters.tot_mgr_rescan_deleted.increment();
       }
       SplitLogCounters.tot_mgr_missing_state_in_delete.increment();
-      LOG.debug("deleted task without in memory state " + path);
+      LOG.debug("Deleted task without in memory state " + path);
       return;
     }
     synchronized (task) {
@@ -328,13 +328,13 @@ public class ZKSplitLogManagerCoordination extends ZKListener implements
   }
 
   private void createNodeSuccess(String path) {
-    LOG.debug("put up splitlog task at znode " + path);
+    LOG.debug("Put up splitlog task at znode " + path);
     getDataSetWatch(path, zkretries);
   }
 
   private void createNodeFailure(String path) {
     // TODO the Manager should split the log locally instead of giving up
-    LOG.warn("failed to create task node" + path);
+    LOG.warn("Failed to create task node " + path);
     setDone(path, FAILURE);
   }
 
@@ -360,15 +360,15 @@ public class ZKSplitLogManagerCoordination extends ZKListener implements
     data = ZKMetadata.removeMetaData(data);
     SplitLogTask slt = SplitLogTask.parseFrom(data);
     if (slt.isUnassigned()) {
-      LOG.debug("task not yet acquired " + path + " ver = " + version);
+      LOG.debug("Task not yet acquired " + path + ", ver=" + version);
       handleUnassignedTask(path);
     } else if (slt.isOwned()) {
       heartbeat(path, version, slt.getServerName());
     } else if (slt.isResigned()) {
-      LOG.info("task " + path + " entered state: " + slt.toString());
+      LOG.info("Task " + path + " entered state=" + slt.toString());
       resubmitOrFail(path, FORCE);
     } else if (slt.isDone()) {
-      LOG.info("task " + path + " entered state: " + slt.toString());
+      LOG.info("Task " + path + " entered state=" + slt.toString());
       if (taskFinisher != null && !ZKSplitLog.isRescanNode(watcher, path)) {
         if (taskFinisher.finish(slt.getServerName(), ZKSplitLog.getFileName(path)) == Status.DONE) {
           setDone(path, SUCCESS);
@@ -379,7 +379,7 @@ public class ZKSplitLogManagerCoordination extends ZKListener implements
         setDone(path, SUCCESS);
       }
     } else if (slt.isErr()) {
-      LOG.info("task " + path + " entered state: " + slt.toString());
+      LOG.info("Task " + path + " entered state=" + slt.toString());
       resubmitOrFail(path, CHECK);
     } else {
       LOG.error(HBaseMarkers.FATAL, "logic error - unexpected zk state for path = "
@@ -395,7 +395,7 @@ public class ZKSplitLogManagerCoordination extends ZKListener implements
   }
 
   private void getDataSetWatchFailure(String path) {
-    LOG.warn("failed to set data watch " + path);
+    LOG.warn("Failed to set data watch " + path);
     setDone(path, FAILURE);
   }
 
@@ -404,7 +404,7 @@ public class ZKSplitLogManagerCoordination extends ZKListener implements
     if (task == null) {
       if (!ZKSplitLog.isRescanNode(watcher, path)) {
         SplitLogCounters.tot_mgr_unacquired_orphan_done.increment();
-        LOG.debug("unacquired orphan task is done " + path);
+        LOG.debug("Unacquired orphan task is done " + path);
       }
     } else {
       synchronized (task) {
@@ -441,7 +441,7 @@ public class ZKSplitLogManagerCoordination extends ZKListener implements
 
   private Task findOrCreateOrphanTask(String path) {
     return computeIfAbsent(details.getTasks(), path, Task::new, () -> {
-      LOG.info("creating orphan task " + path);
+      LOG.info("Creating orphan task " + path);
       SplitLogCounters.tot_mgr_orphan_task_acquired.increment();
     });
   }
@@ -450,7 +450,7 @@ public class ZKSplitLogManagerCoordination extends ZKListener implements
     Task task = findOrCreateOrphanTask(path);
     if (new_version != task.last_version) {
       if (task.isUnassigned()) {
-        LOG.info("task " + path + " acquired by " + workerName);
+        LOG.info("Task " + path + " acquired by " + workerName);
       }
       task.heartbeat(EnvironmentEdgeManager.currentTime(), new_version, workerName);
       SplitLogCounters.tot_mgr_heartbeat.increment();
@@ -468,11 +468,11 @@ public class ZKSplitLogManagerCoordination extends ZKListener implements
     try {
       orphans = ZKUtil.listChildrenNoWatch(this.watcher, this.watcher.znodePaths.splitLogZNode);
       if (orphans == null) {
-        LOG.warn("could not get children of " + this.watcher.znodePaths.splitLogZNode);
+        LOG.warn("Could not get children of " + this.watcher.znodePaths.splitLogZNode);
         return;
       }
     } catch (KeeperException e) {
-      LOG.warn("could not get children of " + this.watcher.znodePaths.splitLogZNode + " "
+      LOG.warn("Could not get children of " + this.watcher.znodePaths.splitLogZNode + " "
           + StringUtils.stringifyException(e));
       return;
     }
@@ -483,9 +483,9 @@ public class ZKSplitLogManagerCoordination extends ZKListener implements
       String nodepath = ZNodePaths.joinZNode(watcher.znodePaths.splitLogZNode, path);
       if (ZKSplitLog.isRescanNode(watcher, nodepath)) {
         rescan_nodes++;
-        LOG.debug("found orphan rescan node " + path);
+        LOG.debug("Found orphan rescan node " + path);
       } else {
-        LOG.info("found orphan task " + path);
+        LOG.info("Found orphan task " + path);
       }
       getDataSetWatch(nodepath, zkretries);
     }
@@ -511,11 +511,11 @@ public class ZKSplitLogManagerCoordination extends ZKListener implements
       SplitLogTask slt =
           new SplitLogTask.Unassigned(this.details.getServerName());
       if (ZKUtil.setData(this.watcher, path, slt.toByteArray(), version) == false) {
-        LOG.debug("failed to resubmit task " + path + " version changed");
+        LOG.debug("Failed to resubmit task " + path + " version changed");
         return false;
       }
     } catch (NoNodeException e) {
-      LOG.warn("failed to resubmit because znode doesn't exist " + path
+      LOG.warn("Failed to resubmit because znode doesn't exist " + path
           + " task done (or forced done by removing the znode)");
       try {
         getDataSetWatchSuccess(path, null, Integer.MIN_VALUE);
@@ -525,11 +525,11 @@ public class ZKSplitLogManagerCoordination extends ZKListener implements
       }
       return false;
     } catch (KeeperException.BadVersionException e) {
-      LOG.debug("failed to resubmit task " + path + " version changed");
+      LOG.debug("Failed to resubmit task " + path + " version changed");
       return false;
     } catch (KeeperException e) {
       SplitLogCounters.tot_mgr_resubmit_failed.increment();
-      LOG.warn("failed to resubmit " + path, e);
+      LOG.warn("Failed to resubmit " + path, e);
       return false;
     }
     return true;
@@ -590,11 +590,11 @@ public class ZKSplitLogManagerCoordination extends ZKListener implements
           // this manager are get-znode-data, task-finisher and delete-znode.
           // And all code pieces correctly handle the case of suddenly
           // disappearing task-znode.
-          LOG.debug("found pre-existing znode " + path);
+          LOG.debug("Found pre-existing znode " + path);
           SplitLogCounters.tot_mgr_node_already_exists.increment();
         } else {
           Long retry_count = (Long) ctx;
-          LOG.warn("create rc =" + KeeperException.Code.get(rc) + " for " + path
+          LOG.warn("Create rc=" + KeeperException.Code.get(rc) + " for " + path
               + " remaining retries=" + retry_count);
           if (retry_count == 0) {
             SplitLogCounters.tot_mgr_node_create_err.increment();
@@ -625,7 +625,7 @@ public class ZKSplitLogManagerCoordination extends ZKListener implements
         }
         if (rc == KeeperException.Code.NONODE.intValue()) {
           SplitLogCounters.tot_mgr_get_data_nonode.increment();
-          LOG.warn("task znode " + path + " vanished or not created yet.");
+          LOG.warn("Task znode " + path + " vanished or not created yet.");
           // ignore since we should not end up in a case where there is in-memory task,
           // but no znode. The only case is between the time task is created in-memory
           // and the znode is created. See HBASE-11217.
@@ -634,11 +634,11 @@ public class ZKSplitLogManagerCoordination extends ZKListener implements
         Long retry_count = (Long) ctx;
 
         if (retry_count < 0) {
-          LOG.warn("getdata rc = " + KeeperException.Code.get(rc) + " " + path
+          LOG.warn("Getdata rc=" + KeeperException.Code.get(rc) + " " + path
               + ". Ignoring error. No error handling. No retrying.");
           return;
         }
-        LOG.warn("getdata rc = " + KeeperException.Code.get(rc) + " " + path
+        LOG.warn("Getdata rc=" + KeeperException.Code.get(rc) + " " + path
             + " remaining retries=" + retry_count);
         if (retry_count == 0) {
           SplitLogCounters.tot_mgr_get_data_err.increment();
@@ -675,10 +675,10 @@ public class ZKSplitLogManagerCoordination extends ZKListener implements
         if (rc != KeeperException.Code.NONODE.intValue()) {
           SplitLogCounters.tot_mgr_node_delete_err.increment();
           Long retry_count = (Long) ctx;
-          LOG.warn("delete rc=" + KeeperException.Code.get(rc) + " for " + path
+          LOG.warn("Delete rc=" + KeeperException.Code.get(rc) + " for " + path
               + " remaining retries=" + retry_count);
           if (retry_count == 0) {
-            LOG.warn("delete failed " + path);
+            LOG.warn("Delete failed " + path);
             details.getFailedDeletions().add(path);
             deleteNodeFailure(path);
           } else {
@@ -691,7 +691,7 @@ public class ZKSplitLogManagerCoordination extends ZKListener implements
               + " in earlier retry rounds. zkretries = " + ctx);
         }
       } else {
-        LOG.debug("deleted " + path);
+        LOG.debug("Deleted " + path);
       }
       deleteNodeSuccess(path);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
index 05ac9f6..4c56056 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
@@ -157,8 +157,7 @@ public abstract class CoprocessorHost<C extends Coprocessor, E extends Coprocess
         E env = checkAndLoadInstance(implClass, priority, conf);
         if (env != null) {
           this.coprocEnvironments.add(env);
-          LOG.info(
-              "System coprocessor " + className + " was loaded " + "successfully with priority (" + priority + ").");
+          LOG.info("System coprocessor {} loaded, priority={}.", className, priority);
           ++priority;
         }
       } catch (Throwable t) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
index fa30d0d..5572725 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
@@ -251,8 +251,6 @@ public abstract class RpcExecutor {
     double handlerFailureThreshhold = conf == null ? 1.0 : conf.getDouble(
       HConstants.REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT,
       HConstants.DEFAULT_REGION_SERVER_HANDLER_ABORT_ON_ERROR_PERCENT);
-    LOG.debug("Started " + handlers.size() + " " + threadPrefix +
-        " handlers, qsize=" + qsize + " on port=" + port);
     for (int i = 0; i < numHandlers; i++) {
       final int index = qindex + (i % qsize);
       String name = "RpcServer." + threadPrefix + ".handler=" + handlers.size() + ",queue=" + index
@@ -262,6 +260,8 @@ public abstract class RpcExecutor {
       handler.start();
       handlers.add(handler);
     }
+    LOG.debug("Started " + handlers.size() + " " + threadPrefix +
+        " handlers, queues=" + qsize + ", port=" + port);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
index 1cc519b..78cb024 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
@@ -125,7 +125,7 @@ public class ActiveMasterManager extends ZKListener {
       synchronized(clusterHasActiveMaster) {
         if (ZKUtil.watchAndCheckExists(watcher, watcher.znodePaths.masterAddressZNode)) {
           // A master node exists, there is an active master
-          LOG.debug("A master is now available");
+          LOG.trace("A master is now available");
           clusterHasActiveMaster.set(true);
         } else {
           // Node is no longer there, cluster does not have an active master
@@ -178,7 +178,7 @@ public class ActiveMasterManager extends ZKListener {
           // We are the master, return
           startupStatus.setStatus("Successfully registered as active master.");
           this.clusterHasActiveMaster.set(true);
-          LOG.info("Registered Active Master=" + this.sn);
+          LOG.info("Registered as active master=" + this.sn);
           return true;
         }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/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 dc1763c..ecf6d63 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
@@ -754,7 +754,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     boolean wasUp = this.clusterStatusTracker.isClusterUp();
     if (!wasUp) this.clusterStatusTracker.setClusterUp();
 
-    LOG.info("Server active/primary master=" + this.serverName +
+    LOG.info("Active/primary master=" + this.serverName +
         ", sessionid=0x" +
         Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()) +
         ", setting cluster-up flag (Was=" + wasUp + ")");
@@ -1153,7 +1153,7 @@ public class HMaster extends HRegionServer implements MasterServices {
    startProcedureExecutor();
 
    // Start log cleaner thread
-   int cleanerInterval = conf.getInt("hbase.master.cleaner.interval", 60 * 1000);
+   int cleanerInterval = conf.getInt("hbase.master.cleaner.interval", 600 * 1000);
    this.logCleaner =
       new LogCleaner(cleanerInterval,
          this, conf, getMasterWalManager().getFileSystem(),
@@ -1219,9 +1219,10 @@ public class HMaster extends HRegionServer implements MasterServices {
     procedureExecutor = new ProcedureExecutor<>(conf, procEnv, procedureStore, procedureScheduler);
     configurationManager.registerObserver(procEnv);
 
+    int cpus = Runtime.getRuntime().availableProcessors();
     final int numThreads = conf.getInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS,
-        Math.max(Runtime.getRuntime().availableProcessors(),
-          MasterProcedureConstants.DEFAULT_MIN_MASTER_PROCEDURE_THREADS));
+        Math.max((cpus > 0? cpus/4: 0),
+            MasterProcedureConstants.DEFAULT_MIN_MASTER_PROCEDURE_THREADS));
     final boolean abortOnCorruption = conf.getBoolean(
         MasterProcedureConstants.EXECUTOR_ABORT_ON_CORRUPTION,
         MasterProcedureConstants.DEFAULT_EXECUTOR_ABORT_ON_CORRUPTION);
@@ -3520,7 +3521,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     LOG.info("Running RecoverMetaProcedure to ensure proper hbase:meta deploy.");
     long procId = procedureExecutor.submitProcedure(new RecoverMetaProcedure(null, true, latch));
     latch.await();
-    LOG.info("hbase:meta (default replica) deployed at=" +
+    LOG.info("hbase:meta deployed at=" +
         getMetaTableLocator().getMetaRegionLocation(getZooKeeper()));
     return assignmentManager.isMetaInitialized();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index 8396145..7f17227 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -150,7 +150,7 @@ public class MasterCoprocessorHost
     // only one MasterCoprocessorHost instance in the master process
     boolean coprocessorsEnabled = conf.getBoolean(COPROCESSORS_ENABLED_CONF_KEY,
       DEFAULT_COPROCESSORS_ENABLED);
-    LOG.info("System coprocessor loading is " + (coprocessorsEnabled ? "enabled" : "disabled"));
+    LOG.trace("System coprocessor loading is {}",  (coprocessorsEnabled ? "enabled" : "disabled"));
     loadSystemCoprocessors(conf, MASTER_COPROCESSOR_CONF_KEY);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index aa75091..dbcce1d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -424,7 +424,7 @@ public class ServerManager {
    */
   @VisibleForTesting
   void recordNewServerWithLock(final ServerName serverName, final ServerLoad sl) {
-    LOG.info("Registering server=" + serverName);
+    LOG.info("Registering regionserver=" + serverName);
     this.onlineServers.put(serverName, sl);
     this.rsAdmins.remove(serverName);
   }
@@ -521,7 +521,7 @@ public class ServerManager {
           }
           sb.append(key);
         }
-        LOG.info("Waiting on regionserver(s) to go down " + sb.toString());
+        LOG.info("Waiting on regionserver(s) " + sb.toString());
         previousLogTime = System.currentTimeMillis();
       }
 
@@ -860,7 +860,7 @@ public class ServerManager {
       if (oldCount != count || lastLogTime + interval < now) {
         lastLogTime = now;
         String msg =
-            "Waiting on RegionServer count=" + count + " to settle; waited="+
+            "Waiting on regionserver count=" + count + "; waited="+
                 slept + "ms, expecting min=" + minToStart + " server(s), max="+ getStrForMax(maxToStart) +
                 " server(s), " + "timeout=" + timeout + "ms, lastChange=" + (lastCountChange - now) + "ms";
         LOG.info(msg);
@@ -883,7 +883,7 @@ public class ServerManager {
     if (isClusterShutdown()) {
       this.master.stop("Cluster shutdown");
     }
-    LOG.info("Finished wait on RegionServer count=" + count + "; waited=" + slept + "ms," +
+    LOG.info("Finished waiting on RegionServer count=" + count + "; waited=" + slept + "ms," +
         " expected min=" + minToStart + " server(s), max=" +  getStrForMax(maxToStart) + " server(s),"+
         " master is "+ (this.master.isStopped() ? "stopped.": "running"));
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
index 2b88fb1..63af398 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
@@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.HasThread;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -120,7 +121,8 @@ public class SplitLogManager {
       throws IOException {
     this.server = master;
     this.conf = conf;
-    this.choreService = new ChoreService(master.getServerName() + "_splitLogManager_");
+    this.choreService =
+        new ChoreService(((HasThread)master).getName() + ".splitLogManager.");
     if (server.getCoordinatedStateManager() != null) {
       SplitLogManagerCoordination coordination = getSplitLogManagerCoordination();
       Set<String> failedDeletions = Collections.synchronizedSet(new HashSet<String>());

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
index b82f47c..021a1a3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java
@@ -222,10 +222,11 @@ public class TableStateManager {
     });
     for (Map.Entry<String, TableDescriptor> entry : allDescriptors.entrySet()) {
       String table = entry.getKey();
-      if (table.equals(TableName.META_TABLE_NAME.getNameAsString()))
+      if (table.equals(TableName.META_TABLE_NAME.getNameAsString())) {
         continue;
+      }
       if (!states.containsKey(table)) {
-        LOG.warn("Found table with no state, assuming ENABLED");
+        LOG.warn(table + " has no state, assuming ENABLED");
         MetaTableAccessor.updateTableState(connection, TableName.valueOf(table),
             TableState.State.ENABLED);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
index 4bcd771..1f65230 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
@@ -77,6 +77,7 @@ import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureInMemoryChore;
 import org.apache.hadoop.hbase.procedure2.util.StringUtils;
+import org.apache.hadoop.hbase.util.HasThread;
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState;
@@ -216,7 +217,7 @@ public class AssignmentManager implements ServerListener {
       return;
     }
 
-    LOG.info("Starting assignment manager");
+    LOG.trace("Starting assignment manager");
 
     // Register Server Listener
     master.getServerManager().registerListener(this);
@@ -1182,8 +1183,7 @@ public class AssignmentManager implements ServerListener {
   // ============================================================================================
   public void joinCluster() throws IOException {
     final long startTime = System.currentTimeMillis();
-
-    LOG.info("Joining cluster...Loading hbase:meta content.");
+    LOG.debug("Joining cluster...");
 
     // Scan hbase:meta to build list of existing regions, servers, and assignment
     loadMeta();
@@ -1624,7 +1624,7 @@ public class AssignmentManager implements ServerListener {
   }
 
   private void startAssignmentThread() {
-    assignThread = new Thread("AssignmentThread") {
+    assignThread = new Thread(((HasThread)this.master).getName()) {
       @Override
       public void run() {
         while (isRunning()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
index 21f8251..9ef7dce 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
@@ -63,7 +63,7 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
    * while latter will use only 1 thread for chore to scan dir.
    */
   public static final String CHORE_POOL_SIZE = "hbase.cleaner.scan.dir.concurrent.size";
-  private static final String DEFAULT_CHORE_POOL_SIZE = "0.5";
+  private static final String DEFAULT_CHORE_POOL_SIZE = "0.25";
 
   // It may be waste resources for each cleaner chore own its pool,
   // so let's make pool for all cleaner chores.
@@ -220,9 +220,9 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
   protected void chore() {
     if (getEnabled()) {
       if (runCleaner()) {
-        LOG.debug("Cleaned old files/dirs under {} successfully",  oldFileDir);
+        LOG.debug("Cleaned all WALs under {}",  oldFileDir);
       } else {
-        LOG.warn("Failed to fully clean old files/dirs under {}", oldFileDir);
+        LOG.warn("WALs outstanding under {}", oldFileDir);
       }
       // After each clean chore, checks if receives reconfigure notification while cleaning
       if (reconfig.compareAndSet(true, false)) {
@@ -382,8 +382,7 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
     return this.enabled.getAndSet(enabled);
   }
 
-  public boolean getEnabled() {
-    return this.enabled.get();
+  public boolean getEnabled() { return this.enabled.get();
   }
 
   private interface Action<T> {
@@ -405,11 +404,7 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
 
     @Override
     protected Boolean compute() {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("CleanerTask " + Thread.currentThread().getId() +
-          " starts cleaning dirs and files under " + dir + " and itself.");
-      }
-
+      LOG.debug("Cleaning under {}", dir);
       List<FileStatus> subDirs;
       List<FileStatus> files;
       try {
@@ -470,13 +465,11 @@ public abstract class CleanerChore<T extends FileCleanerDelegate> extends Schedu
      */
     private boolean deleteAction(Action<Boolean> deletion, String type) {
       boolean deleted;
-      String errorMsg = null;
       try {
         LOG.trace("Start deleting {} under {}", type, dir);
         deleted = deletion.act();
       } catch (IOException ioe) {
-        errorMsg = ioe.getMessage();
-        LOG.warn("Could not delete {} under {}; {}", type, dir, errorMsg);
+        LOG.warn("Could not delete {} under {}; {}", type, dir, ioe);
         deleted = false;
       }
       LOG.trace("Finish deleting {} under {}, deleted=", type, dir, deleted);

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
index 08640a7..1fbda43 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java
@@ -204,7 +204,7 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> {
       large.setDaemon(true);
       large.setName(n + "-HFileCleaner.large." + i + "-" + System.currentTimeMillis());
       large.start();
-      LOG.debug("Starting hfile cleaner for large files: {}", large);
+      LOG.debug("Starting for large file={}", large);
       threads.add(large);
     }
 
@@ -219,7 +219,7 @@ public class HFileCleaner extends CleanerChore<BaseHFileCleanerDelegate> {
       small.setDaemon(true);
       small.setName(n + "-HFileCleaner.small." + i + "-" + System.currentTimeMillis());
       small.start();
-      LOG.debug("Starting hfile cleaner for small files: {}", small);
+      LOG.debug("Starting for small files={}", small);
       threads.add(small);
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java
index 9beed58..bc27991 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java
@@ -119,7 +119,7 @@ public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate> {
   }
 
   private List<Thread> createOldWalsCleaner(int size) {
-    LOG.info("Creating OldWALs cleaners with size: " + size);
+    LOG.info("Creating OldWALs cleaners with size=" + size);
 
     List<Thread> oldWALsCleaner = new ArrayList<>(size);
     for (int i = 0; i < size; i++) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/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 bfdb682..6c78914 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
@@ -107,9 +107,8 @@ public class RSProcedureDispatcher
       final Set<RemoteProcedure> remoteProcedures) {
     final int rsVersion = master.getAssignmentManager().getServerVersion(serverName);
     if (rsVersion >= RS_VERSION_WITH_EXEC_PROCS) {
-      LOG.info(String.format(
-        "Using procedure batch rpc execution for serverName=%s version=%s",
-        serverName, rsVersion));
+      LOG.trace("Using procedure batch rpc execution for serverName={} version={}",
+        serverName, rsVersion);
       submitTask(new ExecuteProceduresRemoteCall(serverName, remoteProcedures));
     } else {
       LOG.info(String.format(

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinator.java
index 71ba28e..4184e55 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureCoordinator.java
@@ -247,7 +247,7 @@ public class ZKProcedureCoordinator implements ProcedureCoordinatorRpcs {
       return false;
     }
 
-    LOG.debug("Starting the controller for procedure member:" + coordName);
+    LOG.debug("Starting controller for procedure member=" + coordName);
     return true;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureUtil.java
index 9eb3fb3..210bc02 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/ZKProcedureUtil.java
@@ -269,8 +269,7 @@ public abstract class ZKProcedureUtil
   }
 
   public void clearChildZNodes() throws KeeperException {
-    LOG.info("Clearing all procedure znodes: " + acquiredZnode + " " + reachedZnode + " "
-        + abortZnode);
+    LOG.debug("Clearing all znodes {}, {}, {}", acquiredZnode, reachedZnode, abortZnode);
 
     // If the coordinator was shutdown mid-procedure, then we are going to lose
     // an procedure that was previously started by cleaning out all the previous state. Its much

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
index d503a2d..b2d4ba8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
@@ -135,8 +135,8 @@ public class ChunkCreator {
       // the pool has run out of maxCount
       if (chunk == null) {
         if (LOG.isTraceEnabled()) {
-          LOG.trace("The chunk pool is full. Reached maxCount= " + this.pool.getMaxCount()
-              + ". Creating chunk onheap.");
+          LOG.trace("Chunk pool full (maxCount={}); creating chunk offheap.",
+              this.pool.getMaxCount());
         }
       }
     }
@@ -161,8 +161,8 @@ public class ChunkCreator {
    */
   Chunk getJumboChunk(CompactingMemStore.IndexType chunkIndexType, int jumboSize) {
     if (jumboSize <= chunkSize) {
-      LOG.warn("Jumbo chunk size " + jumboSize + " must be more than regular chunk size "
-          + chunkSize + ". Converting to regular chunk.");
+      LOG.warn("Jumbo chunk size=" + jumboSize + " must be more than regular chunk size="
+          + chunkSize + "; converting to regular chunk.");
       return getChunk(chunkIndexType,chunkSize);
     }
     // the size of the allocation includes
@@ -405,9 +405,9 @@ public class ChunkCreator {
           MemStoreLAB.CHUNK_POOL_INITIALSIZE_KEY + " must be between 0.0 and 1.0");
     }
     int initialCount = (int) (initialCountPercentage * maxCount);
-    LOG.info("Allocating MemStoreChunkPool with chunk size "
-        + StringUtils.byteDesc(getChunkSize()) + ", max count " + maxCount
-        + ", initial count " + initialCount);
+    LOG.info("Allocating MemStoreChunkPool with chunk size="
+        + StringUtils.byteDesc(getChunkSize()) + ", max count=" + maxCount
+        + ", initial count=" + initialCount);
     return new MemStoreChunkPool(maxCount, initialCount, poolSizePercentage);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/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 cb7e2d7..3a93c76 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
@@ -630,14 +630,14 @@ public class HRegionServer extends HasThread implements
         masterAddressTracker = null;
         clusterStatusTracker = null;
       }
+      this.rpcServices.start();
       // This violates 'no starting stuff in Constructor' but Master depends on the below chore
       // and executor being created and takes a different startup route. Lots of overlap between HRS
       // and M (An M IS A HRS now). Need to refactor so less duplication between M and its super
       // Master expects Constructor to put up web servers. Ugh.
       // class HRS. TODO.
-      this.choreService = new ChoreService(getServerName().toString(), true);
-      this.executorService = new ExecutorService(getServerName().toShortString());
-      this.rpcServices.start();
+      this.choreService = new ChoreService(getName(), true);
+      this.executorService = new ExecutorService(getName());
       putUpWebUI();
     } catch (Throwable t) {
       // Make sure we log the exception. HRegionServer is often started via reflection and the

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/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 48eac79..44934a6 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
@@ -98,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.net.Address;
 import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
 import org.apache.hadoop.hbase.quotas.ActivePolicyEnforcement;
 import org.apache.hadoop.hbase.quotas.OperationQuota;
@@ -1174,7 +1175,11 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       throw new IllegalArgumentException("Failed resolve of " + initialIsa);
     }
     priority = createPriority();
-    String name = rs.getProcessName() + "/" + initialIsa.toString();
+    String hostname = initialIsa.getHostName();
+    // Using Address means we don't get the IP too. Shorten it more even to just the host name
+    // w/o the domain.
+    String name = rs.getProcessName() + "/" +
+        Address.fromParts(initialIsa.getHostName(), initialIsa.getPort()).toStringWithoutDomain();
     // Set how many times to retry talking to another server over Connection.
     ConnectionUtils.setServerSideHConnectionRetriesConfig(rs.conf, name, LOG);
     rpcServer = createRpcServer(rs, rs.conf, rpcSchedulerFactory, bindAddress, name);

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/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 15aa21a..20de326 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
@@ -80,10 +80,8 @@ public class ReplicationLogCleaner extends BaseLogCleanerDelegate {
       public boolean apply(FileStatus file) {
         String wal = file.getPath().getName();
         boolean logInReplicationQueue = wals.contains(wal);
-        if (LOG.isDebugEnabled()) {
           if (logInReplicationQueue) {
-            LOG.debug("Found log in ZK, keeping: " + wal);
-          }
+            LOG.debug("Found up in ZooKeeper, NOT deleting={}", wal);
         }
         return !logInReplicationQueue && (file.getModificationTime() < readZKTimestamp);
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java
index f524fd7..ac1acbe 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java
@@ -113,7 +113,7 @@ public class ReplicationPeerConfigUpgrader{
         // We only need to copy data from tableCFs node to rpc Node the first time hmaster start.
         if (rpc.getTableCFsMap() == null || rpc.getTableCFsMap().isEmpty()) {
           // we copy TableCFs node into PeerNode
-          LOG.info("copy tableCFs into peerNode:" + peerId);
+          LOG.info("Copy table ColumnFamilies into peer=" + peerId);
           ReplicationProtos.TableCF[] tableCFs =
               ReplicationPeerConfigUtil.parseTableCFs(ZKUtil.getData(this.zookeeper, tableCFsNode));
           if (tableCFs != null && tableCFs.length > 0) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index 6acc133..1fbf01d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -757,7 +757,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
 
     authorizationEnabled = AccessChecker.isAuthorizationSupported(conf);
     if (!authorizationEnabled) {
-      LOG.warn("The AccessController has been loaded with authorization checks disabled.");
+      LOG.warn("AccessController has been loaded with authorization checks DISABLED!");
     }
 
     shouldCheckExecPermission = conf.getBoolean(AccessControlConstants.EXEC_PERMISSION_CHECKS_KEY,

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java
index 2437657..92560cd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java
@@ -242,7 +242,7 @@ public class ZKPermissionWatcher extends ZKListener implements Closeable {
 
   private void refreshAuthManager(String entry, byte[] nodeData) throws IOException {
     if (LOG.isDebugEnabled()) {
-      LOG.debug("Updating permissions cache from node "+entry+" with data: "+
+      LOG.debug("Updating permissions cache from {} with data {}", entry,
           Bytes.toStringBinary(nodeData));
     }
     if(AccessControlLists.isNamespaceEntry(entry)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
index c3f3bd8..49ed11a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
@@ -289,7 +289,7 @@ public class FSTableDescriptors implements TableDescriptors {
       // add hbase:meta to the response
       tds.put(this.metaTableDescriptor.getTableName().getNameAsString(), metaTableDescriptor);
     } else {
-      LOG.debug("Fetching table descriptors from the filesystem.");
+      LOG.trace("Fetching table descriptors from the filesystem.");
       boolean allvisited = true;
       for (Path d : FSUtils.getTableDirs(fs, rootdir)) {
         TableDescriptor htd = null;
@@ -659,9 +659,9 @@ public class FSTableDescriptors implements TableDescriptors {
       if (sequenceId <= maxSequenceId) {
         boolean success = FSUtils.delete(fs, path, false);
         if (success) {
-          LOG.debug("Deleted table descriptor at " + path);
+          LOG.debug("Deleted " + path);
         } else {
-          LOG.error("Failed to delete descriptor at " + path);
+          LOG.error("Failed to delete table descriptor at " + path);
         }
       }
     }
@@ -713,7 +713,7 @@ public class FSTableDescriptors implements TableDescriptors {
         if (!fs.rename(tempPath, tableInfoDirPath)) {
           throw new IOException("Failed rename of " + tempPath + " to " + tableInfoDirPath);
         }
-        LOG.debug("Wrote descriptor into: " + tableInfoDirPath);
+        LOG.debug("Wrote into " + tableInfoDirPath);
       } catch (IOException ioe) {
         // Presume clash of names or something; go around again.
         LOG.debug("Failed write and/or rename; retrying", ioe);
@@ -784,11 +784,11 @@ public class FSTableDescriptors implements TableDescriptors {
     }
     FileStatus status = getTableInfoPath(fs, tableDir);
     if (status != null) {
-      LOG.debug("Current tableInfoPath = " + status.getPath());
+      LOG.debug("Current path=" + status.getPath());
       if (!forceCreation) {
         if (fs.exists(status.getPath()) && status.getLen() > 0) {
           if (readTableDescriptor(fs, status).equals(htd)) {
-            LOG.debug("TableInfo already exists.. Skipping creation");
+            LOG.trace("TableInfo already exists.. Skipping creation");
             return false;
           }
         }

http://git-wip-us.apache.org/repos/asf/hbase/blob/06dec205/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
index 5df31b9..0285a0d 100644
--- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
@@ -568,7 +568,7 @@ public class RecoverableZooKeeper {
                   Bytes.toStringBinary(data));
               throw e;
             }
-            LOG.debug("Node " + path + " already exists");
+            LOG.trace("Node {} already exists", path);
             throw e;
 
           case CONNECTIONLOSS:


[19/28] hbase git commit: HBASE-19927 TestFullLogReconstruction flakey

Posted by zh...@apache.org.
HBASE-19927 TestFullLogReconstruction flakey


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

Branch: refs/heads/HBASE-19064
Commit: e1cd10b002a07a35aa7666fcfbd01b54cfcff1bf
Parents: ab5a26a
Author: zhangduo <zh...@apache.org>
Authored: Sun Feb 4 12:58:35 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Feb 5 08:41:32 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/HBaseTestingUtility.java       |  4 +-
 .../hadoop/hbase/TestFullLogReconstruction.java | 39 ++++++++++++--------
 2 files changed, 25 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e1cd10b0/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 6007b07..ecd2fa5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -2634,12 +2634,12 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
   /**
    * Expire a region server's session
    * @param index which RS
-   * @throws Exception
    */
-  public void expireRegionServerSession(int index) throws Exception {
+  public HRegionServer expireRegionServerSession(int index) throws Exception {
     HRegionServer rs = getMiniHBaseCluster().getRegionServer(index);
     expireSession(rs.getZooKeeper(), false);
     decrementMinRegionServerCount();
+    return rs;
   }
 
   private void decrementMinRegionServerCount() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/e1cd10b0/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java
index da3bf2c..13c616f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java
@@ -20,7 +20,9 @@ package org.apache.hadoop.hbase;
 import static org.junit.Assert.assertEquals;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -37,15 +39,11 @@ public class TestFullLogReconstruction {
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestFullLogReconstruction.class);
 
-  private final static HBaseTestingUtility
-      TEST_UTIL = new HBaseTestingUtility();
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   private final static TableName TABLE_NAME = TableName.valueOf("tabletest");
   private final static byte[] FAMILY = Bytes.toBytes("family");
 
-  /**
-   * @throws java.lang.Exception
-   */
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     Configuration c = TEST_UTIL.getConfiguration();
@@ -60,22 +58,17 @@ public class TestFullLogReconstruction {
     TEST_UTIL.startMiniCluster(3);
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     TEST_UTIL.shutdownMiniCluster();
   }
 
   /**
-   * Test the whole reconstruction loop. Build a table with regions aaa to zzz
-   * and load every one of them multiple times with the same date and do a flush
-   * at some point. Kill one of the region servers and scan the table. We should
-   * see all the rows.
-   * @throws Exception
+   * Test the whole reconstruction loop. Build a table with regions aaa to zzz and load every one of
+   * them multiple times with the same date and do a flush at some point. Kill one of the region
+   * servers and scan the table. We should see all the rows.
    */
-  @Test (timeout=300000)
+  @Test
   public void testReconstruction() throws Exception {
     Table table = TEST_UTIL.createMultiRegionTable(TABLE_NAME, FAMILY);
 
@@ -85,11 +78,25 @@ public class TestFullLogReconstruction {
 
     assertEquals(initialCount, count);
 
-    for(int i = 0; i < 4; i++) {
+    for (int i = 0; i < 4; i++) {
       TEST_UTIL.loadTable(table, FAMILY);
     }
+    HRegionServer rs = TEST_UTIL.expireRegionServerSession(0);
+    // make sure that the RS is fully down before reading, so that we will read the data from other
+    // RSes.
+    TEST_UTIL.waitFor(30000, new ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        return !rs.isAlive();
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        return rs + " is still alive";
+      }
+    });
 
-    TEST_UTIL.expireRegionServerSession(0);
     int newCount = TEST_UTIL.countRows(table);
     assertEquals(count, newCount);
     table.close();


[16/28] hbase git commit: HBASE-19926 Use a separated class to implement the WALActionListener for Replication

Posted by zh...@apache.org.
HBASE-19926 Use a separated class to implement the WALActionListener for Replication


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

Branch: refs/heads/HBASE-19064
Commit: 14420e1b415cd468f652bf0137bda575e0a5980a
Parents: 397d347
Author: zhangduo <zh...@apache.org>
Authored: Sun Feb 4 10:42:33 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 20:36:43 2018 +0800

----------------------------------------------------------------------
 .../replication/regionserver/Replication.java   | 22 +----
 .../regionserver/ReplicationSourceManager.java  | 45 ---------
 .../ReplicationSourceWALActionListener.java     | 98 ++++++++++++++++++++
 .../TestReplicationSourceManager.java           | 27 +-----
 4 files changed, 105 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/14420e1b/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 aaf3beb..7803ac4 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
@@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableName;
 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.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
@@ -42,8 +41,6 @@ import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
 import org.apache.hadoop.hbase.replication.ReplicationTracker;
 import org.apache.hadoop.hbase.replication.ReplicationUtils;
 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.WALProvider;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -127,23 +124,8 @@ public class Replication implements ReplicationSourceService, ReplicationSinkSer
         replicationTracker, conf, this.server, fs, logDir, oldLogDir, clusterId,
         walProvider != null ? walProvider.getWALFileLengthProvider() : p -> OptionalLong.empty());
     if (walProvider != null) {
-      walProvider.addWALActionsListener(new WALActionsListener() {
-
-        @Override
-        public void preLogRoll(Path oldPath, Path newPath) throws IOException {
-          replicationManager.preLogRoll(newPath);
-        }
-
-        @Override
-        public void postLogRoll(Path oldPath, Path newPath) throws IOException {
-          replicationManager.postLogRoll(newPath);
-        }
-
-        @Override
-        public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit) throws IOException {
-          replicationManager.scopeWALEdits(logKey, logEdit);
-        }
-      });
+      walProvider
+        .addWALActionsListener(new ReplicationSourceWALActionListener(conf, replicationManager));
     }
     this.statsThreadPeriod =
         this.conf.getInt("replication.stats.thread.period.seconds", 5 * 60);

http://git-wip-us.apache.org/repos/asf/hbase/blob/14420e1b/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 6e87563..85b2e85 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
@@ -43,8 +43,6 @@ import java.util.stream.Collectors;
 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.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.Server;
@@ -60,13 +58,9 @@ 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.ReplicationTracker;
-import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
-import org.apache.hadoop.hbase.wal.WALEdit;
-import org.apache.hadoop.hbase.wal.WALKey;
-import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -74,8 +68,6 @@ import org.slf4j.LoggerFactory;
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
-
 /**
  * This class is responsible to manage all the replication sources. There are two classes of
  * sources:
@@ -609,43 +601,6 @@ public class ReplicationSourceManager implements ReplicationListener {
     }
   }
 
-  void scopeWALEdits(WALKey logKey, WALEdit logEdit) throws IOException {
-    scopeWALEdits(logKey, logEdit, this.conf);
-  }
-
-  /**
-   * Utility method used to set the correct scopes on each log key. Doesn't set a scope on keys from
-   * compaction WAL edits and if the scope is local.
-   * @param logKey Key that may get scoped according to its edits
-   * @param logEdit Edits used to lookup the scopes
-   * @throws IOException If failed to parse the WALEdit
-   */
-  @VisibleForTesting
-  static void scopeWALEdits(WALKey logKey, WALEdit logEdit, Configuration conf) throws IOException {
-    boolean replicationForBulkLoadEnabled =
-      ReplicationUtils.isReplicationForBulkLoadDataEnabled(conf);
-    boolean foundOtherEdits = false;
-    for (Cell cell : logEdit.getCells()) {
-      if (!CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
-        foundOtherEdits = true;
-        break;
-      }
-    }
-
-    if (!foundOtherEdits && logEdit.getCells().size() > 0) {
-      WALProtos.RegionEventDescriptor maybeEvent =
-        WALEdit.getRegionEventDescriptor(logEdit.getCells().get(0));
-      if (maybeEvent != null &&
-        (maybeEvent.getEventType() == WALProtos.RegionEventDescriptor.EventType.REGION_CLOSE)) {
-        // In serially replication, we use scopes when reading close marker.
-        foundOtherEdits = true;
-      }
-    }
-    if ((!replicationForBulkLoadEnabled && !foundOtherEdits) || logEdit.isReplay()) {
-      ((WALKeyImpl) logKey).serializeReplicationScope(false);
-    }
-  }
-
   @Override
   public void regionServerRemoved(String regionserver) {
     transferQueues(ServerName.valueOf(regionserver));

http://git-wip-us.apache.org/repos/asf/hbase/blob/14420e1b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALActionListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALActionListener.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALActionListener.java
new file mode 100644
index 0000000..eb12614
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALActionListener.java
@@ -0,0 +1,98 @@
+/**
+ * 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.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.regionserver.wal.WALActionsListener;
+import org.apache.hadoop.hbase.replication.ReplicationUtils;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
+
+/**
+ * Used to receive new wals.
+ */
+@InterfaceAudience.Private
+class ReplicationSourceWALActionListener implements WALActionsListener {
+
+  private final Configuration conf;
+
+  private final ReplicationSourceManager manager;
+
+  public ReplicationSourceWALActionListener(Configuration conf, ReplicationSourceManager manager) {
+    this.conf = conf;
+    this.manager = manager;
+  }
+
+  @Override
+  public void preLogRoll(Path oldPath, Path newPath) throws IOException {
+    manager.preLogRoll(newPath);
+  }
+
+  @Override
+  public void postLogRoll(Path oldPath, Path newPath) throws IOException {
+    manager.postLogRoll(newPath);
+  }
+
+  @Override
+  public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit) throws IOException {
+    scopeWALEdits(logKey, logEdit, conf);
+  }
+
+  /**
+   * Utility method used to set the correct scopes on each log key. Doesn't set a scope on keys from
+   * compaction WAL edits and if the scope is local.
+   * @param logKey Key that may get scoped according to its edits
+   * @param logEdit Edits used to lookup the scopes
+   * @throws IOException If failed to parse the WALEdit
+   */
+  @VisibleForTesting
+  static void scopeWALEdits(WALKey logKey, WALEdit logEdit, Configuration conf) throws IOException {
+    boolean replicationForBulkLoadEnabled =
+        ReplicationUtils.isReplicationForBulkLoadDataEnabled(conf);
+    boolean foundOtherEdits = false;
+    for (Cell cell : logEdit.getCells()) {
+      if (!CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
+        foundOtherEdits = true;
+        break;
+      }
+    }
+
+    if (!foundOtherEdits && logEdit.getCells().size() > 0) {
+      WALProtos.RegionEventDescriptor maybeEvent =
+          WALEdit.getRegionEventDescriptor(logEdit.getCells().get(0));
+      if (maybeEvent != null &&
+        (maybeEvent.getEventType() == WALProtos.RegionEventDescriptor.EventType.REGION_CLOSE)) {
+        // In serially replication, we use scopes when reading close marker.
+        foundOtherEdits = true;
+      }
+    }
+    if ((!replicationForBulkLoadEnabled && !foundOtherEdits) || logEdit.isReplay()) {
+      ((WALKeyImpl) logKey).serializeReplicationScope(false);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/14420e1b/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 a8afe2d..a53cba3 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
@@ -63,7 +63,6 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
-import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.replication.ReplicationFactory;
 import org.apache.hadoop.hbase.replication.ReplicationPeer;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
@@ -82,7 +81,6 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALFactory;
-import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
@@ -276,23 +274,8 @@ public abstract class TestReplicationSourceManager {
     WALFactory wals =
       new WALFactory(utility.getConfiguration(), URLEncoder.encode("regionserver:60020", "UTF8"));
     ReplicationSourceManager replicationManager = replication.getReplicationManager();
-    wals.getWALProvider().addWALActionsListener(new WALActionsListener() {
-
-      @Override
-      public void preLogRoll(Path oldPath, Path newPath) throws IOException {
-        replicationManager.preLogRoll(newPath);
-      }
-
-      @Override
-      public void postLogRoll(Path oldPath, Path newPath) throws IOException {
-        replicationManager.postLogRoll(newPath);
-      }
-
-      @Override
-      public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit) throws IOException {
-        replicationManager.scopeWALEdits(logKey, logEdit);
-      }
-    });
+    wals.getWALProvider()
+      .addWALActionsListener(new ReplicationSourceWALActionListener(conf, replicationManager));
     final WAL wal = wals.getWAL(hri);
     manager.init();
     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("tableame"));
@@ -450,7 +433,7 @@ public abstract class TestReplicationSourceManager {
     RegionInfo hri = RegionInfoBuilder.newBuilder(tableName).setStartKey(HConstants.EMPTY_START_ROW)
         .setEndKey(HConstants.EMPTY_END_ROW).build();
     WALEdit edit = WALEdit.createCompaction(hri, compactionDescriptor);
-    ReplicationSourceManager.scopeWALEdits(new WALKeyImpl(), edit, conf);
+    ReplicationSourceWALActionListener.scopeWALEdits(new WALKeyImpl(), edit, conf);
   }
 
   @Test
@@ -462,7 +445,7 @@ public abstract class TestReplicationSourceManager {
     WALKeyImpl logKey = new WALKeyImpl(scope);
 
     // 3. Get the scopes for the key
-    ReplicationSourceManager.scopeWALEdits(logKey, logEdit, conf);
+    ReplicationSourceWALActionListener.scopeWALEdits(logKey, logEdit, conf);
 
     // 4. Assert that no bulk load entry scopes are added if bulk load hfile replication is disabled
     assertNull("No bulk load entries scope should be added if bulk load replication is disabled.",
@@ -481,7 +464,7 @@ public abstract class TestReplicationSourceManager {
     bulkLoadConf.setBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY, true);
 
     // 4. Get the scopes for the key
-    ReplicationSourceManager.scopeWALEdits(logKey, logEdit, bulkLoadConf);
+    ReplicationSourceWALActionListener.scopeWALEdits(logKey, logEdit, bulkLoadConf);
 
     NavigableMap<byte[], Integer> scopes = logKey.getReplicationScopes();
     // Assert family with replication scope global is present in the key scopes


[21/28] hbase git commit: HBASE-19917 Improve RSGroupBasedLoadBalancer#filterServers() to be more efficient

Posted by zh...@apache.org.
HBASE-19917 Improve RSGroupBasedLoadBalancer#filterServers() to be more efficient

Signed-off-by: tedyu <yu...@gmail.com>


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

Branch: refs/heads/HBASE-19064
Commit: 7f7f2b2de53d11dc8ddde6954a4af3599a9e0fa5
Parents: 88d6e06
Author: Xiang Li <li...@freewheel.tv>
Authored: Sat Feb 3 04:46:55 2018 +0000
Committer: tedyu <yu...@gmail.com>
Committed: Sun Feb 4 17:22:41 2018 -0800

----------------------------------------------------------------------
 .../hbase/rsgroup/RSGroupBasedLoadBalancer.java | 22 ++++++++++++--------
 1 file changed, 13 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7f7f2b2d/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
index 550f734..c47972d 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.rsgroup;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -29,6 +28,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.HBaseIOException;
@@ -288,16 +288,20 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
    *          List of servers which are online.
    * @return the list
    */
-  private List<ServerName> filterServers(Collection<Address> servers,
-      Collection<ServerName> onlineServers) {
-    ArrayList<ServerName> finalList = new ArrayList<ServerName>();
-    for (Address server : servers) {
-      for(ServerName curr: onlineServers) {
-        if(curr.getAddress().equals(server)) {
-          finalList.add(curr);
-        }
+  private List<ServerName> filterServers(Set<Address> servers,
+                                         List<ServerName> onlineServers) {
+    /**
+     * servers is actually a TreeSet (see {@link org.apache.hadoop.hbase.rsgroup.RSGroupInfo}),
+     * having its contains()'s time complexity as O(logn), which is good enough.
+     * TODO: consider using HashSet to pursue O(1) for contains() throughout the calling chain
+     * if needed. */
+    ArrayList<ServerName> finalList = new ArrayList<>();
+    for (ServerName onlineServer : onlineServers) {
+      if (servers.contains(onlineServer.getAddress())) {
+        finalList.add(onlineServer);
       }
     }
+
     return finalList;
   }
 


[03/28] hbase git commit: HBASE-19918 Promote TestAsyncClusterAdminApi to LargeTests

Posted by zh...@apache.org.
HBASE-19918 Promote TestAsyncClusterAdminApi to LargeTests


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

Branch: refs/heads/HBASE-19064
Commit: ad580acc893bc87845a0f65e5d32b2b18cccf9c6
Parents: 8143d5a
Author: Guanghao Zhang <zg...@apache.org>
Authored: Fri Feb 2 16:27:16 2018 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Sat Feb 3 09:39:40 2018 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/TestAsyncClusterAdminApi.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ad580acc/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 e73b73f..b70d8c0 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
@@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.junit.BeforeClass;
@@ -58,7 +58,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
-@Category({ ClientTests.class, MediumTests.class })
+@Category({ ClientTests.class, LargeTests.class })
 public class TestAsyncClusterAdminApi extends TestAsyncAdminBase {
 
   @ClassRule


[04/28] hbase git commit: HBASE-19915 Create merged/ daughter region/s with initial state CLOSED from MetaTableAccessor#splitRegion()/ mergeRegions()

Posted by zh...@apache.org.
HBASE-19915 Create merged/ daughter region/s with initial state CLOSED from MetaTableAccessor#splitRegion()/ mergeRegions()

If the initial state is not set to CLOSED while creating merged/ children region/s, merged/
children region/s get stored in meta with state OFFLINE(0). If the master gets restarted after
Merge/ Split procedure stores merged/ children region/s, master startup sequence tries to assign
offline region/s. This is followed by re-assignment of merged/ children region from resumed Merge/
Split procedure.

This bug is related to HBASE-19530.

Signed-off-by: Michael Stack <st...@apache.org>


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

Branch: refs/heads/HBASE-19064
Commit: 811afad103dacfe50788f08b41d804e216af5afc
Parents: ad580ac
Author: Umesh Agashe <ua...@cloudera.com>
Authored: Thu Feb 1 15:43:32 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Fri Feb 2 18:25:20 2018 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/MetaTableAccessor.java   | 15 +++++++++++++++
 1 file changed, 15 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/811afad1/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index 938874a..f80bbc0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -1679,6 +1679,13 @@ public class MetaTableAccessor {
               .setValue(RegionInfo.toByteArray(regionB))
               .build());
 
+      // Set initial state to CLOSED
+      // NOTE: If initial state is not set to CLOSED then merged region gets added with the
+      // default OFFLINE state. If Master gets restarted after this step, start up sequence of
+      // master tries to assign this offline region. This is followed by re-assignments of the
+      // merged region from resumed {@link MergeTableRegionsProcedure}
+      addRegionStateToPut(putOfMerged, RegionState.State.CLOSED);
+
       // Deletes for merging regions
       Delete deleteA = makeDeleteFromRegionInfo(regionA, time);
       Delete deleteB = makeDeleteFromRegionInfo(regionB, time);
@@ -1743,6 +1750,14 @@ public class MetaTableAccessor {
       Put putA = makePutFromRegionInfo(splitA);
       Put putB = makePutFromRegionInfo(splitB);
 
+      // Set initial state to CLOSED
+      // NOTE: If initial state is not set to CLOSED then daughter regions get added with the
+      // default OFFLINE state. If Master gets restarted after this step, start up sequence of
+      // master tries to assign these offline regions. This is followed by re-assignments of the
+      // daughter regions from resumed {@link SplitTableRegionProcedure}
+      addRegionStateToPut(putA, RegionState.State.CLOSED);
+      addRegionStateToPut(putA, RegionState.State.CLOSED);
+
       addSequenceNum(putA, 1, -1, splitA.getReplicaId()); //new regions, openSeqNum = 1 is fine.
       addSequenceNum(putB, 1, -1, splitB.getReplicaId());
 


[17/28] hbase git commit: HBASE-19658 make the test testFlatteningToJumboCellChunkMap() stable, by eliminating the possibility of third cell to be added while in-memory-flush is still in progress

Posted by zh...@apache.org.
HBASE-19658 make the test testFlatteningToJumboCellChunkMap() stable, by eliminating the possibility of third cell to be added while in-memory-flush is still in progress


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

Branch: refs/heads/HBASE-19064
Commit: 170ffbba683217bdb30e5c99f0e728e0dc660d56
Parents: 14420e1
Author: anastas <an...@yahoo-inc.com>
Authored: Sun Feb 4 14:59:10 2018 +0200
Committer: anastas <an...@yahoo-inc.com>
Committed: Sun Feb 4 14:59:10 2018 +0200

----------------------------------------------------------------------
 .../TestCompactingToCellFlatMapMemStore.java     | 19 ++++++++++++-------
 1 file changed, 12 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/170ffbba/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellFlatMapMemStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellFlatMapMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellFlatMapMemStore.java
index 91a4b04..9b81c7f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellFlatMapMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellFlatMapMemStore.java
@@ -752,9 +752,9 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore
     // set memstore to flat into CellChunkMap
     MemoryCompactionPolicy compactionType = MemoryCompactionPolicy.BASIC;
     memstore.getConfiguration().set(CompactingMemStore.COMPACTING_MEMSTORE_TYPE_KEY,
-        String.valueOf(compactionType));
-    ((MyCompactingMemStore)memstore).initiateType(compactionType, memstore.getConfiguration());
-    ((CompactingMemStore)memstore).setIndexType(CompactingMemStore.IndexType.CHUNK_MAP);
+            String.valueOf(compactionType));
+    ((MyCompactingMemStore) memstore).initiateType(compactionType, memstore.getConfiguration());
+    ((CompactingMemStore) memstore).setIndexType(CompactingMemStore.IndexType.CHUNK_MAP);
 
     int numOfCells = 1;
     char[] chars = new char[MemStoreLAB.CHUNK_SIZE_DEFAULT];
@@ -762,7 +762,7 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore
       chars[i] = 'A';
     }
     String bigVal = new String(chars);
-    String[] keys1 = { "A"};
+    String[] keys1 = {"A"};
 
     // make one cell
     byte[] row = Bytes.toBytes(keys1[0]);
@@ -782,7 +782,7 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore
     assertEquals(totalCellsLen, regionServicesForStores.getMemStoreSize());
     assertEquals(totalHeapSize, ((CompactingMemStore) memstore).heapSize());
 
-    ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and flatten
+    ((CompactingMemStore) memstore).flushInMemory(); // push keys to pipeline and flatten
     while (((CompactingMemStore) memstore).isMemStoreFlushingInMemory()) {
       Threads.sleep(10);
     }
@@ -807,12 +807,17 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore
 
     memstore.clearSnapshot(snapshot.getId());
 
-    String[] keys2 = { "C", "D", "E"};
+    // Allocating two big cells (too big for being copied into a regular chunk).
+    String[] keys2 = {"C", "D"};
     addRowsByKeys(memstore, keys2, val);
     while (((CompactingMemStore) memstore).isMemStoreFlushingInMemory()) {
       Threads.sleep(10);
     }
-    totalHeapSize = 1 * oneCellOnCSLMHeapSize + MutableSegment.DEEP_OVERHEAD
+
+    // The in-memory flush size is bigger than the size of a single cell,
+    // but smaller than the size of two cells.
+    // Therefore, the two created cells are flattened together.
+    totalHeapSize = MutableSegment.DEEP_OVERHEAD
             + CellChunkImmutableSegment.DEEP_OVERHEAD_CCM
             + 2 * oneCellOnCCMHeapSize;
     assertEquals(totalHeapSize, ((CompactingMemStore) memstore).heapSize());


[15/28] hbase git commit: HBASE-19905 ReplicationSyncUp tool will not exit if a peer replication is disabled

Posted by zh...@apache.org.
HBASE-19905 ReplicationSyncUp tool will not exit if a peer replication is disabled

Signed-off-by: Ashish Singhi <as...@apache.org>


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

Branch: refs/heads/HBASE-19064
Commit: 397d34736e63d7661a2f01524f8b302e1309d40f
Parents: b0e998f
Author: Ashish Singhi <as...@apache.org>
Authored: Sun Feb 4 17:52:38 2018 +0530
Committer: Ashish Singhi <as...@apache.org>
Committed: Sun Feb 4 17:52:38 2018 +0530

----------------------------------------------------------------------
 .../replication/regionserver/ReplicationSourceManager.java   | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/397d3473/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 2147214..6e87563 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
@@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationListener;
 import org.apache.hadoop.hbase.replication.ReplicationPeer;
+import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
@@ -747,6 +748,13 @@ public class ReplicationSourceManager implements ReplicationListener {
             abortWhenFail(() -> queueStorage.removeQueue(server.getServerName(), queueId));
             continue;
           }
+          if (server instanceof ReplicationSyncUp.DummyServer
+              && peer.getPeerState().equals(PeerState.DISABLED)) {
+            LOG.warn("Peer {} is disbaled. ReplicationSyncUp tool will skip "
+                + "replicating data to this peer.",
+              actualPeerId);
+            continue;
+          }
           // track sources in walsByIdRecoveredQueues
           Map<String, SortedSet<String>> walsByGroup = new HashMap<>();
           walsByIdRecoveredQueues.put(queueId, walsByGroup);


[23/28] hbase git commit: HBASE-19781 Add a new cluster state flag for synchronous replication

Posted by zh...@apache.org.
HBASE-19781 Add a new cluster state flag for synchronous replication


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

Branch: refs/heads/HBASE-19064
Commit: d72ecca33b860834fdf4c1ad2e2285179517b4f5
Parents: d945833
Author: Guanghao Zhang <zg...@apache.org>
Authored: Mon Jan 22 11:44:49 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Feb 5 12:10:54 2018 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/client/Admin.java   |  39 +++++
 .../apache/hadoop/hbase/client/AsyncAdmin.java  |  31 ++++
 .../hadoop/hbase/client/AsyncHBaseAdmin.java    |   7 +
 .../hbase/client/ConnectionImplementation.java  |   9 ++
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |  26 +++
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java |  15 ++
 .../client/ShortCircuitMasterConnection.java    |   9 ++
 .../replication/ReplicationPeerConfigUtil.java  |  26 +--
 .../replication/ReplicationPeerDescription.java |  10 +-
 .../hbase/replication/SyncReplicationState.java |  48 ++++++
 .../hbase/shaded/protobuf/RequestConverter.java |  10 ++
 .../src/main/protobuf/Master.proto              |   4 +
 .../src/main/protobuf/MasterProcedure.proto     |   6 +-
 .../src/main/protobuf/Replication.proto         |  20 +++
 .../replication/ReplicationPeerStorage.java     |  18 ++-
 .../hbase/replication/ReplicationUtils.java     |   1 +
 .../replication/ZKReplicationPeerStorage.java   |  60 +++++--
 .../replication/TestReplicationStateBasic.java  |  23 ++-
 .../TestZKReplicationPeerStorage.java           |  12 +-
 .../hbase/coprocessor/MasterObserver.java       |  23 +++
 .../org/apache/hadoop/hbase/master/HMaster.java |  12 ++
 .../hbase/master/MasterCoprocessorHost.java     |  21 +++
 .../hadoop/hbase/master/MasterRpcServices.java  |  17 ++
 .../hadoop/hbase/master/MasterServices.java     |   9 ++
 .../procedure/PeerProcedureInterface.java       |   2 +-
 .../replication/ReplicationPeerManager.java     |  51 +++++-
 ...ransitPeerSyncReplicationStateProcedure.java | 159 +++++++++++++++++++
 .../hbase/security/access/AccessController.java |   8 +
 .../replication/TestReplicationAdmin.java       |  62 ++++++++
 .../hbase/master/MockNoopMasterServices.java    |  11 +-
 .../cleaner/TestReplicationHFileCleaner.java    |   4 +-
 .../TestReplicationTrackerZKImpl.java           |   6 +-
 .../TestReplicationSourceManager.java           |   3 +-
 .../security/access/TestAccessController.java   |  16 ++
 .../hbase/util/TestHBaseFsckReplication.java    |   5 +-
 .../src/main/ruby/hbase/replication_admin.rb    |  15 ++
 hbase-shell/src/main/ruby/shell.rb              |   1 +
 .../src/main/ruby/shell/commands/list_peers.rb  |   6 +-
 .../transit_peer_sync_replication_state.rb      |  44 +++++
 .../test/ruby/hbase/replication_admin_test.rb   |  24 +++
 40 files changed, 818 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/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 b8546fa..167d6f3 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
@@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
 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.SyncReplicationState;
 import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
@@ -2648,6 +2649,44 @@ public interface Admin extends Abortable, Closeable {
   List<ReplicationPeerDescription> listReplicationPeers(Pattern pattern) throws IOException;
 
   /**
+   * Transit current cluster to a new state in a synchronous replication peer.
+   * @param peerId a short name that identifies the peer
+   * @param state a new state of current cluster
+   * @throws IOException if a remote or network exception occurs
+   */
+  void transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState state)
+      throws IOException;
+
+  /**
+   * Transit current cluster to a new state in a synchronous 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 state a new state of current cluster
+   * @throws IOException if a remote or network exception occurs
+   */
+  Future<Void> transitReplicationPeerSyncReplicationStateAsync(String peerId,
+      SyncReplicationState state) throws IOException;
+
+  /**
+   * Get the current cluster state in a synchronous replication peer.
+   * @param peerId a short name that identifies the peer
+   * @return the current cluster state
+   * @throws IOException if a remote or network exception occurs
+   */
+  default SyncReplicationState getReplicationPeerSyncReplicationState(String peerId)
+      throws IOException {
+    List<ReplicationPeerDescription> peers = listReplicationPeers(Pattern.compile(peerId));
+    if (peers.isEmpty() || !peers.get(0).getPeerId().equals(peerId)) {
+      throw new IOException("Replication peer " + peerId + " does not exist");
+    }
+    return peers.get(0).getSyncReplicationState();
+  }
+
+  /**
    * Mark region server(s) as decommissioned to prevent additional regions from getting
    * assigned to them. Optionally unload the regions on the servers. If there are multiple servers
    * to be decommissioned, decommissioning them at the same time can prevent wasteful region

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
index 35cdd3f..895e7ff 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncAdmin.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client;
 
 import com.google.protobuf.RpcChannel;
 
+import java.io.IOException;
 import java.util.Collection;
 import java.util.EnumSet;
 import java.util.List;
@@ -42,6 +43,7 @@ import org.apache.hadoop.hbase.quotas.QuotaFilter;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -600,6 +602,35 @@ public interface AsyncAdmin {
       ReplicationPeerConfig peerConfig);
 
   /**
+   * Transit current cluster to a new state in a synchronous replication peer.
+   * @param peerId a short name that identifies the peer
+   * @param state a new state of current cluster
+   */
+  CompletableFuture<Void> transitReplicationPeerSyncReplicationState(String peerId,
+      SyncReplicationState state);
+
+  /**
+   * Get the current cluster state in a synchronous replication peer.
+   * @param peerId a short name that identifies the peer
+   * @return the current cluster state wrapped by a {@link CompletableFuture}.
+   */
+  default CompletableFuture<SyncReplicationState>
+      getReplicationPeerSyncReplicationState(String peerId) {
+    CompletableFuture<SyncReplicationState> future = new CompletableFuture<>();
+    listReplicationPeers(Pattern.compile(peerId)).whenComplete((peers, error) -> {
+      if (error != null) {
+        future.completeExceptionally(error);
+      } else if (peers.isEmpty() || !peers.get(0).getPeerId().equals(peerId)) {
+        future.completeExceptionally(
+          new IOException("Replication peer " + peerId + " does not exist"));
+      } else {
+        future.complete(peers.get(0).getSyncReplicationState());
+      }
+    });
+    return future;
+  }
+
+  /**
    * Append the replicable table-cf config of the specified peer
    * @param peerId 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/d72ecca3/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
index 9b2390c..44771fd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncHBaseAdmin.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.quotas.QuotaFilter;
 import org.apache.hadoop.hbase.quotas.QuotaSettings;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -414,6 +415,12 @@ class AsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
+  public CompletableFuture<Void> transitReplicationPeerSyncReplicationState(String peerId,
+      SyncReplicationState clusterState) {
+    return wrap(rawAdmin.transitReplicationPeerSyncReplicationState(peerId, clusterState));
+  }
+
+  @Override
   public CompletableFuture<Void> appendReplicationPeerTableCFs(String peerId,
       Map<TableName, List<String>> tableCfs) {
     return wrap(rawAdmin.appendReplicationPeerTableCFs(peerId, tableCfs));

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/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 8807884..5407c6d 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
@@ -123,6 +123,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListR
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
 
@@ -1724,6 +1726,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
           MasterProtos.ClearDeadServersRequest request) throws ServiceException {
         return stub.clearDeadServers(controller, request);
       }
+
+      @Override
+      public TransitReplicationPeerSyncReplicationStateResponse
+        transitReplicationPeerSyncReplicationState(RpcController controller,
+          TransitReplicationPeerSyncReplicationStateRequest request) throws ServiceException {
+        return stub.transitReplicationPeerSyncReplicationState(controller, request);
+      }
     };
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/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 8685984..c01b891 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
@@ -86,6 +86,7 @@ import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
 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.SyncReplicationState;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
@@ -206,6 +207,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Disab
 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.TransitReplicationPeerSyncReplicationStateResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
 
@@ -3991,6 +3993,30 @@ public class HBaseAdmin implements Admin {
   }
 
   @Override
+  public void transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState state)
+      throws IOException {
+    get(transitReplicationPeerSyncReplicationStateAsync(peerId, state), this.syncWaitTimeout,
+      TimeUnit.MILLISECONDS);
+  }
+
+  @Override
+  public Future<Void> transitReplicationPeerSyncReplicationStateAsync(String peerId,
+      SyncReplicationState state) throws IOException {
+    TransitReplicationPeerSyncReplicationStateResponse response =
+        executeCallable(new MasterCallable<TransitReplicationPeerSyncReplicationStateResponse>(
+          getConnection(), getRpcControllerFactory()) {
+          @Override
+          protected TransitReplicationPeerSyncReplicationStateResponse rpcCall() throws Exception {
+            return master.transitReplicationPeerSyncReplicationState(getRpcController(),
+              RequestConverter.buildTransitReplicationPeerSyncReplicationStateRequest(peerId,
+                state));
+          }
+        });
+    return new ReplicationFuture(this, peerId, response.getProcId(),
+      () -> "TRANSIT_REPLICATION_PEER_SYNCHRONOUS_REPLICATION_STATE");
+  }
+
+  @Override
   public void appendReplicationPeerTableCFs(String id,
       Map<TableName, List<String>> tableCfs)
       throws ReplicationException, IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/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 050bfe2..30a372d 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
@@ -78,6 +78,7 @@ import org.apache.hadoop.hbase.quotas.QuotaTableUtil;
 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.SyncReplicationState;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
@@ -255,6 +256,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListR
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
@@ -1613,6 +1616,18 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
   }
 
   @Override
+  public CompletableFuture<Void> transitReplicationPeerSyncReplicationState(String peerId,
+    SyncReplicationState clusterState) {
+    return this
+      .<TransitReplicationPeerSyncReplicationStateRequest, TransitReplicationPeerSyncReplicationStateResponse> procedureCall(
+        RequestConverter.buildTransitReplicationPeerSyncReplicationStateRequest(peerId,
+          clusterState),
+        (s, c, req, done) -> s.transitReplicationPeerSyncReplicationState(c, req, done),
+        (resp) -> resp.getProcId(), new ReplicationProcedureBiConsumer(peerId,
+          () -> "TRANSIT_REPLICATION_PEER_SYNCHRONOUS_REPLICATION_STATE"));
+  }
+
+  @Override
   public CompletableFuture<Void> appendReplicationPeerTableCFs(String id,
       Map<TableName, List<String>> tableCfs) {
     if (tableCfs == null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
index 50690b4..7bb65d2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitMasterConnection.java
@@ -166,6 +166,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListR
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
 
@@ -638,4 +640,11 @@ public class ShortCircuitMasterConnection implements MasterKeepAliveConnection {
       throws ServiceException {
     return stub.splitRegion(controller, request);
   }
+
+  @Override
+  public TransitReplicationPeerSyncReplicationStateResponse
+    transitReplicationPeerSyncReplicationState(RpcController controller,
+      TransitReplicationPeerSyncReplicationStateRequest request) throws ServiceException {
+    return stub.transitReplicationPeerSyncReplicationState(controller, request);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
index 642149b..86b49ea 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
@@ -38,6 +38,7 @@ 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.SyncReplicationState;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
@@ -391,25 +392,28 @@ public final class ReplicationPeerConfigUtil {
     return ProtobufUtil.prependPBMagic(bytes);
   }
 
-  public static ReplicationPeerDescription toReplicationPeerDescription(
-      ReplicationProtos.ReplicationPeerDescription desc) {
-    boolean enabled = ReplicationProtos.ReplicationState.State.ENABLED == desc.getState()
-        .getState();
+  public static ReplicationPeerDescription
+      toReplicationPeerDescription(ReplicationProtos.ReplicationPeerDescription desc) {
+    boolean enabled =
+        ReplicationProtos.ReplicationState.State.ENABLED == desc.getState().getState();
     ReplicationPeerConfig config = convert(desc.getConfig());
-    return new ReplicationPeerDescription(desc.getId(), enabled, config);
+    return new ReplicationPeerDescription(desc.getId(), enabled, config,
+        SyncReplicationState.valueOf(desc.getSyncReplicationState().getNumber()));
   }
 
-  public static ReplicationProtos.ReplicationPeerDescription toProtoReplicationPeerDescription(
-      ReplicationPeerDescription desc) {
+  public static ReplicationProtos.ReplicationPeerDescription
+      toProtoReplicationPeerDescription(ReplicationPeerDescription desc) {
     ReplicationProtos.ReplicationPeerDescription.Builder builder =
         ReplicationProtos.ReplicationPeerDescription.newBuilder();
     builder.setId(desc.getPeerId());
-    ReplicationProtos.ReplicationState.Builder stateBuilder = ReplicationProtos.ReplicationState
-        .newBuilder();
-    stateBuilder.setState(desc.isEnabled() ? ReplicationProtos.ReplicationState.State.ENABLED
-        : ReplicationProtos.ReplicationState.State.DISABLED);
+    ReplicationProtos.ReplicationState.Builder stateBuilder =
+        ReplicationProtos.ReplicationState.newBuilder();
+    stateBuilder.setState(desc.isEnabled() ? ReplicationProtos.ReplicationState.State.ENABLED :
+        ReplicationProtos.ReplicationState.State.DISABLED);
     builder.setState(stateBuilder.build());
     builder.setConfig(convert(desc.getPeerConfig()));
+    builder.setSyncReplicationState(
+      ReplicationProtos.SyncReplicationState.forNumber(desc.getSyncReplicationState().ordinal()));
     return builder.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java
index ba97d07..2d077c5 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java
@@ -28,11 +28,14 @@ public class ReplicationPeerDescription {
   private final String id;
   private final boolean enabled;
   private final ReplicationPeerConfig config;
+  private final SyncReplicationState syncReplicationState;
 
-  public ReplicationPeerDescription(String id, boolean enabled, ReplicationPeerConfig config) {
+  public ReplicationPeerDescription(String id, boolean enabled, ReplicationPeerConfig config,
+      SyncReplicationState syncReplicationState) {
     this.id = id;
     this.enabled = enabled;
     this.config = config;
+    this.syncReplicationState = syncReplicationState;
   }
 
   public String getPeerId() {
@@ -47,11 +50,16 @@ public class ReplicationPeerDescription {
     return this.config;
   }
 
+  public SyncReplicationState getSyncReplicationState() {
+    return this.syncReplicationState;
+  }
+
   @Override
   public String toString() {
     StringBuilder builder = new StringBuilder("id : ").append(id);
     builder.append(", enabled : " + enabled);
     builder.append(", config : " + config);
+    builder.append(", syncReplicationState : " + syncReplicationState);
     return builder.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java
new file mode 100644
index 0000000..bd144e9
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.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.replication;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Used by synchronous replication. Indicate the state of the current cluster in a synchronous
+ * replication peer. The state may be one of {@link SyncReplicationState#ACTIVE},
+ * {@link SyncReplicationState#DOWNGRADE_ACTIVE} or
+ * {@link SyncReplicationState#STANDBY}.
+ * <p>
+ * For asynchronous replication, the state is {@link SyncReplicationState#NONE}.
+ */
+@InterfaceAudience.Public
+public enum SyncReplicationState {
+  NONE, ACTIVE, DOWNGRADE_ACTIVE, STANDBY;
+
+  public static SyncReplicationState valueOf(int value) {
+    switch (value) {
+      case 0:
+        return NONE;
+      case 1:
+        return ACTIVE;
+      case 2:
+        return DOWNGRADE_ACTIVE;
+      case 3:
+        return STANDBY;
+      default:
+        throw new IllegalArgumentException("Unknown synchronous replication state " + value);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
index 8ac7058..659be2a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/RequestConverter.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Pair;
@@ -146,6 +147,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Enabl
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.GetReplicationPeerConfigRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListReplicationPeersRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
 
 /**
@@ -1874,4 +1876,12 @@ public final class RequestConverter {
     }
     return pbServers;
   }
+
+  public static TransitReplicationPeerSyncReplicationStateRequest
+    buildTransitReplicationPeerSyncReplicationStateRequest(String peerId,
+      SyncReplicationState state) {
+    return TransitReplicationPeerSyncReplicationStateRequest.newBuilder().setPeerId(peerId)
+      .setSyncReplicationState(ReplicationProtos.SyncReplicationState.forNumber(state.ordinal()))
+      .build();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/hbase-protocol-shaded/src/main/protobuf/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/protobuf/Master.proto b/hbase-protocol-shaded/src/main/protobuf/Master.proto
index 3a236c0..c2ab180 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Master.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Master.proto
@@ -962,6 +962,10 @@ service MasterService {
   rpc ListReplicationPeers(ListReplicationPeersRequest)
     returns(ListReplicationPeersResponse);
 
+  /** Transit the state of current cluster in a synchronous replication peer */
+  rpc TransitReplicationPeerSyncReplicationState(TransitReplicationPeerSyncReplicationStateRequest)
+    returns(TransitReplicationPeerSyncReplicationStateResponse);
+
   /** Returns a list of ServerNames marked as decommissioned. */
   rpc ListDecommissionedRegionServers(ListDecommissionedRegionServersRequest)
     returns(ListDecommissionedRegionServersResponse);

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/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 ae676ea..b3af029 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -409,4 +409,8 @@ message AddPeerStateData {
 
 message UpdatePeerConfigStateData {
   required ReplicationPeer peer_config = 1;
-}
\ No newline at end of file
+}
+
+message TransitPeerSyncReplicationStateStateData {
+  required SyncReplicationState syncReplicationState = 1;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/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 44295d8..de7b742 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Replication.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Replication.proto
@@ -63,12 +63,23 @@ message ReplicationState {
 }
 
 /**
+ * Indicate the state of the current cluster in a synchronous replication peer.
+ */
+enum SyncReplicationState {
+  NONE = 0;
+  ACTIVE = 1;
+  DOWNGRADE_ACTIVE = 2;
+  STANDBY = 3;
+}
+
+/**
  * Used by replication. Description of the replication peer.
  */
 message ReplicationPeerDescription {
   required string id = 1;
   required ReplicationState state = 2;
   required ReplicationPeer config = 3;
+  optional SyncReplicationState syncReplicationState = 4;
 }
 
 /**
@@ -137,3 +148,12 @@ message ListReplicationPeersRequest {
 message ListReplicationPeersResponse {
   repeated ReplicationPeerDescription peer_desc = 1;
 }
+
+message TransitReplicationPeerSyncReplicationStateRequest {
+  required string peer_id = 1;
+  required SyncReplicationState syncReplicationState = 2;
+}
+
+message TransitReplicationPeerSyncReplicationStateResponse {
+  required uint64 proc_id = 1;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/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
index 1adda02..d2538ab 100644
--- 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
@@ -31,8 +31,8 @@ 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;
+  void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled,
+      SyncReplicationState syncReplicationState) throws ReplicationException;
 
   /**
    * Remove a replication peer.
@@ -70,4 +70,18 @@ public interface ReplicationPeerStorage {
    * @throws ReplicationException if there are errors accessing the storage service.
    */
   ReplicationPeerConfig getPeerConfig(String peerId) throws ReplicationException;
+
+  /**
+   * Set the state of current cluster in a synchronous replication peer.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  void setPeerSyncReplicationState(String peerId, SyncReplicationState state)
+      throws ReplicationException;
+
+  /**
+   * Get the state of current cluster in a synchronous replication peer.
+   * @throws ReplicationException if there are errors accessing the storage service.
+   */
+  SyncReplicationState getPeerSyncReplicationState(String peerId)
+      throws ReplicationException;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java
index 11507aa..d633be9 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationUtils.java
@@ -22,6 +22,7 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CompoundConfiguration;
 import org.apache.hadoop.hbase.HBaseConfiguration;

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/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
index a53500a..338ce3f 100644
--- 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
@@ -22,6 +22,7 @@ import java.util.List;
 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.Bytes;
 import org.apache.hadoop.hbase.util.CollectionUtils;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
@@ -51,6 +52,8 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase
   public static final byte[] DISABLED_ZNODE_BYTES =
     toByteArray(ReplicationProtos.ReplicationState.State.DISABLED);
 
+  public static final String SYNCHRONOUS_REPLICATION_STATE_ZNODE = "sync-rep-state";
+
   /**
    * The name of the znode that contains the replication status of a remote slave (i.e. peer)
    * cluster.
@@ -79,21 +82,29 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase
     return ZNodePaths.joinZNode(peersZNode, peerId);
   }
 
+  @VisibleForTesting
+  public String getSyncReplicationStateNode(String peerId) {
+    return ZNodePaths.joinZNode(getPeerNode(peerId), SYNCHRONOUS_REPLICATION_STATE_ZNODE);
+  }
+
   @Override
-  public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
-      throws ReplicationException {
+  public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled,
+      SyncReplicationState syncReplicationState) throws ReplicationException {
+    List<ZKUtilOp> multiOps = Arrays.asList(
+      ZKUtilOp.createAndFailSilent(getPeerNode(peerId),
+        ReplicationPeerConfigUtil.toByteArray(peerConfig)),
+      ZKUtilOp.createAndFailSilent(getPeerStateNode(peerId),
+        enabled ? ENABLED_ZNODE_BYTES : DISABLED_ZNODE_BYTES),
+      ZKUtilOp.createAndFailSilent(getSyncReplicationStateNode(peerId),
+        Bytes.toBytes(syncReplicationState.ordinal())));
     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);
+      ZKUtil.multiOrSequential(zookeeper, multiOps, false);
     } catch (KeeperException e) {
-      throw new ReplicationException("Could not add peer with id=" + peerId + ", peerConfif=>"
-          + peerConfig + ", state=" + (enabled ? "ENABLED" : "DISABLED"), e);
+      throw new ReplicationException(
+        "Could not add peer with id=" + peerId + ", peerConfig=>" + peerConfig + ", state=" +
+            (enabled ? "ENABLED" : "DISABLED") + ", syncReplicationState=" + syncReplicationState,
+        e);
     }
   }
 
@@ -166,4 +177,31 @@ public class ZKReplicationPeerStorage extends ZKReplicationStorageBase
           "Failed to parse replication peer config for peer with id=" + peerId, e);
     }
   }
+
+  @Override
+  public void setPeerSyncReplicationState(String peerId, SyncReplicationState clusterState)
+      throws ReplicationException {
+    byte[] clusterStateBytes = Bytes.toBytes(clusterState.ordinal());
+    try {
+      ZKUtil.setData(zookeeper, getSyncReplicationStateNode(peerId), clusterStateBytes);
+    } catch (KeeperException e) {
+      throw new ReplicationException(
+          "Unable to change the cluster state for the synchronous replication peer with id=" +
+              peerId,
+          e);
+    }
+  }
+
+  @Override
+  public SyncReplicationState getPeerSyncReplicationState(String peerId)
+      throws ReplicationException {
+    byte[] data;
+    try {
+      data = ZKUtil.getData(zookeeper, getSyncReplicationStateNode(peerId));
+    } catch (KeeperException | InterruptedException e) {
+      throw new ReplicationException(
+          "Error getting cluster state for the synchronous replication peer with id=" + peerId, e);
+    }
+    return SyncReplicationState.valueOf(Bytes.toInt(data));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/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 fccffb5..fe658a3 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
@@ -160,7 +160,8 @@ public abstract class TestReplicationStateBasic {
     assertTrue(rqs.getReplicableHFiles(ID_ONE).isEmpty());
     assertEquals(0, rqs.getAllPeersFromHFileRefsQueue().size());
     rp.getPeerStorage().addPeer(ID_ONE,
-            ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build(), true);
+      ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build(), true,
+      SyncReplicationState.NONE);
     rqs.addPeerToHFileRefs(ID_ONE);
     rqs.addHFileRefs(ID_ONE, files1);
     assertEquals(1, rqs.getAllPeersFromHFileRefsQueue().size());
@@ -183,10 +184,12 @@ public abstract class TestReplicationStateBasic {
   public void testRemovePeerForHFileRefs() throws ReplicationException, KeeperException {
     rp.init();
     rp.getPeerStorage().addPeer(ID_ONE,
-      ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build(), true);
+      ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build(), true,
+      SyncReplicationState.NONE);
     rqs.addPeerToHFileRefs(ID_ONE);
     rp.getPeerStorage().addPeer(ID_TWO,
-      ReplicationPeerConfig.newBuilder().setClusterKey(KEY_TWO).build(), true);
+      ReplicationPeerConfig.newBuilder().setClusterKey(KEY_TWO).build(), true,
+      SyncReplicationState.NONE);
     rqs.addPeerToHFileRefs(ID_TWO);
 
     List<Pair<Path, Path>> files1 = new ArrayList<>(3);
@@ -235,9 +238,13 @@ public abstract class TestReplicationStateBasic {
     assertNumberOfPeers(0);
 
     // Add some peers
-    rp.getPeerStorage().addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), true);
+    rp.getPeerStorage().addPeer(ID_ONE,
+      ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build(), true,
+      SyncReplicationState.NONE);
     assertNumberOfPeers(1);
-    rp.getPeerStorage().addPeer(ID_TWO, new ReplicationPeerConfig().setClusterKey(KEY_TWO), true);
+    rp.getPeerStorage().addPeer(ID_TWO,
+      ReplicationPeerConfig.newBuilder().setClusterKey(KEY_TWO).build(), true,
+      SyncReplicationState.NONE);
     assertNumberOfPeers(2);
 
     assertEquals(KEY_ONE, ZKConfig.getZooKeeperClusterKey(ReplicationUtils
@@ -247,7 +254,9 @@ public abstract class TestReplicationStateBasic {
     assertNumberOfPeers(1);
 
     // Add one peer
-    rp.getPeerStorage().addPeer(ID_ONE, new ReplicationPeerConfig().setClusterKey(KEY_ONE), true);
+    rp.getPeerStorage().addPeer(ID_ONE,
+      ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build(), true,
+      SyncReplicationState.NONE);
     rp.addPeer(ID_ONE);
     assertNumberOfPeers(2);
     assertTrue(rp.getPeer(ID_ONE).isPeerEnabled());
@@ -311,7 +320,7 @@ public abstract class TestReplicationStateBasic {
       // Add peers for the corresponding queues so they are not orphans
       rp.getPeerStorage().addPeer("qId" + i,
         ReplicationPeerConfig.newBuilder().setClusterKey("localhost:2818:/bogus" + i).build(),
-        true);
+        true, SyncReplicationState.NONE);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/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
index 3290fb0..1258695 100644
--- 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
@@ -87,8 +87,9 @@ public class TestZKReplicationPeerStorage {
     Random rand = new Random(seed);
     return ReplicationPeerConfig.newBuilder().setClusterKey(Long.toHexString(rand.nextLong()))
         .setReplicationEndpointImpl(Long.toHexString(rand.nextLong()))
-        .setNamespaces(randNamespaces(rand)).setExcludeNamespaces(randNamespaces(rand))
-        .setTableCFsMap(randTableCFs(rand)).setReplicateAllUserTables(rand.nextBoolean())
+        .setRemoteWALDir(Long.toHexString(rand.nextLong())).setNamespaces(randNamespaces(rand))
+        .setExcludeNamespaces(randNamespaces(rand)).setTableCFsMap(randTableCFs(rand))
+        .setExcludeTableCFsMap(randTableCFs(rand)).setReplicateAllUserTables(rand.nextBoolean())
         .setBandwidth(rand.nextInt(1000)).build();
   }
 
@@ -139,7 +140,8 @@ public class TestZKReplicationPeerStorage {
   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);
+      STORAGE.addPeer(Integer.toString(i), getConfig(i), i % 2 == 0,
+        SyncReplicationState.valueOf(i % 4));
     }
     List<String> peerIds = STORAGE.listPeerIds();
     assertEquals(peerCount, peerIds.size());
@@ -163,6 +165,10 @@ public class TestZKReplicationPeerStorage {
     for (int i = 0; i < peerCount; i++) {
       assertEquals(i % 2 != 0, STORAGE.isPeerEnabled(Integer.toString(i)));
     }
+    for (int i = 0; i < peerCount; i++) {
+      assertEquals(SyncReplicationState.valueOf(i % 4),
+        STORAGE.getPeerSyncReplicationState(Integer.toString(i)));
+    }
     String toRemove = Integer.toString(peerCount / 2);
     STORAGE.removePeer(toRemove);
     peerIds = STORAGE.listPeerIds();

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
index a17bc9f..8d2b55f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.master.RegionPlan;
 import org.apache.hadoop.hbase.net.Address;
 import org.apache.hadoop.hbase.quotas.GlobalQuotaSettings;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 
@@ -1232,6 +1233,28 @@ public interface MasterObserver {
       String regex) throws IOException {}
 
   /**
+   * Called before transit current cluster state for the specified synchronous replication peer
+   * @param ctx the environment to interact with the framework and master
+   * @param peerId a short name that identifies the peer
+   * @param state a new state
+   */
+  default void preTransitReplicationPeerSyncReplicationState(
+      final ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId,
+      SyncReplicationState state) throws IOException {
+  }
+
+  /**
+   * Called after transit current cluster state for the specified synchronous replication peer
+   * @param ctx the environment to interact with the framework and master
+   * @param peerId a short name that identifies the peer
+   * @param state a new state
+   */
+  default void postTransitReplicationPeerSyncReplicationState(
+      final ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId,
+      SyncReplicationState state) throws IOException {
+  }
+
+  /**
    * Called before new LockProcedure is queued.
    * @param ctx the environment to interact with the framework and master
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/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 ecf6d63..ddffbec 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
@@ -135,6 +135,7 @@ 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.ReplicationPeerManager;
+import org.apache.hadoop.hbase.master.replication.TransitPeerSyncReplicationStateProcedure;
 import org.apache.hadoop.hbase.master.replication.UpdatePeerConfigProcedure;
 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
 import org.apache.hadoop.hbase.mob.MobConstants;
@@ -169,6 +170,7 @@ 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.ReplicationUtils;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
 import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
 import org.apache.hadoop.hbase.replication.master.ReplicationPeerConfigUpgrader;
@@ -3416,6 +3418,16 @@ public class HMaster extends HRegionServer implements MasterServices {
     return peers;
   }
 
+  @Override
+  public long transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState state)
+    throws ReplicationException, IOException {
+    LOG.info(
+      getClientIdAuditPrefix() +
+        " transit current cluster state to {} in a synchronous replication peer id={}",
+      state, peerId);
+    return executePeerProcedure(new TransitPeerSyncReplicationStateProcedure(peerId, state));
+  }
+
   /**
    * Mark region server(s) as decommissioned (previously called 'draining') to prevent additional
    * regions from getting assigned to them. Also unload the regions on the servers asynchronously.0

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index 7f17227..ba5e70a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.quotas.GlobalQuotaSettings;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -1524,6 +1525,26 @@ public class MasterCoprocessorHost
     });
   }
 
+  public void preTransitReplicationPeerSyncReplicationState(final String peerId,
+      final SyncReplicationState clusterState) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver observer) throws IOException {
+        observer.preTransitReplicationPeerSyncReplicationState(this, peerId, clusterState);
+      }
+    });
+  }
+
+  public void postTransitReplicationPeerSyncReplicationState(final String peerId,
+      final SyncReplicationState clusterState) throws IOException {
+    execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+      @Override
+      public void call(MasterObserver observer) throws IOException {
+        observer.postTransitReplicationPeerSyncReplicationState(this, peerId, clusterState);
+      }
+    });
+  }
+
   public void preRequestLock(String namespace, TableName tableName, RegionInfo[] regionInfos,
       LockType type, String description) throws IOException {
     execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/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 377a9c6..f5bf117 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
@@ -83,6 +83,7 @@ import org.apache.hadoop.hbase.regionserver.RpcSchedulerFactory;
 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.SyncReplicationState;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.AccessChecker;
 import org.apache.hadoop.hbase.security.access.AccessController;
@@ -285,6 +286,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ListR
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.RemoveReplicationPeerResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.ReplicationState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.TransitReplicationPeerSyncReplicationStateResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
@@ -1958,6 +1961,20 @@ public class MasterRpcServices extends RSRpcServices
   }
 
   @Override
+  public TransitReplicationPeerSyncReplicationStateResponse
+    transitReplicationPeerSyncReplicationState(RpcController controller,
+      TransitReplicationPeerSyncReplicationStateRequest request) throws ServiceException {
+    try {
+      long procId = master.transitReplicationPeerSyncReplicationState(request.getPeerId(),
+        SyncReplicationState.valueOf(request.getSyncReplicationState().getNumber()));
+      return TransitReplicationPeerSyncReplicationStateResponse.newBuilder().setProcId(procId)
+        .build();
+    } catch (ReplicationException | IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
   public ListReplicationPeersResponse listReplicationPeers(RpcController controller,
       ListReplicationPeersRequest request) throws ServiceException {
     ListReplicationPeersResponse.Builder response = ListReplicationPeersResponse.newBuilder();

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/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 9d371bd..5c6f2dd 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
@@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 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.SyncReplicationState;
 import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
@@ -480,6 +481,14 @@ public interface MasterServices extends Server {
       IOException;
 
   /**
+   * Set current cluster state for a synchronous replication peer.
+   * @param peerId a short name that identifies the peer
+   * @param clusterState state of current cluster
+   */
+  long transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState clusterState)
+      throws ReplicationException, IOException;
+
+  /**
    * @return {@link LockManager} to lock namespaces/tables/regions.
    */
   LockManager getLockManager();

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/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
index 4abc9ad..fc5348e 100644
--- 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
@@ -25,7 +25,7 @@ import org.apache.yetus.audience.InterfaceStability;
 public interface PeerProcedureInterface {
 
   enum PeerOperationType {
-    ADD, REMOVE, ENABLE, DISABLE, UPDATE_CONFIG, REFRESH
+    ADD, REMOVE, ENABLE, DISABLE, UPDATE_CONFIG, REFRESH, TRANSIT_SYNC_REPLICATION_STATE
   }
 
   String getPeerId();

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/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 d715e2e..9336fbd 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
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.master.replication;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.EnumMap;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
@@ -43,6 +45,7 @@ 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.replication.ReplicationUtils;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -61,6 +64,16 @@ public class ReplicationPeerManager {
 
   private final ConcurrentMap<String, ReplicationPeerDescription> peers;
 
+  private final EnumMap<SyncReplicationState, EnumSet<SyncReplicationState>> allowedTransition =
+    new EnumMap<SyncReplicationState, EnumSet<SyncReplicationState>>(SyncReplicationState.class) {
+      {
+        put(SyncReplicationState.ACTIVE, EnumSet.of(SyncReplicationState.DOWNGRADE_ACTIVE));
+        put(SyncReplicationState.STANDBY, EnumSet.of(SyncReplicationState.DOWNGRADE_ACTIVE));
+        put(SyncReplicationState.DOWNGRADE_ACTIVE,
+          EnumSet.of(SyncReplicationState.STANDBY, SyncReplicationState.ACTIVE));
+      }
+    };
+
   ReplicationPeerManager(ReplicationPeerStorage peerStorage, ReplicationQueueStorage queueStorage,
       ConcurrentMap<String, ReplicationPeerDescription> peers) {
     this.peerStorage = peerStorage;
@@ -163,6 +176,17 @@ public class ReplicationPeerManager {
     }
   }
 
+  public void preTransitPeerSyncReplicationState(String peerId, SyncReplicationState state)
+      throws DoNotRetryIOException {
+    ReplicationPeerDescription desc = checkPeerExists(peerId);
+    SyncReplicationState fromState = desc.getSyncReplicationState();
+    EnumSet<SyncReplicationState> allowedToStates = allowedTransition.get(fromState);
+    if (allowedToStates == null || !allowedToStates.contains(state)) {
+      throw new DoNotRetryIOException("Can not transit current cluster state from " + fromState +
+          " to " + state + " for peer id=" + peerId);
+    }
+  }
+
   public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
       throws ReplicationException {
     if (peers.containsKey(peerId)) {
@@ -170,8 +194,12 @@ public class ReplicationPeerManager {
       return;
     }
     ReplicationPeerConfig copiedPeerConfig = ReplicationPeerConfig.newBuilder(peerConfig).build();
-    peerStorage.addPeer(peerId, copiedPeerConfig, enabled);
-    peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, copiedPeerConfig));
+    SyncReplicationState syncReplicationState =
+        StringUtils.isBlank(peerConfig.getRemoteWALDir()) ? SyncReplicationState.NONE
+            : SyncReplicationState.DOWNGRADE_ACTIVE;
+    peerStorage.addPeer(peerId, copiedPeerConfig, enabled, syncReplicationState);
+    peers.put(peerId,
+      new ReplicationPeerDescription(peerId, enabled, copiedPeerConfig, syncReplicationState));
   }
 
   public void removePeer(String peerId) throws ReplicationException {
@@ -190,7 +218,8 @@ public class ReplicationPeerManager {
       return;
     }
     peerStorage.setPeerState(peerId, enabled);
-    peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, desc.getPeerConfig()));
+    peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, desc.getPeerConfig(),
+      desc.getSyncReplicationState()));
   }
 
   public void enablePeer(String peerId) throws ReplicationException {
@@ -215,7 +244,8 @@ public class ReplicationPeerManager {
     newPeerConfigBuilder.putAllConfiguration(peerConfig.getConfiguration());
     ReplicationPeerConfig newPeerConfig = newPeerConfigBuilder.build();
     peerStorage.updatePeerConfig(peerId, newPeerConfig);
-    peers.put(peerId, new ReplicationPeerDescription(peerId, desc.isEnabled(), newPeerConfig));
+    peers.put(peerId, new ReplicationPeerDescription(peerId, desc.isEnabled(), newPeerConfig,
+      desc.getSyncReplicationState()));
   }
 
   public List<ReplicationPeerDescription> listPeers(Pattern pattern) {
@@ -231,6 +261,14 @@ public class ReplicationPeerManager {
     return desc != null ? Optional.of(desc.getPeerConfig()) : Optional.empty();
   }
 
+  public void transitPeerSyncReplicationState(String peerId, SyncReplicationState state)
+      throws ReplicationException {
+    ReplicationPeerDescription desc = peers.get(peerId);
+    peerStorage.setPeerSyncReplicationState(peerId, state);
+    peers.put(peerId,
+      new ReplicationPeerDescription(peerId, desc.isEnabled(), desc.getPeerConfig(), state));
+  }
+
   public void removeAllQueuesAndHFileRefs(String peerId) throws ReplicationException {
     // Here we need two passes to address the problem of claimQueue. Maybe a claimQueue is still
     // on-going when the refresh peer config procedure is done, if a RS which has already been
@@ -350,10 +388,11 @@ public class ReplicationPeerManager {
     for (String peerId : peerStorage.listPeerIds()) {
       ReplicationPeerConfig peerConfig = peerStorage.getPeerConfig(peerId);
       boolean enabled = peerStorage.isPeerEnabled(peerId);
-      peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, peerConfig));
+      SyncReplicationState state = peerStorage.getPeerSyncReplicationState(peerId);
+      peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, peerConfig, state));
     }
     return new ReplicationPeerManager(peerStorage,
-        ReplicationStorageFactory.getReplicationQueueStorage(zk, conf), peers);
+      ReplicationStorageFactory.getReplicationQueueStorage(zk, conf), peers);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java
new file mode 100644
index 0000000..d26eecc
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java
@@ -0,0 +1,159 @@
+/**
+ * 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.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+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.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.TransitPeerSyncReplicationStateStateData;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos;
+
+/**
+ * The procedure for transit current cluster state for a synchronous replication peer.
+ */
+@InterfaceAudience.Private
+public class TransitPeerSyncReplicationStateProcedure extends ModifyPeerProcedure {
+
+  private static final Logger LOG =
+    LoggerFactory.getLogger(TransitPeerSyncReplicationStateProcedure.class);
+
+  private SyncReplicationState state;
+
+  public TransitPeerSyncReplicationStateProcedure() {
+  }
+
+  public TransitPeerSyncReplicationStateProcedure(String peerId, SyncReplicationState state) {
+    super(peerId);
+    this.state = state;
+  }
+
+  @Override
+  public PeerOperationType getPeerOperationType() {
+    return PeerOperationType.TRANSIT_SYNC_REPLICATION_STATE;
+  }
+
+  @Override
+  protected void prePeerModification(MasterProcedureEnv env)
+      throws IOException, ReplicationException {
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      cpHost.preTransitReplicationPeerSyncReplicationState(peerId, state);
+    }
+    env.getReplicationPeerManager().preTransitPeerSyncReplicationState(peerId, state);
+  }
+
+  @Override
+  protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
+    env.getReplicationPeerManager().transitPeerSyncReplicationState(peerId, state);
+  }
+
+  @Override
+  protected void postPeerModification(MasterProcedureEnv env)
+    throws IOException, ReplicationException {
+    LOG.info("Successfully transit current cluster state to {} in synchronous replication peer {}",
+      state, peerId);
+    MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
+    if (cpHost != null) {
+      env.getMasterCoprocessorHost().postTransitReplicationPeerSyncReplicationState(peerId, state);
+    }
+  }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.serializeStateData(serializer);
+    serializer.serialize(TransitPeerSyncReplicationStateStateData.newBuilder()
+      .setSyncReplicationState(ReplicationProtos.SyncReplicationState.forNumber(state.ordinal()))
+      .build());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.deserializeStateData(serializer);
+    TransitPeerSyncReplicationStateStateData data =
+      serializer.deserialize(TransitPeerSyncReplicationStateStateData.class);
+    state = SyncReplicationState.valueOf(data.getSyncReplicationState().getNumber());
+  }
+
+  @Override
+  protected Flow executeFromState(MasterProcedureEnv env, PeerModificationState state)
+      throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+    switch (state) {
+      case PRE_PEER_MODIFICATION:
+        try {
+          prePeerModification(env);
+        } catch (IOException e) {
+          LOG.warn("{} failed to call pre CP hook or the pre check is failed for peer {}, " +
+            "mark the procedure as failure and give up", getClass().getName(), peerId, e);
+          setFailure("master-" + getPeerOperationType().name().toLowerCase() + "-peer", e);
+          releaseLatch();
+          return Flow.NO_MORE_STATE;
+        } catch (ReplicationException e) {
+          LOG.warn("{} failed to call prePeerModification for peer {}, retry", getClass().getName(),
+            peerId, e);
+          throw new ProcedureYieldException();
+        }
+        setNextState(PeerModificationState.UPDATE_PEER_STORAGE);
+        return Flow.HAS_MORE_STATE;
+      case UPDATE_PEER_STORAGE:
+        try {
+          updatePeerStorage(env);
+        } catch (ReplicationException e) {
+          LOG.warn("{} update peer storage for peer {} failed, retry", getClass().getName(), peerId,
+            e);
+          throw new ProcedureYieldException();
+        }
+        setNextState(PeerModificationState.REFRESH_PEER_ON_RS);
+        return Flow.HAS_MORE_STATE;
+      case REFRESH_PEER_ON_RS:
+        // TODO: Need add child procedure for every RegionServer
+        setNextState(PeerModificationState.POST_PEER_MODIFICATION);
+        return Flow.HAS_MORE_STATE;
+      case POST_PEER_MODIFICATION:
+        try {
+          postPeerModification(env);
+        } catch (ReplicationException e) {
+          LOG.warn("{} failed to call postPeerModification for peer {}, retry",
+            getClass().getName(), peerId, e);
+          throw new ProcedureYieldException();
+        } catch (IOException e) {
+          LOG.warn("{} failed to call post CP hook for peer {}, " +
+            "ignore since the procedure has already done", getClass().getName(), peerId, e);
+        }
+        releaseLatch();
+        return Flow.NO_MORE_STATE;
+      default:
+        throw new UnsupportedOperationException("unhandled state=" + state);
+    }
+  }
+
+  private void releaseLatch() {
+    ProcedurePrepareLatch.releaseLatch(latch, this);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index 1fbf01d..decfb4c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -112,6 +112,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTrack
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
@@ -2501,6 +2502,13 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
   }
 
   @Override
+  public void preTransitReplicationPeerSyncReplicationState(
+      final ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId,
+      SyncReplicationState clusterState) throws IOException {
+    requirePermission(ctx, "transitSyncReplicationPeerState", Action.ADMIN);
+  }
+
+  @Override
   public void preListReplicationPeers(final ObserverContext<MasterCoprocessorEnvironment> ctx,
       String regex) throws IOException {
     requirePermission(ctx, "listReplicationPeers", Action.ADMIN);

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/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 e471100..a7710e7 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
@@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfigBuilder;
 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.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.replication.TestReplicationEndpoint.InterClusterReplicationEndpointForTest;
 import org.apache.hadoop.hbase.replication.regionserver.TestReplicator.ReplicationEndpointForTest;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
@@ -983,4 +984,65 @@ public class TestReplicationAdmin {
       // OK
     }
   }
+
+  @Test
+  public void testTransitSyncReplicationPeerState() throws Exception {
+    ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder();
+    builder.setClusterKey(KEY_ONE);
+    hbaseAdmin.addReplicationPeer(ID_ONE, builder.build());
+    assertEquals(SyncReplicationState.NONE,
+      hbaseAdmin.getReplicationPeerSyncReplicationState(ID_ONE));
+
+    try {
+      hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_ONE,
+        SyncReplicationState.DOWNGRADE_ACTIVE);
+      fail("Can't transit cluster state if replication peer don't config remote wal dir");
+    } catch (Exception e) {
+      // OK
+    }
+
+    String rootDir = "hdfs://srv1:9999/hbase";
+    builder = ReplicationPeerConfig.newBuilder();
+    builder.setClusterKey(KEY_SECOND);
+    builder.setRemoteWALDir(rootDir);
+    hbaseAdmin.addReplicationPeer(ID_SECOND, builder.build());
+    assertEquals(SyncReplicationState.DOWNGRADE_ACTIVE,
+      hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND));
+
+    // Disable and enable peer don't affect SyncReplicationState
+    hbaseAdmin.disableReplicationPeer(ID_SECOND);
+    assertEquals(SyncReplicationState.DOWNGRADE_ACTIVE,
+      hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND));
+    hbaseAdmin.enableReplicationPeer(ID_SECOND);
+    assertEquals(SyncReplicationState.DOWNGRADE_ACTIVE,
+      hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND));
+
+    hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND, SyncReplicationState.ACTIVE);
+    assertEquals(SyncReplicationState.ACTIVE,
+      hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND));
+
+    try {
+      hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND,
+        SyncReplicationState.STANDBY);
+      fail("Can't transit cluster state from ACTIVE to STANDBY");
+    } catch (Exception e) {
+      // OK
+    }
+
+    hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND,
+      SyncReplicationState.DOWNGRADE_ACTIVE);
+    assertEquals(SyncReplicationState.DOWNGRADE_ACTIVE,
+      hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND));
+
+    hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND, SyncReplicationState.STANDBY);
+    assertEquals(SyncReplicationState.STANDBY,
+      hbaseAdmin.getReplicationPeerSyncReplicationState(ID_SECOND));
+
+    try {
+      hbaseAdmin.transitReplicationPeerSyncReplicationState(ID_SECOND, SyncReplicationState.ACTIVE);
+      fail("Can't transit cluster state from STANDBY to ACTIVE");
+    } catch (Exception e) {
+      // OK
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/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 48e9e8d..b7d54d6 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
@@ -54,11 +54,10 @@ import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 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.SyncReplicationState;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 
-import com.google.protobuf.Service;
-
 public class MockNoopMasterServices implements MasterServices {
   private final Configuration conf;
   private final MetricsMaster metricsMaster;
@@ -481,4 +480,10 @@ public class MockNoopMasterServices implements MasterServices {
   public ReplicationPeerManager getReplicationPeerManager() {
     return null;
   }
-}
+
+  @Override
+  public long transitReplicationPeerSyncReplicationState(String peerId,
+    SyncReplicationState clusterState) throws ReplicationException, IOException {
+    return 0;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/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 08dd428..24b930c 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
@@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -109,7 +110,8 @@ public class TestReplicationHFileCleaner {
   public void setup() throws ReplicationException, IOException {
     root = TEST_UTIL.getDataTestDirOnTestFS();
     rp.getPeerStorage().addPeer(peerId,
-      ReplicationPeerConfig.newBuilder().setClusterKey(TEST_UTIL.getClusterKey()).build(), true);
+      ReplicationPeerConfig.newBuilder().setClusterKey(TEST_UTIL.getClusterKey()).build(), true,
+      SyncReplicationState.NONE);
     rq.addPeerToHFileRefs(peerId);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/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 52b914e..69121f8 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
@@ -154,11 +154,13 @@ public class TestReplicationTrackerZKImpl {
   public void testPeerNameControl() throws Exception {
     int exists = 0;
     rp.getPeerStorage().addPeer("6",
-      ReplicationPeerConfig.newBuilder().setClusterKey(utility.getClusterKey()).build(), true);
+      ReplicationPeerConfig.newBuilder().setClusterKey(utility.getClusterKey()).build(), true,
+      SyncReplicationState.NONE);
 
     try {
       rp.getPeerStorage().addPeer("6",
-        ReplicationPeerConfig.newBuilder().setClusterKey(utility.getClusterKey()).build(), true);
+        ReplicationPeerConfig.newBuilder().setClusterKey(utility.getClusterKey()).build(), true,
+        SyncReplicationState.NONE);
     } catch (ReplicationException e) {
       if (e.getCause() instanceof KeeperException.NodeExistsException) {
         exists++;

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/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 a53cba3..db62b49 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
@@ -70,6 +70,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeers;
 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.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.replication.ZKReplicationPeerStorage;
 import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.NodeFailoverWorker;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -584,7 +585,7 @@ public abstract class TestReplicationSourceManager {
   private void addPeerAndWait(final String peerId, final ReplicationPeerConfig peerConfig,
       final boolean waitForSource) throws Exception {
     final ReplicationPeers rp = manager.getReplicationPeers();
-    rp.getPeerStorage().addPeer(peerId, peerConfig, true);
+    rp.getPeerStorage().addPeer(peerId, peerConfig, true, SyncReplicationState.NONE);
     try {
       manager.addPeer(peerId);
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/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 ab9fb17..b2b5f1c 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
@@ -117,6 +117,7 @@ import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
 import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.security.Superusers;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.Permission.Action;
@@ -2938,6 +2939,21 @@ public class TestAccessController extends SecureTestUtil {
   }
 
   @Test
+  public void testTransitSyncReplicationPeerState() throws Exception {
+    AccessTestAction action = new AccessTestAction() {
+      @Override
+      public Object run() throws Exception {
+        ACCESS_CONTROLLER.preTransitReplicationPeerSyncReplicationState(
+          ObserverContextImpl.createAndPrepare(CP_ENV), "test", SyncReplicationState.NONE);
+        return null;
+      }
+    };
+
+    verifyAllowed(action, SUPERUSER, USER_ADMIN);
+    verifyDenied(action, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
+  }
+
+  @Test
   public void testListReplicationPeers() throws Exception {
     AccessTestAction action = new AccessTestAction() {
       @Override


[06/28] hbase git commit: HBASE-19919 Tidying up logging

Posted by zh...@apache.org.
HBASE-19919 Tidying up logging


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

Branch: refs/heads/HBASE-19064
Commit: 40250f8c5f8efe3016f0c3300d9728b0ee2056c6
Parents: 41974ef
Author: Michael Stack <st...@apache.org>
Authored: Fri Feb 2 00:23:37 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Fri Feb 2 22:52:41 2018 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/net/TestAddress.java    | 50 ++++++++++++++++++++
 .../hbase/regionserver/RSRpcServices.java.rej   |  9 ++++
 .../ReplicationPeerConfigUpgrader.java.rej      | 10 ++++
 3 files changed, 69 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/40250f8c/hbase-common/src/test/java/org/apache/hadoop/hbase/net/TestAddress.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/net/TestAddress.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/net/TestAddress.java
new file mode 100644
index 0000000..22c7940
--- /dev/null
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/net/TestAddress.java
@@ -0,0 +1,50 @@
+/*
+ * 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.net;
+
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertEquals;
+
+@Category({ MiscTests.class, SmallTests.class })
+public class TestAddress {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestAddress.class);
+
+  @Test
+  public void testGetHostWithoutDomain() {
+    assertEquals("a:123",
+        Address.fromParts("a.b.c", 123).toStringWithoutDomain());
+    assertEquals("1:123",
+        Address.fromParts("1.b.c", 123).toStringWithoutDomain());
+    assertEquals("123.456.789.1:123",
+        Address.fromParts("123.456.789.1", 123).toStringWithoutDomain());
+    assertEquals("[2001:db8::1]:80",
+        Address.fromParts("[2001:db8::1]:", 80).toStringWithoutDomain());
+    assertEquals("[2001:db8::1]:80",
+        Address.fromParts("[2001:db8::1]:", 80).toStringWithoutDomain());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/40250f8c/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java.rej
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java.rej b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java.rej
new file mode 100644
index 0000000..3c7237c
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java.rej
@@ -0,0 +1,9 @@
+diff 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	(rejected hunks)
+@@ -100,6 +100,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.net.Address;
+ import org.apache.hadoop.hbase.quotas.ActivePolicyEnforcement;
+ import org.apache.hadoop.hbase.quotas.OperationQuota;
+ import org.apache.hadoop.hbase.quotas.QuotaUtil;

http://git-wip-us.apache.org/repos/asf/hbase/blob/40250f8c/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java.rej
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java.rej b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java.rej
new file mode 100644
index 0000000..a9829ef
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java.rej
@@ -0,0 +1,10 @@
+diff a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java	(rejected hunks)
+@@ -100,7 +100,7 @@ public class ReplicationPeerConfigUpgrader extends ReplicationStateZKBase {
+         // We only need to copy data from tableCFs node to rpc Node the first time hmaster start.
+         if (rpc.getTableCFsMap() == null || rpc.getTableCFsMap().isEmpty()) {
+           // we copy TableCFs node into PeerNode
+-          LOG.info("copy tableCFs into peerNode:" + peerId);
++          LOG.info("Copy table ColumnFamilies into peer=" + peerId);
+           ReplicationProtos.TableCF[] tableCFs =
+                   ReplicationPeerConfigUtil.parseTableCFs(
+                           ZKUtil.getData(this.zookeeper, tableCFsNode));


[22/28] hbase git commit: HBASE-19781 Add a new cluster state flag for synchronous replication

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java
index 8911982..f5eca39 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE;
@@ -67,9 +68,9 @@ public class TestHBaseFsckReplication {
     String peerId1 = "1";
     String peerId2 = "2";
     peerStorage.addPeer(peerId1, ReplicationPeerConfig.newBuilder().setClusterKey("key").build(),
-      true);
+      true, SyncReplicationState.NONE);
     peerStorage.addPeer(peerId2, ReplicationPeerConfig.newBuilder().setClusterKey("key").build(),
-      true);
+      true, SyncReplicationState.NONE);
     for (int i = 0; i < 10; i++) {
       queueStorage.addWAL(ServerName.valueOf("localhost", 10000 + i, 100000 + i), peerId1,
         "file-" + i);

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/hbase-shell/src/main/ruby/hbase/replication_admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/replication_admin.rb b/hbase-shell/src/main/ruby/hbase/replication_admin.rb
index ba7d191..d5d4844 100644
--- a/hbase-shell/src/main/ruby/hbase/replication_admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/replication_admin.rb
@@ -20,6 +20,7 @@
 include Java
 
 java_import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil
+java_import org.apache.hadoop.hbase.replication.SyncReplicationState
 java_import org.apache.hadoop.hbase.replication.ReplicationPeerConfig
 java_import org.apache.hadoop.hbase.util.Bytes
 java_import org.apache.hadoop.hbase.zookeeper.ZKConfig
@@ -329,6 +330,20 @@ module Hbase
       '!' + ReplicationPeerConfigUtil.convertToString(tableCFs)
     end
 
+    # Transit current cluster to a new state in the specified synchronous
+    # replication peer
+    def transit_peer_sync_replication_state(id, state)
+      if 'ACTIVE'.eql?(state)
+        @admin.transitReplicationPeerSyncReplicationState(id, SyncReplicationState::ACTIVE)
+      elsif 'DOWNGRADE_ACTIVE'.eql?(state)
+        @admin.transitReplicationPeerSyncReplicationState(id, SyncReplicationState::DOWNGRADE_ACTIVE)
+      elsif 'STANDBY'.eql?(state)
+        @admin.transitReplicationPeerSyncReplicationState(id, SyncReplicationState::STANDBY)
+      else
+        raise(ArgumentError, 'synchronous replication state must be ACTIVE, DOWNGRADE_ACTIVE or STANDBY')
+      end
+    end
+
     #----------------------------------------------------------------------------------------------
     # Enables a table's replication switch
     def enable_tablerep(table_name)

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/hbase-shell/src/main/ruby/shell.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell.rb b/hbase-shell/src/main/ruby/shell.rb
index 4a74646..28edd6d 100644
--- a/hbase-shell/src/main/ruby/shell.rb
+++ b/hbase-shell/src/main/ruby/shell.rb
@@ -394,6 +394,7 @@ Shell.load_command_group(
     get_peer_config
     list_peer_configs
     update_peer_config
+    transit_peer_sync_replication_state
   ]
 )
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/hbase-shell/src/main/ruby/shell/commands/list_peers.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/list_peers.rb b/hbase-shell/src/main/ruby/shell/commands/list_peers.rb
index caeab86..aa10fda 100644
--- a/hbase-shell/src/main/ruby/shell/commands/list_peers.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/list_peers.rb
@@ -39,8 +39,8 @@ EOF
         peers = replication_admin.list_peers
 
         formatter.header(%w[PEER_ID CLUSTER_KEY ENDPOINT_CLASSNAME
-                            REMOTE_ROOT_DIR STATE REPLICATE_ALL
-                            NAMESPACES TABLE_CFS BANDWIDTH])
+                            REMOTE_ROOT_DIR SYNC_REPLICATION_STATE STATE
+                            REPLICATE_ALL NAMESPACES TABLE_CFS BANDWIDTH])
 
         peers.each do |peer|
           id = peer.getPeerId
@@ -66,7 +66,7 @@ EOF
             remote_root_dir = config.getRemoteWALDir
           end
           formatter.row([id, cluster_key, endpoint_classname,
-                         remote_root_dir, state,
+                         remote_root_dir, peer.getSyncReplicationState, state,
                          config.replicateAllUserTables, namespaces, tableCFs,
                          config.getBandwidth])
         end

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/hbase-shell/src/main/ruby/shell/commands/transit_peer_sync_replication_state.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/transit_peer_sync_replication_state.rb b/hbase-shell/src/main/ruby/shell/commands/transit_peer_sync_replication_state.rb
new file mode 100644
index 0000000..6d4a963
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/transit_peer_sync_replication_state.rb
@@ -0,0 +1,44 @@
+#
+# Copyright The Apache Software Foundation
+#
+# 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.
+#
+
+module Shell
+  module Commands
+    class TransitPeerSyncReplicationState < Command
+      def help
+        <<-EOF
+Transit current cluster to new state in the specified synchronous replication peer.
+Examples:
+
+  # Transit cluster state to DOWNGRADE_ACTIVE in a synchronous replication peer
+  hbase> transit_peer_sync_replication_state '1', 'DOWNGRADE_ACTIVE'
+  # Transit cluster state to ACTIVE in a synchronous replication peer
+  hbase> transit_peer_sync_replication_state '1', 'ACTIVE'
+  # Transit cluster state to STANDBY in a synchronous replication peer
+  hbase> transit_peer_sync_replication_state '1', 'STANDBY'
+
+EOF
+      end
+
+      def command(id, state)
+        replication_admin.transit_peer_sync_replication_state(id, state)
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/d72ecca3/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb b/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
index 7f2b6ae..a758c96 100644
--- a/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
@@ -23,6 +23,9 @@ require 'hbase/hbase'
 require 'hbase/table'
 
 include HBaseConstants
+include Java
+
+java_import org.apache.hadoop.hbase.replication.SyncReplicationState
 
 module Hbase
   class ReplicationAdminTest < Test::Unit::TestCase
@@ -490,6 +493,27 @@ module Hbase
       command(:remove_peer, @peer_id)
     end
 
+    define_test "transit_peer_sync_replication_state: test" do
+      cluster_key = "server1.cie.com:2181:/hbase"
+      remote_wal_dir = "hdfs://srv1:9999/hbase"
+      args = { CLUSTER_KEY => cluster_key, REMOTE_WAL_DIR => remote_wal_dir }
+      command(:add_peer, @peer_id, args)
+
+      assert_equal(1, command(:list_peers).length)
+      peer = command(:list_peers).get(0)
+      assert_equal(@peer_id, peer.getPeerId)
+      assert_equal(SyncReplicationState::DOWNGRADE_ACTIVE, peer.getSyncReplicationState)
+
+      command(:transit_peer_sync_replication_state, @peer_id, 'ACTIVE')
+      assert_equal(1, command(:list_peers).length)
+      peer = command(:list_peers).get(0)
+      assert_equal(@peer_id, peer.getPeerId)
+      assert_equal(SyncReplicationState::ACTIVE, peer.getSyncReplicationState)
+
+      # cleanup for future tests
+      command(:remove_peer, @peer_id)
+    end
+
     define_test "get_peer_config: works with simple clusterKey peer" do
       cluster_key = "localhost:2181:/hbase-test"
       args = { CLUSTER_KEY => cluster_key }


[10/28] hbase git commit: HBASE-19928 TestVisibilityLabelsOnNewVersionBehaviorTable fails

Posted by zh...@apache.org.
HBASE-19928 TestVisibilityLabelsOnNewVersionBehaviorTable fails


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

Branch: refs/heads/HBASE-19064
Commit: bfd74686c75e8481e4662bbdc519bf4f98e9600c
Parents: 12f3c82
Author: Michael Stack <st...@apache.org>
Authored: Sat Feb 3 10:17:16 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Sat Feb 3 10:17:40 2018 -0800

----------------------------------------------------------------------
 .../TestVisibilityLabelsOnNewVersionBehaviorTable.java    | 10 +++++++++-
 1 file changed, 9 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/bfd74686/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOnNewVersionBehaviorTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOnNewVersionBehaviorTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOnNewVersionBehaviorTable.java
index c14438e..d3177f9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOnNewVersionBehaviorTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOnNewVersionBehaviorTable.java
@@ -19,12 +19,21 @@ package org.apache.hadoop.hbase.security.visibility;
 
 import java.io.IOException;
 
+import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.SecurityTests;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
 
+@Category({SecurityTests.class, MediumTests.class})
 public class TestVisibilityLabelsOnNewVersionBehaviorTable extends TestVisibilityLabelsWithDeletes {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestVisibilityLabelsOnNewVersionBehaviorTable.class);
 
   @Override
   protected Table createTable(HColumnDescriptor fam) throws IOException {
@@ -35,5 +44,4 @@ public class TestVisibilityLabelsOnNewVersionBehaviorTable extends TestVisibilit
     TEST_UTIL.getHBaseAdmin().createTable(table);
     return TEST_UTIL.getConnection().getTable(tableName);
   }
-
 }


[02/28] hbase git commit: HBASE-19848 Zookeeper thread leaks in hbase-spark bulkLoad method (Key Hutu)

Posted by zh...@apache.org.
HBASE-19848 Zookeeper thread leaks in hbase-spark bulkLoad method (Key Hutu)


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

Branch: refs/heads/HBASE-19064
Commit: 8143d5afa4a34c5f06a22e30b5017958b8c3f60c
Parents: 2f4d0b9
Author: tedyu <yu...@gmail.com>
Authored: Fri Feb 2 05:45:13 2018 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Fri Feb 2 05:45:13 2018 -0800

----------------------------------------------------------------------
 .../main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala    | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8143d5af/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala
index eb0d683..d188057 100644
--- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala
+++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala
@@ -706,6 +706,7 @@ class HBaseContext(@transient sc: SparkContext,
         compactionExclude)
       rollOverRequested = false
     })
+    if(null != conn) conn.close()
   }
 
   /**
@@ -868,6 +869,7 @@ class HBaseContext(@transient sc: SparkContext,
         compactionExclude)
       rollOverRequested = false
     })
+    if(null != conn) conn.close()
   }
 
   /**


[26/28] hbase git commit: HBASE-19857 Complete the procedure for adding a sync replication peer

Posted by zh...@apache.org.
HBASE-19857 Complete the procedure for adding a sync replication peer


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

Branch: refs/heads/HBASE-19064
Commit: 0b0ecbe831acda8bf71d0f6c4b343a1fc6d174f7
Parents: d72ecca
Author: zhangduo <zh...@apache.org>
Authored: Thu Jan 25 20:09:00 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Feb 5 12:10:54 2018 +0800

----------------------------------------------------------------------
 .../hbase/replication/ReplicationPeer.java      |   9 +
 .../hbase/replication/ReplicationPeerImpl.java  |  28 +--
 .../hbase/replication/ReplicationPeers.java     |   3 +-
 .../regionserver/PeerActionListener.java        |  10 +-
 .../SyncReplicationPeerProvider.java            |  35 +++
 .../SynchronousReplicationPeerProvider.java     |  35 ---
 .../hbase/wal/SyncReplicationWALProvider.java   | 234 +++++++++++++++++++
 .../wal/SynchronousReplicationWALProvider.java  | 225 ------------------
 .../org/apache/hadoop/hbase/wal/WALFactory.java |   8 +-
 .../TestReplicationSourceManager.java           |   3 +
 .../wal/TestSyncReplicationWALProvider.java     | 153 ++++++++++++
 .../TestSynchronousReplicationWALProvider.java  | 153 ------------
 12 files changed, 456 insertions(+), 440 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0b0ecbe8/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 2da3cce..0196a9a 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
@@ -54,6 +54,15 @@ public interface ReplicationPeer {
   PeerState getPeerState();
 
   /**
+   * Returns the sync replication state of the peer by reading local cache.
+   * <p>
+   * If the peer is not a synchronous replication peer, a {@link SyncReplicationState#NONE} will be
+   * returned.
+   * @return the sync replication state
+   */
+  SyncReplicationState getSyncReplicationState();
+
+  /**
    * Test whether the peer is enabled.
    * @return {@code true} if enabled, otherwise {@code false}.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b0ecbe8/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
index 604e0bb..5ec14cd 100644
--- 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
@@ -36,6 +36,8 @@ public class ReplicationPeerImpl implements ReplicationPeer {
 
   private volatile PeerState peerState;
 
+  private volatile SyncReplicationState syncReplicationState;
+
   private final List<ReplicationPeerConfigListener> peerConfigListeners;
 
   /**
@@ -45,12 +47,13 @@ public class ReplicationPeerImpl implements ReplicationPeer {
    * @param id string representation of this peer's identifier
    * @param peerConfig configuration for the replication peer
    */
-  public ReplicationPeerImpl(Configuration conf, String id, boolean peerState,
-      ReplicationPeerConfig peerConfig) {
+  public ReplicationPeerImpl(Configuration conf, String id, ReplicationPeerConfig peerConfig,
+      boolean peerState, SyncReplicationState syncReplicationState) {
     this.conf = conf;
     this.id = id;
     this.peerState = peerState ? PeerState.ENABLED : PeerState.DISABLED;
     this.peerConfig = peerConfig;
+    this.syncReplicationState = syncReplicationState;
     this.peerConfigListeners = new ArrayList<>();
   }
 
@@ -77,37 +80,26 @@ public class ReplicationPeerImpl implements ReplicationPeer {
     return peerState;
   }
 
-  /**
-   * Get the peer config object
-   * @return the ReplicationPeerConfig for this peer
-   */
+  @Override
+  public SyncReplicationState getSyncReplicationState() {
+    return syncReplicationState;
+  }
+
   @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();

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b0ecbe8/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 eacb2f4..f120dbc 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
@@ -129,7 +129,8 @@ public class ReplicationPeers {
   private ReplicationPeerImpl createPeer(String peerId) throws ReplicationException {
     ReplicationPeerConfig peerConfig = peerStorage.getPeerConfig(peerId);
     boolean enabled = peerStorage.isPeerEnabled(peerId);
+    SyncReplicationState syncReplicationState = peerStorage.getPeerSyncReplicationState(peerId);
     return new ReplicationPeerImpl(ReplicationUtils.getPeerClusterConfiguration(peerConfig, conf),
-        peerId, enabled, peerConfig);
+        peerId, peerConfig, enabled, syncReplicationState);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b0ecbe8/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerActionListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerActionListener.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerActionListener.java
index 74ad626..6df2af9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerActionListener.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerActionListener.java
@@ -17,17 +17,19 @@
  */
 package org.apache.hadoop.hbase.replication.regionserver;
 
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Get notification for replication peer events. Mainly used for telling the
- * {@link org.apache.hadoop.hbase.wal.SynchronousReplicationWALProvider} to close some WAL if not
- * used any more.
- * <p>
- * TODO: Also need a synchronous peer state change notification.
+ * {@link org.apache.hadoop.hbase.wal.SyncReplicationWALProvider} to close some WAL if not used any
+ * more.
  */
 @InterfaceAudience.Private
 public interface PeerActionListener {
 
   default void peerRemoved(String peerId) {}
+
+  default void peerSyncReplicationStateChange(String peerId, SyncReplicationState from,
+      SyncReplicationState to) {}
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b0ecbe8/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerProvider.java
new file mode 100644
index 0000000..b97bf7e
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SyncReplicationPeerProvider.java
@@ -0,0 +1,35 @@
+/**
+ * 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.util.Optional;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Get the peer id and remote root dir if the region is synchronously replicated.
+ */
+@InterfaceAudience.Private
+public interface SyncReplicationPeerProvider {
+
+  /**
+   * Return the peer id and remote WAL directory if the region is synchronously replicated.
+   */
+  Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(RegionInfo info);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b0ecbe8/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SynchronousReplicationPeerProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SynchronousReplicationPeerProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SynchronousReplicationPeerProvider.java
deleted file mode 100644
index b4e04fb..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SynchronousReplicationPeerProvider.java
+++ /dev/null
@@ -1,35 +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.regionserver;
-
-import java.util.Optional;
-import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.yetus.audience.InterfaceAudience;
-
-/**
- * Get the peer id and remote root dir if the region is synchronously replicated.
- */
-@InterfaceAudience.Private
-public interface SynchronousReplicationPeerProvider {
-
-  /**
-   * Return the peer id and remote WAL directory if the region is synchronously replicated.
-   */
-  Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(RegionInfo info);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b0ecbe8/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java
new file mode 100644
index 0000000..bccc842
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java
@@ -0,0 +1,234 @@
+/**
+ * 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.wal;
+
+import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.WAL_FILE_NAME_DELIMITER;
+import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.getWALArchiveDirectoryName;
+import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.getWALDirectoryName;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL;
+import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
+import org.apache.hadoop.hbase.replication.SyncReplicationState;
+import org.apache.hadoop.hbase.replication.regionserver.PeerActionListener;
+import org.apache.hadoop.hbase.replication.regionserver.SyncReplicationPeerProvider;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
+import org.apache.hadoop.hbase.util.KeyLocker;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.Streams;
+import org.apache.hbase.thirdparty.io.netty.channel.Channel;
+import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
+
+/**
+ * The special {@link WALProvider} for synchronous replication.
+ * <p>
+ * It works like an interceptor, when getting WAL, first it will check if the given region should be
+ * replicated synchronously, if so it will return a special WAL for it, otherwise it will delegate
+ * the request to the normal {@link WALProvider}.
+ */
+@InterfaceAudience.Private
+public class SyncReplicationWALProvider implements WALProvider, PeerActionListener {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SyncReplicationWALProvider.class);
+
+  private static final String LOG_SUFFIX = ".syncrep";
+
+  private final WALProvider provider;
+
+  private final SyncReplicationPeerProvider peerProvider;
+
+  private WALFactory factory;
+
+  private Configuration conf;
+
+  private List<WALActionsListener> listeners = new ArrayList<>();
+
+  private EventLoopGroup eventLoopGroup;
+
+  private Class<? extends Channel> channelClass;
+
+  private AtomicBoolean initialized = new AtomicBoolean(false);
+
+  private final ConcurrentMap<String, DualAsyncFSWAL> peerId2WAL = new ConcurrentHashMap<>();
+
+  private final KeyLocker<String> createLock = new KeyLocker<>();
+
+  SyncReplicationWALProvider(WALProvider provider, SyncReplicationPeerProvider peerProvider) {
+    this.provider = provider;
+    this.peerProvider = peerProvider;
+  }
+
+  @Override
+  public void init(WALFactory factory, Configuration conf, String providerId) throws IOException {
+    if (!initialized.compareAndSet(false, true)) {
+      throw new IllegalStateException("WALProvider.init should only be called once.");
+    }
+    provider.init(factory, conf, providerId);
+    this.conf = conf;
+    this.factory = factory;
+    Pair<EventLoopGroup, Class<? extends Channel>> eventLoopGroupAndChannelClass =
+        NettyAsyncFSWALConfigHelper.getEventLoopConfig(conf);
+    eventLoopGroup = eventLoopGroupAndChannelClass.getFirst();
+    channelClass = eventLoopGroupAndChannelClass.getSecond();
+  }
+
+  private String getLogPrefix(String peerId) {
+    return factory.factoryId + WAL_FILE_NAME_DELIMITER + peerId;
+  }
+
+  private DualAsyncFSWAL createWAL(String peerId, String remoteWALDir) throws IOException {
+    Path remoteWALDirPath = new Path(remoteWALDir);
+    FileSystem remoteFs = remoteWALDirPath.getFileSystem(conf);
+    return new DualAsyncFSWAL(CommonFSUtils.getWALFileSystem(conf), remoteFs,
+        CommonFSUtils.getWALRootDir(conf), new Path(remoteWALDirPath, peerId),
+        getWALDirectoryName(factory.factoryId), getWALArchiveDirectoryName(conf, factory.factoryId),
+        conf, listeners, true, getLogPrefix(peerId), LOG_SUFFIX, eventLoopGroup, channelClass);
+  }
+
+  private DualAsyncFSWAL getWAL(String peerId, String remoteWALDir) throws IOException {
+    DualAsyncFSWAL wal = peerId2WAL.get(peerId);
+    if (wal != null) {
+      return wal;
+    }
+    Lock lock = createLock.acquireLock(peerId);
+    try {
+      wal = peerId2WAL.get(peerId);
+      if (wal == null) {
+        wal = createWAL(peerId, remoteWALDir);
+        peerId2WAL.put(peerId, wal);
+        wal.init();
+      }
+      return wal;
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public WAL getWAL(RegionInfo region) throws IOException {
+    Optional<Pair<String, String>> peerIdAndRemoteWALDir =
+        peerProvider.getPeerIdAndRemoteWALDir(region);
+    if (peerIdAndRemoteWALDir.isPresent()) {
+      Pair<String, String> pair = peerIdAndRemoteWALDir.get();
+      return getWAL(pair.getFirst(), pair.getSecond());
+    } else {
+      return provider.getWAL(region);
+    }
+  }
+
+  private Stream<WAL> getWALStream() {
+    return Streams.concat(peerId2WAL.values().stream(), provider.getWALs().stream());
+  }
+
+  @Override
+  public List<WAL> getWALs() {
+    return getWALStream().collect(Collectors.toList());
+  }
+
+  @Override
+  public void shutdown() throws IOException {
+    // save the last exception and rethrow
+    IOException failure = null;
+    for (DualAsyncFSWAL wal : peerId2WAL.values()) {
+      try {
+        wal.shutdown();
+      } catch (IOException e) {
+        LOG.error("Shutdown WAL failed", e);
+        failure = e;
+      }
+    }
+    provider.shutdown();
+    if (failure != null) {
+      throw failure;
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    // save the last exception and rethrow
+    IOException failure = null;
+    for (DualAsyncFSWAL wal : peerId2WAL.values()) {
+      try {
+        wal.close();
+      } catch (IOException e) {
+        LOG.error("Close WAL failed", e);
+        failure = e;
+      }
+    }
+    provider.close();
+    if (failure != null) {
+      throw failure;
+    }
+  }
+
+  @Override
+  public long getNumLogFiles() {
+    return peerId2WAL.size() + provider.getNumLogFiles();
+  }
+
+  @Override
+  public long getLogFileSize() {
+    return peerId2WAL.values().stream().mapToLong(DualAsyncFSWAL::getLogFileSize).sum() +
+      provider.getLogFileSize();
+  }
+
+  private void safeClose(WAL wal) {
+    if (wal != null) {
+      try {
+        wal.close();
+      } catch (IOException e) {
+        LOG.error("Close WAL failed", e);
+      }
+    }
+  }
+
+  @Override
+  public void addWALActionsListener(WALActionsListener listener) {
+    listeners.add(listener);
+    provider.addWALActionsListener(listener);
+  }
+
+  @Override
+  public void peerRemoved(String peerId) {
+    safeClose(peerId2WAL.remove(peerId));
+  }
+
+  @Override
+  public void peerSyncReplicationStateChange(String peerId, SyncReplicationState from,
+      SyncReplicationState to) {
+    assert to == SyncReplicationState.DOWNGRADE_ACTIVE;
+    safeClose(peerId2WAL.remove(peerId));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b0ecbe8/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SynchronousReplicationWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SynchronousReplicationWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SynchronousReplicationWALProvider.java
deleted file mode 100644
index f60599f..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SynchronousReplicationWALProvider.java
+++ /dev/null
@@ -1,225 +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.wal;
-
-import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.WAL_FILE_NAME_DELIMITER;
-import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.getWALArchiveDirectoryName;
-import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.getWALDirectoryName;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Optional;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.locks.Lock;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL;
-import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
-import org.apache.hadoop.hbase.replication.regionserver.PeerActionListener;
-import org.apache.hadoop.hbase.replication.regionserver.SynchronousReplicationPeerProvider;
-import org.apache.hadoop.hbase.util.CommonFSUtils;
-import org.apache.hadoop.hbase.util.KeyLocker;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.hbase.thirdparty.com.google.common.collect.Streams;
-import org.apache.hbase.thirdparty.io.netty.channel.Channel;
-import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
-
-/**
- * The special {@link WALProvider} for synchronous replication.
- * <p>
- * It works like an interceptor, when getting WAL, first it will check if the given region should be
- * replicated synchronously, if so it will return a special WAL for it, otherwise it will delegate
- * the request to the normal {@link WALProvider}.
- */
-@InterfaceAudience.Private
-public class SynchronousReplicationWALProvider implements WALProvider, PeerActionListener {
-
-  private static final Logger LOG =
-    LoggerFactory.getLogger(SynchronousReplicationWALProvider.class);
-
-  private static final String LOG_SUFFIX = ".syncrep";
-
-  private final WALProvider provider;
-
-  private final SynchronousReplicationPeerProvider peerProvider;
-
-  private WALFactory factory;
-
-  private Configuration conf;
-
-  private List<WALActionsListener> listeners = new ArrayList<>();
-
-  private EventLoopGroup eventLoopGroup;
-
-  private Class<? extends Channel> channelClass;
-
-  private AtomicBoolean initialized = new AtomicBoolean(false);
-
-  private final ConcurrentMap<String, DualAsyncFSWAL> peerId2WAL = new ConcurrentHashMap<>();
-
-  private final KeyLocker<String> createLock = new KeyLocker<>();
-
-  SynchronousReplicationWALProvider(WALProvider provider,
-      SynchronousReplicationPeerProvider peerProvider) {
-    this.provider = provider;
-    this.peerProvider = peerProvider;
-  }
-
-  @Override
-  public void init(WALFactory factory, Configuration conf, String providerId) throws IOException {
-    if (!initialized.compareAndSet(false, true)) {
-      throw new IllegalStateException("WALProvider.init should only be called once.");
-    }
-    provider.init(factory, conf, providerId);
-    this.conf = conf;
-    this.factory = factory;
-    Pair<EventLoopGroup, Class<? extends Channel>> eventLoopGroupAndChannelClass =
-      NettyAsyncFSWALConfigHelper.getEventLoopConfig(conf);
-    eventLoopGroup = eventLoopGroupAndChannelClass.getFirst();
-    channelClass = eventLoopGroupAndChannelClass.getSecond();
-  }
-
-  private String getLogPrefix(String peerId) {
-    return factory.factoryId + WAL_FILE_NAME_DELIMITER + peerId;
-  }
-
-  private DualAsyncFSWAL createWAL(String peerId, String remoteWALDir) throws IOException {
-    Path remoteWALDirPath = new Path(remoteWALDir);
-    FileSystem remoteFs = remoteWALDirPath.getFileSystem(conf);
-    return new DualAsyncFSWAL(CommonFSUtils.getWALFileSystem(conf), remoteFs,
-        CommonFSUtils.getWALRootDir(conf), new Path(remoteWALDirPath, peerId),
-        getWALDirectoryName(factory.factoryId), getWALArchiveDirectoryName(conf, factory.factoryId),
-        conf, listeners, true, getLogPrefix(peerId), LOG_SUFFIX, eventLoopGroup, channelClass);
-  }
-
-  private DualAsyncFSWAL getWAL(String peerId, String remoteWALDir) throws IOException {
-    DualAsyncFSWAL wal = peerId2WAL.get(peerId);
-    if (wal != null) {
-      return wal;
-    }
-    Lock lock = createLock.acquireLock(peerId);
-    try {
-      wal = peerId2WAL.get(peerId);
-      if (wal == null) {
-        wal = createWAL(peerId, remoteWALDir);
-        peerId2WAL.put(peerId, wal);
-        wal.init();
-      }
-      return wal;
-    } finally {
-      lock.unlock();
-    }
-  }
-
-  @Override
-  public WAL getWAL(RegionInfo region) throws IOException {
-    Optional<Pair<String, String>> peerIdAndRemoteWALDir =
-      peerProvider.getPeerIdAndRemoteWALDir(region);
-    if (peerIdAndRemoteWALDir.isPresent()) {
-      Pair<String, String> pair = peerIdAndRemoteWALDir.get();
-      return getWAL(pair.getFirst(), pair.getSecond());
-    } else {
-      return provider.getWAL(region);
-    }
-  }
-
-  private Stream<WAL> getWALStream() {
-    return Streams.concat(peerId2WAL.values().stream(), provider.getWALs().stream());
-  }
-
-  @Override
-  public List<WAL> getWALs() {
-    return getWALStream().collect(Collectors.toList());
-  }
-
-  @Override
-  public void shutdown() throws IOException {
-    // save the last exception and rethrow
-    IOException failure = null;
-    for (DualAsyncFSWAL wal : peerId2WAL.values()) {
-      try {
-        wal.shutdown();
-      } catch (IOException e) {
-        LOG.error("Shutdown WAL failed", e);
-        failure = e;
-      }
-    }
-    provider.shutdown();
-    if (failure != null) {
-      throw failure;
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    // save the last exception and rethrow
-    IOException failure = null;
-    for (DualAsyncFSWAL wal : peerId2WAL.values()) {
-      try {
-        wal.close();
-      } catch (IOException e) {
-        LOG.error("Close WAL failed", e);
-        failure = e;
-      }
-    }
-    provider.close();
-    if (failure != null) {
-      throw failure;
-    }
-  }
-
-  @Override
-  public long getNumLogFiles() {
-    return peerId2WAL.size() + provider.getNumLogFiles();
-  }
-
-  @Override
-  public long getLogFileSize() {
-    return peerId2WAL.values().stream().mapToLong(DualAsyncFSWAL::getLogFileSize).sum() +
-      provider.getLogFileSize();
-  }
-
-  @Override
-  public void peerRemoved(String peerId) {
-    WAL wal = peerId2WAL.remove(peerId);
-    if (wal != null) {
-      try {
-        wal.close();
-      } catch (IOException e) {
-        LOG.error("Close WAL failed", e);
-      }
-    }
-  }
-
-  @Override
-  public void addWALActionsListener(WALActionsListener listener) {
-    listeners.add(listener);
-    provider.addWALActionsListener(listener);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b0ecbe8/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
index 4e519ee..06999ea 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
 import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
-import org.apache.hadoop.hbase.replication.regionserver.SynchronousReplicationPeerProvider;
+import org.apache.hadoop.hbase.replication.regionserver.SyncReplicationPeerProvider;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.LeaseNotRecoveredException;
@@ -186,7 +186,7 @@ public class WALFactory {
    * Remove it once we can integrate the synchronous replication logic in RS.
    */
   @VisibleForTesting
-  WALFactory(Configuration conf, String factoryId, SynchronousReplicationPeerProvider peerProvider)
+  WALFactory(Configuration conf, String factoryId, SyncReplicationPeerProvider peerProvider)
       throws IOException {
     timeoutMillis = conf.getInt("hbase.hlog.open.timeout", 300000);
     /* TODO Both of these are probably specific to the fs wal provider */
@@ -195,9 +195,9 @@ public class WALFactory {
     this.conf = conf;
     this.factoryId = factoryId;
     WALProvider provider = createProvider(getProviderClass(WAL_PROVIDER, DEFAULT_WAL_PROVIDER));
-    this.provider = new SynchronousReplicationWALProvider(provider, peerProvider);
-    this.provider.addWALActionsListener(new MetricsWAL());
+    this.provider = new SyncReplicationWALProvider(provider, peerProvider);
     this.provider.init(this, conf, null);
+    this.provider.addWALActionsListener(new MetricsWAL());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b0ecbe8/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 db62b49..1204c9b 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
@@ -172,6 +172,9 @@ public abstract class TestReplicationSourceManager {
     ZKUtil.createWithParents(zkw, "/hbase/replication/peers/1/peer-state");
     ZKUtil.setData(zkw, "/hbase/replication/peers/1/peer-state",
       ZKReplicationPeerStorage.ENABLED_ZNODE_BYTES);
+    ZKUtil.createWithParents(zkw, "/hbase/replication/peers/1/sync-rep-state");
+    ZKUtil.setData(zkw, "/hbase/replication/peers/1/sync-rep-state",
+      Bytes.toBytes(SyncReplicationState.NONE.ordinal()));
     ZKUtil.createWithParents(zkw, "/hbase/replication/state");
     ZKUtil.setData(zkw, "/hbase/replication/state", ZKReplicationPeerStorage.ENABLED_ZNODE_BYTES);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b0ecbe8/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSyncReplicationWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSyncReplicationWALProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSyncReplicationWALProvider.java
new file mode 100644
index 0000000..60a9e13
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSyncReplicationWALProvider.java
@@ -0,0 +1,153 @@
+/**
+ * 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.wal;
+
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.not;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+import java.io.IOException;
+import java.util.Optional;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
+import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL;
+import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
+import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogTestHelper;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ RegionServerTests.class, MediumTests.class })
+public class TestSyncReplicationWALProvider {
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static String PEER_ID = "1";
+
+  private static String REMOTE_WAL_DIR = "/RemoteWAL";
+
+  private static TableName TABLE = TableName.valueOf("table");
+
+  private static TableName TABLE_NO_REP = TableName.valueOf("table-no-rep");
+
+  private static RegionInfo REGION = RegionInfoBuilder.newBuilder(TABLE).build();
+
+  private static RegionInfo REGION_NO_REP = RegionInfoBuilder.newBuilder(TABLE_NO_REP).build();
+
+  private static WALFactory FACTORY;
+
+  private static Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(RegionInfo info) {
+    if (info.getTable().equals(TABLE)) {
+      return Optional.of(Pair.newPair(PEER_ID, REMOTE_WAL_DIR));
+    } else {
+      return Optional.empty();
+    }
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    UTIL.startMiniDFSCluster(3);
+    FACTORY = new WALFactory(UTIL.getConfiguration(), "test",
+        TestSyncReplicationWALProvider::getPeerIdAndRemoteWALDir);
+    UTIL.getTestFileSystem().mkdirs(new Path(REMOTE_WAL_DIR, PEER_ID));
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws IOException {
+    FACTORY.close();
+    UTIL.shutdownMiniDFSCluster();
+  }
+
+  private void testReadWrite(DualAsyncFSWAL wal) throws Exception {
+    int recordCount = 100;
+    int columnCount = 10;
+    byte[] row = Bytes.toBytes("testRow");
+    long timestamp = System.currentTimeMillis();
+    MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
+    ProtobufLogTestHelper.doWrite(wal, REGION, TABLE, columnCount, recordCount, row, timestamp,
+      mvcc);
+    Path localFile = wal.getCurrentFileName();
+    Path remoteFile = new Path(REMOTE_WAL_DIR + "/" + PEER_ID, localFile.getName());
+    try (ProtobufLogReader reader =
+      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), localFile)) {
+      ProtobufLogTestHelper.doRead(reader, false, REGION, TABLE, columnCount, recordCount, row,
+        timestamp);
+    }
+    try (ProtobufLogReader reader =
+      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), remoteFile)) {
+      ProtobufLogTestHelper.doRead(reader, false, REGION, TABLE, columnCount, recordCount, row,
+        timestamp);
+    }
+    wal.rollWriter();
+    DistributedFileSystem dfs = (DistributedFileSystem) UTIL.getDFSCluster().getFileSystem();
+    UTIL.waitFor(5000, new ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        return dfs.isFileClosed(localFile) && dfs.isFileClosed(remoteFile);
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        StringBuilder sb = new StringBuilder();
+        if (!dfs.isFileClosed(localFile)) {
+          sb.append(localFile + " has not been closed yet.");
+        }
+        if (!dfs.isFileClosed(remoteFile)) {
+          sb.append(remoteFile + " has not been closed yet.");
+        }
+        return sb.toString();
+      }
+    });
+    try (ProtobufLogReader reader =
+      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), localFile)) {
+      ProtobufLogTestHelper.doRead(reader, true, REGION, TABLE, columnCount, recordCount, row,
+        timestamp);
+    }
+    try (ProtobufLogReader reader =
+      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), remoteFile)) {
+      ProtobufLogTestHelper.doRead(reader, true, REGION, TABLE, columnCount, recordCount, row,
+        timestamp);
+    }
+  }
+
+  @Test
+  public void test() throws Exception {
+    WAL walNoRep = FACTORY.getWAL(REGION_NO_REP);
+    assertThat(walNoRep, not(instanceOf(DualAsyncFSWAL.class)));
+    DualAsyncFSWAL wal = (DualAsyncFSWAL) FACTORY.getWAL(REGION);
+    assertEquals(2, FACTORY.getWALs().size());
+    testReadWrite(wal);
+    SyncReplicationWALProvider walProvider =
+      (SyncReplicationWALProvider) FACTORY.getWALProvider();
+    walProvider.peerRemoved(PEER_ID);
+    assertEquals(1, FACTORY.getWALs().size());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/0b0ecbe8/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSynchronousReplicationWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSynchronousReplicationWALProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSynchronousReplicationWALProvider.java
deleted file mode 100644
index e6031c6..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSynchronousReplicationWALProvider.java
+++ /dev/null
@@ -1,153 +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.wal;
-
-import static org.hamcrest.CoreMatchers.instanceOf;
-import static org.hamcrest.CoreMatchers.not;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
-
-import java.io.IOException;
-import java.util.Optional;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
-import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.client.RegionInfoBuilder;
-import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
-import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL;
-import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
-import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogTestHelper;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.RegionServerTests;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({ RegionServerTests.class, MediumTests.class })
-public class TestSynchronousReplicationWALProvider {
-
-  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
-
-  private static String PEER_ID = "1";
-
-  private static String REMOTE_WAL_DIR = "/RemoteWAL";
-
-  private static TableName TABLE = TableName.valueOf("table");
-
-  private static TableName TABLE_NO_REP = TableName.valueOf("table-no-rep");
-
-  private static RegionInfo REGION = RegionInfoBuilder.newBuilder(TABLE).build();
-
-  private static RegionInfo REGION_NO_REP = RegionInfoBuilder.newBuilder(TABLE_NO_REP).build();
-
-  private static WALFactory FACTORY;
-
-  private static Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(RegionInfo info) {
-    if (info.getTable().equals(TABLE)) {
-      return Optional.of(Pair.newPair(PEER_ID, REMOTE_WAL_DIR));
-    } else {
-      return Optional.empty();
-    }
-  }
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    UTIL.startMiniDFSCluster(3);
-    FACTORY = new WALFactory(UTIL.getConfiguration(), "test",
-        TestSynchronousReplicationWALProvider::getPeerIdAndRemoteWALDir);
-    UTIL.getTestFileSystem().mkdirs(new Path(REMOTE_WAL_DIR, PEER_ID));
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws IOException {
-    FACTORY.close();
-    UTIL.shutdownMiniDFSCluster();
-  }
-
-  private void testReadWrite(DualAsyncFSWAL wal) throws Exception {
-    int recordCount = 100;
-    int columnCount = 10;
-    byte[] row = Bytes.toBytes("testRow");
-    long timestamp = System.currentTimeMillis();
-    MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
-    ProtobufLogTestHelper.doWrite(wal, REGION, TABLE, columnCount, recordCount, row, timestamp,
-      mvcc);
-    Path localFile = wal.getCurrentFileName();
-    Path remoteFile = new Path(REMOTE_WAL_DIR + "/" + PEER_ID, localFile.getName());
-    try (ProtobufLogReader reader =
-      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), localFile)) {
-      ProtobufLogTestHelper.doRead(reader, false, REGION, TABLE, columnCount, recordCount, row,
-        timestamp);
-    }
-    try (ProtobufLogReader reader =
-      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), remoteFile)) {
-      ProtobufLogTestHelper.doRead(reader, false, REGION, TABLE, columnCount, recordCount, row,
-        timestamp);
-    }
-    wal.rollWriter();
-    DistributedFileSystem dfs = (DistributedFileSystem) UTIL.getDFSCluster().getFileSystem();
-    UTIL.waitFor(5000, new ExplainingPredicate<Exception>() {
-
-      @Override
-      public boolean evaluate() throws Exception {
-        return dfs.isFileClosed(localFile) && dfs.isFileClosed(remoteFile);
-      }
-
-      @Override
-      public String explainFailure() throws Exception {
-        StringBuilder sb = new StringBuilder();
-        if (!dfs.isFileClosed(localFile)) {
-          sb.append(localFile + " has not been closed yet.");
-        }
-        if (!dfs.isFileClosed(remoteFile)) {
-          sb.append(remoteFile + " has not been closed yet.");
-        }
-        return sb.toString();
-      }
-    });
-    try (ProtobufLogReader reader =
-      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), localFile)) {
-      ProtobufLogTestHelper.doRead(reader, true, REGION, TABLE, columnCount, recordCount, row,
-        timestamp);
-    }
-    try (ProtobufLogReader reader =
-      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), remoteFile)) {
-      ProtobufLogTestHelper.doRead(reader, true, REGION, TABLE, columnCount, recordCount, row,
-        timestamp);
-    }
-  }
-
-  @Test
-  public void test() throws Exception {
-    WAL walNoRep = FACTORY.getWAL(REGION_NO_REP);
-    assertThat(walNoRep, not(instanceOf(DualAsyncFSWAL.class)));
-    DualAsyncFSWAL wal = (DualAsyncFSWAL) FACTORY.getWAL(REGION);
-    assertEquals(2, FACTORY.getWALs().size());
-    testReadWrite(wal);
-    SynchronousReplicationWALProvider walProvider =
-      (SynchronousReplicationWALProvider) FACTORY.getWALProvider();
-    walProvider.peerRemoved(PEER_ID);
-    assertEquals(1, FACTORY.getWALs().size());
-  }
-}


[27/28] hbase git commit: HBASE-19078 Add a remote peer cluster wal directory config for synchronous replication

Posted by zh...@apache.org.
HBASE-19078 Add a remote peer cluster wal directory config for synchronous replication

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/6e1c8a3c
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/6e1c8a3c
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/6e1c8a3c

Branch: refs/heads/HBASE-19064
Commit: 6e1c8a3cd957348639b9755299148516ce586d76
Parents: 9eb748f
Author: Guanghao Zhang <zg...@apache.org>
Authored: Sat Jan 13 18:55:28 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Feb 5 12:10:54 2018 +0800

----------------------------------------------------------------------
 .../replication/ReplicationPeerConfigUtil.java  |  6 ++
 .../replication/ReplicationPeerConfig.java      | 21 +++++-
 .../ReplicationPeerConfigBuilder.java           |  7 ++
 .../src/main/protobuf/Replication.proto         |  1 +
 .../replication/ReplicationPeerManager.java     | 15 ++++
 .../replication/TestReplicationAdmin.java       | 77 ++++++++++++++++++++
 .../src/main/ruby/hbase/replication_admin.rb    | 17 +++--
 hbase-shell/src/main/ruby/hbase_constants.rb    |  1 +
 .../src/main/ruby/shell/commands/add_peer.rb    | 21 +++++-
 .../src/main/ruby/shell/commands/list_peers.rb  | 19 ++++-
 .../test/ruby/hbase/replication_admin_test.rb   | 16 ++++
 11 files changed, 188 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6e1c8a3c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
index a234a9b..642149b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java
@@ -315,6 +315,9 @@ public final class ReplicationPeerConfigUtil {
         excludeNamespacesList.stream().map(ByteString::toStringUtf8).collect(Collectors.toSet()));
     }
 
+    if (peer.hasRemoteWALDir()) {
+      builder.setRemoteWALDir(peer.getRemoteWALDir());
+    }
     return builder.build();
   }
 
@@ -371,6 +374,9 @@ public final class ReplicationPeerConfigUtil {
       }
     }
 
+    if (peerConfig.getRemoteWALDir() != null) {
+      builder.setRemoteWALDir(peerConfig.getRemoteWALDir());
+    }
     return builder.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6e1c8a3c/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
index bf8d030..4c10c46 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java
@@ -46,6 +46,8 @@ public class ReplicationPeerConfig {
   private Map<TableName, ? extends Collection<String>> excludeTableCFsMap = null;
   private Set<String> excludeNamespaces = null;
   private long bandwidth = 0;
+  // Used by synchronous replication
+  private String remoteWALDir;
 
   private ReplicationPeerConfig(ReplicationPeerConfigBuilderImpl builder) {
     this.clusterKey = builder.clusterKey;
@@ -64,6 +66,7 @@ public class ReplicationPeerConfig {
         builder.excludeNamespaces != null ? Collections.unmodifiableSet(builder.excludeNamespaces)
             : null;
     this.bandwidth = builder.bandwidth;
+    this.remoteWALDir = builder.remoteWALDir;
   }
 
   private Map<TableName, List<String>>
@@ -210,6 +213,10 @@ public class ReplicationPeerConfig {
     return this;
   }
 
+  public String getRemoteWALDir() {
+    return this.remoteWALDir;
+  }
+
   public static ReplicationPeerConfigBuilder newBuilder() {
     return new ReplicationPeerConfigBuilderImpl();
   }
@@ -223,7 +230,8 @@ public class ReplicationPeerConfig {
         .setReplicateAllUserTables(peerConfig.replicateAllUserTables())
         .setExcludeTableCFsMap(peerConfig.getExcludeTableCFsMap())
         .setExcludeNamespaces(peerConfig.getExcludeNamespaces())
-        .setBandwidth(peerConfig.getBandwidth());
+        .setBandwidth(peerConfig.getBandwidth())
+        .setRemoteWALDir(peerConfig.getRemoteWALDir());
     return builder;
   }
 
@@ -250,6 +258,8 @@ public class ReplicationPeerConfig {
 
     private long bandwidth = 0;
 
+    private String remoteWALDir = null;
+
     @Override
     public ReplicationPeerConfigBuilder setClusterKey(String clusterKey) {
       this.clusterKey = clusterKey;
@@ -313,6 +323,12 @@ public class ReplicationPeerConfig {
     }
 
     @Override
+    public ReplicationPeerConfigBuilder setRemoteWALDir(String dir) {
+      this.remoteWALDir = dir;
+      return this;
+    }
+
+    @Override
     public ReplicationPeerConfig build() {
       // It would be nice to validate the configuration, but we have to work with "old" data
       // from ZK which makes it much more difficult.
@@ -341,6 +357,9 @@ public class ReplicationPeerConfig {
       }
     }
     builder.append("bandwidth=").append(bandwidth);
+    if (this.remoteWALDir != null) {
+      builder.append(",remoteWALDir=").append(remoteWALDir);
+    }
     return builder.toString();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6e1c8a3c/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfigBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfigBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfigBuilder.java
index 0b2f2e2..eac98c6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfigBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfigBuilder.java
@@ -138,6 +138,13 @@ public interface ReplicationPeerConfigBuilder {
   ReplicationPeerConfigBuilder setExcludeNamespaces(Set<String> namespaces);
 
   /**
+   * Set the remote peer cluster's wal directory. Used by synchronous replication.
+   * @param dir the remote peer cluster's wal directory
+   * @return {@code this}
+   */
+  ReplicationPeerConfigBuilder setRemoteWALDir(String dir);
+
+  /**
    * Builds the configuration object from the current state of {@code this}.
    * @return A {@link ReplicationPeerConfig} instance.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/6e1c8a3c/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 9f7b4c2..44295d8 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Replication.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Replication.proto
@@ -48,6 +48,7 @@ message ReplicationPeer {
   optional bool replicate_all = 8;
   repeated TableCF exclude_table_cfs = 9;
   repeated bytes exclude_namespaces = 10;
+  optional string remoteWALDir = 11;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/6e1c8a3c/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 19fc7f4..d715e2e 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
@@ -146,6 +146,21 @@ public class ReplicationPeerManager {
           oldPeerConfig.getReplicationEndpointImpl() + "' for peer " + peerId +
           " does not match new class '" + peerConfig.getReplicationEndpointImpl() + "'");
     }
+
+    if (!isStringEquals(peerConfig.getRemoteWALDir(), oldPeerConfig.getRemoteWALDir())) {
+      throw new DoNotRetryIOException(
+          "Changing the remote wal dir on an existing peer is not allowed. Existing remote wal " +
+              "dir '" + oldPeerConfig.getRemoteWALDir() + "' for peer " + peerId +
+              " does not match new remote wal dir '" + peerConfig.getRemoteWALDir() + "'");
+    }
+
+    if (oldPeerConfig.getRemoteWALDir() != null) {
+      if (!ReplicationUtils.isKeyConfigEqual(oldPeerConfig, peerConfig)) {
+        throw new DoNotRetryIOException(
+            "Changing the replicated namespace/table config on a synchronous replication "
+                + "peer(peerId: " + peerId + ") is not allowed.");
+      }
+    }
   }
 
   public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)

http://git-wip-us.apache.org/repos/asf/hbase/blob/6e1c8a3c/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 685c560..e471100 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
@@ -906,4 +906,81 @@ public class TestReplicationAdmin {
       // OK
     }
   }
+
+  @Test
+  public void testPeerRemoteWALDir() throws Exception {
+    String rootDir = "hdfs://srv1:9999/hbase";
+    ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder();
+    builder.setClusterKey(KEY_ONE);
+    hbaseAdmin.addReplicationPeer(ID_ONE, builder.build());
+
+    ReplicationPeerConfig rpc = hbaseAdmin.getReplicationPeerConfig(ID_ONE);
+    assertNull(rpc.getRemoteWALDir());
+
+    try {
+      builder.setRemoteWALDir("hdfs://srv2:8888/hbase");
+      hbaseAdmin.updateReplicationPeerConfig(ID_ONE, builder.build());
+      fail("Change remote wal dir is not allowed");
+    } catch (Exception e) {
+      // OK
+    }
+
+    builder = ReplicationPeerConfig.newBuilder();
+    builder.setClusterKey(KEY_SECOND);
+    builder.setRemoteWALDir(rootDir);
+    hbaseAdmin.addReplicationPeer(ID_SECOND, builder.build());
+
+    rpc = hbaseAdmin.getReplicationPeerConfig(ID_SECOND);
+    assertEquals(rootDir, rpc.getRemoteWALDir());
+
+    try {
+      builder.setRemoteWALDir("hdfs://srv2:8888/hbase");
+      hbaseAdmin.updateReplicationPeerConfig(ID_SECOND, builder.build());
+      fail("Change remote wal dir is not allowed");
+    } catch (Exception e) {
+      // OK
+    }
+
+    try {
+      builder.setRemoteWALDir(null);
+      hbaseAdmin.updateReplicationPeerConfig(ID_SECOND, builder.build());
+      fail("Change remote wal dir is not allowed");
+    } catch (Exception e) {
+      // OK
+    }
+
+    try {
+      builder = ReplicationPeerConfig.newBuilder(rpc);
+      builder.setReplicateAllUserTables(false);
+      hbaseAdmin.updateReplicationPeerConfig(ID_SECOND, builder.build());
+      fail(
+        "Change replicated namespace/table config on an existing synchronous peer is not allowed");
+    } catch (Exception e) {
+      // OK
+    }
+
+    try {
+      builder = ReplicationPeerConfig.newBuilder(rpc);
+      Set<String> namespaces = new HashSet<>();
+      namespaces.add("ns1");
+      builder.setExcludeNamespaces(namespaces);
+      hbaseAdmin.updateReplicationPeerConfig(ID_SECOND, builder.build());
+      fail(
+        "Change replicated namespace/table config on an existing synchronous peer is not allowed");
+    } catch (Exception e) {
+      // OK
+    }
+
+    try {
+      builder = ReplicationPeerConfig.newBuilder(rpc);
+      Map<TableName, List<String>> tableCfs = new HashMap<>();
+      tableCfs.put(TableName.valueOf(name.getMethodName()), new ArrayList<>());
+      builder.setExcludeTableCFsMap(tableCfs);
+      hbaseAdmin.updateReplicationPeerConfig(ID_SECOND, builder.build());
+      fail(
+        "Change replicated namespace/table config on an existing synchronous peer is not allowed");
+    } catch (Exception e) {
+      // OK
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6e1c8a3c/hbase-shell/src/main/ruby/hbase/replication_admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/replication_admin.rb b/hbase-shell/src/main/ruby/hbase/replication_admin.rb
index b9d4a0c..ba7d191 100644
--- a/hbase-shell/src/main/ruby/hbase/replication_admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/replication_admin.rb
@@ -64,16 +64,20 @@ module Hbase
         table_cfs = args.fetch(TABLE_CFS, nil)
         namespaces = args.fetch(NAMESPACES, nil)
         peer_state = args.fetch(STATE, nil)
+        remote_wal_dir = args.fetch(REMOTE_WAL_DIR, nil)
 
         # Create and populate a ReplicationPeerConfig
-        builder = org.apache.hadoop.hbase.replication.ReplicationPeerConfig
-          .newBuilder()
+        builder = ReplicationPeerConfig.newBuilder()
         builder.set_cluster_key(cluster_key)
 
         unless endpoint_classname.nil?
           builder.set_replication_endpoint_impl(endpoint_classname)
         end
 
+        unless remote_wal_dir.nil?
+          builder.setRemoteWALDir(remote_wal_dir)
+        end
+
         unless config.nil?
           builder.putAllConfiguration(config)
         end
@@ -228,8 +232,7 @@ module Hbase
           namespaces.each do |n|
             ns_set.add(n)
           end
-          builder = org.apache.hadoop.hbase.replication.ReplicationPeerConfig
-            .newBuilder(rpc)
+          builder = ReplicationPeerConfig.newBuilder(rpc)
           builder.setNamespaces(ns_set)
           @admin.updateReplicationPeerConfig(id, builder.build)
         end
@@ -248,8 +251,7 @@ module Hbase
               ns_set.remove(n)
             end
           end
-          builder = org.apache.hadoop.hbase.replication.ReplicationPeerConfig
-            .newBuilder(rpc)
+          builder = ReplicationPeerConfig.newBuilder(rpc)
           builder.setNamespaces(ns_set)
           @admin.updateReplicationPeerConfig(id, builder.build)
         end
@@ -361,8 +363,7 @@ module Hbase
 
       # Create and populate a ReplicationPeerConfig
       replication_peer_config = get_peer_config(id)
-      builder = org.apache.hadoop.hbase.replication.ReplicationPeerConfig
-        .newBuilder(replication_peer_config)
+      builder = ReplicationPeerConfig.newBuilder(replication_peer_config)
       unless config.nil?
         builder.putAllConfiguration(config)
       end

http://git-wip-us.apache.org/repos/asf/hbase/blob/6e1c8a3c/hbase-shell/src/main/ruby/hbase_constants.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase_constants.rb b/hbase-shell/src/main/ruby/hbase_constants.rb
index 28484cb..2870dfb 100644
--- a/hbase-shell/src/main/ruby/hbase_constants.rb
+++ b/hbase-shell/src/main/ruby/hbase_constants.rb
@@ -77,6 +77,7 @@ module HBaseConstants
   VALUE = 'VALUE'.freeze
   ENDPOINT_CLASSNAME = 'ENDPOINT_CLASSNAME'.freeze
   CLUSTER_KEY = 'CLUSTER_KEY'.freeze
+  REMOTE_WAL_DIR = 'REMOTE_WAL_DIR'.freeze
   TABLE_CFS = 'TABLE_CFS'.freeze
   NAMESPACES = 'NAMESPACES'.freeze
   STATE = 'STATE'.freeze

http://git-wip-us.apache.org/repos/asf/hbase/blob/6e1c8a3c/hbase-shell/src/main/ruby/shell/commands/add_peer.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/add_peer.rb b/hbase-shell/src/main/ruby/shell/commands/add_peer.rb
index eb2da83..4b6f294 100644
--- a/hbase-shell/src/main/ruby/shell/commands/add_peer.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/add_peer.rb
@@ -35,7 +35,7 @@ to the peer cluster.
 An optional parameter for table column families identifies which tables and/or column families
 will be replicated to the peer cluster.
 
-Notice: Set a namespace in the peer config means that all tables in this namespace
+Note: Set a namespace in the peer config means that all tables in this namespace
 will be replicated to the peer cluster. So if you already have set a namespace in peer config,
 then you can't set this namespace's tables in the peer config again.
 
@@ -74,6 +74,25 @@ the key TABLE_CFS.
 Note: Either CLUSTER_KEY or ENDPOINT_CLASSNAME must be specified. If ENDPOINT_CLASSNAME is specified, CLUSTER_KEY is
 optional and should only be specified if a particular custom endpoint requires it.
 
+The default replication peer is asynchronous. You can also add a synchronous replication peer
+with REMOTE_WAL_DIR parameter. Meanwhile, synchronous replication peer also support other optional
+config for asynchronous replication peer.
+
+Examples:
+
+  hbase> add_peer '1', CLUSTER_KEY => "server1.cie.com:2181:/hbase",
+    REMOTE_WAL_DIR => "hdfs://srv1:9999/hbase"
+  hbase> add_peer '1', CLUSTER_KEY => "server1.cie.com:2181:/hbase",
+    STATE => "ENABLED", REMOTE_WAL_DIR => "hdfs://srv1:9999/hbase"
+  hbase> add_peer '1', CLUSTER_KEY => "server1.cie.com:2181:/hbase",
+    STATE => "DISABLED", REMOTE_WAL_DIR => "hdfs://srv1:9999/hbase"
+  hbase> add_peer '1', CLUSTER_KEY => "server1.cie.com:2181:/hbase",
+    REMOTE_WAL_DIR => "hdfs://srv1:9999/hbase", NAMESPACES => ["ns1", "ns2"]
+  hbase> add_peer '1', CLUSTER_KEY => "server1.cie.com:2181:/hbase",
+    REMOTE_WAL_DIR => "hdfs://srv1:9999/hbase", TABLE_CFS => { "table1" => [] }
+
+Note: The REMOTE_WAL_DIR is not allowed to change.
+
 EOF
       end
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6e1c8a3c/hbase-shell/src/main/ruby/shell/commands/list_peers.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/list_peers.rb b/hbase-shell/src/main/ruby/shell/commands/list_peers.rb
index 522d23d..caeab86 100644
--- a/hbase-shell/src/main/ruby/shell/commands/list_peers.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/list_peers.rb
@@ -39,7 +39,8 @@ EOF
         peers = replication_admin.list_peers
 
         formatter.header(%w[PEER_ID CLUSTER_KEY ENDPOINT_CLASSNAME
-                            STATE REPLICATE_ALL NAMESPACES TABLE_CFS BANDWIDTH])
+                            REMOTE_ROOT_DIR STATE REPLICATE_ALL
+                            NAMESPACES TABLE_CFS BANDWIDTH])
 
         peers.each do |peer|
           id = peer.getPeerId
@@ -52,8 +53,20 @@ EOF
             namespaces = replication_admin.show_peer_namespaces(config)
             tableCFs = replication_admin.show_peer_tableCFs_by_config(config)
           end
-          formatter.row([id, config.getClusterKey,
-                         config.getReplicationEndpointImpl, state,
+          cluster_key = 'nil'
+          unless config.getClusterKey.nil?
+            cluster_key = config.getClusterKey
+          end
+          endpoint_classname = 'nil'
+          unless config.getReplicationEndpointImpl.nil?
+            endpoint_classname = config.getReplicationEndpointImpl
+          end
+          remote_root_dir = 'nil'
+          unless config.getRemoteWALDir.nil?
+            remote_root_dir = config.getRemoteWALDir
+          end
+          formatter.row([id, cluster_key, endpoint_classname,
+                         remote_root_dir, state,
                          config.replicateAllUserTables, namespaces, tableCFs,
                          config.getBandwidth])
         end

http://git-wip-us.apache.org/repos/asf/hbase/blob/6e1c8a3c/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb b/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
index 0f84396..7f2b6ae 100644
--- a/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
@@ -97,6 +97,22 @@ module Hbase
       command(:remove_peer, @peer_id)
     end
 
+    define_test "add_peer: remote wal dir" do
+      cluster_key = "server1.cie.com:2181:/hbase"
+      remote_wal_dir = "hdfs://srv1:9999/hbase"
+      args = { CLUSTER_KEY => cluster_key, REMOTE_WAL_DIR => remote_wal_dir }
+      command(:add_peer, @peer_id, args)
+
+      assert_equal(1, command(:list_peers).length)
+      peer = command(:list_peers).get(0)
+      assert_equal(@peer_id, peer.getPeerId)
+      assert_equal(cluster_key, peer.getPeerConfig.getClusterKey)
+      assert_equal(remote_wal_dir, peer.getPeerConfig.getRemoteWALDir)
+
+      # cleanup for future tests
+      command(:remove_peer, @peer_id)
+    end
+
     define_test "add_peer: single zk cluster key with enabled/disabled state" do
       cluster_key = "server1.cie.com:2181:/hbase"
 


[13/28] hbase git commit: HBASE-19914 Refactor TestVisibilityLabelsOnNewVersionBehaviorTable

Posted by zh...@apache.org.
HBASE-19914 Refactor TestVisibilityLabelsOnNewVersionBehaviorTable


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

Branch: refs/heads/HBASE-19064
Commit: 2e1ec3d3d8dc4ef771463decb814e7c118523bf9
Parents: bfd7468
Author: zhangduo <zh...@apache.org>
Authored: Sun Feb 4 08:38:46 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sun Feb 4 13:27:28 2018 +0800

----------------------------------------------------------------------
 .../client/ColumnFamilyDescriptorBuilder.java   |    5 +
 ...sibilityLabelsOnNewVersionBehaviorTable.java |   25 +-
 ...ibilityLabelsWithDefaultVisLabelService.java |    4 +-
 .../TestVisibilityLabelsWithDeletes.java        | 1476 +++++++-----------
 .../VisibilityLabelsWithDeletesTestBase.java    |  313 ++++
 5 files changed, 886 insertions(+), 937 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2e1ec3d3/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java
index aaa460b..d7feea6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.java
@@ -552,6 +552,11 @@ public class ColumnFamilyDescriptorBuilder {
     return this;
   }
 
+  public ColumnFamilyDescriptorBuilder setNewVersionBehavior(final boolean value) {
+    desc.setNewVersionBehavior(value);
+    return this;
+  }
+
   public ColumnFamilyDescriptorBuilder setValue(final Bytes key, final Bytes value) {
     desc.setValue(key, value);
     return this;

http://git-wip-us.apache.org/repos/asf/hbase/blob/2e1ec3d3/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOnNewVersionBehaviorTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOnNewVersionBehaviorTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOnNewVersionBehaviorTable.java
index d3177f9..4093ace 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOnNewVersionBehaviorTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsOnNewVersionBehaviorTable.java
@@ -18,30 +18,33 @@
 package org.apache.hadoop.hbase.security.visibility;
 
 import java.io.IOException;
-
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.SecurityTests;
 import org.junit.ClassRule;
 import org.junit.experimental.categories.Category;
 
-@Category({SecurityTests.class, MediumTests.class})
-public class TestVisibilityLabelsOnNewVersionBehaviorTable extends TestVisibilityLabelsWithDeletes {
+
+@Category({ SecurityTests.class, MediumTests.class })
+public class TestVisibilityLabelsOnNewVersionBehaviorTable
+    extends VisibilityLabelsWithDeletesTestBase {
+
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestVisibilityLabelsOnNewVersionBehaviorTable.class);
 
   @Override
-  protected Table createTable(HColumnDescriptor fam) throws IOException {
-    fam.setNewVersionBehavior(true);
-    TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
-    HTableDescriptor table = new HTableDescriptor(tableName);
-    table.addFamily(fam);
-    TEST_UTIL.getHBaseAdmin().createTable(table);
+  protected Table createTable(byte[] fam) throws IOException {
+    TableName tableName = TableName.valueOf(testName.getMethodName());
+    TEST_UTIL.getAdmin()
+        .createTable(TableDescriptorBuilder.newBuilder(tableName)
+            .addColumnFamily(
+              ColumnFamilyDescriptorBuilder.newBuilder(fam).setNewVersionBehavior(true).build())
+            .build());
     return TEST_UTIL.getConnection().getTable(tableName);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2e1ec3d3/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java
index 632665f..0d7ff68 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabelsWithDefaultVisLabelService.java
@@ -114,7 +114,7 @@ public class TestVisibilityLabelsWithDefaultVisLabelService extends TestVisibili
     SUPERUSER.runAs(action);
   }
 
-  @Test(timeout = 60 * 1000)
+  @Test
   public void testAddVisibilityLabelsOnRSRestart() throws Exception {
     List<RegionServerThread> regionServerThreads = TEST_UTIL.getHBaseCluster()
         .getRegionServerThreads();
@@ -227,7 +227,7 @@ public class TestVisibilityLabelsWithDefaultVisLabelService extends TestVisibili
     SUPERUSER.runAs(action);
   }
 
-  @Test(timeout = 60 * 1000)
+  @Test
   public void testVisibilityLabelsOnWALReplay() throws Exception {
     final TableName tableName = TableName.valueOf(TEST_NAME.getMethodName());
     try (Table table = createTableAndWriteDataWithLabels(tableName,


[28/28] hbase git commit: HBASE-19747 Introduce a special WALProvider for synchronous replication

Posted by zh...@apache.org.
HBASE-19747 Introduce a special WALProvider for synchronous replication


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

Branch: refs/heads/HBASE-19064
Commit: d945833963b5ead47dbac4ebefbcd829806acdeb
Parents: 6e1c8a3
Author: zhangduo <zh...@apache.org>
Authored: Fri Jan 19 18:38:39 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Feb 5 12:10:54 2018 +0800

----------------------------------------------------------------------
 .../hbase/regionserver/wal/AbstractFSWAL.java   |   7 +
 .../hbase/regionserver/wal/AsyncFSWAL.java      |   1 -
 .../hbase/regionserver/wal/DualAsyncFSWAL.java  |   4 +-
 .../hadoop/hbase/regionserver/wal/FSHLog.java   |   3 -
 .../regionserver/PeerActionListener.java        |  33 +++
 .../SynchronousReplicationPeerProvider.java     |  35 +++
 .../hadoop/hbase/wal/AbstractFSWALProvider.java |   1 +
 .../hadoop/hbase/wal/AsyncFSWALProvider.java    |  18 +-
 .../hbase/wal/NettyAsyncFSWALConfigHelper.java  |   8 +-
 .../hbase/wal/RegionGroupingProvider.java       |  13 +-
 .../wal/SynchronousReplicationWALProvider.java  | 225 +++++++++++++++++++
 .../org/apache/hadoop/hbase/wal/WALFactory.java |  37 ++-
 .../org/apache/hadoop/hbase/wal/WALKeyImpl.java |  16 +-
 .../regionserver/TestCompactionPolicy.java      |   1 +
 .../regionserver/TestFailedAppendAndSync.java   | 122 +++++-----
 .../hadoop/hbase/regionserver/TestHRegion.java  |  24 +-
 .../TestHRegionWithInMemoryFlush.java           |   7 -
 .../hbase/regionserver/TestRegionIncrement.java |  20 +-
 .../hbase/regionserver/TestWALLockup.java       |  11 +-
 .../regionserver/wal/AbstractTestWALReplay.java |   1 +
 .../regionserver/wal/ProtobufLogTestHelper.java |  44 +++-
 .../hbase/regionserver/wal/TestAsyncFSWAL.java  |  13 +-
 .../regionserver/wal/TestAsyncWALReplay.java    |   4 +-
 .../wal/TestCombinedAsyncWriter.java            |   3 +-
 .../hbase/regionserver/wal/TestFSHLog.java      |  15 +-
 .../hbase/regionserver/wal/TestWALReplay.java   |   1 +
 .../apache/hadoop/hbase/wal/IOTestProvider.java |   2 -
 .../TestSynchronousReplicationWALProvider.java  | 153 +++++++++++++
 28 files changed, 663 insertions(+), 159 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
index 54a5cd3..a37bda3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
@@ -435,6 +435,13 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
     this.implClassName = getClass().getSimpleName();
   }
 
+  /**
+   * Used to initialize the WAL. Usually just call rollWriter to create the first log writer.
+   */
+  public void init() throws IOException {
+    rollWriter();
+  }
+
   @Override
   public void registerWALActionsListener(WALActionsListener listener) {
     this.listeners.add(listener);

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
index 24094e0..b741260 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
@@ -247,7 +247,6 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     batchSize = conf.getLong(WAL_BATCH_SIZE, DEFAULT_WAL_BATCH_SIZE);
     waitOnShutdownInSeconds = conf.getInt(ASYNC_WAL_WAIT_ON_SHUTDOWN_IN_SECONDS,
       DEFAULT_ASYNC_WAL_WAIT_ON_SHUTDOWN_IN_SECONDS);
-    rollWriter();
   }
 
   private static boolean waitingRoll(int epochAndState) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java
index 6bf9e02..f92ce93 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java
@@ -38,14 +38,14 @@ public class DualAsyncFSWAL extends AsyncFSWAL {
 
   private final Path remoteWalDir;
 
-  public DualAsyncFSWAL(FileSystem fs, FileSystem remoteFs, Path rootDir, Path remoteRootDir,
+  public DualAsyncFSWAL(FileSystem fs, FileSystem remoteFs, Path rootDir, Path remoteWalDir,
       String logDir, String archiveDir, Configuration conf, List<WALActionsListener> listeners,
       boolean failIfWALExists, String prefix, String suffix, EventLoopGroup eventLoopGroup,
       Class<? extends Channel> channelClass) throws FailedLogCloseException, IOException {
     super(fs, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix, suffix,
         eventLoopGroup, channelClass);
     this.remoteFs = remoteFs;
-    this.remoteWalDir = new Path(remoteRootDir, logDir);
+    this.remoteWalDir = remoteWalDir;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
index 7ff0453..3fef09b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
@@ -212,9 +212,6 @@ public class FSHLog extends AbstractFSWAL<Writer> {
       5);
     this.closeErrorsTolerated = conf.getInt("hbase.regionserver.logroll.errors.tolerated", 2);
 
-    // rollWriter sets this.hdfs_out if it can.
-    rollWriter();
-
     // This is the 'writer' -- a single threaded executor. This single thread 'consumes' what is
     // put on the ring buffer.
     String hostingThreadName = Thread.currentThread().getName();

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerActionListener.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerActionListener.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerActionListener.java
new file mode 100644
index 0000000..74ad626
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerActionListener.java
@@ -0,0 +1,33 @@
+/**
+ * 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.yetus.audience.InterfaceAudience;
+
+/**
+ * Get notification for replication peer events. Mainly used for telling the
+ * {@link org.apache.hadoop.hbase.wal.SynchronousReplicationWALProvider} to close some WAL if not
+ * used any more.
+ * <p>
+ * TODO: Also need a synchronous peer state change notification.
+ */
+@InterfaceAudience.Private
+public interface PeerActionListener {
+
+  default void peerRemoved(String peerId) {}
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SynchronousReplicationPeerProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SynchronousReplicationPeerProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SynchronousReplicationPeerProvider.java
new file mode 100644
index 0000000..b4e04fb
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SynchronousReplicationPeerProvider.java
@@ -0,0 +1,35 @@
+/**
+ * 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.util.Optional;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Get the peer id and remote root dir if the region is synchronously replicated.
+ */
+@InterfaceAudience.Private
+public interface SynchronousReplicationPeerProvider {
+
+  /**
+   * Return the peer id and remote WAL directory if the region is synchronously replicated.
+   */
+  Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(RegionInfo info);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
index 231afd5..3eb8f8f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
@@ -137,6 +137,7 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
         if (walCopy == null) {
           walCopy = createWAL();
           wal = walCopy;
+          walCopy.init();
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java
index 9c62bed..84e859d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.wal;
 
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -31,12 +30,10 @@ import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
 import org.apache.hbase.thirdparty.io.netty.channel.Channel;
 import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
-import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
-import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel;
-import org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultThreadFactory;
 
 /**
  * A WAL provider that use {@link AsyncFSWAL}.
@@ -61,6 +58,7 @@ public class AsyncFSWALProvider extends AbstractFSWALProvider<AsyncFSWAL> {
   private EventLoopGroup eventLoopGroup;
 
   private Class<? extends Channel> channelClass;
+
   @Override
   protected AsyncFSWAL createWAL() throws IOException {
     return new AsyncFSWAL(CommonFSUtils.getWALFileSystem(conf), CommonFSUtils.getWALRootDir(conf),
@@ -73,15 +71,9 @@ public class AsyncFSWALProvider extends AbstractFSWALProvider<AsyncFSWAL> {
   @Override
   protected void doInit(Configuration conf) throws IOException {
     Pair<EventLoopGroup, Class<? extends Channel>> eventLoopGroupAndChannelClass =
-        NettyAsyncFSWALConfigHelper.getEventLoopConfig(conf);
-    if (eventLoopGroupAndChannelClass != null) {
-      eventLoopGroup = eventLoopGroupAndChannelClass.getFirst();
-      channelClass = eventLoopGroupAndChannelClass.getSecond();
-    } else {
-      eventLoopGroup = new NioEventLoopGroup(1,
-          new DefaultThreadFactory("AsyncFSWAL", true, Thread.MAX_PRIORITY));
-      channelClass = NioSocketChannel.class;
-    }
+      NettyAsyncFSWALConfigHelper.getEventLoopConfig(conf);
+    eventLoopGroup = eventLoopGroupAndChannelClass.getFirst();
+    channelClass = eventLoopGroupAndChannelClass.getSecond();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/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 0836b5d..00ccb71 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
@@ -26,6 +26,9 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hbase.thirdparty.io.netty.channel.Channel;
 import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
+import org.apache.hbase.thirdparty.io.netty.channel.nio.NioEventLoopGroup;
+import org.apache.hbase.thirdparty.io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultThreadFactory;
 
 /**
  * Helper class for passing netty event loop config to {@link AsyncFSWALProvider}.
@@ -55,7 +58,10 @@ public class NettyAsyncFSWALConfigHelper {
   static Pair<EventLoopGroup, Class<? extends Channel>> getEventLoopConfig(Configuration conf) {
     String name = conf.get(EVENT_LOOP_CONFIG);
     if (StringUtils.isBlank(name)) {
-      return null;
+      // create new event loop group if config is empty
+      return Pair.<EventLoopGroup, Class<? extends Channel>> newPair(
+        new NioEventLoopGroup(0, new DefaultThreadFactory("AsyncFSWAL", true, Thread.MAX_PRIORITY)),
+        NioSocketChannel.class);
     }
     return EVENT_LOOP_CONFIG_MAP.get(name);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
index 28817e9..0b7b8da 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
@@ -31,6 +31,7 @@ import java.util.stream.Collectors;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
 // imports for classes still in regionserver.wal
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -132,6 +133,7 @@ public class RegionGroupingProvider implements WALProvider {
 
   private RegionGroupingStrategy strategy;
   private WALFactory factory;
+  private Configuration conf;
   private List<WALActionsListener> listeners = new ArrayList<>();
   private String providerId;
   private Class<? extends WALProvider> providerClass;
@@ -141,6 +143,7 @@ public class RegionGroupingProvider implements WALProvider {
     if (null != strategy) {
       throw new IllegalStateException("WALProvider.init should only be called once.");
     }
+    this.conf = conf;
     this.factory = factory;
     StringBuilder sb = new StringBuilder().append(factory.factoryId);
     if (providerId != null) {
@@ -156,11 +159,11 @@ public class RegionGroupingProvider implements WALProvider {
   }
 
   private WALProvider createProvider(String group) throws IOException {
-    if (META_WAL_PROVIDER_ID.equals(providerId)) {
-      return factory.createProvider(providerClass, META_WAL_PROVIDER_ID);
-    } else {
-      return factory.createProvider(providerClass, group);
-    }
+    WALProvider provider = WALFactory.createProvider(providerClass);
+    provider.init(factory, conf,
+      META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : group);
+    provider.addWALActionsListener(new MetricsWAL());
+    return provider;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SynchronousReplicationWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SynchronousReplicationWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SynchronousReplicationWALProvider.java
new file mode 100644
index 0000000..f60599f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SynchronousReplicationWALProvider.java
@@ -0,0 +1,225 @@
+/**
+ * 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.wal;
+
+import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.WAL_FILE_NAME_DELIMITER;
+import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.getWALArchiveDirectoryName;
+import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.getWALDirectoryName;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.Lock;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL;
+import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
+import org.apache.hadoop.hbase.replication.regionserver.PeerActionListener;
+import org.apache.hadoop.hbase.replication.regionserver.SynchronousReplicationPeerProvider;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
+import org.apache.hadoop.hbase.util.KeyLocker;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.Streams;
+import org.apache.hbase.thirdparty.io.netty.channel.Channel;
+import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
+
+/**
+ * The special {@link WALProvider} for synchronous replication.
+ * <p>
+ * It works like an interceptor, when getting WAL, first it will check if the given region should be
+ * replicated synchronously, if so it will return a special WAL for it, otherwise it will delegate
+ * the request to the normal {@link WALProvider}.
+ */
+@InterfaceAudience.Private
+public class SynchronousReplicationWALProvider implements WALProvider, PeerActionListener {
+
+  private static final Logger LOG =
+    LoggerFactory.getLogger(SynchronousReplicationWALProvider.class);
+
+  private static final String LOG_SUFFIX = ".syncrep";
+
+  private final WALProvider provider;
+
+  private final SynchronousReplicationPeerProvider peerProvider;
+
+  private WALFactory factory;
+
+  private Configuration conf;
+
+  private List<WALActionsListener> listeners = new ArrayList<>();
+
+  private EventLoopGroup eventLoopGroup;
+
+  private Class<? extends Channel> channelClass;
+
+  private AtomicBoolean initialized = new AtomicBoolean(false);
+
+  private final ConcurrentMap<String, DualAsyncFSWAL> peerId2WAL = new ConcurrentHashMap<>();
+
+  private final KeyLocker<String> createLock = new KeyLocker<>();
+
+  SynchronousReplicationWALProvider(WALProvider provider,
+      SynchronousReplicationPeerProvider peerProvider) {
+    this.provider = provider;
+    this.peerProvider = peerProvider;
+  }
+
+  @Override
+  public void init(WALFactory factory, Configuration conf, String providerId) throws IOException {
+    if (!initialized.compareAndSet(false, true)) {
+      throw new IllegalStateException("WALProvider.init should only be called once.");
+    }
+    provider.init(factory, conf, providerId);
+    this.conf = conf;
+    this.factory = factory;
+    Pair<EventLoopGroup, Class<? extends Channel>> eventLoopGroupAndChannelClass =
+      NettyAsyncFSWALConfigHelper.getEventLoopConfig(conf);
+    eventLoopGroup = eventLoopGroupAndChannelClass.getFirst();
+    channelClass = eventLoopGroupAndChannelClass.getSecond();
+  }
+
+  private String getLogPrefix(String peerId) {
+    return factory.factoryId + WAL_FILE_NAME_DELIMITER + peerId;
+  }
+
+  private DualAsyncFSWAL createWAL(String peerId, String remoteWALDir) throws IOException {
+    Path remoteWALDirPath = new Path(remoteWALDir);
+    FileSystem remoteFs = remoteWALDirPath.getFileSystem(conf);
+    return new DualAsyncFSWAL(CommonFSUtils.getWALFileSystem(conf), remoteFs,
+        CommonFSUtils.getWALRootDir(conf), new Path(remoteWALDirPath, peerId),
+        getWALDirectoryName(factory.factoryId), getWALArchiveDirectoryName(conf, factory.factoryId),
+        conf, listeners, true, getLogPrefix(peerId), LOG_SUFFIX, eventLoopGroup, channelClass);
+  }
+
+  private DualAsyncFSWAL getWAL(String peerId, String remoteWALDir) throws IOException {
+    DualAsyncFSWAL wal = peerId2WAL.get(peerId);
+    if (wal != null) {
+      return wal;
+    }
+    Lock lock = createLock.acquireLock(peerId);
+    try {
+      wal = peerId2WAL.get(peerId);
+      if (wal == null) {
+        wal = createWAL(peerId, remoteWALDir);
+        peerId2WAL.put(peerId, wal);
+        wal.init();
+      }
+      return wal;
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public WAL getWAL(RegionInfo region) throws IOException {
+    Optional<Pair<String, String>> peerIdAndRemoteWALDir =
+      peerProvider.getPeerIdAndRemoteWALDir(region);
+    if (peerIdAndRemoteWALDir.isPresent()) {
+      Pair<String, String> pair = peerIdAndRemoteWALDir.get();
+      return getWAL(pair.getFirst(), pair.getSecond());
+    } else {
+      return provider.getWAL(region);
+    }
+  }
+
+  private Stream<WAL> getWALStream() {
+    return Streams.concat(peerId2WAL.values().stream(), provider.getWALs().stream());
+  }
+
+  @Override
+  public List<WAL> getWALs() {
+    return getWALStream().collect(Collectors.toList());
+  }
+
+  @Override
+  public void shutdown() throws IOException {
+    // save the last exception and rethrow
+    IOException failure = null;
+    for (DualAsyncFSWAL wal : peerId2WAL.values()) {
+      try {
+        wal.shutdown();
+      } catch (IOException e) {
+        LOG.error("Shutdown WAL failed", e);
+        failure = e;
+      }
+    }
+    provider.shutdown();
+    if (failure != null) {
+      throw failure;
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    // save the last exception and rethrow
+    IOException failure = null;
+    for (DualAsyncFSWAL wal : peerId2WAL.values()) {
+      try {
+        wal.close();
+      } catch (IOException e) {
+        LOG.error("Close WAL failed", e);
+        failure = e;
+      }
+    }
+    provider.close();
+    if (failure != null) {
+      throw failure;
+    }
+  }
+
+  @Override
+  public long getNumLogFiles() {
+    return peerId2WAL.size() + provider.getNumLogFiles();
+  }
+
+  @Override
+  public long getLogFileSize() {
+    return peerId2WAL.values().stream().mapToLong(DualAsyncFSWAL::getLogFileSize).sum() +
+      provider.getLogFileSize();
+  }
+
+  @Override
+  public void peerRemoved(String peerId) {
+    WAL wal = peerId2WAL.remove(peerId);
+    if (wal != null) {
+      try {
+        wal.close();
+      } catch (IOException e) {
+        LOG.error("Close WAL failed", e);
+      }
+    }
+  }
+
+  @Override
+  public void addWALActionsListener(WALActionsListener listener) {
+    listeners.add(listener);
+    provider.addWALActionsListener(listener);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
index 1410b53..4e519ee 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
 import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
+import org.apache.hadoop.hbase.replication.regionserver.SynchronousReplicationPeerProvider;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.LeaseNotRecoveredException;
@@ -130,13 +131,10 @@ public class WALFactory {
     }
   }
 
-  WALProvider createProvider(Class<? extends WALProvider> clazz, String providerId)
-      throws IOException {
-    LOG.info("Instantiating WALProvider of type " + clazz);
+  static WALProvider createProvider(Class<? extends WALProvider> clazz) throws IOException {
+    LOG.info("Instantiating WALProvider of type {}", clazz);
     try {
-      final WALProvider result = clazz.getDeclaredConstructor().newInstance();
-      result.init(this, conf, providerId);
-      return result;
+      return clazz.newInstance();
     } catch (Exception e) {
       LOG.error("couldn't set up WALProvider, the configured class is " + clazz);
       LOG.debug("Exception details for failure to load WALProvider.", e);
@@ -148,9 +146,10 @@ public class WALFactory {
    * instantiate a provider from a config property. requires conf to have already been set (as well
    * as anything the provider might need to read).
    */
-  WALProvider getProvider(String key, String defaultValue, String providerId) throws IOException {
-    Class<? extends WALProvider> clazz = getProviderClass(key, defaultValue);
-    WALProvider provider = createProvider(clazz, providerId);
+  private WALProvider getProvider(String key, String defaultValue, String providerId)
+      throws IOException {
+    WALProvider provider = createProvider(getProviderClass(key, defaultValue));
+    provider.init(this, conf, providerId);
     provider.addWALActionsListener(new MetricsWAL());
     return provider;
   }
@@ -182,6 +181,26 @@ public class WALFactory {
   }
 
   /**
+   * A temporary constructor for testing synchronous replication.
+   * <p>
+   * Remove it once we can integrate the synchronous replication logic in RS.
+   */
+  @VisibleForTesting
+  WALFactory(Configuration conf, String factoryId, SynchronousReplicationPeerProvider peerProvider)
+      throws IOException {
+    timeoutMillis = conf.getInt("hbase.hlog.open.timeout", 300000);
+    /* TODO Both of these are probably specific to the fs wal provider */
+    logReaderClass = conf.getClass("hbase.regionserver.hlog.reader.impl", ProtobufLogReader.class,
+      AbstractFSWALProvider.Reader.class);
+    this.conf = conf;
+    this.factoryId = factoryId;
+    WALProvider provider = createProvider(getProviderClass(WAL_PROVIDER, DEFAULT_WAL_PROVIDER));
+    this.provider = new SynchronousReplicationWALProvider(provider, peerProvider);
+    this.provider.addWALActionsListener(new MetricsWAL());
+    this.provider.init(this, conf, null);
+  }
+
+  /**
    * Shutdown all WALs and clean up any underlying storage.
    * Use only when you will not need to replay and edits that have gone to any wals from this
    * factory.

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java
index 7abd4a5..d24bd2e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALKeyImpl.java
@@ -135,13 +135,21 @@ public class WALKeyImpl implements WALKey {
   }
 
   @VisibleForTesting
-  public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename,
-                long logSeqNum,
+  public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename, long logSeqNum,
       final long now, UUID clusterId) {
     List<UUID> clusterIds = new ArrayList<>(1);
     clusterIds.add(clusterId);
-    init(encodedRegionName, tablename, logSeqNum, now, clusterIds,
-        HConstants.NO_NONCE, HConstants.NO_NONCE, null, null);
+    init(encodedRegionName, tablename, logSeqNum, now, clusterIds, HConstants.NO_NONCE,
+      HConstants.NO_NONCE, null, null);
+  }
+
+  @VisibleForTesting
+  public WALKeyImpl(final byte[] encodedRegionName, final TableName tablename, long logSeqNum,
+      final long now, UUID clusterId, MultiVersionConcurrencyControl mvcc) {
+    List<UUID> clusterIds = new ArrayList<>(1);
+    clusterIds.add(clusterId);
+    init(encodedRegionName, tablename, logSeqNum, now, clusterIds, HConstants.NO_NONCE,
+      HConstants.NO_NONCE, mvcc, null);
   }
 
   // TODO: Fix being able to pass in sequenceid.

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
index ca4b227..939f35c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java
@@ -100,6 +100,7 @@ public class TestCompactionPolicy {
     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
 
     hlog = new FSHLog(fs, basedir, logName, conf);
+    hlog.init();
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     region = HRegion.createHRegion(info, basedir, conf, htd, hlog);
     region.close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
index e27a986..f9eb534 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFailedAppendAndSync.java
@@ -102,65 +102,64 @@ public class TestFailedAppendAndSync {
     return name.getMethodName();
   }
 
-  /**
-   * Reproduce locking up that happens when we get an exceptions appending and syncing.
-   * See HBASE-14317.
-   * First I need to set up some mocks for Server and RegionServerServices. I also need to
-   * set up a dodgy WAL that will throw an exception when we go to append to it.
-   */
-  @Test (timeout=300000)
-  public void testLockupAroundBadAssignSync() throws IOException {
+  // Dodgy WAL. Will throw exceptions when flags set.
+  class DodgyFSLog extends FSHLog {
+    volatile boolean throwSyncException = false;
+    volatile boolean throwAppendException = false;
     final AtomicLong rolls = new AtomicLong(0);
-    // Dodgy WAL. Will throw exceptions when flags set.
-    class DodgyFSLog extends FSHLog {
-      volatile boolean throwSyncException = false;
-      volatile boolean throwAppendException = false;
 
-      public DodgyFSLog(FileSystem fs, Path root, String logDir, Configuration conf)
-      throws IOException {
-        super(fs, root, logDir, conf);
-      }
-
-      @Override
-      public byte[][] rollWriter(boolean force) throws FailedLogCloseException, IOException {
-        byte [][] regions = super.rollWriter(force);
-        rolls.getAndIncrement();
-        return regions;
-      }
+    public DodgyFSLog(FileSystem fs, Path root, String logDir, Configuration conf)
+        throws IOException {
+      super(fs, root, logDir, conf);
+    }
 
-      @Override
-      protected Writer createWriterInstance(Path path) throws IOException {
-        final Writer w = super.createWriterInstance(path);
-          return new Writer() {
-            @Override
-            public void close() throws IOException {
-              w.close();
-            }
+    @Override
+    public byte[][] rollWriter(boolean force) throws FailedLogCloseException, IOException {
+      byte[][] regions = super.rollWriter(force);
+      rolls.getAndIncrement();
+      return regions;
+    }
 
-            @Override
-            public void sync() throws IOException {
-              if (throwSyncException) {
-                throw new IOException("FAKE! Failed to replace a bad datanode...");
-              }
-              w.sync();
-            }
+    @Override
+    protected Writer createWriterInstance(Path path) throws IOException {
+      final Writer w = super.createWriterInstance(path);
+      return new Writer() {
+        @Override
+        public void close() throws IOException {
+          w.close();
+        }
 
-            @Override
-            public void append(Entry entry) throws IOException {
-              if (throwAppendException) {
-                throw new IOException("FAKE! Failed to replace a bad datanode...");
-              }
-              w.append(entry);
-            }
+        @Override
+        public void sync() throws IOException {
+          if (throwSyncException) {
+            throw new IOException("FAKE! Failed to replace a bad datanode...");
+          }
+          w.sync();
+        }
 
-            @Override
-            public long getLength() {
-              return w.getLength();
-              }
-            };
+        @Override
+        public void append(Entry entry) throws IOException {
+          if (throwAppendException) {
+            throw new IOException("FAKE! Failed to replace a bad datanode...");
           }
-      }
+          w.append(entry);
+        }
 
+        @Override
+        public long getLength() {
+          return w.getLength();
+        }
+      };
+    }
+  }
+  /**
+   * Reproduce locking up that happens when we get an exceptions appending and syncing.
+   * See HBASE-14317.
+   * First I need to set up some mocks for Server and RegionServerServices. I also need to
+   * set up a dodgy WAL that will throw an exception when we go to append to it.
+   */
+  @Test (timeout=300000)
+  public void testLockupAroundBadAssignSync() throws IOException {
     // Make up mocked server and services.
     Server server = mock(Server.class);
     when(server.getConfiguration()).thenReturn(CONF);
@@ -172,6 +171,7 @@ public class TestFailedAppendAndSync {
     FileSystem fs = FileSystem.get(CONF);
     Path rootDir = new Path(dir + getName());
     DodgyFSLog dodgyWAL = new DodgyFSLog(fs, rootDir, getName(), CONF);
+    dodgyWAL.init();
     LogRoller logRoller = new LogRoller(server, services);
     logRoller.addWAL(dodgyWAL);
     logRoller.start();
@@ -192,7 +192,7 @@ public class TestFailedAppendAndSync {
       } catch (IOException ioe) {
         fail();
       }
-      long rollsCount = rolls.get();
+      long rollsCount = dodgyWAL.rolls.get();
       try {
         dodgyWAL.throwAppendException = true;
         dodgyWAL.throwSyncException = false;
@@ -202,8 +202,10 @@ public class TestFailedAppendAndSync {
       } catch (IOException ioe) {
         threwOnAppend = true;
       }
-      while (rollsCount == rolls.get()) Threads.sleep(100);
-      rollsCount = rolls.get();
+      while (rollsCount == dodgyWAL.rolls.get()) {
+        Threads.sleep(100);
+      }
+      rollsCount = dodgyWAL.rolls.get();
 
       // When we get to here.. we should be ok. A new WAL has been put in place. There were no
       // appends to sync. We should be able to continue.
@@ -217,14 +219,16 @@ public class TestFailedAppendAndSync {
       } catch (IOException ioe) {
         threwOnBoth = true;
       }
-      while (rollsCount == rolls.get()) Threads.sleep(100);
+      while (rollsCount == dodgyWAL.rolls.get()) {
+        Threads.sleep(100);
+      }
 
       // Again, all should be good. New WAL and no outstanding unsync'd edits so we should be able
       // to just continue.
 
       // So, should be no abort at this stage. Verify.
-      Mockito.verify(server, Mockito.atLeast(0)).
-        abort(Mockito.anyString(), (Throwable)Mockito.anyObject());
+      Mockito.verify(server, Mockito.atLeast(0)).abort(Mockito.anyString(),
+        Mockito.any(Throwable.class));
       try {
         dodgyWAL.throwAppendException = false;
         dodgyWAL.throwSyncException = true;
@@ -239,8 +243,8 @@ public class TestFailedAppendAndSync {
       // happens. If it don't we'll timeout the whole test. That is fine.
       while (true) {
         try {
-          Mockito.verify(server, Mockito.atLeast(1)).
-            abort(Mockito.anyString(), (Throwable)Mockito.anyObject());
+          Mockito.verify(server, Mockito.atLeast(1)).abort(Mockito.anyString(),
+            Mockito.any(Throwable.class));
           break;
         } catch (WantedButNotInvoked t) {
           Threads.sleep(1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 88e1aa2..df24e0a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -217,7 +217,6 @@ public class TestHRegion {
   protected static HBaseTestingUtility TEST_UTIL;
   public static Configuration CONF ;
   private String dir;
-  private static FileSystem FILESYSTEM;
   private final int MAX_VERSIONS = 2;
 
   // Test names
@@ -239,7 +238,6 @@ public class TestHRegion {
   @Before
   public void setup() throws IOException {
     TEST_UTIL = HBaseTestingUtility.createLocalHTU();
-    FILESYSTEM = TEST_UTIL.getTestFileSystem();
     CONF = TEST_UTIL.getConfiguration();
     dir = TEST_UTIL.getDataTestDir("TestHRegion").toString();
     method = name.getMethodName();
@@ -341,6 +339,7 @@ public class TestHRegion {
     FileSystem fs = FileSystem.get(CONF);
     Path rootDir = new Path(dir + "testMemstoreSnapshotSize");
     MyFaultyFSLog faultyLog = new MyFaultyFSLog(fs, rootDir, "testMemstoreSnapshotSize", CONF);
+    faultyLog.init();
     HRegion region = initHRegion(tableName, null, null, false, Durability.SYNC_WAL, faultyLog,
         COLUMN_FAMILY_BYTES);
 
@@ -352,7 +351,6 @@ public class TestHRegion {
     Put put = new Put(value);
     put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("abc"), value);
     faultyLog.setFailureType(FaultyFSLog.FailureType.SYNC);
-
     boolean threwIOE = false;
     try {
       region.put(put);
@@ -388,6 +386,7 @@ public class TestHRegion {
     FileSystem fs = FileSystem.get(CONF);
     Path rootDir = new Path(dir + testName);
     FSHLog hLog = new FSHLog(fs, rootDir, testName, CONF);
+    hLog.init();
     HRegion region = initHRegion(tableName, null, null, false, Durability.SYNC_WAL, hLog,
         COLUMN_FAMILY_BYTES);
     HStore store = region.getStore(COLUMN_FAMILY_BYTES);
@@ -1162,6 +1161,7 @@ public class TestHRegion {
     FailAppendFlushMarkerWAL wal =
       new FailAppendFlushMarkerWAL(FileSystem.get(walConf), FSUtils.getRootDir(walConf),
         method, walConf);
+    wal.init();
     this.region = initHRegion(tableName, HConstants.EMPTY_START_ROW,
       HConstants.EMPTY_END_ROW, false, Durability.USE_DEFAULT, wal, family);
     try {
@@ -1193,7 +1193,7 @@ public class TestHRegion {
       wal.flushActions = new FlushAction [] {FlushAction.COMMIT_FLUSH};
       wal = new FailAppendFlushMarkerWAL(FileSystem.get(walConf), FSUtils.getRootDir(walConf),
             method, walConf);
-
+      wal.init();
       this.region = initHRegion(tableName, HConstants.EMPTY_START_ROW,
         HConstants.EMPTY_END_ROW, false, Durability.USE_DEFAULT, wal, family);
       region.put(put);
@@ -2445,6 +2445,7 @@ public class TestHRegion {
     FileSystem fs = FileSystem.get(CONF);
     Path rootDir = new Path(dir + "testDataInMemoryWithoutWAL");
     FSHLog hLog = new FSHLog(fs, rootDir, "testDataInMemoryWithoutWAL", CONF);
+    hLog.init();
     // This chunk creation is done throughout the code base. Do we want to move it into core?
     // It is missing from this test. W/o it we NPE.
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
@@ -2497,9 +2498,9 @@ public class TestHRegion {
     RegionCoprocessorHost mockedCPHost = Mockito.mock(RegionCoprocessorHost.class);
     // Because the preBatchMutate returns void, we can't do usual Mockito when...then form. Must
     // do below format (from Mockito doc).
-    Mockito.doAnswer(new Answer() {
+    Mockito.doAnswer(new Answer<Void>() {
       @Override
-      public Object answer(InvocationOnMock invocation) throws Throwable {
+      public Void answer(InvocationOnMock invocation) throws Throwable {
         MiniBatchOperationInProgress<Mutation> mb = invocation.getArgument(0);
         mb.addOperationsFromCP(0, new Mutation[]{addPut});
         return null;
@@ -3793,9 +3794,12 @@ public class TestHRegion {
 
         boolean previousEmpty = res.isEmpty();
         res.clear();
-        InternalScanner scanner = region.getScanner(scan);
-        while (scanner.next(res))
-          ;
+        try (InternalScanner scanner = region.getScanner(scan)) {
+          boolean moreRows;
+          do {
+            moreRows = scanner.next(res);
+          } while (moreRows);
+        }
         if (!res.isEmpty() || !previousEmpty || i > compactInterval) {
           assertEquals("i=" + i, expectedCount, res.size());
           long timestamp = res.get(0).getTimestamp();
@@ -3891,7 +3895,7 @@ public class TestHRegion {
             region.put(put);
             numPutsFinished++;
             if (numPutsFinished > 0 && numPutsFinished % 47 == 0) {
-              System.out.println("put iteration = " + numPutsFinished);
+              LOG.debug("put iteration = {}", numPutsFinished);
               Delete delete = new Delete(row, (long) numPutsFinished - 30);
               region.delete(delete);
             }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java
index dfe52d0..58f62e3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionWithInMemoryFlush.java
@@ -27,25 +27,18 @@ import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.junit.ClassRule;
 import org.junit.experimental.categories.Category;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * A test similar to TestHRegion, but with in-memory flush families.
  * Also checks wal truncation after in-memory compaction.
  */
 @Category({VerySlowRegionServerTests.class, LargeTests.class})
-@SuppressWarnings("deprecation")
 public class TestHRegionWithInMemoryFlush extends TestHRegion{
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestHRegionWithInMemoryFlush.class);
 
-  // Do not spin up clusters in here. If you need to spin up a cluster, do it
-  // over in TestHRegionOnCluster.
-  private static final Logger LOG = LoggerFactory.getLogger(TestHRegionWithInMemoryFlush.class);
-
   /**
    * @return A region on which you must call
    *         {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when done.

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
index 8b96fa7..e5006ea 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionIncrement.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Scan;
@@ -36,7 +37,6 @@ import org.apache.hadoop.hbase.client.TestIncrementsFromClientSide;
 import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.wal.WAL;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.ClassRule;
@@ -81,12 +81,12 @@ public class TestRegionIncrement {
   }
 
   private HRegion getRegion(final Configuration conf, final String tableName) throws IOException {
-    WAL wal = new FSHLog(FileSystem.get(conf), TEST_UTIL.getDataTestDir(),
-      TEST_UTIL.getDataTestDir().toString(), conf);
+    FSHLog wal = new FSHLog(FileSystem.get(conf), TEST_UTIL.getDataTestDir(),
+        TEST_UTIL.getDataTestDir().toString(), conf);
+    wal.init();
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
-    return (HRegion)TEST_UTIL.createLocalHRegion(Bytes.toBytes(tableName),
-      HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, tableName, conf,
-      false, Durability.SKIP_WAL, wal, INCREMENT_BYTES);
+    return TEST_UTIL.createLocalHRegion(TableName.valueOf(tableName), HConstants.EMPTY_BYTE_ARRAY,
+      HConstants.EMPTY_BYTE_ARRAY, false, Durability.SKIP_WAL, wal, INCREMENT_BYTES);
   }
 
   private void closeRegion(final HRegion region) throws IOException {
@@ -170,8 +170,6 @@ public class TestRegionIncrement {
 
   /**
    * Have each thread update its own Cell. Avoid contention with another thread.
-   * @throws IOException
-   * @throws InterruptedException
    */
   @Test
   public void testUnContendedSingleCellIncrement()
@@ -209,13 +207,9 @@ public class TestRegionIncrement {
 
   /**
    * Have each thread update its own Cell. Avoid contention with another thread.
-   * This is
-   * @throws IOException
-   * @throws InterruptedException
    */
   @Test
-  public void testContendedAcrossCellsIncrement()
-  throws IOException, InterruptedException {
+  public void testContendedAcrossCellsIncrement() throws IOException, InterruptedException {
     final HRegion region = getRegion(TEST_UTIL.getConfiguration(),
         TestIncrementsFromClientSide.filterStringSoTableNameSafe(this.name.getMethodName()));
     long startTime = System.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
index 1e59248..2e8c05f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.CoordinatedStateManager;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -215,6 +214,7 @@ public class TestWALLockup {
     FileSystem fs = FileSystem.get(CONF);
     Path rootDir = new Path(dir + getName());
     DodgyFSLog dodgyWAL = new DodgyFSLog(fs, rootDir, getName(), CONF);
+    dodgyWAL.init();
     Path originalWAL = dodgyWAL.getCurrentFileName();
     // I need a log roller running.
     LogRoller logRoller = new LogRoller(server, services);
@@ -222,7 +222,6 @@ public class TestWALLockup {
     // There is no 'stop' once a logRoller is running.. it just dies.
     logRoller.start();
     // Now get a region and start adding in edits.
-    HTableDescriptor htd = new HTableDescriptor(TableName.META_TABLE_NAME);
     final HRegion region = initHRegion(tableName, null, null, dodgyWAL);
     byte [] bytes = Bytes.toBytes(getName());
     NavigableMap<byte[], Integer> scopes = new TreeMap<>(
@@ -236,7 +235,7 @@ public class TestWALLockup {
       Put put = new Put(bytes);
       put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("1"), bytes);
       WALKeyImpl key = new WALKeyImpl(region.getRegionInfo().getEncodedNameAsBytes(),
-          htd.getTableName(), System.currentTimeMillis(), mvcc, scopes);
+          TableName.META_TABLE_NAME, System.currentTimeMillis(), mvcc, scopes);
       WALEdit edit = new WALEdit();
       CellScanner CellScanner = put.cellScanner();
       assertTrue(CellScanner.advance());
@@ -389,10 +388,11 @@ public class TestWALLockup {
     FileSystem fs = FileSystem.get(CONF);
     Path rootDir = new Path(dir + getName());
     DodgyFSLog dodgyWAL1 = new DodgyFSLog(fs, rootDir, getName(), CONF);
-
+    dodgyWAL1.init();
     Path rootDir2 = new Path(dir + getName() + "2");
     final DodgyFSLog dodgyWAL2 = new DodgyFSLog(fs, rootDir2, getName() + "2",
         CONF);
+    dodgyWAL2.init();
     // Add a listener to force ringbuffer event handler sleep for a while
     dodgyWAL2.registerWALActionsListener(new DummyWALActionsListener());
 
@@ -403,7 +403,6 @@ public class TestWALLockup {
     // There is no 'stop' once a logRoller is running.. it just dies.
     logRoller.start();
     // Now get a region and start adding in edits.
-    HTableDescriptor htd = new HTableDescriptor(TableName.META_TABLE_NAME);
     final HRegion region = initHRegion(tableName, null, null, dodgyWAL1);
     byte[] bytes = Bytes.toBytes(getName());
     NavigableMap<byte[], Integer> scopes = new TreeMap<>(
@@ -414,7 +413,7 @@ public class TestWALLockup {
       Put put = new Put(bytes);
       put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("1"), bytes);
       WALKeyImpl key = new WALKeyImpl(region.getRegionInfo().getEncodedNameAsBytes(),
-          htd.getTableName(), System.currentTimeMillis(), mvcc, scopes);
+          TableName.META_TABLE_NAME, System.currentTimeMillis(), mvcc, scopes);
       WALEdit edit = new WALEdit();
       CellScanner CellScanner = put.cellScanner();
       assertTrue(CellScanner.advance());

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
index b1e304e..7600fe9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java
@@ -1097,6 +1097,7 @@ public abstract class AbstractTestWALReplay {
 
   private MockWAL createMockWAL() throws IOException {
     MockWAL wal = new MockWAL(fs, hbaseRootDir, logName, conf);
+    wal.init();
     // Set down maximum recovery so we dfsclient doesn't linger retrying something
     // long gone.
     HBaseTestingUtility.setMaxRecoveryErrorCount(wal.getOutputStream(), 1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java
index ecd8e6c..49633cb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogTestHelper.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WALEdit;
@@ -41,7 +42,7 @@ import org.apache.hadoop.hbase.wal.WALProvider;
 /**
  * Helper class for testing protobuf log.
  */
-final class ProtobufLogTestHelper {
+public final class ProtobufLogTestHelper {
 
   private ProtobufLogTestHelper() {
   }
@@ -54,17 +55,22 @@ final class ProtobufLogTestHelper {
     return RegionInfoBuilder.newBuilder(tableName).setRegionId(1024).build();
   }
 
+  private static WAL.Entry generateEdit(int i, RegionInfo hri, TableName tableName, byte[] row,
+      int columnCount, long timestamp, MultiVersionConcurrencyControl mvcc) {
+    WALKeyImpl key = new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, i, timestamp,
+        HConstants.DEFAULT_CLUSTER_ID, mvcc);
+    WALEdit edit = new WALEdit();
+    int prefix = i;
+    IntStream.range(0, columnCount).mapToObj(j -> toValue(prefix, j))
+        .map(value -> new KeyValue(row, row, row, timestamp, value)).forEachOrdered(edit::add);
+    return new WAL.Entry(key, edit);
+  }
+
   public static void doWrite(WALProvider.Writer writer, boolean withTrailer, TableName tableName,
       int columnCount, int recordCount, byte[] row, long timestamp) throws IOException {
     RegionInfo hri = toRegionInfo(tableName);
     for (int i = 0; i < recordCount; i++) {
-      WALKeyImpl key = new WALKeyImpl(hri.getEncodedNameAsBytes(), tableName, i, timestamp,
-          HConstants.DEFAULT_CLUSTER_ID);
-      WALEdit edit = new WALEdit();
-      int prefix = i;
-      IntStream.range(0, columnCount).mapToObj(j -> toValue(prefix, j))
-          .map(value -> new KeyValue(row, row, row, timestamp, value)).forEachOrdered(edit::add);
-      writer.append(new WAL.Entry(key, edit));
+      writer.append(generateEdit(i, hri, tableName, row, columnCount, timestamp, null));
     }
     writer.sync();
     if (withTrailer) {
@@ -72,14 +78,24 @@ final class ProtobufLogTestHelper {
     }
   }
 
-  public static void doRead(ProtobufLogReader reader, boolean withTrailer, TableName tableName,
-      int columnCount, int recordCount, byte[] row, long timestamp) throws IOException {
+  public static void doWrite(WAL wal, RegionInfo hri, TableName tableName, int columnCount,
+      int recordCount, byte[] row, long timestamp, MultiVersionConcurrencyControl mvcc)
+      throws IOException {
+    for (int i = 0; i < recordCount; i++) {
+      WAL.Entry entry = generateEdit(i, hri, tableName, row, columnCount, timestamp, mvcc);
+      wal.append(hri, entry.getKey(), entry.getEdit(), true);
+    }
+    wal.sync();
+  }
+
+  public static void doRead(ProtobufLogReader reader, boolean withTrailer, RegionInfo hri,
+      TableName tableName, int columnCount, int recordCount, byte[] row, long timestamp)
+      throws IOException {
     if (withTrailer) {
       assertNotNull(reader.trailer);
     } else {
       assertNull(reader.trailer);
     }
-    RegionInfo hri = toRegionInfo(tableName);
     for (int i = 0; i < recordCount; ++i) {
       WAL.Entry entry = reader.next();
       assertNotNull(entry);
@@ -96,4 +112,10 @@ final class ProtobufLogTestHelper {
     }
     assertNull(reader.next());
   }
+
+  public static void doRead(ProtobufLogReader reader, boolean withTrailer, TableName tableName,
+      int columnCount, int recordCount, byte[] row, long timestamp) throws IOException {
+    doRead(reader, withTrailer, toRegionInfo(tableName), tableName, columnCount, recordCount, row,
+      timestamp);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java
index 450c01b..5f0f77c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWAL.java
@@ -67,8 +67,10 @@ public class TestAsyncFSWAL extends AbstractTestFSWAL {
   protected AbstractFSWAL<?> newWAL(FileSystem fs, Path rootDir, String logDir, String archiveDir,
       Configuration conf, List<WALActionsListener> listeners, boolean failIfWALExists,
       String prefix, String suffix) throws IOException {
-    return new AsyncFSWAL(fs, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix,
-        suffix, GROUP, CHANNEL_CLASS);
+    AsyncFSWAL wal = new AsyncFSWAL(fs, rootDir, logDir, archiveDir, conf, listeners,
+        failIfWALExists, prefix, suffix, GROUP, CHANNEL_CLASS);
+    wal.init();
+    return wal;
   }
 
   @Override
@@ -76,15 +78,16 @@ public class TestAsyncFSWAL extends AbstractTestFSWAL {
       String archiveDir, Configuration conf, List<WALActionsListener> listeners,
       boolean failIfWALExists, String prefix, String suffix, final Runnable action)
       throws IOException {
-    return new AsyncFSWAL(fs, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix,
-        suffix, GROUP, CHANNEL_CLASS) {
+    AsyncFSWAL wal = new AsyncFSWAL(fs, rootDir, logDir, archiveDir, conf, listeners,
+        failIfWALExists, prefix, suffix, GROUP, CHANNEL_CLASS) {
 
       @Override
       void atHeadOfRingBufferEventHandlerAppend() {
         action.run();
         super.atHeadOfRingBufferEventHandlerAppend();
       }
-
     };
+    wal.init();
+    return wal;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java
index 80b7477..0740954 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncWALReplay.java
@@ -66,7 +66,9 @@ public class TestAsyncWALReplay extends AbstractTestWALReplay {
 
   @Override
   protected WAL createWAL(Configuration c, Path hbaseRootDir, String logName) throws IOException {
-    return new AsyncFSWAL(FileSystem.get(c), hbaseRootDir, logName,
+    AsyncFSWAL wal = new AsyncFSWAL(FileSystem.get(c), hbaseRootDir, logName,
         HConstants.HREGION_OLDLOGDIR_NAME, c, null, true, null, null, GROUP, CHANNEL_CLASS);
+    wal.init();
+    return wal;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCombinedAsyncWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCombinedAsyncWriter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCombinedAsyncWriter.java
index d74f9d8..36dbe0f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCombinedAsyncWriter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestCombinedAsyncWriter.java
@@ -77,8 +77,7 @@ public class TestCombinedAsyncWriter {
     CHANNEL_CLASS = NioSocketChannel.class;
     UTIL.startMiniDFSCluster(3);
     UTIL.getTestFileSystem().mkdirs(UTIL.getDataTestDirOnTestFS());
-    WALS =
-      new WALFactory(UTIL.getConfiguration(), null, TestCombinedAsyncWriter.class.getSimpleName());
+    WALS = new WALFactory(UTIL.getConfiguration(), TestCombinedAsyncWriter.class.getSimpleName());
   }
 
   @AfterClass

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java
index 9c789ba..3502428 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java
@@ -72,8 +72,10 @@ public class TestFSHLog extends AbstractTestFSWAL {
   protected AbstractFSWAL<?> newWAL(FileSystem fs, Path rootDir, String walDir, String archiveDir,
       Configuration conf, List<WALActionsListener> listeners, boolean failIfWALExists,
       String prefix, String suffix) throws IOException {
-    return new FSHLog(fs, rootDir, walDir, archiveDir, conf, listeners, failIfWALExists, prefix,
-        suffix);
+    FSHLog wal =
+      new FSHLog(fs, rootDir, walDir, archiveDir, conf, listeners, failIfWALExists, prefix, suffix);
+    wal.init();
+    return wal;
   }
 
   @Override
@@ -81,8 +83,8 @@ public class TestFSHLog extends AbstractTestFSWAL {
       String archiveDir, Configuration conf, List<WALActionsListener> listeners,
       boolean failIfWALExists, String prefix, String suffix, final Runnable action)
       throws IOException {
-    return new FSHLog(fs, rootDir, walDir, archiveDir, conf, listeners, failIfWALExists, prefix,
-        suffix) {
+    FSHLog wal = new FSHLog(fs, rootDir, walDir, archiveDir, conf, listeners, failIfWALExists,
+        prefix, suffix) {
 
       @Override
       void atHeadOfRingBufferEventHandlerAppend() {
@@ -90,6 +92,8 @@ public class TestFSHLog extends AbstractTestFSWAL {
         super.atHeadOfRingBufferEventHandlerAppend();
       }
     };
+    wal.init();
+    return wal;
   }
 
   @Test
@@ -98,6 +102,7 @@ public class TestFSHLog extends AbstractTestFSWAL {
     final String name = this.name.getMethodName();
     FSHLog log = new FSHLog(FS, FSUtils.getRootDir(CONF), name, HConstants.HREGION_OLDLOGDIR_NAME,
         CONF, null, true, null, null);
+    log.init();
     try {
       Field ringBufferEventHandlerField = FSHLog.class.getDeclaredField("ringBufferEventHandler");
       ringBufferEventHandlerField.setAccessible(true);
@@ -140,7 +145,7 @@ public class TestFSHLog extends AbstractTestFSWAL {
     try (FSHLog log =
         new FSHLog(FS, FSUtils.getRootDir(CONF), name, HConstants.HREGION_OLDLOGDIR_NAME, CONF,
             null, true, null, null)) {
-
+      log.init();
       log.registerWALActionsListener(new WALActionsListener() {
         @Override
         public void visitLogEntryBeforeWrite(WALKey logKey, WALEdit logEdit)

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
index 649e981..66e19a8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java
@@ -48,6 +48,7 @@ public class TestWALReplay extends AbstractTestWALReplay {
   @Override
   protected WAL createWAL(Configuration c, Path hbaseRootDir, String logName) throws IOException {
     FSHLog wal = new FSHLog(FileSystem.get(c), hbaseRootDir, logName, c);
+    wal.init();
     // Set down maximum recovery so we dfsclient doesn't linger retrying something
     // long gone.
     HBaseTestingUtility.setMaxRecoveryErrorCount(wal.getOutputStream(), 1);

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
index 3928d9c..f996ce0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/IOTestProvider.java
@@ -104,8 +104,6 @@ public class IOTestProvider implements WALProvider {
     this.factory = factory;
     this.conf = conf;
     this.providerId = providerId != null ? providerId : DEFAULT_PROVIDER_ID;
-
-
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/d9458339/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSynchronousReplicationWALProvider.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSynchronousReplicationWALProvider.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSynchronousReplicationWALProvider.java
new file mode 100644
index 0000000..e6031c6
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestSynchronousReplicationWALProvider.java
@@ -0,0 +1,153 @@
+/**
+ * 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.wal;
+
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.not;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+import java.io.IOException;
+import java.util.Optional;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
+import org.apache.hadoop.hbase.regionserver.wal.DualAsyncFSWAL;
+import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
+import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogTestHelper;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ RegionServerTests.class, MediumTests.class })
+public class TestSynchronousReplicationWALProvider {
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static String PEER_ID = "1";
+
+  private static String REMOTE_WAL_DIR = "/RemoteWAL";
+
+  private static TableName TABLE = TableName.valueOf("table");
+
+  private static TableName TABLE_NO_REP = TableName.valueOf("table-no-rep");
+
+  private static RegionInfo REGION = RegionInfoBuilder.newBuilder(TABLE).build();
+
+  private static RegionInfo REGION_NO_REP = RegionInfoBuilder.newBuilder(TABLE_NO_REP).build();
+
+  private static WALFactory FACTORY;
+
+  private static Optional<Pair<String, String>> getPeerIdAndRemoteWALDir(RegionInfo info) {
+    if (info.getTable().equals(TABLE)) {
+      return Optional.of(Pair.newPair(PEER_ID, REMOTE_WAL_DIR));
+    } else {
+      return Optional.empty();
+    }
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    UTIL.startMiniDFSCluster(3);
+    FACTORY = new WALFactory(UTIL.getConfiguration(), "test",
+        TestSynchronousReplicationWALProvider::getPeerIdAndRemoteWALDir);
+    UTIL.getTestFileSystem().mkdirs(new Path(REMOTE_WAL_DIR, PEER_ID));
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws IOException {
+    FACTORY.close();
+    UTIL.shutdownMiniDFSCluster();
+  }
+
+  private void testReadWrite(DualAsyncFSWAL wal) throws Exception {
+    int recordCount = 100;
+    int columnCount = 10;
+    byte[] row = Bytes.toBytes("testRow");
+    long timestamp = System.currentTimeMillis();
+    MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
+    ProtobufLogTestHelper.doWrite(wal, REGION, TABLE, columnCount, recordCount, row, timestamp,
+      mvcc);
+    Path localFile = wal.getCurrentFileName();
+    Path remoteFile = new Path(REMOTE_WAL_DIR + "/" + PEER_ID, localFile.getName());
+    try (ProtobufLogReader reader =
+      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), localFile)) {
+      ProtobufLogTestHelper.doRead(reader, false, REGION, TABLE, columnCount, recordCount, row,
+        timestamp);
+    }
+    try (ProtobufLogReader reader =
+      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), remoteFile)) {
+      ProtobufLogTestHelper.doRead(reader, false, REGION, TABLE, columnCount, recordCount, row,
+        timestamp);
+    }
+    wal.rollWriter();
+    DistributedFileSystem dfs = (DistributedFileSystem) UTIL.getDFSCluster().getFileSystem();
+    UTIL.waitFor(5000, new ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        return dfs.isFileClosed(localFile) && dfs.isFileClosed(remoteFile);
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        StringBuilder sb = new StringBuilder();
+        if (!dfs.isFileClosed(localFile)) {
+          sb.append(localFile + " has not been closed yet.");
+        }
+        if (!dfs.isFileClosed(remoteFile)) {
+          sb.append(remoteFile + " has not been closed yet.");
+        }
+        return sb.toString();
+      }
+    });
+    try (ProtobufLogReader reader =
+      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), localFile)) {
+      ProtobufLogTestHelper.doRead(reader, true, REGION, TABLE, columnCount, recordCount, row,
+        timestamp);
+    }
+    try (ProtobufLogReader reader =
+      (ProtobufLogReader) FACTORY.createReader(UTIL.getTestFileSystem(), remoteFile)) {
+      ProtobufLogTestHelper.doRead(reader, true, REGION, TABLE, columnCount, recordCount, row,
+        timestamp);
+    }
+  }
+
+  @Test
+  public void test() throws Exception {
+    WAL walNoRep = FACTORY.getWAL(REGION_NO_REP);
+    assertThat(walNoRep, not(instanceOf(DualAsyncFSWAL.class)));
+    DualAsyncFSWAL wal = (DualAsyncFSWAL) FACTORY.getWAL(REGION);
+    assertEquals(2, FACTORY.getWALs().size());
+    testReadWrite(wal);
+    SynchronousReplicationWALProvider walProvider =
+      (SynchronousReplicationWALProvider) FACTORY.getWALProvider();
+    walProvider.peerRemoved(PEER_ID);
+    assertEquals(1, FACTORY.getWALs().size());
+  }
+}


[07/28] hbase git commit: Revert "HBASE-19919 Tidying up logging"

Posted by zh...@apache.org.
Revert "HBASE-19919 Tidying up logging"

This reverts commit 40250f8c5f8efe3016f0c3300d9728b0ee2056c6.

Wrong patch. Revert to fix compile error.


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

Branch: refs/heads/HBASE-19064
Commit: 6519b98ac3115c4442a2778f6ed7b39ce5cd3b83
Parents: 40250f8
Author: zhangduo <zh...@apache.org>
Authored: Sat Feb 3 18:15:01 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Sat Feb 3 18:18:09 2018 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/net/TestAddress.java    | 50 --------------------
 .../hbase/regionserver/RSRpcServices.java.rej   |  9 ----
 .../ReplicationPeerConfigUpgrader.java.rej      | 10 ----
 3 files changed, 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6519b98a/hbase-common/src/test/java/org/apache/hadoop/hbase/net/TestAddress.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/net/TestAddress.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/net/TestAddress.java
deleted file mode 100644
index 22c7940..0000000
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/net/TestAddress.java
+++ /dev/null
@@ -1,50 +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.net;
-
-
-import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.Assert;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import static org.junit.Assert.assertEquals;
-
-@Category({ MiscTests.class, SmallTests.class })
-public class TestAddress {
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestAddress.class);
-
-  @Test
-  public void testGetHostWithoutDomain() {
-    assertEquals("a:123",
-        Address.fromParts("a.b.c", 123).toStringWithoutDomain());
-    assertEquals("1:123",
-        Address.fromParts("1.b.c", 123).toStringWithoutDomain());
-    assertEquals("123.456.789.1:123",
-        Address.fromParts("123.456.789.1", 123).toStringWithoutDomain());
-    assertEquals("[2001:db8::1]:80",
-        Address.fromParts("[2001:db8::1]:", 80).toStringWithoutDomain());
-    assertEquals("[2001:db8::1]:80",
-        Address.fromParts("[2001:db8::1]:", 80).toStringWithoutDomain());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/6519b98a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java.rej
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java.rej b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java.rej
deleted file mode 100644
index 3c7237c..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java.rej
+++ /dev/null
@@ -1,9 +0,0 @@
-diff 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	(rejected hunks)
-@@ -100,6 +100,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.net.Address;
- import org.apache.hadoop.hbase.quotas.ActivePolicyEnforcement;
- import org.apache.hadoop.hbase.quotas.OperationQuota;
- import org.apache.hadoop.hbase.quotas.QuotaUtil;

http://git-wip-us.apache.org/repos/asf/hbase/blob/6519b98a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java.rej
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java.rej b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java.rej
deleted file mode 100644
index a9829ef..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java.rej
+++ /dev/null
@@ -1,10 +0,0 @@
-diff a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/master/ReplicationPeerConfigUpgrader.java	(rejected hunks)
-@@ -100,7 +100,7 @@ public class ReplicationPeerConfigUpgrader extends ReplicationStateZKBase {
-         // We only need to copy data from tableCFs node to rpc Node the first time hmaster start.
-         if (rpc.getTableCFsMap() == null || rpc.getTableCFsMap().isEmpty()) {
-           // we copy TableCFs node into PeerNode
--          LOG.info("copy tableCFs into peerNode:" + peerId);
-+          LOG.info("Copy table ColumnFamilies into peer=" + peerId);
-           ReplicationProtos.TableCF[] tableCFs =
-                   ReplicationPeerConfigUtil.parseTableCFs(
-                           ZKUtil.getData(this.zookeeper, tableCFsNode));


[09/28] hbase git commit: HBASE-19919 Tidying up logging; ADDENDUM Fix tests w/ mocked Servers

Posted by zh...@apache.org.
HBASE-19919 Tidying up logging; ADDENDUM Fix tests w/ mocked Servers


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

Branch: refs/heads/HBASE-19064
Commit: 12f3c82a866eee0436c22136909882581fd19905
Parents: 06dec20
Author: Michael Stack <st...@apache.org>
Authored: Sat Feb 3 09:25:16 2018 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Sat Feb 3 09:25:16 2018 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/master/SplitLogManager.java   | 6 +++++-
 .../hadoop/hbase/master/assignment/AssignmentManager.java      | 6 +++++-
 2 files changed, 10 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/12f3c82a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
index 63af398..2e2f8bf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
@@ -121,8 +121,12 @@ public class SplitLogManager {
       throws IOException {
     this.server = master;
     this.conf = conf;
+    // Get Server Thread name. Sometimes the Server is mocked so may not implement HasThread.
+    // For example, in tests.
+    String name = master instanceof HasThread? ((HasThread)master).getName():
+        master.getServerName().toShortString();
     this.choreService =
-        new ChoreService(((HasThread)master).getName() + ".splitLogManager.");
+        new ChoreService(name + ".splitLogManager.");
     if (server.getCoordinatedStateManager() != null) {
       SplitLogManagerCoordination coordination = getSplitLogManagerCoordination();
       Set<String> failedDeletions = Collections.synchronizedSet(new HashSet<String>());

http://git-wip-us.apache.org/repos/asf/hbase/blob/12f3c82a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
index 1f65230..e09b29b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
@@ -1624,7 +1624,11 @@ public class AssignmentManager implements ServerListener {
   }
 
   private void startAssignmentThread() {
-    assignThread = new Thread(((HasThread)this.master).getName()) {
+    // Get Server Thread name. Sometimes the Server is mocked so may not implement HasThread.
+    // For example, in tests.
+    String name = master instanceof HasThread? ((HasThread)master).getName():
+        master.getServerName().toShortString();
+    assignThread = new Thread(name) {
       @Override
       public void run() {
         while (isRunning()) {