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/04/09 07:19:21 UTC

[01/22] hbase git commit: HBASE-20361 Non-successive TableInputSplits may wrongly be merged by auto balancing feature [Forced Update!]

Repository: hbase
Updated Branches:
  refs/heads/HBASE-20046-branch-2 e821b9ac5 -> 74ab10c35 (forced update)


HBASE-20361 Non-successive TableInputSplits may wrongly be merged by auto balancing feature

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

Branch: refs/heads/HBASE-20046-branch-2
Commit: 1ec708bf6cbe6cead0270a59f0b7d1e18dd3ac38
Parents: 10ed3f4
Author: Yuki Tawara <yk...@gmail.com>
Authored: Sun Apr 8 01:46:52 2018 +0900
Committer: tedyu <yu...@gmail.com>
Committed: Sun Apr 8 10:58:56 2018 -0700

----------------------------------------------------------------------
 .../hbase/mapreduce/TableInputFormatBase.java   |  12 +-
 .../mapreduce/TestTableInputFormatBase.java     | 232 +++++++++++++++++++
 2 files changed, 242 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1ec708bf/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
index d8031d9..24973c9 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableInputFormatBase.java
@@ -53,6 +53,7 @@ import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.net.DNS;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
 /**
  * A base for {@link TableInputFormat}s. Receives a {@link Connection}, a {@link TableName},
@@ -291,7 +292,7 @@ public abstract class TableInputFormatBase
    */
   private List<InputSplit> oneInputSplitPerRegion() throws IOException {
     RegionSizeCalculator sizeCalculator =
-        new RegionSizeCalculator(getRegionLocator(), getAdmin());
+        createRegionSizeCalculator(getRegionLocator(), getAdmin());
 
     TableName tableName = getTable().getName();
 
@@ -478,7 +479,8 @@ public abstract class TableInputFormatBase
         while (j < splits.size()) {
           TableSplit nextRegion = (TableSplit) splits.get(j);
           long nextRegionSize = nextRegion.getLength();
-          if (totalSize + nextRegionSize <= averageRegionSize) {
+          if (totalSize + nextRegionSize <= averageRegionSize
+              && Bytes.equals(splitEndKey, nextRegion.getStartRow())) {
             totalSize = totalSize + nextRegionSize;
             splitEndKey = nextRegion.getEndRow();
             j++;
@@ -586,6 +588,12 @@ public abstract class TableInputFormatBase
     this.connection = connection;
   }
 
+  @VisibleForTesting
+  protected RegionSizeCalculator createRegionSizeCalculator(RegionLocator locator, Admin admin)
+      throws IOException {
+    return new RegionSizeCalculator(locator, admin);
+  }
+
   /**
    * Gets the scan defining the actual details like columns etc.
    *

http://git-wip-us.apache.org/repos/asf/hbase/blob/1ec708bf/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java
index 5fa4b54..29a92ee 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java
@@ -18,15 +18,45 @@
 package org.apache.hadoop.hbase.mapreduce;
 
 import static org.junit.Assert.*;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.anyBoolean;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
+import java.io.IOException;
 import java.net.Inet6Address;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.ExecutorService;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.BufferedMutator;
+import org.apache.hadoop.hbase.client.BufferedMutatorParams;
+import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableBuilder;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.mapreduce.JobContext;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 @Category({SmallTests.class})
 public class TestTableInputFormatBase {
@@ -55,4 +85,206 @@ public class TestTableInputFormatBase {
     assertEquals("Should retrun the hostname for this host. Expected : " +
         localhost + " Actual : " + actualHostName, localhost, actualHostName);
   }
+
+  @Test
+  public void testNonSuccessiveSplitsAreNotMerged() throws IOException {
+    JobContext context = mock(JobContext.class);
+    Configuration conf = HBaseConfiguration.create();
+    conf.set(ClusterConnection.HBASE_CLIENT_CONNECTION_IMPL,
+        ConnectionForMergeTesting.class.getName());
+    conf.set(TableInputFormat.INPUT_TABLE, "testTable");
+    conf.setBoolean(TableInputFormatBase.MAPREDUCE_INPUT_AUTOBALANCE, true);
+    when(context.getConfiguration()).thenReturn(conf);
+
+    TableInputFormat tifExclude = new TableInputFormatForMergeTesting();
+    tifExclude.setConf(conf);
+    // split["b", "c"] is excluded, split["o", "p"] and split["p", "q"] are merged,
+    // but split["a", "b"] and split["c", "d"] are not merged.
+    assertEquals(ConnectionForMergeTesting.START_KEYS.length - 1 - 1,
+        tifExclude.getSplits(context).size());
+  }
+
+  /**
+   * Subclass of {@link TableInputFormat} to use in {@link #testNonSuccessiveSplitsAreNotMerged}.
+   * This class overrides {@link TableInputFormatBase#includeRegionInSplit}
+   * to exclude specific splits.
+   */
+  private static class TableInputFormatForMergeTesting extends TableInputFormat {
+    private byte[] prefixStartKey = Bytes.toBytes("b");
+    private byte[] prefixEndKey = Bytes.toBytes("c");
+    private RegionSizeCalculator sizeCalculator;
+
+    /**
+     * Exclude regions which contain rows starting with "b".
+     */
+    @Override
+    protected boolean includeRegionInSplit(final byte[] startKey, final byte [] endKey) {
+      if (Bytes.compareTo(startKey, prefixEndKey) < 0
+          && (Bytes.compareTo(prefixStartKey, endKey) < 0
+              || Bytes.equals(endKey, HConstants.EMPTY_END_ROW))) {
+        return false;
+      } else {
+        return true;
+      }
+    }
+
+    @Override
+    protected void initializeTable(Connection connection, TableName tableName) throws IOException {
+      super.initializeTable(connection, tableName);
+      ConnectionForMergeTesting cft = (ConnectionForMergeTesting) connection;
+      sizeCalculator = cft.getRegionSizeCalculator();
+    }
+
+    @Override
+    protected RegionSizeCalculator createRegionSizeCalculator(RegionLocator locator, Admin admin)
+      throws IOException {
+      return sizeCalculator;
+    }
+  }
+
+  /**
+   * Connection class to use in {@link #testNonSuccessiveSplitsAreNotMerged}.
+   * This class returns mocked {@link Table}, {@link RegionLocator}, {@link RegionSizeCalculator},
+   * and {@link Admin}.
+   */
+  private static class ConnectionForMergeTesting implements Connection {
+    public static final byte[][] SPLITS = new byte[][] {
+      Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c"), Bytes.toBytes("d"),
+      Bytes.toBytes("e"), Bytes.toBytes("f"), Bytes.toBytes("g"), Bytes.toBytes("h"),
+      Bytes.toBytes("i"), Bytes.toBytes("j"), Bytes.toBytes("k"), Bytes.toBytes("l"),
+      Bytes.toBytes("m"), Bytes.toBytes("n"), Bytes.toBytes("o"), Bytes.toBytes("p"),
+      Bytes.toBytes("q"), Bytes.toBytes("r"), Bytes.toBytes("s"), Bytes.toBytes("t"),
+      Bytes.toBytes("u"), Bytes.toBytes("v"), Bytes.toBytes("w"), Bytes.toBytes("x"),
+      Bytes.toBytes("y"), Bytes.toBytes("z")
+    };
+
+    public static final byte[][] START_KEYS;
+    public static final byte[][] END_KEYS;
+    static {
+      START_KEYS = new byte[SPLITS.length + 1][];
+      START_KEYS[0] = HConstants.EMPTY_BYTE_ARRAY;
+      for (int i = 0; i < SPLITS.length; i++) {
+        START_KEYS[i + 1] = SPLITS[i];
+      }
+
+      END_KEYS = new byte[SPLITS.length + 1][];
+      for (int i = 0; i < SPLITS.length; i++) {
+        END_KEYS[i] = SPLITS[i];
+      }
+      END_KEYS[SPLITS.length] = HConstants.EMPTY_BYTE_ARRAY;
+    }
+
+    public static final Map<byte[], Long> SIZE_MAP = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+    static {
+      for (byte[] startKey : START_KEYS) {
+        SIZE_MAP.put(startKey, 1024L * 1024L * 1024L);
+      }
+      SIZE_MAP.put(Bytes.toBytes("a"), 200L * 1024L * 1024L);
+      SIZE_MAP.put(Bytes.toBytes("b"), 200L * 1024L * 1024L);
+      SIZE_MAP.put(Bytes.toBytes("c"), 200L * 1024L * 1024L);
+      SIZE_MAP.put(Bytes.toBytes("o"), 200L * 1024L * 1024L);
+      SIZE_MAP.put(Bytes.toBytes("p"), 200L * 1024L * 1024L);
+    }
+
+    ConnectionForMergeTesting(Configuration conf, ExecutorService pool, User user)
+        throws IOException {
+    }
+
+    @Override
+    public void abort(String why, Throwable e) {
+    }
+
+    @Override
+    public boolean isAborted() {
+      return false;
+    }
+
+    @Override
+    public Configuration getConfiguration() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Table getTable(TableName tableName) throws IOException {
+      Table table = mock(Table.class);
+      when(table.getName()).thenReturn(tableName);
+      return table;
+    }
+
+    @Override
+    public Table getTable(TableName tableName, ExecutorService pool) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public BufferedMutator getBufferedMutator(TableName tableName) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public BufferedMutator getBufferedMutator(BufferedMutatorParams params) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public RegionLocator getRegionLocator(TableName tableName) throws IOException {
+      final Map<byte[], HRegionLocation> locationMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
+      for (byte[] startKey : START_KEYS) {
+        HRegionLocation hrl = new HRegionLocation(
+            RegionInfoBuilder.newBuilder(tableName).setStartKey(startKey).build(),
+            ServerName.valueOf("localhost", 0, 0));
+        locationMap.put(startKey, hrl);
+      }
+
+      RegionLocator locator = mock(RegionLocator.class);
+      when(locator.getRegionLocation(any(byte [].class), anyBoolean())).
+        thenAnswer(new Answer<HRegionLocation>() {
+          @Override
+          public HRegionLocation answer(InvocationOnMock invocationOnMock) throws Throwable {
+            Object [] args = invocationOnMock.getArguments();
+            byte [] key = (byte [])args[0];
+            return locationMap.get(key);
+          }
+        });
+      when(locator.getStartEndKeys()).
+        thenReturn(new Pair<byte[][], byte[][]>(START_KEYS, END_KEYS));
+      return locator;
+    }
+
+    public RegionSizeCalculator getRegionSizeCalculator() {
+      RegionSizeCalculator sizeCalculator = mock(RegionSizeCalculator.class);
+      when(sizeCalculator.getRegionSize(any(byte[].class))).
+        thenAnswer(new Answer<Long>() {
+          @Override
+          public Long answer(InvocationOnMock invocationOnMock) throws Throwable {
+            Object [] args = invocationOnMock.getArguments();
+            byte [] regionId = (byte [])args[0];
+            byte[] startKey = RegionInfo.getStartKey(regionId);
+            return SIZE_MAP.get(startKey);
+          }
+        });
+      return sizeCalculator;
+    }
+
+    @Override
+    public Admin getAdmin() throws IOException {
+      Admin admin = mock(Admin.class);
+      // return non-null admin to pass null checks
+      return admin;
+    }
+
+    @Override
+    public void close() throws IOException {
+    }
+
+    @Override
+    public boolean isClosed() {
+      return false;
+    }
+
+    @Override
+    public TableBuilder getTableBuilder(TableName tableName, ExecutorService pool) {
+      throw new UnsupportedOperationException();
+    }
+  }
 }


[10/22] hbase git commit: HBASE-20129 Add UT for serial replication checker

Posted by zh...@apache.org.
HBASE-20129 Add UT for serial replication checker


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

Branch: refs/heads/HBASE-20046-branch-2
Commit: 8b61a061d3d8cdf9d4bd271cc6211d669e925e60
Parents: f29bf1d
Author: zhangduo <zh...@apache.org>
Authored: Tue Mar 6 08:40:31 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Apr 9 15:18:44 2018 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/MetaTableAccessor.java  |  71 ++++++++--
 .../regionserver/SerialReplicationChecker.java  |  18 +++
 .../TestSerialReplicationChecker.java           | 133 ++++++++++++++++++-
 3 files changed, 208 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8b61a061/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 109f2d0..2a88b56 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
@@ -19,12 +19,14 @@ package org.apache.hadoop.hbase;
 
 import edu.umd.cs.findbugs.annotations.NonNull;
 import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.ByteArrayOutputStream;
 import java.io.Closeable;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
@@ -35,7 +37,6 @@ import java.util.TreeMap;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
-import java.util.stream.Stream;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell.Type;
 import org.apache.hadoop.hbase.client.Connection;
@@ -150,11 +151,13 @@ public class MetaTableAccessor {
       META_REGION_PREFIX, 0, len);
   }
 
-  private static final byte[] REPLICATION_PARENT_QUALIFIER = Bytes.toBytes("parent");
+  @VisibleForTesting
+  public static final byte[] REPLICATION_PARENT_QUALIFIER = Bytes.toBytes("parent");
+
+  private static final byte ESCAPE_BYTE = (byte) 0xFF;
 
-  private static final String REPLICATION_PARENT_SEPARATOR = "|";
+  private static final byte SEPARATED_BYTE = 0x00;
 
-  private static final String REPLICATION_PARENT_SEPARATOR_REGEX = "\\|";
   /**
    * Lists all of the table regions currently in META.
    * Deprecated, keep there until some test use this.
@@ -1921,10 +1924,51 @@ public class MetaTableAccessor {
               .build());
   }
 
+  private static void writeRegionName(ByteArrayOutputStream out, byte[] regionName) {
+    for (byte b : regionName) {
+      if (b == ESCAPE_BYTE) {
+        out.write(ESCAPE_BYTE);
+      }
+      out.write(b);
+    }
+  }
+
+  @VisibleForTesting
+  public static byte[] getParentsBytes(List<RegionInfo> parents) {
+    ByteArrayOutputStream bos = new ByteArrayOutputStream();
+    Iterator<RegionInfo> iter = parents.iterator();
+    writeRegionName(bos, iter.next().getRegionName());
+    while (iter.hasNext()) {
+      bos.write(ESCAPE_BYTE);
+      bos.write(SEPARATED_BYTE);
+      writeRegionName(bos, iter.next().getRegionName());
+    }
+    return bos.toByteArray();
+  }
+
+  private static List<byte[]> parseParentsBytes(byte[] bytes) {
+    List<byte[]> parents = new ArrayList<>();
+    ByteArrayOutputStream bos = new ByteArrayOutputStream();
+    for (int i = 0; i < bytes.length; i++) {
+      if (bytes[i] == ESCAPE_BYTE) {
+        i++;
+        if (bytes[i] == SEPARATED_BYTE) {
+          parents.add(bos.toByteArray());
+          bos.reset();
+          continue;
+        }
+        // fall through to append the byte
+      }
+      bos.write(bytes[i]);
+    }
+    if (bos.size() > 0) {
+      parents.add(bos.toByteArray());
+    }
+    return parents;
+  }
+
   private static void addReplicationParent(Put put, List<RegionInfo> parents) throws IOException {
-    byte[] value = parents.stream().map(RegionReplicaUtil::getRegionInfoForDefaultReplica)
-      .map(RegionInfo::getRegionNameAsString).collect(Collectors
-        .collectingAndThen(Collectors.joining(REPLICATION_PARENT_SEPARATOR), Bytes::toBytes));
+    byte[] value = getParentsBytes(parents);
     put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
       .setFamily(HConstants.REPLICATION_BARRIER_FAMILY).setQualifier(REPLICATION_PARENT_QUALIFIER)
       .setTimestamp(put.getTimeStamp()).setType(Type.Put).setValue(value).build());
@@ -1995,6 +2039,14 @@ public class MetaTableAccessor {
     public List<byte[]> getParentRegionNames() {
       return parentRegionNames;
     }
+
+    @Override
+    public String toString() {
+      return "ReplicationBarrierResult [barriers=" + Arrays.toString(barriers) + ", state=" +
+        state + ", parentRegionNames=" +
+        parentRegionNames.stream().map(Bytes::toStringBinary).collect(Collectors.joining(", ")) +
+        "]";
+    }
   }
 
   private static long getReplicationBarrier(Cell c) {
@@ -2014,10 +2066,7 @@ public class MetaTableAccessor {
     byte[] parentRegionsBytes =
       result.getValue(HConstants.REPLICATION_BARRIER_FAMILY, REPLICATION_PARENT_QUALIFIER);
     List<byte[]> parentRegionNames =
-      parentRegionsBytes != null
-        ? Stream.of(Bytes.toString(parentRegionsBytes).split(REPLICATION_PARENT_SEPARATOR_REGEX))
-          .map(Bytes::toBytes).collect(Collectors.toList())
-        : Collections.emptyList();
+      parentRegionsBytes != null ? parseParentsBytes(parentRegionsBytes) : Collections.emptyList();
     return new ReplicationBarrierResult(barriers, state, parentRegionNames);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/8b61a061/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SerialReplicationChecker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SerialReplicationChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SerialReplicationChecker.java
index 95f3868..9276359 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SerialReplicationChecker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SerialReplicationChecker.java
@@ -35,6 +35,8 @@ import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.cache.Cache;
 import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
@@ -108,6 +110,8 @@ import org.apache.hbase.thirdparty.com.google.common.cache.LoadingCache;
 @InterfaceAudience.Private
 class SerialReplicationChecker {
 
+  private static final Logger LOG = LoggerFactory.getLogger(SerialReplicationChecker.class);
+
   public static final String REPLICATION_SERIALLY_WAITING_KEY =
     "hbase.serial.replication.waiting.ms";
   public static final long REPLICATION_SERIALLY_WAITING_DEFAULT = 10000;
@@ -182,9 +186,11 @@ class SerialReplicationChecker {
     long seqId = entry.getKey().getSequenceId();
     ReplicationBarrierResult barrierResult = MetaTableAccessor.getReplicationBarrierResult(conn,
       entry.getKey().getTableName(), row, entry.getKey().getEncodedRegionName());
+    LOG.debug("Replication barrier for {}: {}", entry, barrierResult);
     long[] barriers = barrierResult.getBarriers();
     int index = Arrays.binarySearch(barriers, seqId);
     if (index == -1) {
+      LOG.debug("{} is before the first barrier, pass", entry);
       // This means we are in the range before the first record openSeqNum, this usually because the
       // wal is written before we enable serial replication for this table, just return true since
       // we can not guarantee the order.
@@ -203,22 +209,29 @@ class SerialReplicationChecker {
       // we are in the first range, check whether we have parents
       for (byte[] regionName : barrierResult.getParentRegionNames()) {
         if (!isParentFinished(regionName)) {
+          LOG.debug("Parent {} has not been finished yet for entry {}, give up",
+            Bytes.toStringBinary(regionName), entry);
           return false;
         }
       }
       if (isLastRangeAndOpening(barrierResult, index)) {
+        LOG.debug("{} is in the last range and the region is opening, give up", entry);
         return false;
       }
+      LOG.debug("{} is in the first range, pass", entry);
       recordCanPush(encodedNameAsString, seqId, barriers, 1);
       return true;
     }
     // check whether the previous range is finished
     if (!isRangeFinished(barriers[index - 1], encodedNameAsString)) {
+      LOG.debug("Previous range for {} has not been finished yet, give up", entry);
       return false;
     }
     if (isLastRangeAndOpening(barrierResult, index)) {
+      LOG.debug("{} is in the last range and the region is opening, give up", entry);
       return false;
     }
+    LOG.debug("The previous range for {} has been finished, pass", entry);
     recordCanPush(encodedNameAsString, seqId, barriers, index);
     return true;
   }
@@ -229,8 +242,11 @@ class SerialReplicationChecker {
     Long canReplicateUnderSeqId = canPushUnder.getIfPresent(encodedNameAsString);
     if (canReplicateUnderSeqId != null) {
       if (seqId < canReplicateUnderSeqId.longValue()) {
+        LOG.trace("{} is before the end barrier {}, pass", entry, canReplicateUnderSeqId);
         return true;
       }
+      LOG.debug("{} is beyond the previous end barrier {}, remove from cache", entry,
+        canReplicateUnderSeqId);
       // we are already beyond the last safe point, remove
       canPushUnder.invalidate(encodedNameAsString);
     }
@@ -239,6 +255,7 @@ class SerialReplicationChecker {
     // has been moved to another RS and then back, so we need to check the barrier.
     MutableLong previousPushedSeqId = pushed.getUnchecked(encodedNameAsString);
     if (seqId == previousPushedSeqId.longValue() + 1) {
+      LOG.trace("The sequence id for {} is continuous, pass");
       previousPushedSeqId.increment();
       return true;
     }
@@ -249,6 +266,7 @@ class SerialReplicationChecker {
       throws IOException, InterruptedException {
     byte[] row = CellUtil.cloneRow(firstCellInEdit);
     while (!canPush(entry, row)) {
+      LOG.debug("Can not push{}, wait", entry);
       Thread.sleep(waitTimeMs);
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/8b61a061/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java
index c8387c5..58e9543 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java
@@ -19,10 +19,16 @@ package org.apache.hadoop.hbase.replication.regionserver;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
 import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.Cell.Type;
 import org.apache.hadoop.hbase.CellBuilderFactory;
@@ -30,8 +36,10 @@ import org.apache.hadoop.hbase.CellBuilderType;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
@@ -54,6 +62,8 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
 
@@ -72,6 +82,8 @@ public class TestSerialReplicationChecker {
 
   private static String WAL_FILE_NAME = "test.wal";
 
+  private Connection conn;
+
   private SerialReplicationChecker checker;
 
   @Rule
@@ -98,8 +110,18 @@ public class TestSerialReplicationChecker {
     ReplicationSource source = mock(ReplicationSource.class);
     when(source.getPeerId()).thenReturn(PEER_ID);
     when(source.getQueueStorage()).thenReturn(QUEUE_STORAGE);
+    conn = mock(Connection.class);
+    when(conn.isClosed()).thenReturn(false);
+    doAnswer(new Answer<Table>() {
+
+      @Override
+      public Table answer(InvocationOnMock invocation) throws Throwable {
+        return UTIL.getConnection().getTable((TableName) invocation.getArgument(0));
+      }
+
+    }).when(conn).getTable(any(TableName.class));
     Server server = mock(Server.class);
-    when(server.getConnection()).thenReturn(UTIL.getConnection());
+    when(server.getConnection()).thenReturn(conn);
     when(source.getServer()).thenReturn(server);
     checker = new SerialReplicationChecker(UTIL.getConfiguration(), source);
     tableName = TableName.valueOf(name.getMethodName());
@@ -129,8 +151,10 @@ public class TestSerialReplicationChecker {
   private void addStateAndBarrier(RegionInfo region, RegionState.State state, long... barriers)
       throws IOException {
     Put put = new Put(region.getRegionName(), EnvironmentEdgeManager.currentTime());
-    put.addColumn(HConstants.CATALOG_FAMILY, HConstants.STATE_QUALIFIER,
-      Bytes.toBytes(state.name()));
+    if (state != null) {
+      put.addColumn(HConstants.CATALOG_FAMILY, HConstants.STATE_QUALIFIER,
+        Bytes.toBytes(state.name()));
+    }
     for (int i = 0; i < barriers.length; i++) {
       put.addColumn(HConstants.REPLICATION_BARRIER_FAMILY, HConstants.SEQNUM_QUALIFIER,
         put.getTimeStamp() - i, Bytes.toBytes(barriers[i]));
@@ -154,6 +178,15 @@ public class TestSerialReplicationChecker {
       PEER_ID, WAL_FILE_NAME, 10, ImmutableMap.of(region.getEncodedName(), seqId));
   }
 
+  private void addParents(RegionInfo region, List<RegionInfo> parents) throws IOException {
+    Put put = new Put(region.getRegionName(), EnvironmentEdgeManager.currentTime());
+    put.addColumn(HConstants.REPLICATION_BARRIER_FAMILY,
+      MetaTableAccessor.REPLICATION_PARENT_QUALIFIER, MetaTableAccessor.getParentsBytes(parents));
+    try (Table table = UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) {
+      table.put(put);
+    }
+  }
+
   @Test
   public void testLastRegionAndOpeningCanNotPush() throws IOException, ReplicationException {
     RegionInfo region = RegionInfoBuilder.newBuilder(tableName).build();
@@ -173,4 +206,98 @@ public class TestSerialReplicationChecker {
     setState(region, RegionState.State.OPENING);
     assertFalse(checker.canPush(createEntry(region, 104), cell));
   }
+
+  @Test
+  public void testCanPushUnder() throws IOException, ReplicationException {
+    RegionInfo region = RegionInfoBuilder.newBuilder(tableName).build();
+    addStateAndBarrier(region, RegionState.State.OPEN, 10, 100);
+    updatePushedSeqId(region, 9);
+    Cell cell = createCell(region);
+    assertTrue(checker.canPush(createEntry(region, 20), cell));
+    verify(conn, times(1)).getTable(any(TableName.class));
+    // not continuous
+    for (int i = 22; i < 100; i += 2) {
+      assertTrue(checker.canPush(createEntry(region, i), cell));
+    }
+    // verify that we do not go to meta table
+    verify(conn, times(1)).getTable(any(TableName.class));
+  }
+
+  @Test
+  public void testCanPushIfContinuous() throws IOException, ReplicationException {
+    RegionInfo region = RegionInfoBuilder.newBuilder(tableName).build();
+    addStateAndBarrier(region, RegionState.State.OPEN, 10);
+    updatePushedSeqId(region, 9);
+    Cell cell = createCell(region);
+    assertTrue(checker.canPush(createEntry(region, 20), cell));
+    verify(conn, times(1)).getTable(any(TableName.class));
+    // continuous
+    for (int i = 21; i < 100; i++) {
+      assertTrue(checker.canPush(createEntry(region, i), cell));
+    }
+    // verify that we do not go to meta table
+    verify(conn, times(1)).getTable(any(TableName.class));
+  }
+
+  @Test
+  public void testCanPushAfterMerge() throws IOException, ReplicationException {
+    // 0xFF is the escape byte when storing region name so let's make sure it can work.
+    byte[] endKey = new byte[] { (byte) 0xFF, 0x00, (byte) 0xFF, (byte) 0xFF, 0x01 };
+    RegionInfo regionA =
+      RegionInfoBuilder.newBuilder(tableName).setEndKey(endKey).setRegionId(1).build();
+    RegionInfo regionB =
+      RegionInfoBuilder.newBuilder(tableName).setStartKey(endKey).setRegionId(2).build();
+    RegionInfo region = RegionInfoBuilder.newBuilder(tableName).setRegionId(3).build();
+    addStateAndBarrier(regionA, null, 10, 100);
+    addStateAndBarrier(regionB, null, 20, 200);
+    addStateAndBarrier(region, RegionState.State.OPEN, 200);
+    addParents(region, Arrays.asList(regionA, regionB));
+    Cell cell = createCell(region);
+    // can not push since both parents have not been finished yet
+    assertFalse(checker.canPush(createEntry(region, 300), cell));
+    updatePushedSeqId(regionB, 199);
+    // can not push since regionA has not been finished yet
+    assertFalse(checker.canPush(createEntry(region, 300), cell));
+    updatePushedSeqId(regionA, 99);
+    // can push since all parents have been finished
+    assertTrue(checker.canPush(createEntry(region, 300), cell));
+  }
+
+  @Test
+  public void testCanPushAfterSplit() throws IOException, ReplicationException {
+    // 0xFF is the escape byte when storing region name so let's make sure it can work.
+    byte[] endKey = new byte[] { (byte) 0xFF, 0x00, (byte) 0xFF, (byte) 0xFF, 0x01 };
+    RegionInfo region = RegionInfoBuilder.newBuilder(tableName).setRegionId(1).build();
+    RegionInfo regionA =
+      RegionInfoBuilder.newBuilder(tableName).setEndKey(endKey).setRegionId(2).build();
+    RegionInfo regionB =
+      RegionInfoBuilder.newBuilder(tableName).setStartKey(endKey).setRegionId(3).build();
+    addStateAndBarrier(region, null, 10, 100);
+    addStateAndBarrier(regionA, RegionState.State.OPEN, 100, 200);
+    addStateAndBarrier(regionB, RegionState.State.OPEN, 100, 300);
+    addParents(regionA, Arrays.asList(region));
+    addParents(regionB, Arrays.asList(region));
+    Cell cellA = createCell(regionA);
+    Cell cellB = createCell(regionB);
+    // can not push since parent has not been finished yet
+    assertFalse(checker.canPush(createEntry(regionA, 150), cellA));
+    assertFalse(checker.canPush(createEntry(regionB, 200), cellB));
+    updatePushedSeqId(region, 99);
+    // can push since parent has been finished
+    assertTrue(checker.canPush(createEntry(regionA, 150), cellA));
+    assertTrue(checker.canPush(createEntry(regionB, 200), cellB));
+  }
+
+  @Test
+  public void testCanPushEqualsToBarrier() throws IOException, ReplicationException {
+    // For binary search, equals to an element will result to a positive value, let's test whether
+    // it works.
+    RegionInfo region = RegionInfoBuilder.newBuilder(tableName).build();
+    addStateAndBarrier(region, RegionState.State.OPEN, 10, 100);
+    Cell cell = createCell(region);
+    assertTrue(checker.canPush(createEntry(region, 10), cell));
+    assertFalse(checker.canPush(createEntry(region, 100), cell));
+    updatePushedSeqId(region, 99);
+    assertTrue(checker.canPush(createEntry(region, 100), cell));
+  }
 }


[18/22] hbase git commit: HBASE-20271 ReplicationSourceWALReader.switched should use the file name instead of the path object directly

Posted by zh...@apache.org.
HBASE-20271 ReplicationSourceWALReader.switched should use the file name instead of the path object directly


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

Branch: refs/heads/HBASE-20046-branch-2
Commit: 6f13b28c1988630e4ae7de70f613502cdbc3dfa1
Parents: 9369cf2
Author: zhangduo <zh...@apache.org>
Authored: Sat Mar 24 16:25:20 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Apr 9 15:18:44 2018 +0800

----------------------------------------------------------------------
 .../replication/regionserver/ReplicationSourceWALReader.java      | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6f13b28c/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
index 2154856..7ba347f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
@@ -174,7 +174,8 @@ class ReplicationSourceWALReader extends Thread {
   }
 
   protected static final boolean switched(WALEntryStream entryStream, Path path) {
-    return !path.equals(entryStream.getCurrentPath());
+    Path newPath = entryStream.getCurrentPath();
+    return newPath == null || !path.getName().equals(newPath.getName());
   }
 
   protected WALEntryBatch readWALEntries(WALEntryStream entryStream)


[22/22] hbase git commit: HBASE-20127 Add UT for serial replication after failover

Posted by zh...@apache.org.
HBASE-20127 Add UT for serial replication after failover


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

Branch: refs/heads/HBASE-20046-branch-2
Commit: 8de95963555aecf49952a1c7a8b3d17c76dceb6f
Parents: 3b015d6
Author: zhangduo <zh...@apache.org>
Authored: Mon Mar 26 16:08:20 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Apr 9 15:18:44 2018 +0800

----------------------------------------------------------------------
 .../replication/SerialReplicationTestBase.java  |  7 ++
 .../replication/TestSerialReplication.java      |  5 --
 .../TestSerialReplicationFailover.java          | 76 ++++++++++++++++++++
 3 files changed, 83 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8de95963/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SerialReplicationTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SerialReplicationTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SerialReplicationTestBase.java
index 83afd81..b5aae85 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SerialReplicationTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SerialReplicationTestBase.java
@@ -113,6 +113,7 @@ public class SerialReplicationTestBase {
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     UTIL.getConfiguration().setInt("replication.source.nb.capacity", 10);
+    UTIL.getConfiguration().setLong("replication.sleep.before.failover", 1000);
     UTIL.startMiniCluster(3);
     // disable balancer
     UTIL.getAdmin().balancerSwitch(false, true);
@@ -200,6 +201,11 @@ public class SerialReplicationTestBase {
     });
   }
 
+  protected final void enablePeerAndWaitUntilReplicationDone(int expectedEntries) throws Exception {
+    UTIL.getAdmin().enableReplicationPeer(PEER_ID);
+    waitUntilReplicationDone(expectedEntries);
+  }
+
   protected final void addPeer(boolean enabled) throws IOException {
     UTIL.getAdmin().addReplicationPeer(PEER_ID,
       ReplicationPeerConfig.newBuilder().setClusterKey("127.0.0.1:2181:/hbase")
@@ -221,6 +227,7 @@ public class SerialReplicationTestBase {
         assertTrue(
           "Sequence id go backwards from " + seqId + " to " + entry.getKey().getSequenceId(),
           entry.getKey().getSequenceId() >= seqId);
+        seqId = entry.getKey().getSequenceId();
         count++;
       }
       assertEquals(expectedEntries, count);

http://git-wip-us.apache.org/repos/asf/hbase/blob/8de95963/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
index 94b79d9..bedb2ec 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
@@ -63,11 +63,6 @@ public class TestSerialReplication extends SerialReplicationTestBase {
     addPeer(false);
   }
 
-  private void enablePeerAndWaitUntilReplicationDone(int expectedEntries) throws Exception {
-    UTIL.getAdmin().enableReplicationPeer(PEER_ID);
-    waitUntilReplicationDone(expectedEntries);
-  }
-
   @Test
   public void testRegionMove() throws Exception {
     TableName tableName = TableName.valueOf(name.getMethodName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/8de95963/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplicationFailover.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplicationFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplicationFailover.java
new file mode 100644
index 0000000..324a69f
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplicationFailover.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Put;
+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.ReplicationTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ ReplicationTests.class, MediumTests.class })
+public class TestSerialReplicationFailover extends SerialReplicationTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestSerialReplicationFailover.class);
+
+  @Before
+  public void setUp() throws IOException, StreamLacksCapabilityException {
+    setupWALWriter();
+    // add in disable state, so later when enabling it all sources will start push together.
+    addPeer(false);
+  }
+
+  @Test
+  public void testKillRS() throws Exception {
+    TableName tableName = TableName.valueOf(name.getMethodName());
+    UTIL.getAdmin().createTable(
+      TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(ColumnFamilyDescriptorBuilder
+        .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build());
+    UTIL.waitTableAvailable(tableName);
+    try (Table table = UTIL.getConnection().getTable(tableName)) {
+      for (int i = 0; i < 100; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+    RegionServerThread thread = UTIL.getMiniHBaseCluster().getRegionServerThreads().stream()
+      .filter(t -> !t.getRegionServer().getRegions(tableName).isEmpty()).findFirst().get();
+    thread.getRegionServer().abort("for testing");
+    thread.join();
+    try (Table table = UTIL.getConnection().getTable(tableName)) {
+      for (int i = 100; i < 200; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+    enablePeerAndWaitUntilReplicationDone(200);
+    checkOrder(200);
+  }
+}


[03/22] hbase git commit: HBASE-20138 Find a way to deal with the conflicts when updating replication position

Posted by zh...@apache.org.
HBASE-20138 Find a way to deal with the conflicts when updating replication position


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

Branch: refs/heads/HBASE-20046-branch-2
Commit: 83488b866f59d579bc4e9448b4953bf6427c549d
Parents: 8de9596
Author: huzheng <op...@gmail.com>
Authored: Wed Mar 21 17:34:10 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Apr 9 15:18:44 2018 +0800

----------------------------------------------------------------------
 .../replication/ZKReplicationQueueStorage.java  | 110 +++++++++++++------
 .../replication/TestReplicationStateBasic.java  |   6 +
 .../TestZKReplicationQueueStorage.java          |  48 +++++++-
 .../apache/hadoop/hbase/zookeeper/ZKUtil.java   |  34 ++++--
 .../hadoop/hbase/zookeeper/TestZKUtil.java      |  25 +++++
 5 files changed, 178 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/83488b86/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
index 2ab08ae..2e7a012 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
@@ -203,20 +203,25 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
   }
 
   private void addLastSeqIdsToOps(String queueId, Map<String, Long> lastSeqIds,
-      List<ZKUtilOp> listOfOps) throws KeeperException {
+      List<ZKUtilOp> listOfOps) throws KeeperException, ReplicationException {
+    String peerId = new ReplicationQueueInfo(queueId).getPeerId();
     for (Entry<String, Long> lastSeqEntry : lastSeqIds.entrySet()) {
-      String peerId = new ReplicationQueueInfo(queueId).getPeerId();
       String path = getSerialReplicationRegionPeerNode(lastSeqEntry.getKey(), peerId);
-      /*
-       * Make sure the existence of path
-       * /hbase/replication/regions/<hash>/<encoded-region-name>-<peer-id>. As the javadoc in
-       * multiOrSequential() method said, if received a NodeExistsException, all operations will
-       * fail. So create the path here, and in fact, no need to add this operation to listOfOps,
-       * because only need to make sure that update file position and sequence id atomically.
-       */
-      ZKUtil.createWithParents(zookeeper, path);
-      // Persist the max sequence id of region to zookeeper.
-      listOfOps.add(ZKUtilOp.setData(path, ZKUtil.positionToByteArray(lastSeqEntry.getValue())));
+      Pair<Long, Integer> p = getLastSequenceIdWithVersion(lastSeqEntry.getKey(), peerId);
+      byte[] data = ZKUtil.positionToByteArray(lastSeqEntry.getValue());
+      if (p.getSecond() < 0) { // ZNode does not exist.
+        ZKUtil.createWithParents(zookeeper,
+          path.substring(0, path.lastIndexOf(ZNodePaths.ZNODE_PATH_SEPARATOR)));
+        listOfOps.add(ZKUtilOp.createAndFailSilent(path, data));
+        continue;
+      }
+      // Perform CAS in a specific version v0 (HBASE-20138)
+      int v0 = p.getSecond();
+      long lastPushedSeqId = p.getFirst();
+      if (lastSeqEntry.getValue() <= lastPushedSeqId) {
+        continue;
+      }
+      listOfOps.add(ZKUtilOp.setData(path, data, v0));
     }
   }
 
@@ -224,50 +229,85 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
   public void setWALPosition(ServerName serverName, String queueId, String fileName, long position,
       Map<String, Long> lastSeqIds) throws ReplicationException {
     try {
-      List<ZKUtilOp> listOfOps = new ArrayList<>();
-      if (position > 0) {
-        listOfOps.add(ZKUtilOp.setData(getFileNode(serverName, queueId, fileName),
-          ZKUtil.positionToByteArray(position)));
+      for (int retry = 0;; retry++) {
+        List<ZKUtilOp> listOfOps = new ArrayList<>();
+        if (position > 0) {
+          listOfOps.add(ZKUtilOp.setData(getFileNode(serverName, queueId, fileName),
+            ZKUtil.positionToByteArray(position)));
+        }
+        // Persist the max sequence id(s) of regions for serial replication atomically.
+        addLastSeqIdsToOps(queueId, lastSeqIds, listOfOps);
+        if (listOfOps.isEmpty()) {
+          return;
+        }
+        try {
+          ZKUtil.multiOrSequential(zookeeper, listOfOps, false);
+          return;
+        } catch (KeeperException.BadVersionException | KeeperException.NodeExistsException e) {
+          LOG.warn(
+            "Bad version(or node exist) when persist the last pushed sequence id to zookeeper storage, "
+                + "Retry = " + retry + ", serverName=" + serverName + ", queueId=" + queueId
+                + ", fileName=" + fileName);
+        }
       }
-      // Persist the max sequence id(s) of regions for serial replication atomically.
-      addLastSeqIdsToOps(queueId, lastSeqIds, listOfOps);
-      ZKUtil.multiOrSequential(zookeeper, listOfOps, false);
     } catch (KeeperException e) {
       throw new ReplicationException("Failed to set log position (serverName=" + serverName
           + ", queueId=" + queueId + ", fileName=" + fileName + ", position=" + position + ")", e);
     }
   }
 
-  @Override
-  public long getLastSequenceId(String encodedRegionName, String peerId)
-      throws ReplicationException {
-    byte[] data;
-    try {
-      data =
-          ZKUtil.getData(zookeeper, getSerialReplicationRegionPeerNode(encodedRegionName, peerId));
-    } catch (KeeperException | InterruptedException e) {
-      throw new ReplicationException("Failed to get the last sequence id(region="
-          + encodedRegionName + ", peerId=" + peerId + ")");
+  /**
+   * Return the {lastPushedSequenceId, ZNodeDataVersion} pair. if ZNodeDataVersion is -1, it means
+   * that the ZNode does not exist.
+   */
+  @VisibleForTesting
+  protected Pair<Long, Integer> getLastSequenceIdWithVersion(String encodedRegionName,
+      String peerId) throws KeeperException {
+    Stat stat = new Stat();
+    String path = getSerialReplicationRegionPeerNode(encodedRegionName, peerId);
+    byte[] data = ZKUtil.getDataNoWatch(zookeeper, path, stat);
+    if (data == null) {
+      // ZNode does not exist, so just return version -1 to indicate that no node exist.
+      return Pair.newPair(HConstants.NO_SEQNUM, -1);
     }
     try {
-      return ZKUtil.parseWALPositionFrom(data);
+      return Pair.newPair(ZKUtil.parseWALPositionFrom(data), stat.getVersion());
     } catch (DeserializationException de) {
       LOG.warn("Failed to parse log position (region=" + encodedRegionName + ", peerId=" + peerId
           + "), data=" + Bytes.toStringBinary(data));
     }
-    return HConstants.NO_SEQNUM;
+    return Pair.newPair(HConstants.NO_SEQNUM, stat.getVersion());
+  }
+
+  @Override
+  public long getLastSequenceId(String encodedRegionName, String peerId)
+      throws ReplicationException {
+    try {
+      return getLastSequenceIdWithVersion(encodedRegionName, peerId).getFirst();
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to get last pushed sequence id (encodedRegionName="
+          + encodedRegionName + ", peerId=" + peerId + ")", e);
+    }
   }
 
   @Override
   public void setLastSequenceIds(String peerId, Map<String, Long> lastSeqIds)
       throws ReplicationException {
     try {
+      // No need CAS and retry here, because it'll call setLastSequenceIds() for disabled peers
+      // only, so no conflict happen.
       List<ZKUtilOp> listOfOps = new ArrayList<>();
-      addLastSeqIdsToOps(peerId, lastSeqIds, listOfOps);
-      ZKUtil.multiOrSequential(zookeeper, listOfOps, false);
+      for (Entry<String, Long> lastSeqEntry : lastSeqIds.entrySet()) {
+        String path = getSerialReplicationRegionPeerNode(lastSeqEntry.getKey(), peerId);
+        ZKUtil.createWithParents(zookeeper, path);
+        listOfOps.add(ZKUtilOp.setData(path, ZKUtil.positionToByteArray(lastSeqEntry.getValue())));
+      }
+      if (!listOfOps.isEmpty()) {
+        ZKUtil.multiOrSequential(zookeeper, listOfOps, true);
+      }
     } catch (KeeperException e) {
-      throw new ReplicationException("Failed to set last sequence ids, peerId=" + peerId +
-        ", lastSeqIds.size=" + lastSeqIds.size(), e);
+      throw new ReplicationException("Failed to set last sequence ids, peerId=" + peerId
+          + ", size of lastSeqIds=" + lastSeqIds.size(), e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/83488b86/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 3ed4121..437804c 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
@@ -316,6 +316,12 @@ public abstract class TestReplicationStateBasic {
     }
     assertEquals(900L, rqs.getLastSequenceId(region0, queue1));
     assertEquals(1000L, rqs.getLastSequenceId(region1, queue1));
+
+    // Try to decrease the last pushed id by setWALPosition method.
+    rqs.setWALPosition(serverName1, queue1, getFileName("file1", 0), 11 * 100,
+      ImmutableMap.of(region0, 899L, region1, 1001L));
+    assertEquals(900L, rqs.getLastSequenceId(region0, queue1));
+    assertEquals(1001L, rqs.getLastSequenceId(region1, queue1));
   }
 
   protected void assertConnectedPeerStatus(boolean status, String peerId) throws Exception {

http://git-wip-us.apache.org/repos/asf/hbase/blob/83488b86/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
index ca86a05..5821271 100644
--- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
@@ -28,6 +28,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 import java.util.SortedSet;
+
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseZKTestingUtility;
@@ -35,6 +36,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -44,6 +46,8 @@ import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
+
 @Category({ ReplicationTests.class, MediumTests.class })
 public class TestZKReplicationQueueStorage {
 
@@ -215,10 +219,11 @@ public class TestZKReplicationQueueStorage {
     assertEquals(1, v1 - v0);
   }
 
-  private ZKReplicationQueueStorage createWithUnstableCversion() throws IOException {
+  private ZKReplicationQueueStorage createWithUnstableVersion() throws IOException {
     return new ZKReplicationQueueStorage(UTIL.getZooKeeperWatcher(), UTIL.getConfiguration()) {
 
       private int called = 0;
+      private int getLastSeqIdOpIndex = 0;
 
       @Override
       protected int getQueuesZNodeCversion() throws KeeperException {
@@ -227,12 +232,26 @@ public class TestZKReplicationQueueStorage {
         }
         return called;
       }
+
+      @Override
+      protected Pair<Long, Integer> getLastSequenceIdWithVersion(String encodedRegionName,
+          String peerId) throws KeeperException {
+        Pair<Long, Integer> oldPair = super.getLastSequenceIdWithVersion(encodedRegionName, peerId);
+        if (getLastSeqIdOpIndex < 100) {
+          // Let the ZNode version increase.
+          String path = getSerialReplicationRegionPeerNode(encodedRegionName, peerId);
+          ZKUtil.createWithParents(zookeeper, path);
+          ZKUtil.setData(zookeeper, path, ZKUtil.positionToByteArray(100L));
+        }
+        getLastSeqIdOpIndex++;
+        return oldPair;
+      }
     };
   }
 
   @Test
   public void testGetAllWALsCversionChange() throws IOException, ReplicationException {
-    ZKReplicationQueueStorage storage = createWithUnstableCversion();
+    ZKReplicationQueueStorage storage = createWithUnstableVersion();
     storage.addWAL(getServerName(0), "1", "file");
     // This should return eventually when cversion stabilizes
     Set<String> allWals = storage.getAllWALs();
@@ -243,7 +262,7 @@ public class TestZKReplicationQueueStorage {
   // For HBASE-14621
   @Test
   public void testGetAllHFileRefsCversionChange() throws IOException, ReplicationException {
-    ZKReplicationQueueStorage storage = createWithUnstableCversion();
+    ZKReplicationQueueStorage storage = createWithUnstableVersion();
     storage.addPeerToHFileRefs("1");
     Path p = new Path("/test");
     storage.addHFileRefs("1", Arrays.asList(Pair.newPair(p, p)));
@@ -253,6 +272,29 @@ public class TestZKReplicationQueueStorage {
     assertThat(allHFileRefs, hasItems("test"));
   }
 
+  // For HBASE-20138
+  @Test
+  public void testSetWALPositionBadVersion() throws IOException, ReplicationException {
+    ZKReplicationQueueStorage storage = createWithUnstableVersion();
+    ServerName serverName1 = ServerName.valueOf("128.0.0.1", 8000, 10000);
+    assertTrue(storage.getAllQueues(serverName1).isEmpty());
+    String queue1 = "1";
+    String fileName = getFileName("file1", 0);
+    String encodedRegionName = "31d9792f4435b99d9fb1016f6fbc8dc6";
+    storage.addWAL(serverName1, queue1, fileName);
+
+    List<String> wals1 = storage.getWALsInQueue(serverName1, queue1);
+    assertEquals(1, wals1.size());
+
+    assertEquals(0, storage.getWALPosition(serverName1, queue1, fileName));
+    // This should return eventually when data version stabilizes
+    storage.setWALPosition(serverName1, queue1, fileName, 100,
+      ImmutableMap.of(encodedRegionName, 120L));
+
+    assertEquals(100, storage.getWALPosition(serverName1, queue1, fileName));
+    assertEquals(120L, storage.getLastSequenceId(encodedRegionName, queue1));
+  }
+
   @Test
   public void testRegionsZNodeLayout() throws Exception {
     String peerId = "1";

http://git-wip-us.apache.org/repos/asf/hbase/blob/83488b86/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
index dda5319..f1cacbf 100644
--- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
@@ -1500,11 +1500,18 @@ public final class ZKUtil {
     /**
      * @return a setData ZKUtilOp
      */
-    public static ZKUtilOp setData(String path, byte [] data) {
+    public static ZKUtilOp setData(String path, byte[] data) {
       return new SetData(path, data);
     }
 
     /**
+     * @return a setData ZKUtilOp
+     */
+    public static ZKUtilOp setData(String path, byte[] data, int version) {
+      return new SetData(path, data, version);
+    }
+
+    /**
      * @return path to znode where the ZKOp will occur
      */
     public String getPath() {
@@ -1578,17 +1585,28 @@ public final class ZKUtil {
      * ZKUtilOp representing setData in ZooKeeper
      */
     public static final class SetData extends ZKUtilOp {
-      private byte [] data;
+      private byte[] data;
+      private int version = -1;
+
+      private SetData(String path, byte[] data) {
+        super(path);
+        this.data = data;
+      }
 
-      private SetData(String path, byte [] data) {
+      private SetData(String path, byte[] data, int version) {
         super(path);
         this.data = data;
+        this.version = version;
       }
 
       public byte[] getData() {
         return data;
       }
 
+      public int getVersion() {
+        return version;
+      }
+
       @Override
       public boolean equals(Object o) {
         if (this == o) {
@@ -1599,13 +1617,15 @@ public final class ZKUtil {
         }
 
         SetData op = (SetData) o;
-        return getPath().equals(op.getPath()) && Arrays.equals(data, op.data);
+        return getPath().equals(op.getPath()) && Arrays.equals(data, op.data)
+            && getVersion() == op.getVersion();
       }
 
       @Override
       public int hashCode() {
         int ret = getPath().hashCode();
-        return ret * 31 + Bytes.hashCode(data);
+        ret = ret * 31 + Bytes.hashCode(data);
+        return ret * 31 + Integer.hashCode(version);
       }
     }
   }
@@ -1626,8 +1646,8 @@ public final class ZKUtil {
       DeleteNodeFailSilent dnfs = (DeleteNodeFailSilent)op;
       return Op.delete(dnfs.getPath(), -1);
     } else if (op instanceof SetData) {
-      SetData sd = (SetData)op;
-      return Op.setData(sd.getPath(), sd.getData(), -1);
+      SetData sd = (SetData) op;
+      return Op.setData(sd.getPath(), sd.getData(), sd.getVersion());
     } else {
       throw new UnsupportedOperationException("Unexpected ZKUtilOp type: "
         + op.getClass().getName());

http://git-wip-us.apache.org/repos/asf/hbase/blob/83488b86/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java
----------------------------------------------------------------------
diff --git a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java
index 6c3279a..1508441 100644
--- a/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java
+++ b/hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.zookeeper;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
@@ -32,6 +33,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ZKTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooDefs;
@@ -46,6 +48,7 @@ import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
 import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
 
 @Category({ ZKTests.class, MediumTests.class })
@@ -117,6 +120,28 @@ public class TestZKUtil {
     assertNull(ZKUtil.getDataNoWatch(ZKW, "/l1/l2", null));
   }
 
+  private int getZNodeDataVersion(String znode) throws KeeperException {
+    Stat stat = new Stat();
+    ZKUtil.getDataNoWatch(ZKW, znode, stat);
+    return stat.getVersion();
+  }
+
+  @Test
+  public void testSetDataWithVersion() throws Exception {
+    ZKUtil.createWithParents(ZKW, "/s1/s2/s3");
+    int v0 = getZNodeDataVersion("/s1/s2/s3");
+    assertEquals(0, v0);
+
+    ZKUtil.setData(ZKW, "/s1/s2/s3", Bytes.toBytes(12L));
+    int v1 = getZNodeDataVersion("/s1/s2/s3");
+    assertEquals(1, v1);
+
+    ZKUtil.multiOrSequential(ZKW,
+      ImmutableList.of(ZKUtilOp.setData("/s1/s2/s3", Bytes.toBytes(13L), v1)), false);
+    int v2 = getZNodeDataVersion("/s1/s2/s3");
+    assertEquals(2, v2);
+  }
+
   /**
    * A test for HBASE-3238
    * @throws IOException A connection attempt to zk failed


[17/22] hbase git commit: HBASE-20242 The open sequence number will grow if we fail to open a region after writing the max sequence id file

Posted by zh...@apache.org.
HBASE-20242 The open sequence number will grow if we fail to open a region after writing the max sequence id file


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

Branch: refs/heads/HBASE-20046-branch-2
Commit: aec43bb207a34a1c8830bc0bd649ed060200cac6
Parents: 2b9fed8
Author: zhangduo <zh...@apache.org>
Authored: Wed Mar 21 21:35:34 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Apr 9 15:18:44 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      |   5 +-
 .../TestOpenSeqNumUnexpectedIncrease.java       | 111 +++++++++++++++++++
 2 files changed, 114 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/aec43bb2/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 23284be..9b9136b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -968,7 +968,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       WALSplitter.getMaxRegionSequenceId(fs.getFileSystem(), fs.getRegionDir());
     long nextSeqId = Math.max(maxSeqId, maxSeqIdFromFile) + 1;
     if (writestate.writesEnabled) {
-      WALSplitter.writeRegionSequenceIdFile(fs.getFileSystem(), fs.getRegionDir(), nextSeqId);
+      WALSplitter.writeRegionSequenceIdFile(fs.getFileSystem(), fs.getRegionDir(), nextSeqId - 1);
     }
 
     LOG.info("Opened {}; next sequenceid={}", this.getRegionInfo().getShortNameToLog(), nextSeqId);
@@ -1097,7 +1097,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     return allStoreFiles;
   }
 
-  private void writeRegionOpenMarker(WAL wal, long openSeqId) throws IOException {
+  @VisibleForTesting
+  protected void writeRegionOpenMarker(WAL wal, long openSeqId) throws IOException {
     Map<byte[], List<Path>> storeFiles = getStoreFiles();
     RegionEventDescriptor regionOpenDesc = ProtobufUtil.toRegionEventDescriptor(
       RegionEventDescriptor.EventType.REGION_OPEN, getRegionInfo(), openSeqId,

http://git-wip-us.apache.org/repos/asf/hbase/blob/aec43bb2/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestOpenSeqNumUnexpectedIncrease.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestOpenSeqNumUnexpectedIncrease.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestOpenSeqNumUnexpectedIncrease.java
new file mode 100644
index 0000000..14d5a98
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestOpenSeqNumUnexpectedIncrease.java
@@ -0,0 +1,111 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+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.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+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.WAL;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Testcase for HBASE-20242
+ */
+@Category({ RegionServerTests.class, MediumTests.class })
+public class TestOpenSeqNumUnexpectedIncrease {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestOpenSeqNumUnexpectedIncrease.class);
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static AtomicInteger FAILED_OPEN = new AtomicInteger(0);
+
+  private static TableName TABLE_NAME = TableName.valueOf("test");
+
+  private static byte[] CF = Bytes.toBytes("CF");
+
+  public static final class MockHRegion extends HRegion {
+
+    @SuppressWarnings("deprecation")
+    public MockHRegion(Path tableDir, WAL wal, FileSystem fs, Configuration confParam,
+        RegionInfo regionInfo, TableDescriptor htd, RegionServerServices rsServices) {
+      super(tableDir, wal, fs, confParam, regionInfo, htd, rsServices);
+    }
+
+    @Override
+    protected void writeRegionOpenMarker(WAL wal, long openSeqId) throws IOException {
+      if (getRegionInfo().getTable().equals(TABLE_NAME) && FAILED_OPEN.get() > 0) {
+        FAILED_OPEN.decrementAndGet();
+        rsServices.abort("for testing", new Exception("Inject error for testing"));
+        throw new IOException("Inject error for testing");
+      }
+    }
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.getConfiguration().setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 600000);
+    UTIL.getConfiguration().setClass(HConstants.REGION_IMPL, MockHRegion.class, HRegion.class);
+    UTIL.startMiniCluster(3);
+    UTIL.createTable(TABLE_NAME, CF);
+    UTIL.getAdmin().balancerSwitch(false, true);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void test() throws IOException, InterruptedException {
+    HRegion region = UTIL.getMiniHBaseCluster().getRegions(TABLE_NAME).get(0);
+    long openSeqNum = region.getOpenSeqNum();
+    HRegionServer src = UTIL.getRSForFirstRegionInTable(TABLE_NAME);
+    HRegionServer dst = UTIL.getOtherRegionServer(src);
+
+    // will fail two times, and then verify that the open sequence number is still openSeqNum + 2
+    FAILED_OPEN.set(2);
+    UTIL.getAdmin().move(region.getRegionInfo().getEncodedNameAsBytes(),
+      Bytes.toBytes(dst.getServerName().getServerName()));
+    UTIL.waitTableAvailable(TABLE_NAME);
+
+    HRegion region1 = UTIL.getMiniHBaseCluster().getRegions(TABLE_NAME).get(0);
+    long openSeqNum1 = region1.getOpenSeqNum();
+
+    assertEquals(openSeqNum + 2, openSeqNum1);
+  }
+}


[12/22] hbase git commit: HBASE-20125 Add UT for serial replication after region split and merge

Posted by zh...@apache.org.
HBASE-20125 Add UT for serial replication after region split and merge


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

Branch: refs/heads/HBASE-20046-branch-2
Commit: 6b5412759fdc2accc15f2fb4dd3ed0138f34e555
Parents: 8b61a06
Author: zhangduo <zh...@apache.org>
Authored: Tue Mar 6 21:31:05 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Apr 9 15:18:44 2018 +0800

----------------------------------------------------------------------
 .../hbase/replication/WALEntryFilter.java       |  17 +-
 .../regionserver/ReplicationSourceShipper.java  |   4 +-
 .../ReplicationSourceWALReader.java             |  54 +++--
 .../regionserver/WALEntryStream.java            |  73 ++++---
 .../replication/TestSerialReplication.java      | 200 ++++++++++++++++---
 5 files changed, 270 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6b541275/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/WALEntryFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/WALEntryFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/WALEntryFilter.java
index 417f868..cd3f1bd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/WALEntryFilter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/WALEntryFilter.java
@@ -15,7 +15,6 @@
  * 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;
@@ -35,12 +34,20 @@ import org.apache.hadoop.hbase.wal.WAL.Entry;
  */
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.REPLICATION)
 public interface WALEntryFilter {
+
   /**
-   * Applies the filter, possibly returning a different Entry instance.
-   * If null is returned, the entry will be skipped.
+   * <p>
+   * Applies the filter, possibly returning a different Entry instance. If null is returned, the
+   * entry will be skipped.
+   * </p>
+   * <p>
+   * Notice that you are free to modify the cell list of the give entry, but do not change the
+   * content of the cell, it may be used by others at the same time(and usually you can not modify a
+   * cell unless you cast it to the implementation class, which is not a good idea).
+   * </p>
    * @param entry Entry to filter
-   * @return a (possibly modified) Entry to use. Returning null or an entry with
-   * no cells will cause the entry to be skipped for replication.
+   * @return a (possibly modified) Entry to use. Returning null or an entry with no cells will cause
+   *         the entry to be skipped for replication.
    */
   public Entry filter(Entry entry);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b541275/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
index d207d77..50aaf95 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
@@ -120,7 +120,7 @@ public class ReplicationSourceShipper extends Thread {
   /**
    * Do the shipping logic
    */
-  protected void shipEdits(WALEntryBatch entryBatch) {
+  protected final void shipEdits(WALEntryBatch entryBatch) {
     List<Entry> entries = entryBatch.getWalEntries();
     long lastReadPosition = entryBatch.getLastWalPosition();
     currentPath = entryBatch.getLastWalPath();
@@ -253,7 +253,7 @@ public class ReplicationSourceShipper extends Thread {
     return 0;
   }
 
-  protected boolean isActive() {
+  protected final boolean isActive() {
     return source.isSourceActive() && state == WorkerState.RUNNING && !isInterrupted();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b541275/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
index fe87aec..ad3baaf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
@@ -71,6 +71,13 @@ public class ReplicationSourceWALReader extends Thread {
   private final int maxRetriesMultiplier;
   private final boolean eofAutoRecovery;
 
+  // used to store the first cell in an entry before filtering. This is because that if serial
+  // replication is enabled, we may find out that an entry can not be pushed after filtering. And
+  // when we try the next time, the cells maybe null since the entry has already been filtered,
+  // especially for region event wal entries. And this can also used to determine whether we can
+  // skip filtering.
+  private Cell firstCellInEntryBeforeFiltering;
+
   //Indicates whether this particular worker is running
   private boolean isReaderRunning = true;
 
@@ -162,37 +169,52 @@ public class ReplicationSourceWALReader extends Thread {
     }
   }
 
+  private void removeEntryFromStream(WALEntryStream entryStream, WALEntryBatch batch)
+      throws IOException {
+    entryStream.next();
+    firstCellInEntryBeforeFiltering = null;
+    batch.setLastWalPosition(entryStream.getPosition());
+  }
+
   private WALEntryBatch readWALEntries(WALEntryStream entryStream)
       throws IOException, InterruptedException {
     if (!entryStream.hasNext()) {
       return null;
     }
+    long positionBefore = entryStream.getPosition();
     WALEntryBatch batch =
       new WALEntryBatch(replicationBatchCountCapacity, entryStream.getCurrentPath());
     do {
       Entry entry = entryStream.peek();
-      batch.setLastWalPosition(entryStream.getPosition());
       boolean hasSerialReplicationScope = entry.getKey().hasSerialReplicationScope();
-      // Used to locate the region record in meta table. In WAL we only have the table name and
-      // encoded region name which can not be mapping to region name without scanning all the
-      // records for a table, so we need a start key, just like what we have done at client side
-      // when locating a region. For the markers, we will use the start key of the region as the row
-      // key for the edit. And we need to do this before filtering since all the cells may be
-      // filtered out, especially that for the markers.
-      Cell firstCellInEdit = null;
+      boolean doFiltering = true;
       if (hasSerialReplicationScope) {
-        assert !entry.getEdit().isEmpty() : "should not write empty edits";
-        firstCellInEdit = entry.getEdit().getCells().get(0);
+        if (firstCellInEntryBeforeFiltering == null) {
+          assert !entry.getEdit().isEmpty() : "should not write empty edits";
+          // Used to locate the region record in meta table. In WAL we only have the table name and
+          // encoded region name which can not be mapping to region name without scanning all the
+          // records for a table, so we need a start key, just like what we have done at client side
+          // when locating a region. For the markers, we will use the start key of the region as the
+          // row key for the edit. And we need to do this before filtering since all the cells may
+          // be filtered out, especially that for the markers.
+          firstCellInEntryBeforeFiltering = entry.getEdit().getCells().get(0);
+        } else {
+          // if this is not null then we know that the entry has already been filtered.
+          doFiltering = false;
+        }
+      }
+
+      if (doFiltering) {
+        entry = filterEntry(entry);
       }
-      entry = filterEntry(entry);
       if (entry != null) {
         if (hasSerialReplicationScope) {
-          if (!serialReplicationChecker.canPush(entry, firstCellInEdit)) {
-            if (batch.getNbEntries() > 0) {
+          if (!serialReplicationChecker.canPush(entry, firstCellInEntryBeforeFiltering)) {
+            if (batch.getLastWalPosition() > positionBefore) {
               // we have something that can push, break
               break;
             } else {
-              serialReplicationChecker.waitUntilCanPush(entry, firstCellInEdit);
+              serialReplicationChecker.waitUntilCanPush(entry, firstCellInEntryBeforeFiltering);
             }
           }
           // arrive here means we can push the entry, record the last sequence id
@@ -200,7 +222,7 @@ public class ReplicationSourceWALReader extends Thread {
             entry.getKey().getSequenceId());
         }
         // actually remove the entry.
-        entryStream.next();
+        removeEntryFromStream(entryStream, batch);
         WALEdit edit = entry.getEdit();
         if (edit != null && !edit.isEmpty()) {
           long entrySize = getEntrySize(entry);
@@ -215,7 +237,7 @@ public class ReplicationSourceWALReader extends Thread {
         }
       } else {
         // actually remove the entry.
-        entryStream.next();
+        removeEntryFromStream(entryStream, batch);
       }
     } while (entryStream.hasNext());
     return batch;

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b541275/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
index bcab9b4..c639a48 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
@@ -56,8 +56,12 @@ class WALEntryStream implements Closeable {
   private Path currentPath;
   // cache of next entry for hasNext()
   private Entry currentEntry;
+  // position for the current entry. As now we support peek, which means that the upper layer may
+  // choose to return before reading the current entry, so it is not safe to return the value below
+  // in getPosition.
+  private long currentPositionOfEntry = 0;
   // position after reading current entry
-  private long currentPosition = 0;
+  private long currentPositionOfReader = 0;
   private final PriorityBlockingQueue<Path> logQueue;
   private final FileSystem fs;
   private final Configuration conf;
@@ -82,7 +86,7 @@ class WALEntryStream implements Closeable {
     this.logQueue = logQueue;
     this.fs = fs;
     this.conf = conf;
-    this.currentPosition = startPosition;
+    this.currentPositionOfEntry = startPosition;
     this.walFileLengthProvider = walFileLengthProvider;
     this.serverName = serverName;
     this.metrics = metrics;
@@ -110,6 +114,7 @@ class WALEntryStream implements Closeable {
    */
   public Entry next() throws IOException {
     Entry save = peek();
+    currentPositionOfEntry = currentPositionOfReader;
     currentEntry = null;
     return save;
   }
@@ -126,7 +131,7 @@ class WALEntryStream implements Closeable {
    * @return the position of the last Entry returned by next()
    */
   public long getPosition() {
-    return currentPosition;
+    return currentPositionOfEntry;
   }
 
   /**
@@ -140,7 +145,7 @@ class WALEntryStream implements Closeable {
     StringBuilder sb = new StringBuilder();
     if (currentPath != null) {
       sb.append("currently replicating from: ").append(currentPath).append(" at position: ")
-          .append(currentPosition).append("\n");
+          .append(currentPositionOfEntry).append("\n");
     } else {
       sb.append("no replication ongoing, waiting for new log");
     }
@@ -159,7 +164,7 @@ class WALEntryStream implements Closeable {
   }
 
   private void setPosition(long position) {
-    currentPosition = position;
+    currentPositionOfEntry = position;
   }
 
   private void setCurrentPath(Path path) {
@@ -168,19 +173,19 @@ class WALEntryStream implements Closeable {
 
   private void tryAdvanceEntry() throws IOException {
     if (checkReader()) {
-      boolean beingWritten = readNextEntryAndSetPosition();
+      boolean beingWritten = readNextEntryAndRecordReaderPosition();
       if (currentEntry == null && !beingWritten) {
         // no more entries in this log file, and the file is already closed, i.e, rolled
         // Before dequeueing, we should always get one more attempt at reading.
         // This is in case more entries came in after we opened the reader, and the log is rolled
         // while we were reading. See HBASE-6758
         resetReader();
-        readNextEntryAndSetPosition();
+        readNextEntryAndRecordReaderPosition();
         if (currentEntry == null) {
           if (checkAllBytesParsed()) { // now we're certain we're done with this log file
             dequeueCurrentLog();
             if (openNextLog()) {
-              readNextEntryAndSetPosition();
+              readNextEntryAndRecordReaderPosition();
             }
           }
         }
@@ -201,45 +206,49 @@ class WALEntryStream implements Closeable {
     try {
       stat = fs.getFileStatus(this.currentPath);
     } catch (IOException exception) {
-      LOG.warn("Couldn't get file length information about log " + this.currentPath + ", it "
-          + (trailerSize < 0 ? "was not" : "was") + " closed cleanly " + getCurrentPathStat());
+      LOG.warn("Couldn't get file length information about log {}, it {} closed cleanly {}",
+        currentPath, trailerSize < 0 ? "was not" : "was", getCurrentPathStat());
       metrics.incrUnknownFileLengthForClosedWAL();
     }
+    // Here we use currentPositionOfReader instead of currentPositionOfEntry.
+    // We only call this method when currentEntry is null so usually they are the same, but there
+    // are two exceptions. One is we have nothing in the file but only a header, in this way
+    // the currentPositionOfEntry will always be 0 since we have no change to update it. The other
+    // is that we reach the end of file, then currentPositionOfEntry will point to the tail of the
+    // last valid entry, and the currentPositionOfReader will usually point to the end of the file.
     if (stat != null) {
       if (trailerSize < 0) {
-        if (currentPosition < stat.getLen()) {
-          final long skippedBytes = stat.getLen() - currentPosition;
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Reached the end of WAL file '" + currentPath
-                + "'. It was not closed cleanly, so we did not parse " + skippedBytes
-                + " bytes of data. This is normally ok.");
-          }
+        if (currentPositionOfReader < stat.getLen()) {
+          final long skippedBytes = stat.getLen() - currentPositionOfReader;
+          LOG.debug(
+            "Reached the end of WAL file '{}'. It was not closed cleanly," +
+              " so we did not parse {} bytes of data. This is normally ok.",
+            currentPath, skippedBytes);
           metrics.incrUncleanlyClosedWALs();
           metrics.incrBytesSkippedInUncleanlyClosedWALs(skippedBytes);
         }
-      } else if (currentPosition + trailerSize < stat.getLen()) {
-        LOG.warn("Processing end of WAL file '" + currentPath + "'. At position " + currentPosition
-            + ", which is too far away from reported file length " + stat.getLen()
-            + ". Restarting WAL reading (see HBASE-15983 for details). " + getCurrentPathStat());
+      } else if (currentPositionOfReader + trailerSize < stat.getLen()) {
+        LOG.warn(
+          "Processing end of WAL file '{}'. At position {}, which is too far away from" +
+            " reported file length {}. Restarting WAL reading (see HBASE-15983 for details). {}",
+          currentPath, currentPositionOfReader, stat.getLen(), getCurrentPathStat());
         setPosition(0);
         resetReader();
         metrics.incrRestartedWALReading();
-        metrics.incrRepeatedFileBytes(currentPosition);
+        metrics.incrRepeatedFileBytes(currentPositionOfReader);
         return false;
       }
     }
     if (LOG.isTraceEnabled()) {
-      LOG.trace("Reached the end of log " + this.currentPath + ", and the length of the file is "
-          + (stat == null ? "N/A" : stat.getLen()));
+      LOG.trace("Reached the end of log " + this.currentPath + ", and the length of the file is " +
+        (stat == null ? "N/A" : stat.getLen()));
     }
     metrics.incrCompletedWAL();
     return true;
   }
 
   private void dequeueCurrentLog() throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Reached the end of log " + currentPath);
-    }
+    LOG.debug("Reached the end of log {}", currentPath);
     closeReader();
     logQueue.remove();
     setPosition(0);
@@ -249,7 +258,7 @@ class WALEntryStream implements Closeable {
   /**
    * Returns whether the file is opened for writing.
    */
-  private boolean readNextEntryAndSetPosition() throws IOException {
+  private boolean readNextEntryAndRecordReaderPosition() throws IOException {
     Entry readEntry = reader.next();
     long readerPos = reader.getPosition();
     OptionalLong fileLength = walFileLengthProvider.getLogFileSizeIfBeingWritten(currentPath);
@@ -265,10 +274,10 @@ class WALEntryStream implements Closeable {
     }
     if (readEntry != null) {
       metrics.incrLogEditsRead();
-      metrics.incrLogReadInBytes(readerPos - currentPosition);
+      metrics.incrLogReadInBytes(readerPos - currentPositionOfEntry);
     }
     currentEntry = readEntry; // could be null
-    setPosition(readerPos);
+    this.currentPositionOfReader = readerPos;
     return fileLength.isPresent();
   }
 
@@ -401,8 +410,8 @@ class WALEntryStream implements Closeable {
   }
 
   private void seek() throws IOException {
-    if (currentPosition != 0) {
-      reader.seek(currentPosition);
+    if (currentPositionOfEntry != 0) {
+      reader.seek(currentPositionOfEntry);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6b541275/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
index 1408cf0..9d8e7fe 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
@@ -18,11 +18,18 @@
 package org.apache.hadoop.hbase.replication;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.io.UncheckedIOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
@@ -40,6 +47,7 @@ import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.hbase.wal.WALFactory;
@@ -124,6 +132,8 @@ public class TestSerialReplication {
   public static void setUpBeforeClass() throws Exception {
     UTIL.getConfiguration().setInt("replication.source.nb.capacity", 10);
     UTIL.startMiniCluster(3);
+    // disable balancer
+    UTIL.getAdmin().balancerSwitch(false, true);
     LOG_DIR = UTIL.getDataTestDirOnTestFS("replicated");
     FS = UTIL.getTestFileSystem();
     FS.mkdirs(LOG_DIR);
@@ -141,7 +151,6 @@ public class TestSerialReplication {
 
   @Before
   public void setUp() throws IOException, StreamLacksCapabilityException {
-    UTIL.ensureSomeRegionServersAvailable(3);
     logPath = new Path(LOG_DIR, name.getMethodName());
     WRITER = WALFactory.createWALWriter(FS, logPath, UTIL.getConfiguration());
     // add in disable state, so later when enabling it all sources will start push together.
@@ -152,35 +161,41 @@ public class TestSerialReplication {
   }
 
   @After
-  public void tearDown() throws IOException {
+  public void tearDown() throws Exception {
     UTIL.getAdmin().removeReplicationPeer(PEER_ID);
+    for (RegionServerThread t : UTIL.getMiniHBaseCluster().getLiveRegionServerThreads()) {
+      t.getRegionServer().getWalRoller().requestRollAll();
+    }
+    UTIL.waitFor(30000, new ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        return UTIL.getMiniHBaseCluster().getLiveRegionServerThreads().stream()
+          .map(t -> t.getRegionServer()).allMatch(HRegionServer::walRollRequestFinished);
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        return "Log roll has not finished yet";
+      }
+    });
+    for (RegionServerThread t : UTIL.getMiniHBaseCluster().getLiveRegionServerThreads()) {
+      t.getRegionServer().getWalRoller().requestRollAll();
+    }
     if (WRITER != null) {
       WRITER.close();
       WRITER = null;
     }
   }
 
-  @Test
-  public void testRegionMove() throws Exception {
-    TableName tableName = TableName.valueOf(name.getMethodName());
-    UTIL.getAdmin().createTable(
-      TableDescriptorBuilder.newBuilder(tableName).addColumnFamily(ColumnFamilyDescriptorBuilder
-        .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_SERIAL).build()).build());
-    UTIL.waitTableAvailable(tableName);
-    try (Table table = UTIL.getConnection().getTable(tableName)) {
-      for (int i = 0; i < 100; i++) {
-        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
-      }
-    }
-    RegionInfo region = UTIL.getAdmin().getRegions(tableName).get(0);
-    HRegionServer rs = UTIL.getOtherRegionServer(UTIL.getRSForFirstRegionInTable(tableName));
+  private void moveRegion(RegionInfo region, HRegionServer rs) throws Exception {
     UTIL.getAdmin().move(region.getEncodedNameAsBytes(),
       Bytes.toBytes(rs.getServerName().getServerName()));
     UTIL.waitFor(30000, new ExplainingPredicate<Exception>() {
 
       @Override
       public boolean evaluate() throws Exception {
-        return !rs.getRegions(tableName).isEmpty();
+        return rs.getRegion(region.getEncodedName()) != null;
       }
 
       @Override
@@ -188,11 +203,9 @@ public class TestSerialReplication {
         return region + " is still not on " + rs;
       }
     });
-    try (Table table = UTIL.getConnection().getTable(tableName)) {
-      for (int i = 100; i < 200; i++) {
-        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
-      }
-    }
+  }
+
+  private void enablePeerAndWaitUntilReplicationDone(int expectedEntries) throws Exception {
     UTIL.getAdmin().enableReplicationPeer(PEER_ID);
     UTIL.waitFor(30000, new ExplainingPredicate<Exception>() {
 
@@ -203,7 +216,7 @@ public class TestSerialReplication {
           while (reader.next() != null) {
             count++;
           }
-          return count >= 200;
+          return count >= expectedEntries;
         } catch (IOException e) {
           return false;
         }
@@ -214,6 +227,29 @@ public class TestSerialReplication {
         return "Not enough entries replicated";
       }
     });
+  }
+
+  @Test
+  public void testRegionMove() throws Exception {
+    TableName tableName = TableName.valueOf(name.getMethodName());
+    UTIL.getAdmin().createTable(
+      TableDescriptorBuilder.newBuilder(tableName).addColumnFamily(ColumnFamilyDescriptorBuilder
+        .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_SERIAL).build()).build());
+    UTIL.waitTableAvailable(tableName);
+    try (Table table = UTIL.getConnection().getTable(tableName)) {
+      for (int i = 0; i < 100; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+    RegionInfo region = UTIL.getAdmin().getRegions(tableName).get(0);
+    HRegionServer rs = UTIL.getOtherRegionServer(UTIL.getRSForFirstRegionInTable(tableName));
+    moveRegion(region, rs);
+    try (Table table = UTIL.getConnection().getTable(tableName)) {
+      for (int i = 100; i < 200; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+    enablePeerAndWaitUntilReplicationDone(200);
     try (WAL.Reader reader =
       WALFactory.createReader(UTIL.getTestFileSystem(), logPath, UTIL.getConfiguration())) {
       long seqId = -1L;
@@ -231,4 +267,122 @@ public class TestSerialReplication {
       assertEquals(200, count);
     }
   }
+
+  @Test
+  public void testRegionSplit() throws Exception {
+    TableName tableName = TableName.valueOf(name.getMethodName());
+    UTIL.getAdmin().createTable(
+      TableDescriptorBuilder.newBuilder(tableName).addColumnFamily(ColumnFamilyDescriptorBuilder
+        .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_SERIAL).build()).build());
+    UTIL.waitTableAvailable(tableName);
+    try (Table table = UTIL.getConnection().getTable(tableName)) {
+      for (int i = 0; i < 100; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+    UTIL.flush(tableName);
+    RegionInfo region = UTIL.getAdmin().getRegions(tableName).get(0);
+    UTIL.getAdmin().splitRegionAsync(region.getEncodedNameAsBytes(), Bytes.toBytes(50)).get(30,
+      TimeUnit.SECONDS);
+    UTIL.waitUntilNoRegionsInTransition(30000);
+    List<RegionInfo> regions = UTIL.getAdmin().getRegions(tableName);
+    assertEquals(2, regions.size());
+    try (Table table = UTIL.getConnection().getTable(tableName)) {
+      for (int i = 0; i < 100; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+    enablePeerAndWaitUntilReplicationDone(200);
+    Map<String, Long> regionsToSeqId = new HashMap<>();
+    regionsToSeqId.put(region.getEncodedName(), -1L);
+    regions.stream().map(RegionInfo::getEncodedName).forEach(n -> regionsToSeqId.put(n, -1L));
+    try (WAL.Reader reader =
+      WALFactory.createReader(UTIL.getTestFileSystem(), logPath, UTIL.getConfiguration())) {
+      int count = 0;
+      for (Entry entry;;) {
+        entry = reader.next();
+        if (entry == null) {
+          break;
+        }
+        String encodedName = Bytes.toString(entry.getKey().getEncodedRegionName());
+        Long seqId = regionsToSeqId.get(encodedName);
+        assertNotNull(
+          "Unexcepted entry " + entry + ", expected regions " + region + ", or " + regions, seqId);
+        assertTrue("Sequence id go backwards from " + seqId + " to " +
+          entry.getKey().getSequenceId() + " for " + encodedName,
+          entry.getKey().getSequenceId() >= seqId.longValue());
+        if (count < 100) {
+          assertEquals(encodedName + " is pushed before parent " + region.getEncodedName(),
+            region.getEncodedName(), encodedName);
+        } else {
+          assertNotEquals(region.getEncodedName(), encodedName);
+        }
+        count++;
+      }
+      assertEquals(200, count);
+    }
+  }
+
+  @Test
+  public void testRegionMerge() throws Exception {
+    byte[] splitKey = Bytes.toBytes(50);
+    TableName tableName = TableName.valueOf(name.getMethodName());
+    UTIL.getAdmin().createTable(
+      TableDescriptorBuilder.newBuilder(tableName)
+        .addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(CF)
+          .setScope(HConstants.REPLICATION_SCOPE_SERIAL).build())
+        .build(),
+      new byte[][] { splitKey });
+    UTIL.waitTableAvailable(tableName);
+    try (Table table = UTIL.getConnection().getTable(tableName)) {
+      for (int i = 0; i < 100; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+    List<RegionInfo> regions = UTIL.getAdmin().getRegions(tableName);
+    UTIL.getAdmin()
+      .mergeRegionsAsync(
+        regions.stream().map(RegionInfo::getEncodedNameAsBytes).toArray(byte[][]::new), false)
+      .get(30, TimeUnit.SECONDS);
+    UTIL.waitUntilNoRegionsInTransition(30000);
+    List<RegionInfo> regionsAfterMerge = UTIL.getAdmin().getRegions(tableName);
+    assertEquals(1, regionsAfterMerge.size());
+    try (Table table = UTIL.getConnection().getTable(tableName)) {
+      for (int i = 0; i < 100; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+    enablePeerAndWaitUntilReplicationDone(200);
+    Map<String, Long> regionsToSeqId = new HashMap<>();
+    RegionInfo region = regionsAfterMerge.get(0);
+    regionsToSeqId.put(region.getEncodedName(), -1L);
+    regions.stream().map(RegionInfo::getEncodedName).forEach(n -> regionsToSeqId.put(n, -1L));
+    try (WAL.Reader reader =
+      WALFactory.createReader(UTIL.getTestFileSystem(), logPath, UTIL.getConfiguration())) {
+      int count = 0;
+      for (Entry entry;;) {
+        entry = reader.next();
+        if (entry == null) {
+          break;
+        }
+        String encodedName = Bytes.toString(entry.getKey().getEncodedRegionName());
+        Long seqId = regionsToSeqId.get(encodedName);
+        assertNotNull(
+          "Unexcepted entry " + entry + ", expected regions " + region + ", or " + regions, seqId);
+        assertTrue("Sequence id go backwards from " + seqId + " to " +
+          entry.getKey().getSequenceId() + " for " + encodedName,
+          entry.getKey().getSequenceId() >= seqId.longValue());
+        if (count < 100) {
+          assertNotEquals(
+            encodedName + " is pushed before parents " +
+              regions.stream().map(RegionInfo::getEncodedName).collect(Collectors.joining(" and ")),
+            region.getEncodedName(), encodedName);
+        } else {
+          assertEquals(region.getEncodedName(), encodedName);
+        }
+        count++;
+      }
+      assertEquals(200, count);
+    }
+  }
 }


[16/22] hbase git commit: HBASE-20285 Delete all last pushed sequence ids when removing a peer or removing the serial flag for a peer

Posted by zh...@apache.org.
HBASE-20285 Delete all last pushed sequence ids when removing a peer or removing the serial flag for a peer


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

Branch: refs/heads/HBASE-20046-branch-2
Commit: ead569c9515368c2c7e1932561fd05c77ccd9482
Parents: 83488b8
Author: zhangduo <zh...@apache.org>
Authored: Mon Mar 26 22:17:00 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Apr 9 15:18:44 2018 +0800

----------------------------------------------------------------------
 .../src/main/protobuf/MasterProcedure.proto     | 10 +++
 .../replication/ReplicationQueueStorage.java    |  5 ++
 .../replication/ZKReplicationQueueStorage.java  | 37 ++++++++++-
 .../TestZKReplicationQueueStorage.java          | 31 ++++++++-
 .../replication/DisablePeerProcedure.java       | 15 +++++
 .../master/replication/EnablePeerProcedure.java | 15 +++++
 .../master/replication/RemovePeerProcedure.java | 31 ++++++++-
 .../replication/ReplicationPeerManager.java     |  8 ++-
 .../replication/UpdatePeerConfigProcedure.java  |  3 +
 .../replication/SerialReplicationTestBase.java  | 19 +++++-
 .../TestAddToSerialReplicationPeer.java         | 28 ++------
 .../replication/TestSerialReplication.java      | 68 ++++++++++++++++----
 12 files changed, 227 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ead569c9/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 f710759..b37557c 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -421,3 +421,13 @@ message UpdatePeerConfigStateData {
   required ReplicationPeer peer_config = 1;
   optional ReplicationPeer old_peer_config = 2;
 }
+
+message RemovePeerStateData {
+  optional ReplicationPeer peer_config = 1;
+}
+
+message EnablePeerStateData {
+}
+
+message DisablePeerStateData {
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ead569c9/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
index 99a1e97..cd37ac2 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
@@ -87,6 +87,11 @@ public interface ReplicationQueueStorage {
   void setLastSequenceIds(String peerId, Map<String, Long> lastSeqIds) throws ReplicationException;
 
   /**
+   * Remove all the max sequence id record for the given peer.
+   * @param peerId peer id
+   */
+  void removeLastSequenceIds(String peerId) throws ReplicationException;
+  /**
    * Get the current position for a specific WAL in a given queue for a given regionserver.
    * @param serverName the name of the regionserver
    * @param queueId a String that identifies the queue

http://git-wip-us.apache.org/repos/asf/hbase/blob/ead569c9/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
index 2e7a012..96b0b91 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
@@ -102,7 +102,8 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
    */
   private final String hfileRefsZNode;
 
-  private final String regionsZNode;
+  @VisibleForTesting
+  final String regionsZNode;
 
   public ZKReplicationQueueStorage(ZKWatcher zookeeper, Configuration conf) {
     super(zookeeper, conf);
@@ -312,6 +313,40 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
   }
 
   @Override
+  public void removeLastSequenceIds(String peerId) throws ReplicationException {
+    String suffix = "-" + peerId;
+    try {
+      StringBuilder sb = new StringBuilder(regionsZNode);
+      int regionsZNodeLength = regionsZNode.length();
+      int levelOneLength = regionsZNodeLength + 3;
+      int levelTwoLength = levelOneLength + 3;
+      List<String> levelOneDirs = ZKUtil.listChildrenNoWatch(zookeeper, regionsZNode);
+      // it is possible that levelOneDirs is null if we haven't write any last pushed sequence ids
+      // yet, so we need an extra check here.
+      if (CollectionUtils.isEmpty(levelOneDirs)) {
+        return;
+      }
+      for (String levelOne : levelOneDirs) {
+        sb.append(ZNodePaths.ZNODE_PATH_SEPARATOR).append(levelOne);
+        for (String levelTwo : ZKUtil.listChildrenNoWatch(zookeeper, sb.toString())) {
+          sb.append(ZNodePaths.ZNODE_PATH_SEPARATOR).append(levelTwo);
+          for (String znode : ZKUtil.listChildrenNoWatch(zookeeper, sb.toString())) {
+            if (znode.endsWith(suffix)) {
+              sb.append(ZNodePaths.ZNODE_PATH_SEPARATOR).append(znode);
+              ZKUtil.deleteNode(zookeeper, sb.toString());
+              sb.setLength(levelTwoLength);
+            }
+          }
+          sb.setLength(levelOneLength);
+        }
+        sb.setLength(regionsZNodeLength);
+      }
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to remove all last sequence ids, peerId=" + peerId, e);
+    }
+  }
+
+  @Override
   public long getWALPosition(ServerName serverName, String queueId, String fileName)
       throws ReplicationException {
     byte[] bytes;

http://git-wip-us.apache.org/repos/asf/hbase/blob/ead569c9/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
index 5821271..74a24ac 100644
--- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
@@ -32,15 +32,17 @@ import java.util.SortedSet;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseZKTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.MD5Hash;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -71,7 +73,7 @@ public class TestZKReplicationQueueStorage {
   }
 
   @After
-  public void tearDownAfterTest() throws ReplicationException {
+  public void tearDownAfterTest() throws ReplicationException, KeeperException, IOException {
     for (ServerName serverName : STORAGE.getListOfReplicators()) {
       for (String queue : STORAGE.getAllQueues(serverName)) {
         STORAGE.removeQueue(serverName, queue);
@@ -301,6 +303,29 @@ public class TestZKReplicationQueueStorage {
     String encodedRegionName = "31d9792f4435b99d9fb1016f6fbc8dc7";
     String expectedPath = "/hbase/replication/regions/31/d9/792f4435b99d9fb1016f6fbc8dc7-" + peerId;
     String path = STORAGE.getSerialReplicationRegionPeerNode(encodedRegionName, peerId);
-    Assert.assertEquals(expectedPath, path);
+    assertEquals(expectedPath, path);
+  }
+
+  @Test
+  public void testRemoveAllLastPushedSeqIdsForPeer() throws Exception {
+    String peerId = "1";
+    String peerIdToDelete = "2";
+    for (int i = 0; i < 100; i++) {
+      String encodedRegionName = MD5Hash.getMD5AsHex(Bytes.toBytes(i));
+      STORAGE.setLastSequenceIds(peerId, ImmutableMap.of(encodedRegionName, (long) i));
+      STORAGE.setLastSequenceIds(peerIdToDelete, ImmutableMap.of(encodedRegionName, (long) i));
+    }
+    for (int i = 0; i < 100; i++) {
+      String encodedRegionName = MD5Hash.getMD5AsHex(Bytes.toBytes(i));
+      assertEquals(i, STORAGE.getLastSequenceId(encodedRegionName, peerId));
+      assertEquals(i, STORAGE.getLastSequenceId(encodedRegionName, peerIdToDelete));
+    }
+    STORAGE.removeLastSequenceIds(peerIdToDelete);
+    for (int i = 0; i < 100; i++) {
+      String encodedRegionName = MD5Hash.getMD5AsHex(Bytes.toBytes(i));
+      assertEquals(i, STORAGE.getLastSequenceId(encodedRegionName, peerId));
+      assertEquals(HConstants.NO_SEQNUM,
+        STORAGE.getLastSequenceId(encodedRegionName, peerIdToDelete));
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ead569c9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
index 0871575..7bda1d7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java
@@ -20,11 +20,14 @@ 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.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DisablePeerStateData;
+
 /**
  * The procedure for disabling a replication peer.
  */
@@ -67,4 +70,16 @@ public class DisablePeerProcedure extends ModifyPeerProcedure {
       cpHost.postDisableReplicationPeer(peerId);
     }
   }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.serializeStateData(serializer);
+    serializer.serialize(DisablePeerStateData.getDefaultInstance());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.deserializeStateData(serializer);
+    serializer.deserialize(DisablePeerStateData.class);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ead569c9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
index 890462f..530d4cb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
@@ -20,11 +20,14 @@ 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.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.EnablePeerStateData;
+
 /**
  * The procedure for enabling a replication peer.
  */
@@ -67,4 +70,16 @@ public class EnablePeerProcedure extends ModifyPeerProcedure {
       cpHost.postEnableReplicationPeer(peerId);
     }
   }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.serializeStateData(serializer);
+    serializer.serialize(EnablePeerStateData.getDefaultInstance());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.deserializeStateData(serializer);
+    serializer.deserialize(EnablePeerStateData.class);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ead569c9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
index 64faf2b..82dc07e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/RemovePeerProcedure.java
@@ -18,13 +18,18 @@
 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.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RemovePeerStateData;
+
 /**
  * The procedure for removing a replication peer.
  */
@@ -33,6 +38,8 @@ public class RemovePeerProcedure extends ModifyPeerProcedure {
 
   private static final Logger LOG = LoggerFactory.getLogger(RemovePeerProcedure.class);
 
+  private ReplicationPeerConfig peerConfig;
+
   public RemovePeerProcedure() {
   }
 
@@ -51,7 +58,7 @@ public class RemovePeerProcedure extends ModifyPeerProcedure {
     if (cpHost != null) {
       cpHost.preRemoveReplicationPeer(peerId);
     }
-    env.getReplicationPeerManager().preRemovePeer(peerId);
+    peerConfig = env.getReplicationPeerManager().preRemovePeer(peerId);
   }
 
   @Override
@@ -63,10 +70,32 @@ public class RemovePeerProcedure extends ModifyPeerProcedure {
   protected void postPeerModification(MasterProcedureEnv env)
       throws IOException, ReplicationException {
     env.getReplicationPeerManager().removeAllQueuesAndHFileRefs(peerId);
+    if (peerConfig.isSerial()) {
+      env.getReplicationPeerManager().removeAllLastPushedSeqIds(peerId);
+    }
     LOG.info("Successfully removed peer {}", peerId);
     MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
     if (cpHost != null) {
       cpHost.postRemoveReplicationPeer(peerId);
     }
   }
+
+  @Override
+  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.serializeStateData(serializer);
+    RemovePeerStateData.Builder builder = RemovePeerStateData.newBuilder();
+    if (peerConfig != null) {
+      builder.setPeerConfig(ReplicationPeerConfigUtil.convert(peerConfig));
+    }
+    serializer.serialize(builder.build());
+  }
+
+  @Override
+  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+    super.deserializeStateData(serializer);
+    RemovePeerStateData data = serializer.deserialize(RemovePeerStateData.class);
+    if (data.hasPeerConfig()) {
+      this.peerConfig = ReplicationPeerConfigUtil.convert(data.getPeerConfig());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ead569c9/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 a0e01e0..87d0111 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
@@ -109,8 +109,8 @@ public class ReplicationPeerManager {
     return desc;
   }
 
-  void preRemovePeer(String peerId) throws DoNotRetryIOException {
-    checkPeerExists(peerId);
+  ReplicationPeerConfig preRemovePeer(String peerId) throws DoNotRetryIOException {
+    return checkPeerExists(peerId).getPeerConfig();
   }
 
   void preEnablePeer(String peerId) throws DoNotRetryIOException {
@@ -220,6 +220,10 @@ public class ReplicationPeerManager {
     return desc != null ? Optional.of(desc.getPeerConfig()) : Optional.empty();
   }
 
+  void removeAllLastPushedSeqIds(String peerId) throws ReplicationException {
+    queueStorage.removeLastSequenceIds(peerId);
+  }
+
   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

http://git-wip-us.apache.org/repos/asf/hbase/blob/ead569c9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
index b7e670a..ccfd4a0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
@@ -107,6 +107,9 @@ public class UpdatePeerConfigProcedure extends ModifyPeerProcedure {
   @Override
   protected void postPeerModification(MasterProcedureEnv env)
       throws IOException, ReplicationException {
+    if (oldPeerConfig.isSerial() && !peerConfig.isSerial()) {
+      env.getReplicationPeerManager().removeAllLastPushedSeqIds(peerId);
+    }
     LOG.info("Successfully updated peer config of {} to {}", peerId, peerConfig);
     MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
     if (cpHost != null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/ead569c9/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SerialReplicationTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SerialReplicationTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SerialReplicationTestBase.java
index b5aae85..4b7fa87 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SerialReplicationTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SerialReplicationTestBase.java
@@ -26,8 +26,13 @@ import java.util.UUID;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
@@ -129,7 +134,10 @@ public class SerialReplicationTestBase {
 
   @After
   public void tearDown() throws Exception {
-    UTIL.getAdmin().removeReplicationPeer(PEER_ID);
+    Admin admin = UTIL.getAdmin();
+    for (ReplicationPeerDescription pd : admin.listReplicationPeers()) {
+      admin.removeReplicationPeer(pd.getPeerId());
+    }
     rollAllWALs();
     if (WRITER != null) {
       WRITER.close();
@@ -233,4 +241,13 @@ public class SerialReplicationTestBase {
       assertEquals(expectedEntries, count);
     }
   }
+
+  protected final TableName createTable() throws IOException, InterruptedException {
+    TableName tableName = TableName.valueOf(name.getMethodName());
+    UTIL.getAdmin().createTable(
+      TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(ColumnFamilyDescriptorBuilder
+        .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build());
+    UTIL.waitTableAvailable(tableName);
+    return tableName;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ead569c9/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestAddToSerialReplicationPeer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestAddToSerialReplicationPeer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestAddToSerialReplicationPeer.java
index 64b5bb1..317c120 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestAddToSerialReplicationPeer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestAddToSerialReplicationPeer.java
@@ -21,14 +21,11 @@ import java.io.IOException;
 import java.util.Collections;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
@@ -88,11 +85,7 @@ public class TestAddToSerialReplicationPeer extends SerialReplicationTestBase {
 
   @Test
   public void testAddPeer() throws Exception {
-    TableName tableName = TableName.valueOf(name.getMethodName());
-    UTIL.getAdmin().createTable(
-      TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(ColumnFamilyDescriptorBuilder
-        .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build());
-    UTIL.waitTableAvailable(tableName);
+    TableName tableName = createTable();
     try (Table table = UTIL.getConnection().getTable(tableName)) {
       for (int i = 0; i < 100; i++) {
         table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
@@ -118,12 +111,7 @@ public class TestAddToSerialReplicationPeer extends SerialReplicationTestBase {
         .setReplicationEndpointImpl(LocalReplicationEndpoint.class.getName()).build();
     UTIL.getAdmin().addReplicationPeer(PEER_ID, peerConfig, true);
 
-    TableName tableName = TableName.valueOf(name.getMethodName());
-
-    UTIL.getAdmin().createTable(
-      TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(ColumnFamilyDescriptorBuilder
-        .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build());
-    UTIL.waitTableAvailable(tableName);
+    TableName tableName = createTable();
     try (Table table = UTIL.getConnection().getTable(tableName)) {
       for (int i = 0; i < 100; i++) {
         table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
@@ -159,11 +147,7 @@ public class TestAddToSerialReplicationPeer extends SerialReplicationTestBase {
         .setReplicateAllUserTables(false).setSerial(true).build();
     UTIL.getAdmin().addReplicationPeer(PEER_ID, peerConfig, true);
 
-    TableName tableName = TableName.valueOf(name.getMethodName());
-    UTIL.getAdmin().createTable(
-      TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(ColumnFamilyDescriptorBuilder
-        .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build());
-    UTIL.waitTableAvailable(tableName);
+    TableName tableName = createTable();
     try (Table table = UTIL.getConnection().getTable(tableName)) {
       for (int i = 0; i < 100; i++) {
         table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
@@ -190,11 +174,7 @@ public class TestAddToSerialReplicationPeer extends SerialReplicationTestBase {
 
   @Test
   public void testDisabledTable() throws Exception {
-    TableName tableName = TableName.valueOf(name.getMethodName());
-    UTIL.getAdmin().createTable(
-      TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(ColumnFamilyDescriptorBuilder
-        .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build());
-    UTIL.waitTableAvailable(tableName);
+    TableName tableName = createTable();
     try (Table table = UTIL.getConnection().getTable(tableName)) {
       for (int i = 0; i < 100; i++) {
         table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));

http://git-wip-us.apache.org/repos/asf/hbase/blob/ead569c9/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
index bedb2ec..07e626b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
@@ -65,11 +65,7 @@ public class TestSerialReplication extends SerialReplicationTestBase {
 
   @Test
   public void testRegionMove() throws Exception {
-    TableName tableName = TableName.valueOf(name.getMethodName());
-    UTIL.getAdmin().createTable(
-      TableDescriptorBuilder.newBuilder(tableName).addColumnFamily(ColumnFamilyDescriptorBuilder
-        .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build());
-    UTIL.waitTableAvailable(tableName);
+    TableName tableName = createTable();
     try (Table table = UTIL.getConnection().getTable(tableName)) {
       for (int i = 0; i < 100; i++) {
         table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
@@ -89,11 +85,7 @@ public class TestSerialReplication extends SerialReplicationTestBase {
 
   @Test
   public void testRegionSplit() throws Exception {
-    TableName tableName = TableName.valueOf(name.getMethodName());
-    UTIL.getAdmin().createTable(
-      TableDescriptorBuilder.newBuilder(tableName).addColumnFamily(ColumnFamilyDescriptorBuilder
-        .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build());
-    UTIL.waitTableAvailable(tableName);
+    TableName tableName = createTable();
     try (Table table = UTIL.getConnection().getTable(tableName)) {
       for (int i = 0; i < 100; i++) {
         table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
@@ -148,7 +140,7 @@ public class TestSerialReplication extends SerialReplicationTestBase {
     TableName tableName = TableName.valueOf(name.getMethodName());
     UTIL.getAdmin().createTable(
       TableDescriptorBuilder.newBuilder(tableName)
-        .addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(CF)
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(CF)
           .setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build())
         .build(),
       new byte[][] { splitKey });
@@ -204,4 +196,58 @@ public class TestSerialReplication extends SerialReplicationTestBase {
       assertEquals(200, count);
     }
   }
+
+  @Test
+  public void testRemovePeerNothingReplicated() throws Exception {
+    TableName tableName = createTable();
+    String encodedRegionName =
+      UTIL.getMiniHBaseCluster().getRegions(tableName).get(0).getRegionInfo().getEncodedName();
+    ReplicationQueueStorage queueStorage =
+      UTIL.getMiniHBaseCluster().getMaster().getReplicationPeerManager().getQueueStorage();
+    assertEquals(HConstants.NO_SEQNUM, queueStorage.getLastSequenceId(encodedRegionName, PEER_ID));
+    UTIL.getAdmin().removeReplicationPeer(PEER_ID);
+    assertEquals(HConstants.NO_SEQNUM, queueStorage.getLastSequenceId(encodedRegionName, PEER_ID));
+  }
+
+  @Test
+  public void testRemovePeer() throws Exception {
+    TableName tableName = createTable();
+    try (Table table = UTIL.getConnection().getTable(tableName)) {
+      for (int i = 0; i < 100; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+    enablePeerAndWaitUntilReplicationDone(100);
+    checkOrder(100);
+    String encodedRegionName =
+      UTIL.getMiniHBaseCluster().getRegions(tableName).get(0).getRegionInfo().getEncodedName();
+    ReplicationQueueStorage queueStorage =
+      UTIL.getMiniHBaseCluster().getMaster().getReplicationPeerManager().getQueueStorage();
+    assertTrue(queueStorage.getLastSequenceId(encodedRegionName, PEER_ID) > 0);
+    UTIL.getAdmin().removeReplicationPeer(PEER_ID);
+    // confirm that we delete the last pushed sequence id
+    assertEquals(HConstants.NO_SEQNUM, queueStorage.getLastSequenceId(encodedRegionName, PEER_ID));
+  }
+
+  @Test
+  public void testRemoveSerialFlag() throws Exception {
+    TableName tableName = createTable();
+    try (Table table = UTIL.getConnection().getTable(tableName)) {
+      for (int i = 0; i < 100; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+    enablePeerAndWaitUntilReplicationDone(100);
+    checkOrder(100);
+    String encodedRegionName =
+      UTIL.getMiniHBaseCluster().getRegions(tableName).get(0).getRegionInfo().getEncodedName();
+    ReplicationQueueStorage queueStorage =
+      UTIL.getMiniHBaseCluster().getMaster().getReplicationPeerManager().getQueueStorage();
+    assertTrue(queueStorage.getLastSequenceId(encodedRegionName, PEER_ID) > 0);
+    ReplicationPeerConfig peerConfig = UTIL.getAdmin().getReplicationPeerConfig(PEER_ID);
+    UTIL.getAdmin().updateReplicationPeerConfig(PEER_ID,
+      ReplicationPeerConfig.newBuilder(peerConfig).setSerial(false).build());
+    // confirm that we delete the last pushed sequence id
+    assertEquals(HConstants.NO_SEQNUM, queueStorage.getLastSequenceId(encodedRegionName, PEER_ID));
+  }
 }


[15/22] hbase git commit: HBASE-20227 Add UT for ReplicationUtils.contains method

Posted by zh...@apache.org.
HBASE-20227 Add UT for ReplicationUtils.contains method

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

Branch: refs/heads/HBASE-20046-branch-2
Commit: 3b015d60a48927363cf1510f0f6dbea7b3abee18
Parents: 6f13b28
Author: tianjingyun <ti...@gmail.com>
Authored: Sat Mar 24 18:57:48 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Apr 9 15:18:44 2018 +0800

----------------------------------------------------------------------
 .../hbase/replication/ReplicationUtils.java     |   2 +-
 .../hbase/replication/TestReplicationUtil.java  | 235 +++++++++++++++++++
 2 files changed, 236 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3b015d60/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 1c42de4..c7568bb 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
@@ -145,7 +145,7 @@ public final class ReplicationUtils {
       if (excludeNamespaces != null && excludeNamespaces.contains(namespace)) {
         return false;
       }
-      Map<TableName, List<String>> excludedTableCFs = peerConfig.getTableCFsMap();
+      Map<TableName, List<String>> excludedTableCFs = peerConfig.getExcludeTableCFsMap();
       // trap here, must check existence first since HashMap allows null value.
       if (excludedTableCFs == null || !excludedTableCFs.containsKey(tableName)) {
         return true;

http://git-wip-us.apache.org/repos/asf/hbase/blob/3b015d60/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationUtil.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationUtil.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationUtil.java
new file mode 100644
index 0000000..f8543fe
--- /dev/null
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationUtil.java
@@ -0,0 +1,235 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+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;
+
+@Category({ ReplicationTests.class, SmallTests.class })
+public class TestReplicationUtil {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestReplicationUtil.class);
+
+  private static TableName TABLE_A = TableName.valueOf("replication", "testA");
+  private static TableName TABLE_B = TableName.valueOf("replication", "testB");
+
+  @Test
+  public void testContainsWithReplicatingAll() {
+    ReplicationPeerConfig peerConfig;
+    ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl builder =
+      new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl();
+    Map<TableName, List<String>> tableCfs = new HashMap<>();
+    Set<String> namespaces = new HashSet<>();
+
+    // 1. replication_all flag is true, no namespaces and table-cfs config
+    builder.setReplicateAllUserTables(true);
+    peerConfig = builder.build();
+    Assert.assertTrue(ReplicationUtils.contains(peerConfig, TABLE_A));
+
+    // 2. replicate_all flag is true, and config in excludedTableCfs
+    builder.setExcludeNamespaces(null);
+    // empty map
+    tableCfs = new HashMap<>();
+    builder.setReplicateAllUserTables(true);
+    builder.setExcludeTableCFsMap(tableCfs);
+    peerConfig = builder.build();
+    Assert.assertTrue(ReplicationUtils.contains(peerConfig, TABLE_A));
+
+    // table testB
+    tableCfs = new HashMap<>();
+    tableCfs.put(TABLE_B, null);
+    builder.setReplicateAllUserTables(true);
+    builder.setExcludeTableCFsMap(tableCfs);
+    peerConfig = builder.build();
+    Assert.assertTrue(ReplicationUtils.contains(peerConfig, TABLE_A));
+
+    // table testA
+    tableCfs = new HashMap<>();
+    tableCfs.put(TABLE_A, null);
+    builder.setReplicateAllUserTables(true);
+    builder.setExcludeTableCFsMap(tableCfs);
+    peerConfig = builder.build();
+    Assert.assertFalse(ReplicationUtils.contains(peerConfig, TABLE_A));
+
+    // 3. replicate_all flag is true, and config in excludeNamespaces
+    builder.setExcludeTableCFsMap(null);
+    // empty set
+    namespaces = new HashSet<>();
+    builder.setReplicateAllUserTables(true);
+    builder.setExcludeNamespaces(namespaces);
+    peerConfig = builder.build();
+    Assert.assertTrue(ReplicationUtils.contains(peerConfig, TABLE_A));
+
+    // namespace default
+    namespaces = new HashSet<>();
+    namespaces.add("default");
+    builder.setReplicateAllUserTables(true);
+    builder.setExcludeNamespaces(namespaces);
+    peerConfig = builder.build();
+    Assert.assertTrue(ReplicationUtils.contains(peerConfig, TABLE_A));
+
+    // namespace replication
+    namespaces = new HashSet<>();
+    namespaces.add("replication");
+    builder.setReplicateAllUserTables(true);
+    builder.setExcludeNamespaces(namespaces);
+    peerConfig = builder.build();
+    Assert.assertFalse(ReplicationUtils.contains(peerConfig, TABLE_A));
+
+    // 4. replicate_all flag is true, and config excludeNamespaces and excludedTableCfs both
+    // Namespaces config doesn't conflict with table-cfs config
+    namespaces = new HashSet<>();
+    tableCfs = new HashMap<>();
+    namespaces.add("replication");
+    tableCfs.put(TABLE_A, null);
+    builder.setReplicateAllUserTables(true);
+    builder.setExcludeTableCFsMap(tableCfs);
+    builder.setExcludeNamespaces(namespaces);
+    peerConfig = builder.build();
+    Assert.assertFalse(ReplicationUtils.contains(peerConfig, TABLE_A));
+
+    // Namespaces config conflicts with table-cfs config
+    namespaces = new HashSet<>();
+    tableCfs = new HashMap<>();
+    namespaces.add("default");
+    tableCfs.put(TABLE_A, null);
+    builder.setReplicateAllUserTables(true);
+    builder.setExcludeTableCFsMap(tableCfs);
+    builder.setExcludeNamespaces(namespaces);
+    peerConfig = builder.build();
+    Assert.assertFalse(ReplicationUtils.contains(peerConfig, TABLE_A));
+
+    namespaces = new HashSet<>();
+    tableCfs = new HashMap<>();
+    namespaces.add("replication");
+    tableCfs.put(TABLE_B, null);
+    builder.setReplicateAllUserTables(true);
+    builder.setExcludeTableCFsMap(tableCfs);
+    builder.setExcludeNamespaces(namespaces);
+    peerConfig = builder.build();
+    Assert.assertFalse(ReplicationUtils.contains(peerConfig, TABLE_A));
+
+  }
+
+  @Test
+  public void testContainsWithoutReplicatingAll() {
+    ReplicationPeerConfig peerConfig;
+    ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl builder =
+      new ReplicationPeerConfig.ReplicationPeerConfigBuilderImpl();
+    Map<TableName, List<String>> tableCfs = new HashMap<>();
+    Set<String> namespaces = new HashSet<>();
+
+    // 1. replication_all flag is false, no namespaces and table-cfs config
+    builder.setReplicateAllUserTables(false);
+    peerConfig = builder.build();
+    Assert.assertFalse(ReplicationUtils.contains(peerConfig, TABLE_A));
+
+    // 2. replicate_all flag is false, and only config table-cfs in peer
+    // empty map
+    builder.setReplicateAllUserTables(false);
+    builder.setTableCFsMap(tableCfs);
+    peerConfig = builder.build();
+    Assert.assertFalse(ReplicationUtils.contains(peerConfig, TABLE_A));
+
+    // table testB
+    tableCfs = new HashMap<>();
+    tableCfs.put(TABLE_B, null);
+    builder.setReplicateAllUserTables(false);
+    builder.setTableCFsMap(tableCfs);
+    peerConfig = builder.build();
+    Assert.assertFalse(ReplicationUtils.contains(peerConfig, TABLE_A));
+
+    // table testA
+    tableCfs = new HashMap<>();
+    tableCfs.put(TABLE_A, null);
+    builder.setReplicateAllUserTables(false);
+    builder.setTableCFsMap(tableCfs);
+    peerConfig = builder.build();
+    Assert.assertTrue(ReplicationUtils.contains(peerConfig, TABLE_A));
+
+    // 3. replication_all flag is false, and only config namespace in peer
+    builder.setTableCFsMap(null);
+    // empty set
+    builder.setReplicateAllUserTables(false);
+    builder.setNamespaces(namespaces);
+    peerConfig = builder.build();
+    Assert.assertFalse(ReplicationUtils.contains(peerConfig, TABLE_A));
+
+    // namespace default
+    namespaces = new HashSet<>();
+    namespaces.add("default");
+    builder.setReplicateAllUserTables(false);
+    builder.setNamespaces(namespaces);
+    peerConfig = builder.build();
+    Assert.assertFalse(ReplicationUtils.contains(peerConfig, TABLE_A));
+
+    // namespace replication
+    namespaces = new HashSet<>();
+    namespaces.add("replication");
+    builder.setReplicateAllUserTables(false);
+    builder.setNamespaces(namespaces);
+    peerConfig = builder.build();
+    Assert.assertTrue(ReplicationUtils.contains(peerConfig, TABLE_A));
+
+    // 4. replicate_all flag is false, and config namespaces and table-cfs both
+    // Namespaces config doesn't conflict with table-cfs config
+    namespaces = new HashSet<>();
+    tableCfs = new HashMap<>();
+    namespaces.add("replication");
+    tableCfs.put(TABLE_A, null);
+    builder.setReplicateAllUserTables(false);
+    builder.setTableCFsMap(tableCfs);
+    builder.setNamespaces(namespaces);
+    peerConfig = builder.build();
+    Assert.assertTrue(ReplicationUtils.contains(peerConfig, TABLE_A));
+
+    // Namespaces config conflicts with table-cfs config
+    namespaces = new HashSet<>();
+    tableCfs = new HashMap<>();
+    namespaces.add("default");
+    tableCfs.put(TABLE_A, null);
+    builder.setReplicateAllUserTables(false);
+    builder.setTableCFsMap(tableCfs);
+    builder.setNamespaces(namespaces);
+    peerConfig = builder.build();
+    Assert.assertTrue(ReplicationUtils.contains(peerConfig, TABLE_A));
+
+    namespaces = new HashSet<>();
+    tableCfs = new HashMap<>();
+    namespaces.add("replication");
+    tableCfs.put(TABLE_B, null);
+    builder.setReplicateAllUserTables(false);
+    builder.setTableCFsMap(tableCfs);
+    builder.setNamespaces(namespaces);
+    peerConfig = builder.build();
+    Assert.assertTrue(ReplicationUtils.contains(peerConfig, TABLE_A));
+  }
+}


[05/22] hbase git commit: HBASE-20148 Make serial replication as a option for a peer instead of a table

Posted by zh...@apache.org.
HBASE-20148 Make serial replication as a option for a peer instead of a table


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

Branch: refs/heads/HBASE-20046-branch-2
Commit: cea5199ea1b2cc07ea2cc773ece2887ff57ab7cf
Parents: 6b54127
Author: zhangduo <zh...@apache.org>
Authored: Fri Mar 9 15:00:59 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Apr 9 15:18:44 2018 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/HTableDescriptor.java   |  8 -----
 .../hadoop/hbase/client/TableDescriptor.java    | 19 +++++++-----
 .../hbase/client/TableDescriptorBuilder.java    |  9 ------
 .../replication/ReplicationPeerConfigUtil.java  |  5 +++
 .../replication/ReplicationPeerConfig.java      | 32 +++++++++++++++-----
 .../ReplicationPeerConfigBuilder.java           | 12 ++++++++
 .../org/apache/hadoop/hbase/HConstants.java     |  6 ----
 .../src/main/protobuf/Replication.proto         |  1 +
 .../hbase/replication/ReplicationUtils.java     |  3 ++
 .../master/assignment/RegionStateStore.java     | 14 ++++-----
 .../hbase/replication/ScopeWALEntryFilter.java  | 32 ++++++++++----------
 .../regionserver/ReplicationSource.java         |  4 +++
 .../ReplicationSourceWALActionListener.java     | 10 +-----
 .../ReplicationSourceWALReader.java             |  6 ++--
 .../regionserver/SerialReplicationChecker.java  |  2 +-
 .../org/apache/hadoop/hbase/wal/WALKeyImpl.java |  8 -----
 .../TestReplicationWALEntryFilters.java         | 15 ++++++---
 .../replication/TestSerialReplication.java      |  9 +++---
 18 files changed, 104 insertions(+), 91 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/cea5199e/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
index ca0cb91..960b91f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
@@ -539,14 +539,6 @@ public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescr
   }
 
   /**
-   * Return true if there are at least one cf whose replication scope is serial.
-   */
-  @Override
-  public boolean hasSerialReplicationScope() {
-    return delegatee.hasSerialReplicationScope();
-  }
-
-  /**
    * Returns the configured replicas per region
    */
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/cea5199e/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
index 13ad0e2..0a0683b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
@@ -25,6 +25,7 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.Set;
 import java.util.stream.Collectors;
+import java.util.stream.Stream;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -245,11 +246,6 @@ public interface TableDescriptor {
   boolean hasRegionMemStoreReplication();
 
   /**
-   * @return true if there are at least one cf whose replication scope is serial.
-   */
-  boolean hasSerialReplicationScope();
-
-  /**
    * Check if the compaction enable flag of the table is true. If flag is false
    * then no minor/major compactions will be done in real.
    *
@@ -288,6 +284,16 @@ public interface TableDescriptor {
   boolean isReadOnly();
 
   /**
+   * Check if any of the table's cfs' replication scope are set to
+   * {@link HConstants#REPLICATION_SCOPE_GLOBAL}.
+   * @return {@code true} if we have, otherwise {@code false}.
+   */
+  default boolean hasGlobalReplicationScope() {
+    return Stream.of(getColumnFamilies())
+      .anyMatch(cf -> cf.getScope() == HConstants.REPLICATION_SCOPE_GLOBAL);
+  }
+
+  /**
    * Check if the table's cfs' replication scope matched with the replication state
    * @param enabled replication state
    * @return true if matched, otherwise false
@@ -297,8 +303,7 @@ public interface TableDescriptor {
     boolean hasDisabled = false;
 
     for (ColumnFamilyDescriptor cf : getColumnFamilies()) {
-      if (cf.getScope() != HConstants.REPLICATION_SCOPE_GLOBAL &&
-        cf.getScope() != HConstants.REPLICATION_SCOPE_SERIAL) {
+      if (cf.getScope() != HConstants.REPLICATION_SCOPE_GLOBAL) {
         hasDisabled = true;
       } else {
         hasEnabled = true;

http://git-wip-us.apache.org/repos/asf/hbase/blob/cea5199e/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
index 2d6bfaf..02901ac 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
@@ -1128,15 +1128,6 @@ public class TableDescriptorBuilder {
     }
 
     /**
-     * Return true if there are at least one cf whose replication scope is serial.
-     */
-    @Override
-    public boolean hasSerialReplicationScope() {
-      return families.values().stream()
-        .anyMatch(column -> column.getScope() == HConstants.REPLICATION_SCOPE_SERIAL);
-    }
-
-    /**
      * Returns the configured replicas per region
      */
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/cea5199e/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..b1c1713 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
@@ -303,6 +303,10 @@ public final class ReplicationPeerConfigUtil {
       builder.setReplicateAllUserTables(peer.getReplicateAll());
     }
 
+    if (peer.hasSerial()) {
+      builder.setSerial(peer.getSerial());
+    }
+
     Map<TableName, List<String>> excludeTableCFsMap = convert2Map(peer.getExcludeTableCfsList()
         .toArray(new ReplicationProtos.TableCF[peer.getExcludeTableCfsCount()]));
     if (excludeTableCFsMap != null) {
@@ -357,6 +361,7 @@ public final class ReplicationPeerConfigUtil {
 
     builder.setBandwidth(peerConfig.getBandwidth());
     builder.setReplicateAll(peerConfig.replicateAllUserTables());
+    builder.setSerial(peerConfig.isSerial());
 
     ReplicationProtos.TableCF[] excludeTableCFs = convert(peerConfig.getExcludeTableCFsMap());
     if (excludeTableCFs != null) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/cea5199e/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..e0d9a4c 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,7 @@ public class ReplicationPeerConfig {
   private Map<TableName, ? extends Collection<String>> excludeTableCFsMap = null;
   private Set<String> excludeNamespaces = null;
   private long bandwidth = 0;
+  private final boolean serial;
 
   private ReplicationPeerConfig(ReplicationPeerConfigBuilderImpl builder) {
     this.clusterKey = builder.clusterKey;
@@ -64,6 +65,7 @@ public class ReplicationPeerConfig {
         builder.excludeNamespaces != null ? Collections.unmodifiableSet(builder.excludeNamespaces)
             : null;
     this.bandwidth = builder.bandwidth;
+    this.serial = builder.serial;
   }
 
   private Map<TableName, List<String>>
@@ -82,6 +84,7 @@ public class ReplicationPeerConfig {
   public ReplicationPeerConfig() {
     this.peerData = new TreeMap<>(Bytes.BYTES_COMPARATOR);
     this.configuration = new HashMap<>(0);
+    this.serial = false;
   }
 
   /**
@@ -214,16 +217,20 @@ public class ReplicationPeerConfig {
     return new ReplicationPeerConfigBuilderImpl();
   }
 
+  public boolean isSerial() {
+    return serial;
+  }
+
   public static ReplicationPeerConfigBuilder newBuilder(ReplicationPeerConfig peerConfig) {
     ReplicationPeerConfigBuilderImpl builder = new ReplicationPeerConfigBuilderImpl();
     builder.setClusterKey(peerConfig.getClusterKey())
-        .setReplicationEndpointImpl(peerConfig.getReplicationEndpointImpl())
-        .putAllPeerData(peerConfig.getPeerData()).putAllConfiguration(peerConfig.getConfiguration())
-        .setTableCFsMap(peerConfig.getTableCFsMap()).setNamespaces(peerConfig.getNamespaces())
-        .setReplicateAllUserTables(peerConfig.replicateAllUserTables())
-        .setExcludeTableCFsMap(peerConfig.getExcludeTableCFsMap())
-        .setExcludeNamespaces(peerConfig.getExcludeNamespaces())
-        .setBandwidth(peerConfig.getBandwidth());
+      .setReplicationEndpointImpl(peerConfig.getReplicationEndpointImpl())
+      .putAllPeerData(peerConfig.getPeerData()).putAllConfiguration(peerConfig.getConfiguration())
+      .setTableCFsMap(peerConfig.getTableCFsMap()).setNamespaces(peerConfig.getNamespaces())
+      .setReplicateAllUserTables(peerConfig.replicateAllUserTables())
+      .setExcludeTableCFsMap(peerConfig.getExcludeTableCFsMap())
+      .setExcludeNamespaces(peerConfig.getExcludeNamespaces())
+      .setBandwidth(peerConfig.getBandwidth()).setSerial(peerConfig.isSerial());
     return builder;
   }
 
@@ -250,6 +257,8 @@ public class ReplicationPeerConfig {
 
     private long bandwidth = 0;
 
+    private boolean serial = false;
+
     @Override
     public ReplicationPeerConfigBuilder setClusterKey(String clusterKey) {
       this.clusterKey = clusterKey;
@@ -313,6 +322,12 @@ public class ReplicationPeerConfig {
     }
 
     @Override
+    public ReplicationPeerConfigBuilder setSerial(boolean serial) {
+      this.serial = serial;
+      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.
@@ -340,7 +355,8 @@ public class ReplicationPeerConfig {
         builder.append("tableCFs=").append(tableCFsMap.toString()).append(",");
       }
     }
-    builder.append("bandwidth=").append(bandwidth);
+    builder.append("bandwidth=").append(bandwidth).append(",");
+    builder.append("serial=").append(serial);
     return builder.toString();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/cea5199e/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..4c531c5 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,18 @@ public interface ReplicationPeerConfigBuilder {
   ReplicationPeerConfigBuilder setExcludeNamespaces(Set<String> namespaces);
 
   /**
+   * <p>
+   * Sets whether we should preserve order when replicating, i.e, serial replication.
+   * </p>
+   * <p>
+   * Default {@code false}.
+   * </p>
+   * @param serial {@code true} means preserve order, otherwise {@code false}.
+   * @return {@code this}
+   */
+  ReplicationPeerConfigBuilder setSerial(boolean serial);
+
+  /**
    * 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/cea5199e/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index edf8f9c..5904df6 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -682,12 +682,6 @@ public final class HConstants {
   public static final int REPLICATION_SCOPE_GLOBAL = 1;
 
   /**
-   * Scope tag for serially scoped data
-   * This data will be replicated to all peers by the order of sequence id.
-   */
-  public static final int REPLICATION_SCOPE_SERIAL = 2;
-
-  /**
    * Default cluster ID, cannot be used to identify a cluster so a key with
    * this value means it wasn't meant for replication.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/cea5199e/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..557b87c 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 bool serial = 11;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/cea5199e/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 ca871ea..857b385 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
@@ -124,6 +124,9 @@ public final class ReplicationUtils {
     if (rpc1.replicateAllUserTables() != rpc2.replicateAllUserTables()) {
       return false;
     }
+    if (rpc1.isSerial() != rpc2.isSerial()) {
+      return false;
+    }
     if (rpc1.replicateAllUserTables()) {
       return isNamespacesEqual(rpc1.getExcludeNamespaces(), rpc2.getExcludeNamespaces()) &&
         isTableCFsEqual(rpc1.getExcludeTableCFsMap(), rpc2.getExcludeTableCFsMap());

http://git-wip-us.apache.org/repos/asf/hbase/blob/cea5199e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
index c98a2d1..207ce86 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
@@ -165,7 +165,7 @@ public class RegionStateStore {
       MetaTableAccessor.addLocation(put, regionLocation, openSeqNum, replicaId);
       // only update replication barrier for default replica
       if (regionInfo.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID &&
-        hasSerialReplicationScope(regionInfo.getTable())) {
+        hasGlobalReplicationScope(regionInfo.getTable())) {
         MetaTableAccessor.addReplicationBarrier(put, openSeqNum);
       }
       info.append(", openSeqNum=").append(openSeqNum);
@@ -224,7 +224,7 @@ public class RegionStateStore {
       ServerName serverName) throws IOException {
     TableDescriptor htd = getTableDescriptor(parent.getTable());
     long parentOpenSeqNum = HConstants.NO_SEQNUM;
-    if (htd.hasSerialReplicationScope()) {
+    if (htd.hasGlobalReplicationScope()) {
       parentOpenSeqNum = getOpenSeqNumForParentRegion(parent);
     }
     MetaTableAccessor.splitRegion(master.getConnection(), parent, parentOpenSeqNum, hriA, hriB,
@@ -239,7 +239,7 @@ public class RegionStateStore {
     TableDescriptor htd = getTableDescriptor(child.getTable());
     long regionAOpenSeqNum = -1L;
     long regionBOpenSeqNum = -1L;
-    if (htd.hasSerialReplicationScope()) {
+    if (htd.hasGlobalReplicationScope()) {
       regionAOpenSeqNum = getOpenSeqNumForParentRegion(hriA);
       regionBOpenSeqNum = getOpenSeqNumForParentRegion(hriB);
     }
@@ -261,12 +261,12 @@ public class RegionStateStore {
   // ==========================================================================
   //  Table Descriptors helpers
   // ==========================================================================
-  private boolean hasSerialReplicationScope(TableName tableName) throws IOException {
-    return hasSerialReplicationScope(getTableDescriptor(tableName));
+  private boolean hasGlobalReplicationScope(TableName tableName) throws IOException {
+    return hasGlobalReplicationScope(getTableDescriptor(tableName));
   }
 
-  private boolean hasSerialReplicationScope(TableDescriptor htd) {
-    return htd != null ? htd.hasSerialReplicationScope() : false;
+  private boolean hasGlobalReplicationScope(TableDescriptor htd) {
+    return htd != null ? htd.hasGlobalReplicationScope() : false;
   }
 
   private int getRegionReplication(TableDescriptor htd) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/cea5199e/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ScopeWALEntryFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ScopeWALEntryFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ScopeWALEntryFilter.java
index 6a2fbcf..f8722eb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ScopeWALEntryFilter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ScopeWALEntryFilter.java
@@ -37,31 +37,31 @@ public class ScopeWALEntryFilter implements WALEntryFilter, WALCellFilter {
 
   @Override
   public Entry filter(Entry entry) {
-    NavigableMap<byte[], Integer> scopes = entry.getKey().getReplicationScopes();
-    if (scopes == null || scopes.isEmpty()) {
-      return null;
-    }
+    // Do not filter out an entire entry by replication scopes. As now we support serial
+    // replication, the sequence id of a marker is also needed by upper layer. We will filter out
+    // all the cells in the filterCell method below if the replication scopes is null or empty.
     return entry;
   }
 
+  private boolean hasGlobalScope(NavigableMap<byte[], Integer> scopes, byte[] family) {
+    Integer scope = scopes.get(family);
+    return scope != null && scope.intValue() == HConstants.REPLICATION_SCOPE_GLOBAL;
+  }
   @Override
   public Cell filterCell(Entry entry, Cell cell) {
-    final NavigableMap<byte[], Integer> scopes = entry.getKey().getReplicationScopes();
-    // The scope will be null or empty if
-    // there's nothing to replicate in that WALEdit
-    byte[] fam = CellUtil.cloneFamily(cell);
+    NavigableMap<byte[], Integer> scopes = entry.getKey().getReplicationScopes();
+    if (scopes == null || scopes.isEmpty()) {
+      return null;
+    }
+    byte[] family = CellUtil.cloneFamily(cell);
     if (CellUtil.matchingColumn(cell, WALEdit.METAFAMILY, WALEdit.BULK_LOAD)) {
-      cell = bulkLoadFilter.filterCell(cell, new Predicate<byte[]>() {
+      return bulkLoadFilter.filterCell(cell, new Predicate<byte[]>() {
         @Override
-        public boolean apply(byte[] fam) {
-          return !scopes.containsKey(fam) || scopes.get(fam) == HConstants.REPLICATION_SCOPE_LOCAL;
+        public boolean apply(byte[] family) {
+          return !hasGlobalScope(scopes, family);
         }
       });
-    } else {
-      if (!scopes.containsKey(fam) || scopes.get(fam) == HConstants.REPLICATION_SCOPE_LOCAL) {
-        return null;
-      }
     }
-    return cell;
+    return hasGlobalScope(scopes, family) ? cell : null;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/cea5199e/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index 3b65b25..f5e4185 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -392,6 +392,10 @@ public class ReplicationSource implements ReplicationSourceInterface {
     return replicationPeer.isPeerEnabled();
   }
 
+  public boolean isSerial() {
+    return replicationPeer.getPeerConfig().isSerial();
+  }
+
   private void initialize() {
     int sleepMultiplier = 1;
     while (this.isSourceActive()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/cea5199e/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
index 95fc6a0..27b25c4 100644
--- 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
@@ -72,18 +72,10 @@ class ReplicationSourceWALActionListener implements WALActionsListener {
     if (ReplicationUtils.isReplicationForBulkLoadDataEnabled(conf)) {
       return;
     }
-    WALKeyImpl keyImpl = (WALKeyImpl) logKey;
-    // For serial replication we need to count all the sequence ids even for markers, so here we
-    // always need to retain the replication scopes to let the replication wal reader to know that
-    // we need serial replication. The ScopeWALEntryFilter will help filtering out the cell for
-    // WALEdit.METAFAMILY.
-    if (keyImpl.hasSerialReplicationScope()) {
-      return;
-    }
     // For replay, or if all the cells are markers, do not need to store replication scope.
     if (logEdit.isReplay() ||
       logEdit.getCells().stream().allMatch(c -> CellUtil.matchingFamily(c, WALEdit.METAFAMILY))) {
-      keyImpl.clearReplicationScope();
+      ((WALKeyImpl) logKey).clearReplicationScope();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/cea5199e/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
index ad3baaf..da92a09 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
@@ -186,9 +186,9 @@ public class ReplicationSourceWALReader extends Thread {
       new WALEntryBatch(replicationBatchCountCapacity, entryStream.getCurrentPath());
     do {
       Entry entry = entryStream.peek();
-      boolean hasSerialReplicationScope = entry.getKey().hasSerialReplicationScope();
+      boolean isSerial = source.isSerial();
       boolean doFiltering = true;
-      if (hasSerialReplicationScope) {
+      if (isSerial) {
         if (firstCellInEntryBeforeFiltering == null) {
           assert !entry.getEdit().isEmpty() : "should not write empty edits";
           // Used to locate the region record in meta table. In WAL we only have the table name and
@@ -208,7 +208,7 @@ public class ReplicationSourceWALReader extends Thread {
         entry = filterEntry(entry);
       }
       if (entry != null) {
-        if (hasSerialReplicationScope) {
+        if (isSerial) {
           if (!serialReplicationChecker.canPush(entry, firstCellInEntryBeforeFiltering)) {
             if (batch.getLastWalPosition() > positionBefore) {
               // we have something that can push, break

http://git-wip-us.apache.org/repos/asf/hbase/blob/cea5199e/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SerialReplicationChecker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SerialReplicationChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SerialReplicationChecker.java
index 9276359..b775d25 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SerialReplicationChecker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SerialReplicationChecker.java
@@ -266,7 +266,7 @@ class SerialReplicationChecker {
       throws IOException, InterruptedException {
     byte[] row = CellUtil.cloneRow(firstCellInEdit);
     while (!canPush(entry, row)) {
-      LOG.debug("Can not push{}, wait", entry);
+      LOG.debug("Can not push {}, wait", entry);
       Thread.sleep(waitTimeMs);
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/cea5199e/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 ac23d1d..8828239 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
@@ -419,14 +419,6 @@ public class WALKeyImpl implements WALKey {
     setReplicationScope(null);
   }
 
-  public boolean hasSerialReplicationScope() {
-    if (replicationScope == null || replicationScope.isEmpty()) {
-      return false;
-    }
-    return replicationScope.values().stream()
-      .anyMatch(scope -> scope.intValue() == HConstants.REPLICATION_SCOPE_SERIAL);
-  }
-
   /**
    * Marks that the cluster with the given clusterId has consumed the change
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/cea5199e/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java
index 67a2551..f2c5e50 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.replication;
 
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -32,9 +33,9 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparatorImpl;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -48,7 +49,7 @@ import org.junit.experimental.categories.Category;
 
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
-@Category({ReplicationTests.class, SmallTests.class})
+@Category({ ReplicationTests.class, SmallTests.class })
 public class TestReplicationWALEntryFilters {
 
   @ClassRule
@@ -65,7 +66,8 @@ public class TestReplicationWALEntryFilters {
     SystemTableWALEntryFilter filter = new SystemTableWALEntryFilter();
 
     // meta
-    WALKeyImpl key1 = new WALKeyImpl(HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
+    WALKeyImpl key1 =
+      new WALKeyImpl(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
         TableName.META_TABLE_NAME, System.currentTimeMillis());
     Entry metaEntry = new Entry(key1, null);
 
@@ -96,12 +98,15 @@ public class TestReplicationWALEntryFilters {
     Entry userEntryEmpty = createEntry(null);
 
     // no scopes
-    assertEquals(null, filter.filter(userEntry));
+    // now we will not filter out entries without a replication scope since serial replication still
+    // need the sequence id, but the cells will all be filtered out.
+    assertTrue(filter.filter(userEntry).getEdit().isEmpty());
 
     // empty scopes
+    // ditto
     TreeMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
     userEntry = createEntry(scopes, a, b);
-    assertEquals(null, filter.filter(userEntry));
+    assertTrue(filter.filter(userEntry).getEdit().isEmpty());
 
     // different scope
     scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);

http://git-wip-us.apache.org/repos/asf/hbase/blob/cea5199e/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
index 9d8e7fe..37f11f6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
@@ -156,7 +156,8 @@ public class TestSerialReplication {
     // add in disable state, so later when enabling it all sources will start push together.
     UTIL.getAdmin().addReplicationPeer(PEER_ID,
       ReplicationPeerConfig.newBuilder().setClusterKey("127.0.0.1:2181:/hbase")
-        .setReplicationEndpointImpl(LocalReplicationEndpoint.class.getName()).build(),
+        .setReplicationEndpointImpl(LocalReplicationEndpoint.class.getName()).setSerial(true)
+        .build(),
       false);
   }
 
@@ -234,7 +235,7 @@ public class TestSerialReplication {
     TableName tableName = TableName.valueOf(name.getMethodName());
     UTIL.getAdmin().createTable(
       TableDescriptorBuilder.newBuilder(tableName).addColumnFamily(ColumnFamilyDescriptorBuilder
-        .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_SERIAL).build()).build());
+        .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build());
     UTIL.waitTableAvailable(tableName);
     try (Table table = UTIL.getConnection().getTable(tableName)) {
       for (int i = 0; i < 100; i++) {
@@ -273,7 +274,7 @@ public class TestSerialReplication {
     TableName tableName = TableName.valueOf(name.getMethodName());
     UTIL.getAdmin().createTable(
       TableDescriptorBuilder.newBuilder(tableName).addColumnFamily(ColumnFamilyDescriptorBuilder
-        .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_SERIAL).build()).build());
+        .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build());
     UTIL.waitTableAvailable(tableName);
     try (Table table = UTIL.getConnection().getTable(tableName)) {
       for (int i = 0; i < 100; i++) {
@@ -330,7 +331,7 @@ public class TestSerialReplication {
     UTIL.getAdmin().createTable(
       TableDescriptorBuilder.newBuilder(tableName)
         .addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(CF)
-          .setScope(HConstants.REPLICATION_SCOPE_SERIAL).build())
+          .setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build())
         .build(),
       new byte[][] { splitKey });
     UTIL.waitTableAvailable(tableName);


[09/22] hbase git commit: HBASE-20115 Reimplement serial replication based on the new replication storage layer

Posted by zh...@apache.org.
HBASE-20115 Reimplement serial replication based on the new replication storage layer


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

Branch: refs/heads/HBASE-20046-branch-2
Commit: f29bf1d7786fb9fedb3b735f4a8134b61283e1d4
Parents: 1d11cdb
Author: zhangduo <zh...@apache.org>
Authored: Mon Mar 5 16:47:03 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Apr 9 15:18:44 2018 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/HTableDescriptor.java   |   8 +
 .../apache/hadoop/hbase/MetaTableAccessor.java  | 211 ++++++++++++---
 .../hadoop/hbase/client/TableDescriptor.java    |   8 +-
 .../hbase/client/TableDescriptorBuilder.java    |   9 +
 .../org/apache/hadoop/hbase/HConstants.java     |  12 +
 .../hadoop/hbase/master/MasterFileSystem.java   |  11 +-
 .../master/assignment/AssignmentManager.java    |   3 +-
 .../master/assignment/RegionStateStore.java     |  60 +++--
 .../assignment/SplitTableRegionProcedure.java   |   4 +-
 .../AbstractStateMachineTableProcedure.java     |   8 +-
 .../hbase/regionserver/HRegionFileSystem.java   |  11 +-
 .../NamespaceTableCfWALEntryFilter.java         |   8 +-
 .../hbase/replication/ScopeWALEntryFilter.java  |  34 ++-
 .../RecoveredReplicationSource.java             |   5 +
 .../RecoveredReplicationSourceShipper.java      |  12 +-
 .../RecoveredReplicationSourceWALReader.java    |   9 +-
 .../regionserver/ReplicationSource.java         |   8 +
 .../ReplicationSourceInterface.java             |   7 +
 .../regionserver/ReplicationSourceManager.java  |   4 +-
 .../regionserver/ReplicationSourceShipper.java  |  17 +-
 .../ReplicationSourceWALActionListener.java     |  39 ++-
 .../ReplicationSourceWALReader.java             | 188 +++++---------
 .../regionserver/SerialReplicationChecker.java  | 255 +++++++++++++++++++
 .../replication/regionserver/WALEntryBatch.java | 138 ++++++++++
 .../regionserver/WALEntryStream.java            |  29 +--
 .../hadoop/hbase/util/FSTableDescriptors.java   |   8 +
 .../org/apache/hadoop/hbase/util/FSUtils.java   |  28 +-
 .../org/apache/hadoop/hbase/wal/WALKeyImpl.java |  12 +-
 .../hadoop/hbase/TestMetaTableAccessor.java     |  14 +-
 .../regionserver/TestHRegionFileSystem.java     |  14 +-
 .../regionserver/TestRegionServerMetrics.java   |   4 +-
 .../TestReplicationDroppedTables.java           |   8 +-
 .../replication/TestSerialReplication.java      | 234 +++++++++++++++++
 .../TestReplicationSourceManager.java           |   2 +-
 .../TestSerialReplicationChecker.java           | 176 +++++++++++++
 .../regionserver/TestWALEntryStream.java        |  19 +-
 36 files changed, 1279 insertions(+), 338 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
index 960b91f..ca0cb91 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
@@ -539,6 +539,14 @@ public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescr
   }
 
   /**
+   * Return true if there are at least one cf whose replication scope is serial.
+   */
+  @Override
+  public boolean hasSerialReplicationScope() {
+    return delegatee.hasSerialReplicationScope();
+  }
+
+  /**
    * Returns the configured replicas per region
    */
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/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 d6bbf53..109f2d0 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
@@ -34,6 +34,8 @@ import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell.Type;
 import org.apache.hadoop.hbase.client.Connection;
@@ -56,6 +58,7 @@ import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
@@ -137,7 +140,7 @@ public class MetaTableAccessor {
   private static final Logger LOG = LoggerFactory.getLogger(MetaTableAccessor.class);
   private static final Logger METALOG = LoggerFactory.getLogger("org.apache.hadoop.hbase.META");
 
-  static final byte [] META_REGION_PREFIX;
+  private static final byte[] META_REGION_PREFIX;
   static {
     // Copy the prefix from FIRST_META_REGIONINFO into META_REGION_PREFIX.
     // FIRST_META_REGIONINFO == 'hbase:meta,,1'.  META_REGION_PREFIX == 'hbase:meta,'
@@ -147,6 +150,11 @@ public class MetaTableAccessor {
       META_REGION_PREFIX, 0, len);
   }
 
+  private static final byte[] REPLICATION_PARENT_QUALIFIER = Bytes.toBytes("parent");
+
+  private static final String REPLICATION_PARENT_SEPARATOR = "|";
+
+  private static final String REPLICATION_PARENT_SEPARATOR_REGEX = "\\|";
   /**
    * Lists all of the table regions currently in META.
    * Deprecated, keep there until some test use this.
@@ -838,7 +846,7 @@ public class MetaTableAccessor {
 
   /**
    * Returns the column qualifier for serialized region state
-   * @return HConstants.TABLE_STATE_QUALIFIER
+   * @return HConstants.STATE_QUALIFIER
    */
   private static byte[] getRegionStateColumn() {
     return HConstants.STATE_QUALIFIER;
@@ -1266,7 +1274,6 @@ public class MetaTableAccessor {
   ////////////////////////
   // Editing operations //
   ////////////////////////
-
   /**
    * Generates and returns a Put containing the region into for the catalog table
    */
@@ -1438,7 +1445,7 @@ public class MetaTableAccessor {
    * Adds daughter region infos to hbase:meta row for the specified region. Note that this does not
    * add its daughter's as different rows, but adds information about the daughters in the same row
    * as the parent. Use
-   * {@link #splitRegion(Connection, RegionInfo, RegionInfo, RegionInfo, ServerName,int)}
+   * {@link #splitRegion(Connection, RegionInfo, long, RegionInfo, RegionInfo, ServerName, int)}
    * if you want to do that.
    * @param connection connection we're using
    * @param regionInfo RegionInfo of parent region
@@ -1464,7 +1471,7 @@ public class MetaTableAccessor {
    * Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this
    * does not add its daughter's as different rows, but adds information about the daughters
    * in the same row as the parent. Use
-   * {@link #splitRegion(Connection, RegionInfo, RegionInfo, RegionInfo, ServerName, int)}
+   * {@link #splitRegion(Connection, RegionInfo, long, RegionInfo, RegionInfo, ServerName, int)}
    * if you want to do that.
    * @param connection connection we're using
    * @param regionInfo region information
@@ -1519,20 +1526,37 @@ public class MetaTableAccessor {
   }
 
   /**
-   * Merge the two regions into one in an atomic operation. Deletes the two
-   * merging regions in hbase:meta and adds the merged region with the information of
-   * two merging regions.
+   * Merge the two regions into one in an atomic operation. Deletes the two merging regions in
+   * hbase:meta and adds the merged region with the information of two merging regions.
    * @param connection connection we're using
    * @param mergedRegion the merged region
    * @param regionA merge parent region A
+   * @param regionAOpenSeqNum the next open sequence id for region A, used by serial replication. -1
+   *          if not necessary.
    * @param regionB merge parent region B
+   * @param regionBOpenSeqNum the next open sequence id for region B, used by serial replication. -1
+   *          if not necessary.
    * @param sn the location of the region
    */
-  public static void mergeRegions(final Connection connection, RegionInfo mergedRegion,
-      RegionInfo regionA, RegionInfo regionB, ServerName sn, int regionReplication)
-      throws IOException {
+  public static void mergeRegions(Connection connection, RegionInfo mergedRegion,
+      RegionInfo regionA, long regionAOpenSeqNum, RegionInfo regionB, long regionBOpenSeqNum,
+      ServerName sn, int regionReplication) throws IOException {
     try (Table meta = getMetaHTable(connection)) {
       long time = EnvironmentEdgeManager.currentTime();
+      List<Mutation> mutations = new ArrayList<>();
+
+      List<RegionInfo> replicationParents = new ArrayList<>(2);
+      // Deletes for merging regions
+      mutations.add(makeDeleteFromRegionInfo(regionA, time));
+      if (regionAOpenSeqNum > 0) {
+        mutations.add(makePutForReplicationBarrier(regionA, regionAOpenSeqNum, time));
+        replicationParents.add(regionA);
+      }
+      mutations.add(makeDeleteFromRegionInfo(regionB, time));
+      if (regionBOpenSeqNum > 0) {
+        mutations.add(makePutForReplicationBarrier(regionB, regionBOpenSeqNum, time));
+        replicationParents.add(regionB);
+      }
 
       // Put for parent
       Put putOfMerged = makePutFromRegionInfo(mergedRegion, time);
@@ -1552,18 +1576,13 @@ public class MetaTableAccessor {
               .setType(Type.Put)
               .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);
-
+      mutations.add(putOfMerged);
       // The merged is a new region, openSeqNum = 1 is fine. ServerName may be null
       // if crash after merge happened but before we got to here.. means in-memory
       // locations of offlined merged, now-closed, regions is lost. Should be ok. We
@@ -1577,26 +1596,30 @@ public class MetaTableAccessor {
       for (int i = 1; i < regionReplication; i++) {
         addEmptyLocation(putOfMerged, i);
       }
-
-      byte[] tableRow = Bytes.toBytes(mergedRegion.getRegionNameAsString()
-        + HConstants.DELIMITER);
-      multiMutate(connection, meta, tableRow, putOfMerged, deleteA, deleteB);
+      // add parent reference for serial replication
+      if (!replicationParents.isEmpty()) {
+        addReplicationParent(putOfMerged, replicationParents);
+      }
+      byte[] tableRow = Bytes.toBytes(mergedRegion.getRegionNameAsString() + HConstants.DELIMITER);
+      multiMutate(connection, meta, tableRow, mutations);
     }
   }
 
   /**
-   * Splits the region into two in an atomic operation. Offlines the parent
-   * region with the information that it is split into two, and also adds
-   * the daughter regions. Does not add the location information to the daughter
-   * regions since they are not open yet.
+   * Splits the region into two in an atomic operation. Offlines the parent region with the
+   * information that it is split into two, and also adds the daughter regions. Does not add the
+   * location information to the daughter regions since they are not open yet.
    * @param connection connection we're using
    * @param parent the parent region which is split
+   * @param parentOpenSeqNum the next open sequence id for parent region, used by serial
+   *          replication. -1 if not necessary.
    * @param splitA Split daughter region A
    * @param splitB Split daughter region B
    * @param sn the location of the region
    */
-  public static void splitRegion(final Connection connection, RegionInfo parent, RegionInfo splitA,
-      RegionInfo splitB, ServerName sn, int regionReplication) throws IOException {
+  public static void splitRegion(Connection connection, RegionInfo parent, long parentOpenSeqNum,
+      RegionInfo splitA, RegionInfo splitB, ServerName sn, int regionReplication)
+      throws IOException {
     try (Table meta = getMetaHTable(connection)) {
       long time = EnvironmentEdgeManager.currentTime();
       // Put for parent
@@ -1608,7 +1631,11 @@ public class MetaTableAccessor {
       // Puts for daughters
       Put putA = makePutFromRegionInfo(splitA, time);
       Put putB = makePutFromRegionInfo(splitB, time);
-
+      if (parentOpenSeqNum > 0) {
+        addReplicationBarrier(putParent, parentOpenSeqNum);
+        addReplicationParent(putA, Collections.singletonList(parent));
+        addReplicationParent(putB, Collections.singletonList(parent));
+      }
       // 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
@@ -1668,20 +1695,15 @@ public class MetaTableAccessor {
   }
 
   private static void multiMutate(Connection connection, Table table, byte[] row,
-      Mutation... mutations)
-  throws IOException {
+      Mutation... mutations) throws IOException {
     multiMutate(connection, table, row, Arrays.asList(mutations));
   }
 
   /**
    * Performs an atomic multi-mutate operation against the given table.
    */
-  // Used by the RSGroup Coprocessor Endpoint. It had a copy/paste of the below. Need to reveal
-  // this facility for CPEP use or at least those CPEPs that are on their way to becoming part of
-  // core as is the intent for RSGroup eventually.
-  public static void multiMutate(Connection connection, final Table table, byte[] row,
-      final List<Mutation> mutations)
-  throws IOException {
+  private static void multiMutate(Connection connection, final Table table, byte[] row,
+      final List<Mutation> mutations) throws IOException {
     debugLogMutations(mutations);
     // TODO: Need rollback!!!!
     // TODO: Need Retry!!!
@@ -1782,9 +1804,7 @@ public class MetaTableAccessor {
    * @param regionInfo region to be deleted from META
    * @throws IOException
    */
-  public static void deleteRegion(Connection connection,
-                                  RegionInfo regionInfo)
-    throws IOException {
+  public static void deleteRegion(Connection connection, RegionInfo regionInfo) throws IOException {
     long time = EnvironmentEdgeManager.currentTime();
     Delete delete = new Delete(regionInfo.getRegionName());
     delete.addFamily(getCatalogFamily(), time);
@@ -1901,6 +1921,33 @@ public class MetaTableAccessor {
               .build());
   }
 
+  private static void addReplicationParent(Put put, List<RegionInfo> parents) throws IOException {
+    byte[] value = parents.stream().map(RegionReplicaUtil::getRegionInfoForDefaultReplica)
+      .map(RegionInfo::getRegionNameAsString).collect(Collectors
+        .collectingAndThen(Collectors.joining(REPLICATION_PARENT_SEPARATOR), Bytes::toBytes));
+    put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
+      .setFamily(HConstants.REPLICATION_BARRIER_FAMILY).setQualifier(REPLICATION_PARENT_QUALIFIER)
+      .setTimestamp(put.getTimeStamp()).setType(Type.Put).setValue(value).build());
+  }
+
+  private static Put makePutForReplicationBarrier(RegionInfo regionInfo, long openSeqNum, long ts)
+      throws IOException {
+    Put put = new Put(regionInfo.getRegionName(), ts);
+    addReplicationBarrier(put, openSeqNum);
+    return put;
+  }
+
+  public static void addReplicationBarrier(Put put, long openSeqNum) throws IOException {
+    put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
+      .setRow(put.getRow())
+      .setFamily(HConstants.REPLICATION_BARRIER_FAMILY)
+      .setQualifier(HConstants.SEQNUM_QUALIFIER)
+      .setTimestamp(put.getTimeStamp())
+      .setType(Type.Put)
+      .setValue(Bytes.toBytes(openSeqNum))
+      .build());
+  }
+
   private static Put addEmptyLocation(Put p, int replicaId) throws IOException {
     CellBuilder builder = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
     return p.add(builder.clear()
@@ -1926,6 +1973,92 @@ public class MetaTableAccessor {
                 .build());
   }
 
+  public static final class ReplicationBarrierResult {
+    private final long[] barriers;
+    private final RegionState.State state;
+    private final List<byte[]> parentRegionNames;
+
+    public ReplicationBarrierResult(long[] barriers, State state, List<byte[]> parentRegionNames) {
+      this.barriers = barriers;
+      this.state = state;
+      this.parentRegionNames = parentRegionNames;
+    }
+
+    public long[] getBarriers() {
+      return barriers;
+    }
+
+    public RegionState.State getState() {
+      return state;
+    }
+
+    public List<byte[]> getParentRegionNames() {
+      return parentRegionNames;
+    }
+  }
+
+  private static long getReplicationBarrier(Cell c) {
+    return Bytes.toLong(c.getValueArray(), c.getValueOffset(), c.getValueLength());
+  }
+
+  private static long[] getReplicationBarriers(Result result) {
+    return result.getColumnCells(HConstants.REPLICATION_BARRIER_FAMILY, HConstants.SEQNUM_QUALIFIER)
+      .stream().mapToLong(MetaTableAccessor::getReplicationBarrier).sorted().distinct().toArray();
+  }
+
+  private static ReplicationBarrierResult getReplicationBarrierResult(Result result) {
+    long[] barriers = getReplicationBarriers(result);
+    byte[] stateBytes = result.getValue(getCatalogFamily(), getRegionStateColumn());
+    RegionState.State state =
+      stateBytes != null ? RegionState.State.valueOf(Bytes.toString(stateBytes)) : null;
+    byte[] parentRegionsBytes =
+      result.getValue(HConstants.REPLICATION_BARRIER_FAMILY, REPLICATION_PARENT_QUALIFIER);
+    List<byte[]> parentRegionNames =
+      parentRegionsBytes != null
+        ? Stream.of(Bytes.toString(parentRegionsBytes).split(REPLICATION_PARENT_SEPARATOR_REGEX))
+          .map(Bytes::toBytes).collect(Collectors.toList())
+        : Collections.emptyList();
+    return new ReplicationBarrierResult(barriers, state, parentRegionNames);
+  }
+
+  public static ReplicationBarrierResult getReplicationBarrierResult(Connection conn,
+      TableName tableName, byte[] row, byte[] encodedRegionName) throws IOException {
+    byte[] metaStartKey = RegionInfo.createRegionName(tableName, row, HConstants.NINES, false);
+    byte[] metaStopKey =
+      RegionInfo.createRegionName(tableName, HConstants.EMPTY_START_ROW, "", false);
+    Scan scan = new Scan().withStartRow(metaStartKey).withStopRow(metaStopKey)
+      .addColumn(getCatalogFamily(), getRegionStateColumn())
+      .addFamily(HConstants.REPLICATION_BARRIER_FAMILY).readAllVersions().setReversed(true)
+      .setCaching(10);
+    try (Table table = getMetaHTable(conn); ResultScanner scanner = table.getScanner(scan)) {
+      for (Result result;;) {
+        result = scanner.next();
+        if (result == null) {
+          return new ReplicationBarrierResult(new long[0], null, Collections.emptyList());
+        }
+        byte[] regionName = result.getRow();
+        // TODO: we may look up a region which has already been split or merged so we need to check
+        // whether the encoded name matches. Need to find a way to quit earlier when there is no
+        // record for the given region, for now it will scan to the end of the table.
+        if (!Bytes.equals(encodedRegionName,
+          Bytes.toBytes(RegionInfo.encodeRegionName(regionName)))) {
+          continue;
+        }
+        return getReplicationBarrierResult(result);
+      }
+    }
+  }
+
+  public static long[] getReplicationBarrier(Connection conn, byte[] regionName)
+      throws IOException {
+    try (Table table = getMetaHTable(conn)) {
+      Result result = table.get(new Get(regionName)
+        .addColumn(HConstants.REPLICATION_BARRIER_FAMILY, HConstants.SEQNUM_QUALIFIER)
+        .readAllVersions());
+      return getReplicationBarriers(result);
+    }
+  }
+
   private static void debugLogMutations(List<? extends Mutation> mutations) throws IOException {
     if (!METALOG.isDebugEnabled()) {
       return;

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
index 9456fd4..13ad0e2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
@@ -245,6 +245,11 @@ public interface TableDescriptor {
   boolean hasRegionMemStoreReplication();
 
   /**
+   * @return true if there are at least one cf whose replication scope is serial.
+   */
+  boolean hasSerialReplicationScope();
+
+  /**
    * Check if the compaction enable flag of the table is true. If flag is false
    * then no minor/major compactions will be done in real.
    *
@@ -292,7 +297,8 @@ public interface TableDescriptor {
     boolean hasDisabled = false;
 
     for (ColumnFamilyDescriptor cf : getColumnFamilies()) {
-      if (cf.getScope() != HConstants.REPLICATION_SCOPE_GLOBAL) {
+      if (cf.getScope() != HConstants.REPLICATION_SCOPE_GLOBAL &&
+        cf.getScope() != HConstants.REPLICATION_SCOPE_SERIAL) {
         hasDisabled = true;
       } else {
         hasEnabled = true;

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
index 02901ac..2d6bfaf 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
@@ -1128,6 +1128,15 @@ public class TableDescriptorBuilder {
     }
 
     /**
+     * Return true if there are at least one cf whose replication scope is serial.
+     */
+    @Override
+    public boolean hasSerialReplicationScope() {
+      return families.values().stream()
+        .anyMatch(column -> column.getScope() == HConstants.REPLICATION_SCOPE_SERIAL);
+    }
+
+    /**
      * Returns the configured replicas per region
      */
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index ac56ce5..edf8f9c 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -535,6 +535,12 @@ public final class HConstants {
   /** The serialized table state qualifier */
   public static final byte[] TABLE_STATE_QUALIFIER = Bytes.toBytes("state");
 
+  /** The replication barrier family as a string*/
+  public static final String REPLICATION_BARRIER_FAMILY_STR = "rep_barrier";
+
+  /** The replication barrier family */
+  public static final byte[] REPLICATION_BARRIER_FAMILY =
+      Bytes.toBytes(REPLICATION_BARRIER_FAMILY_STR);
 
   /**
    * The meta table version column qualifier.
@@ -676,6 +682,12 @@ public final class HConstants {
   public static final int REPLICATION_SCOPE_GLOBAL = 1;
 
   /**
+   * Scope tag for serially scoped data
+   * This data will be replicated to all peers by the order of sequence id.
+   */
+  public static final int REPLICATION_SCOPE_SERIAL = 2;
+
+  /**
    * Default cluster ID, cannot be used to identify a cluster so a key with
    * this value means it wasn't meant for replication.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
index a37fd4e..864be02 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterFileSystem.java
@@ -208,7 +208,16 @@ public class MasterFileSystem {
   /**
    * @return HBase root log dir.
    */
-  public Path getWALRootDir() { return this.walRootDir; }
+  public Path getWALRootDir() {
+    return this.walRootDir;
+  }
+
+  /**
+   * @return the directory for a give {@code region}.
+   */
+  public Path getRegionDir(RegionInfo region) {
+    return FSUtils.getRegionDir(FSUtils.getTableDir(getRootDir(), region.getTable()), region);
+  }
 
   /**
    * @return HBase temp dir.

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/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 754731b..0e47065 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
@@ -1571,8 +1571,7 @@ public class AssignmentManager implements ServerListener {
   }
 
   public void markRegionAsSplit(final RegionInfo parent, final ServerName serverName,
-      final RegionInfo daughterA, final RegionInfo daughterB)
-  throws IOException {
+      final RegionInfo daughterA, final RegionInfo daughterB) throws IOException {
     // Update hbase:meta. Parent will be marked offline and split up in hbase:meta.
     // The parent stays in regionStates until cleared when removed by CatalogJanitor.
     // Update its state in regionStates to it shows as offline and split when read

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
index 1eaa4c6..c98a2d1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
@@ -1,5 +1,4 @@
 /**
- *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -16,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.master.assignment;
 
 import java.io.IOException;
@@ -36,11 +34,13 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.procedure2.util.StringUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.wal.WALSplitter;
 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
@@ -163,6 +163,11 @@ public class RegionStateStore {
       Preconditions.checkArgument(state == State.OPEN && regionLocation != null,
           "Open region should be on a server");
       MetaTableAccessor.addLocation(put, regionLocation, openSeqNum, replicaId);
+      // only update replication barrier for default replica
+      if (regionInfo.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID &&
+        hasSerialReplicationScope(regionInfo.getTable())) {
+        MetaTableAccessor.addReplicationBarrier(put, openSeqNum);
+      }
       info.append(", openSeqNum=").append(openSeqNum);
       info.append(", regionLocation=").append(regionLocation);
     } else if (regionLocation != null && !regionLocation.equals(lastHost)) {
@@ -205,24 +210,41 @@ public class RegionStateStore {
     }
   }
 
+  private long getOpenSeqNumForParentRegion(RegionInfo region) throws IOException {
+    MasterFileSystem mfs = master.getMasterFileSystem();
+    long maxSeqId =
+        WALSplitter.getMaxRegionSequenceId(mfs.getFileSystem(), mfs.getRegionDir(region));
+    return maxSeqId > 0 ? maxSeqId + 1 : HConstants.NO_SEQNUM;
+  }
+
   // ============================================================================================
   //  Update Region Splitting State helpers
   // ============================================================================================
-  public void splitRegion(final RegionInfo parent, final RegionInfo hriA,
-      final RegionInfo hriB, final ServerName serverName)  throws IOException {
-    final TableDescriptor htd = getTableDescriptor(parent.getTable());
-    MetaTableAccessor.splitRegion(master.getConnection(), parent, hriA, hriB, serverName,
-        getRegionReplication(htd));
+  public void splitRegion(RegionInfo parent, RegionInfo hriA, RegionInfo hriB,
+      ServerName serverName) throws IOException {
+    TableDescriptor htd = getTableDescriptor(parent.getTable());
+    long parentOpenSeqNum = HConstants.NO_SEQNUM;
+    if (htd.hasSerialReplicationScope()) {
+      parentOpenSeqNum = getOpenSeqNumForParentRegion(parent);
+    }
+    MetaTableAccessor.splitRegion(master.getConnection(), parent, parentOpenSeqNum, hriA, hriB,
+      serverName, getRegionReplication(htd));
   }
 
   // ============================================================================================
   //  Update Region Merging State helpers
   // ============================================================================================
-  public void mergeRegions(final RegionInfo parent, final RegionInfo hriA, final RegionInfo hriB,
-      final ServerName serverName) throws IOException {
-    final TableDescriptor htd = getTableDescriptor(parent.getTable());
-    MetaTableAccessor.mergeRegions(master.getConnection(), parent, hriA, hriB, serverName,
-      getRegionReplication(htd));
+  public void mergeRegions(RegionInfo child, RegionInfo hriA, RegionInfo hriB,
+      ServerName serverName) throws IOException {
+    TableDescriptor htd = getTableDescriptor(child.getTable());
+    long regionAOpenSeqNum = -1L;
+    long regionBOpenSeqNum = -1L;
+    if (htd.hasSerialReplicationScope()) {
+      regionAOpenSeqNum = getOpenSeqNumForParentRegion(hriA);
+      regionBOpenSeqNum = getOpenSeqNumForParentRegion(hriB);
+    }
+    MetaTableAccessor.mergeRegions(master.getConnection(), child, hriA, regionAOpenSeqNum, hriB,
+      regionBOpenSeqNum, serverName, getRegionReplication(htd));
   }
 
   // ============================================================================================
@@ -239,11 +261,19 @@ public class RegionStateStore {
   // ==========================================================================
   //  Table Descriptors helpers
   // ==========================================================================
-  private int getRegionReplication(final TableDescriptor htd) {
-    return (htd != null) ? htd.getRegionReplication() : 1;
+  private boolean hasSerialReplicationScope(TableName tableName) throws IOException {
+    return hasSerialReplicationScope(getTableDescriptor(tableName));
+  }
+
+  private boolean hasSerialReplicationScope(TableDescriptor htd) {
+    return htd != null ? htd.hasSerialReplicationScope() : false;
+  }
+
+  private int getRegionReplication(TableDescriptor htd) {
+    return htd != null ? htd.getRegionReplication() : 1;
   }
 
-  private TableDescriptor getTableDescriptor(final TableName tableName) throws IOException {
+  private TableDescriptor getTableDescriptor(TableName tableName) throws IOException {
     return master.getTableDescriptors().get(tableName);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
index 994983f..341affb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
@@ -253,7 +253,7 @@ public class SplitTableRegionProcedure
           setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_UPDATE_META);
           break;
         case SPLIT_TABLE_REGION_UPDATE_META:
-          updateMetaForDaughterRegions(env);
+          updateMeta(env);
           setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_META);
           break;
         case SPLIT_TABLE_REGION_PRE_OPERATION_AFTER_META:
@@ -762,7 +762,7 @@ public class SplitTableRegionProcedure
    * Add daughter regions to META
    * @param env MasterProcedureEnv
    */
-  private void updateMetaForDaughterRegions(final MasterProcedureEnv env) throws IOException {
+  private void updateMeta(final MasterProcedureEnv env) throws IOException {
     env.getAssignmentManager().markRegionAsSplit(getParentRegion(), getParentRegionServerName(env),
       daughter_1_RI, daughter_2_RI);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java
index 60436c2..d296828 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AbstractStateMachineTableProcedure.java
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.master.procedure;
 
 import java.io.IOException;
@@ -31,15 +30,12 @@ import org.apache.hadoop.hbase.client.DoNotRetryRegionException;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionOfflineException;
 import org.apache.hadoop.hbase.client.TableState;
-import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.MasterServices;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.master.TableStateManager;
 import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -131,9 +127,7 @@ public abstract class AbstractStateMachineTableProcedure<TState>
   }
 
   protected final Path getRegionDir(MasterProcedureEnv env, RegionInfo region) throws IOException {
-    MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
-    Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), getTableName());
-    return new Path(tableDir, ServerRegionReplicaUtil.getRegionInfoForFs(region).getEncodedName());
+    return env.getMasterServices().getMasterFileSystem().getRegionDir(region);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
index 37a4309..9666aa5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
@@ -1,5 +1,4 @@
 /**
- *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -16,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.regionserver;
 
 import java.io.FileNotFoundException;
@@ -25,6 +23,7 @@ import java.io.InterruptedIOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
+import java.util.Objects;
 import java.util.Optional;
 import java.util.UUID;
 import org.apache.hadoop.conf.Configuration;
@@ -84,6 +83,7 @@ public class HRegionFileSystem {
   private final Configuration conf;
   private final Path tableDir;
   private final FileSystem fs;
+  private final Path regionDir;
 
   /**
    * In order to handle NN connectivity hiccups, one need to retry non-idempotent operation at the
@@ -105,9 +105,10 @@ public class HRegionFileSystem {
       final RegionInfo regionInfo) {
     this.fs = fs;
     this.conf = conf;
-    this.tableDir = tableDir;
-    this.regionInfo = regionInfo;
+    this.tableDir = Objects.requireNonNull(tableDir, "tableDir is null");
+    this.regionInfo = Objects.requireNonNull(regionInfo, "regionInfo is null");
     this.regionInfoForFs = ServerRegionReplicaUtil.getRegionInfoForFs(regionInfo);
+    this.regionDir = FSUtils.getRegionDir(tableDir, regionInfo);
     this.hdfsClientRetriesNumber = conf.getInt("hdfs.client.retries.number",
       DEFAULT_HDFS_CLIENT_RETRIES_NUMBER);
     this.baseSleepBeforeRetries = conf.getInt("hdfs.client.sleep.before.retries",
@@ -135,7 +136,7 @@ public class HRegionFileSystem {
 
   /** @return {@link Path} to the region directory. */
   public Path getRegionDir() {
-    return new Path(this.tableDir, this.regionInfoForFs.getEncodedName());
+    return regionDir;
   }
 
   // ===========================================================================

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/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 ad6e5a6..08c9f37 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
@@ -21,16 +21,13 @@ package org.apache.hadoop.hbase.replication;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Filter a WAL Entry by the peer config: replicate_all flag, namespaces config, table-cfs config,
@@ -47,7 +44,6 @@ import org.apache.hadoop.hbase.wal.WAL.Entry;
 @InterfaceAudience.Private
 public class NamespaceTableCfWALEntryFilter implements WALEntryFilter, WALCellFilter {
 
-  private static final Logger LOG = LoggerFactory.getLogger(NamespaceTableCfWALEntryFilter.class);
   private final ReplicationPeer peer;
   private BulkLoadCellFilter bulkLoadFilter = new BulkLoadCellFilter();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ScopeWALEntryFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ScopeWALEntryFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ScopeWALEntryFilter.java
index 5cde40c..6a2fbcf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ScopeWALEntryFilter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ScopeWALEntryFilter.java
@@ -15,17 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.hbase.replication;
 
 import java.util.NavigableMap;
-
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WALEdit;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import org.apache.hbase.thirdparty.com.google.common.base.Predicate;
 
@@ -35,7 +33,7 @@ import org.apache.hbase.thirdparty.com.google.common.base.Predicate;
 @InterfaceAudience.Private
 public class ScopeWALEntryFilter implements WALEntryFilter, WALCellFilter {
 
-  BulkLoadCellFilter bulkLoadFilter = new BulkLoadCellFilter();
+  private final BulkLoadCellFilter bulkLoadFilter = new BulkLoadCellFilter();
 
   @Override
   public Entry filter(Entry entry) {
@@ -49,21 +47,21 @@ public class ScopeWALEntryFilter implements WALEntryFilter, WALCellFilter {
   @Override
   public Cell filterCell(Entry entry, Cell cell) {
     final NavigableMap<byte[], Integer> scopes = entry.getKey().getReplicationScopes();
-      // The scope will be null or empty if
-      // there's nothing to replicate in that WALEdit
-      byte[] fam = CellUtil.cloneFamily(cell);
-      if (CellUtil.matchingColumn(cell, WALEdit.METAFAMILY, WALEdit.BULK_LOAD)) {
-        cell = bulkLoadFilter.filterCell(cell, new Predicate<byte[]>() {
-          @Override
-          public boolean apply(byte[] fam) {
-            return !scopes.containsKey(fam) || scopes.get(fam) == HConstants.REPLICATION_SCOPE_LOCAL;
-          }
-        });
-      } else {
-        if (!scopes.containsKey(fam) || scopes.get(fam) == HConstants.REPLICATION_SCOPE_LOCAL) {
-          return null;
+    // The scope will be null or empty if
+    // there's nothing to replicate in that WALEdit
+    byte[] fam = CellUtil.cloneFamily(cell);
+    if (CellUtil.matchingColumn(cell, WALEdit.METAFAMILY, WALEdit.BULK_LOAD)) {
+      cell = bulkLoadFilter.filterCell(cell, new Predicate<byte[]>() {
+        @Override
+        public boolean apply(byte[] fam) {
+          return !scopes.containsKey(fam) || scopes.get(fam) == HConstants.REPLICATION_SCOPE_LOCAL;
         }
+      });
+    } else {
+      if (!scopes.containsKey(fam) || scopes.get(fam) == HConstants.REPLICATION_SCOPE_LOCAL) {
+        return null;
       }
+    }
     return cell;
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
index 3cae0f2..d9506c0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
@@ -194,4 +194,9 @@ public class RecoveredReplicationSource extends ReplicationSource {
   public ServerName getServerWALsBelongTo() {
     return this.replicationQueueInfo.getDeadRegionServers().get(0);
   }
+
+  @Override
+  public boolean isRecovered() {
+    return true;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
index 38bbb48..9c36497 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -20,12 +19,10 @@ package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.io.IOException;
 import java.util.concurrent.PriorityBlockingQueue;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
-import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.WALEntryBatch;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -127,13 +124,6 @@ public class RecoveredReplicationSourceShipper extends ReplicationSourceShipper
     return startPosition;
   }
 
-  @Override
-  protected void updateLogPosition(long lastReadPosition) {
-    source.getSourceManager().logPositionAndCleanOldLogs(currentPath, source.getQueueId(),
-      lastReadPosition, true);
-    lastLoggedPosition = lastReadPosition;
-  }
-
   private void terminate(String reason, Exception cause) {
     if (cause == null) {
       LOG.info("Closing worker for wal group " + this.walGroupId + " because: " + reason);

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceWALReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceWALReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceWALReader.java
index 0af3f5c..114f139 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceWALReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceWALReader.java
@@ -35,8 +35,9 @@ import org.apache.hadoop.hbase.replication.WALEntryFilter;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class RecoveredReplicationSourceWALReader extends ReplicationSourceWALReader {
+
   private static final Logger LOG =
-      LoggerFactory.getLogger(RecoveredReplicationSourceWALReader.class);
+    LoggerFactory.getLogger(RecoveredReplicationSourceWALReader.class);
 
   public RecoveredReplicationSourceWALReader(FileSystem fs, Configuration conf,
       PriorityBlockingQueue<Path> logQueue, long startPosition, WALEntryFilter filter,
@@ -45,13 +46,11 @@ public class RecoveredReplicationSourceWALReader extends ReplicationSourceWALRea
   }
 
   @Override
-  protected void handleEmptyWALEntryBatch(WALEntryBatch batch, Path currentPath)
-      throws InterruptedException {
+  protected void handleEmptyWALEntryBatch(Path currentPath) throws InterruptedException {
     LOG.trace("Didn't read any new entries from WAL");
     // we're done with queue recovery, shut ourself down
     setReaderRunning(false);
     // shuts down shipper thread immediately
-    entryBatchQueue.put(batch != null ? batch
-        : new WALEntryBatch(replicationBatchCountCapacity, currentPath));
+    entryBatchQueue.put(new WALEntryBatch(replicationBatchCountCapacity, currentPath));
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index 73d4652..3b65b25 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -607,4 +607,12 @@ public class ReplicationSource implements ReplicationSourceInterface {
   public ServerName getServerWALsBelongTo() {
     return server.getServerName();
   }
+
+  Server getServer() {
+    return server;
+  }
+
+  ReplicationQueueStorage getQueueStorage() {
+    return queueStorage;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
index d7cf9a3..090b465 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceInterface.java
@@ -166,4 +166,11 @@ public interface ReplicationSourceInterface {
    * @return the server name which all WALs belong to
    */
   ServerName getServerWALsBelongTo();
+
+  /**
+   * @return whether this is a replication source for recovery.
+   */
+  default boolean isRecovered() {
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/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 eb9dba2..06fe977 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
@@ -480,10 +480,10 @@ public class ReplicationSourceManager implements ReplicationListener {
    * @param queueRecovered indicates if this queue comes from another region server
    */
   public void logPositionAndCleanOldLogs(Path log, String queueId, long position,
-      boolean queueRecovered) {
+      Map<String, Long> lastSeqIds, boolean queueRecovered) {
     String fileName = log.getName();
     abortWhenFail(() -> this.queueStorage.setWALPosition(server.getServerName(), queueId, fileName,
-      position, null));
+      position, lastSeqIds));
     cleanOldLogs(fileName, queueId, queueRecovered);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
index 959f676..d207d77 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
@@ -1,5 +1,4 @@
-/*
- *
+/**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -20,13 +19,13 @@ package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.PriorityBlockingQueue;
 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.replication.ReplicationEndpoint;
-import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.WALEntryBatch;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
@@ -128,7 +127,7 @@ public class ReplicationSourceShipper extends Thread {
     int sleepMultiplier = 0;
     if (entries.isEmpty()) {
       if (lastLoggedPosition != lastReadPosition) {
-        updateLogPosition(lastReadPosition);
+        updateLogPosition(lastReadPosition, entryBatch.getLastSeqIds());
         // if there was nothing to ship and it's not an error
         // set "ageOfLastShippedOp" to <now> to indicate that we're current
         source.getSourceMetrics().setAgeOfLastShippedOp(EnvironmentEdgeManager.currentTime(),
@@ -168,13 +167,13 @@ public class ReplicationSourceShipper extends Thread {
         }
 
         if (this.lastLoggedPosition != lastReadPosition) {
-          //Clean up hfile references
+          // Clean up hfile references
           int size = entries.size();
           for (int i = 0; i < size; i++) {
             cleanUpHFileRefs(entries.get(i).getEdit());
           }
-          //Log and clean up WAL logs
-          updateLogPosition(lastReadPosition);
+          // Log and clean up WAL logs
+          updateLogPosition(lastReadPosition, entryBatch.getLastSeqIds());
         }
 
         source.postShipEdits(entries, currentSize);
@@ -222,9 +221,9 @@ public class ReplicationSourceShipper extends Thread {
     }
   }
 
-  protected void updateLogPosition(long lastReadPosition) {
+  private void updateLogPosition(long lastReadPosition, Map<String, Long> lastSeqIds) {
     source.getSourceManager().logPositionAndCleanOldLogs(currentPath, source.getQueueId(),
-      lastReadPosition, false);
+      lastReadPosition, lastSeqIds, source.isRecovered());
     lastLoggedPosition = lastReadPosition;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/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
index eb12614..95fc6a0 100644
--- 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
@@ -20,7 +20,6 @@ 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;
@@ -31,8 +30,6 @@ 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.
  */
@@ -68,31 +65,25 @@ class ReplicationSourceWALActionListener implements WALActionsListener {
    * 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;
-      }
+  static void scopeWALEdits(WALKey logKey, WALEdit logEdit, Configuration conf) {
+    // For bulk load replication we need meta family to know the file we want to replicate.
+    if (ReplicationUtils.isReplicationForBulkLoadDataEnabled(conf)) {
+      return;
     }
-
-    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;
-      }
+    WALKeyImpl keyImpl = (WALKeyImpl) logKey;
+    // For serial replication we need to count all the sequence ids even for markers, so here we
+    // always need to retain the replication scopes to let the replication wal reader to know that
+    // we need serial replication. The ScopeWALEntryFilter will help filtering out the cell for
+    // WALEdit.METAFAMILY.
+    if (keyImpl.hasSerialReplicationScope()) {
+      return;
     }
-    if ((!replicationForBulkLoadEnabled && !foundOtherEdits) || logEdit.isReplay()) {
-      ((WALKeyImpl) logKey).serializeReplicationScope(false);
+    // For replay, or if all the cells are markers, do not need to store replication scope.
+    if (logEdit.isReplay() ||
+      logEdit.getCells().stream().allMatch(c -> CellUtil.matchingFamily(c, WALEdit.METAFAMILY))) {
+      keyImpl.clearReplicationScope();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
index 579d20f..fe87aec 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.io.EOFException;
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
@@ -33,6 +32,7 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.replication.WALEntryFilter;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
@@ -46,8 +46,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.BulkLoadDescr
 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
 
 /**
- * Reads and filters WAL entries, groups the filtered entries into batches, and puts the batches onto a queue
- *
+ * Reads and filters WAL entries, groups the filtered entries into batches, and puts the batches
+ * onto a queue
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
@@ -77,6 +77,8 @@ public class ReplicationSourceWALReader extends Thread {
   private AtomicLong totalBufferUsed;
   private long totalBufferQuota;
 
+  private final SerialReplicationChecker serialReplicationChecker;
+
   /**
    * Creates a reader worker for a given WAL queue. Reads WAL entries off a given queue, batches the
    * entries, and puts them on a batch queue.
@@ -111,6 +113,7 @@ public class ReplicationSourceWALReader extends Thread {
         this.conf.getInt("replication.source.maxretriesmultiplier", 300); // 5 minutes @ 1 sec per
     this.eofAutoRecovery = conf.getBoolean("replication.source.eof.autorecovery", false);
     this.entryBatchQueue = new LinkedBlockingQueue<>(batchCount);
+    this.serialReplicationChecker = new SerialReplicationChecker(conf, source);
     LOG.info("peerClusterZnode=" + source.getQueueId()
         + ", ReplicationSourceWALReaderThread : " + source.getPeerId()
         + " inited, replicationBatchSizeCapacity=" + replicationBatchSizeCapacity
@@ -131,15 +134,14 @@ public class ReplicationSourceWALReader extends Thread {
             continue;
           }
           WALEntryBatch batch = readWALEntries(entryStream);
-          if (batch != null && batch.getNbEntries() > 0) {
-            if (LOG.isTraceEnabled()) {
-              LOG.trace(String.format("Read %s WAL entries eligible for replication",
-                batch.getNbEntries()));
-            }
+          if (batch != null) {
+            // need to propagate the batch even it has no entries since it may carry the last
+            // sequence id information for serial replication.
+            LOG.trace("Read {} WAL entries eligible for replication", batch.getNbEntries());
             entryBatchQueue.put(batch);
             sleepMultiplier = 1;
           } else { // got no entries and didn't advance position in WAL
-            handleEmptyWALEntryBatch(batch, entryStream.getCurrentPath());
+            handleEmptyWALEntryBatch(entryStream.getCurrentPath());
           }
           currentPosition = entryStream.getPosition();
           entryStream.reset(); // reuse stream
@@ -160,34 +162,66 @@ public class ReplicationSourceWALReader extends Thread {
     }
   }
 
-  private WALEntryBatch readWALEntries(WALEntryStream entryStream) throws IOException {
-    WALEntryBatch batch = null;
-    while (entryStream.hasNext()) {
-      if (batch == null) {
-        batch = new WALEntryBatch(replicationBatchCountCapacity, entryStream.getCurrentPath());
+  private WALEntryBatch readWALEntries(WALEntryStream entryStream)
+      throws IOException, InterruptedException {
+    if (!entryStream.hasNext()) {
+      return null;
+    }
+    WALEntryBatch batch =
+      new WALEntryBatch(replicationBatchCountCapacity, entryStream.getCurrentPath());
+    do {
+      Entry entry = entryStream.peek();
+      batch.setLastWalPosition(entryStream.getPosition());
+      boolean hasSerialReplicationScope = entry.getKey().hasSerialReplicationScope();
+      // Used to locate the region record in meta table. In WAL we only have the table name and
+      // encoded region name which can not be mapping to region name without scanning all the
+      // records for a table, so we need a start key, just like what we have done at client side
+      // when locating a region. For the markers, we will use the start key of the region as the row
+      // key for the edit. And we need to do this before filtering since all the cells may be
+      // filtered out, especially that for the markers.
+      Cell firstCellInEdit = null;
+      if (hasSerialReplicationScope) {
+        assert !entry.getEdit().isEmpty() : "should not write empty edits";
+        firstCellInEdit = entry.getEdit().getCells().get(0);
       }
-      Entry entry = entryStream.next();
       entry = filterEntry(entry);
       if (entry != null) {
+        if (hasSerialReplicationScope) {
+          if (!serialReplicationChecker.canPush(entry, firstCellInEdit)) {
+            if (batch.getNbEntries() > 0) {
+              // we have something that can push, break
+              break;
+            } else {
+              serialReplicationChecker.waitUntilCanPush(entry, firstCellInEdit);
+            }
+          }
+          // arrive here means we can push the entry, record the last sequence id
+          batch.setLastSeqId(Bytes.toString(entry.getKey().getEncodedRegionName()),
+            entry.getKey().getSequenceId());
+        }
+        // actually remove the entry.
+        entryStream.next();
         WALEdit edit = entry.getEdit();
         if (edit != null && !edit.isEmpty()) {
           long entrySize = getEntrySize(entry);
           batch.addEntry(entry);
-          updateBatchStats(batch, entry, entryStream.getPosition(), entrySize);
+          updateBatchStats(batch, entry, entrySize);
           boolean totalBufferTooLarge = acquireBufferQuota(entrySize);
           // Stop if too many entries or too big
-          if (totalBufferTooLarge || batch.getHeapSize() >= replicationBatchSizeCapacity
-              || batch.getNbEntries() >= replicationBatchCountCapacity) {
+          if (totalBufferTooLarge || batch.getHeapSize() >= replicationBatchSizeCapacity ||
+            batch.getNbEntries() >= replicationBatchCountCapacity) {
             break;
           }
         }
+      } else {
+        // actually remove the entry.
+        entryStream.next();
       }
-    }
+    } while (entryStream.hasNext());
     return batch;
   }
 
-  protected void handleEmptyWALEntryBatch(WALEntryBatch batch, Path currentPath)
-      throws InterruptedException {
+  protected void handleEmptyWALEntryBatch(Path currentPath) throws InterruptedException {
     LOG.trace("Didn't read any new entries from WAL");
     Thread.sleep(sleepForRetries);
   }
@@ -214,7 +248,7 @@ public class ReplicationSourceWALReader extends Thread {
     // if we've read some WAL entries, get the Path we read from
     WALEntryBatch batchQueueHead = entryBatchQueue.peek();
     if (batchQueueHead != null) {
-      return batchQueueHead.lastWalPath;
+      return batchQueueHead.getLastWalPath();
     }
     // otherwise, we must be currently reading from the head of the log queue
     return logQueue.peek();
@@ -253,15 +287,12 @@ public class ReplicationSourceWALReader extends Thread {
     return edit.heapSize() + calculateTotalSizeOfStoreFiles(edit);
   }
 
-  private void updateBatchStats(WALEntryBatch batch, Entry entry, long entryPosition, long entrySize) {
+  private void updateBatchStats(WALEntryBatch batch, Entry entry, long entrySize) {
     WALEdit edit = entry.getEdit();
-    if (edit != null && !edit.isEmpty()) {
-      batch.incrementHeapSize(entrySize);
-      Pair<Integer, Integer> nbRowsAndHFiles = countDistinctRowKeysAndHFiles(edit);
-      batch.incrementNbRowKeys(nbRowsAndHFiles.getFirst());
-      batch.incrementNbHFiles(nbRowsAndHFiles.getSecond());
-    }
-    batch.lastWalPosition = entryPosition;
+    batch.incrementHeapSize(entrySize);
+    Pair<Integer, Integer> nbRowsAndHFiles = countDistinctRowKeysAndHFiles(edit);
+    batch.incrementNbRowKeys(nbRowsAndHFiles.getFirst());
+    batch.incrementNbHFiles(nbRowsAndHFiles.getSecond());
   }
 
   /**
@@ -355,101 +386,4 @@ public class ReplicationSourceWALReader extends Thread {
   public void setReaderRunning(boolean readerRunning) {
     this.isReaderRunning = readerRunning;
   }
-
-  /**
-   * Holds a batch of WAL entries to replicate, along with some statistics
-   *
-   */
-  static class WALEntryBatch {
-    private List<Entry> walEntries;
-    // last WAL that was read
-    private Path lastWalPath;
-    // position in WAL of last entry in this batch
-    private long lastWalPosition = 0;
-    // number of distinct row keys in this batch
-    private int nbRowKeys = 0;
-    // number of HFiles
-    private int nbHFiles = 0;
-    // heap size of data we need to replicate
-    private long heapSize = 0;
-
-    /**
-     * @param walEntries
-     * @param lastWalPath Path of the WAL the last entry in this batch was read from
-     * @param lastWalPosition Position in the WAL the last entry in this batch was read from
-     */
-    WALEntryBatch(int maxNbEntries, Path lastWalPath) {
-      this.walEntries = new ArrayList<>(maxNbEntries);
-      this.lastWalPath = lastWalPath;
-    }
-
-    public void addEntry(Entry entry) {
-      walEntries.add(entry);
-    }
-
-    /**
-     * @return the WAL Entries.
-     */
-    public List<Entry> getWalEntries() {
-      return walEntries;
-    }
-
-    /**
-     * @return the path of the last WAL that was read.
-     */
-    public Path getLastWalPath() {
-      return lastWalPath;
-    }
-
-    /**
-     * @return the position in the last WAL that was read.
-     */
-    public long getLastWalPosition() {
-      return lastWalPosition;
-    }
-
-    public int getNbEntries() {
-      return walEntries.size();
-    }
-
-    /**
-     * @return the number of distinct row keys in this batch
-     */
-    public int getNbRowKeys() {
-      return nbRowKeys;
-    }
-
-    /**
-     * @return the number of HFiles in this batch
-     */
-    public int getNbHFiles() {
-      return nbHFiles;
-    }
-
-    /**
-     * @return total number of operations in this batch
-     */
-    public int getNbOperations() {
-      return getNbRowKeys() + getNbHFiles();
-    }
-
-    /**
-     * @return the heap size of this batch
-     */
-    public long getHeapSize() {
-      return heapSize;
-    }
-
-    private void incrementNbRowKeys(int increment) {
-      nbRowKeys += increment;
-    }
-
-    private void incrementNbHFiles(int increment) {
-      nbHFiles += increment;
-    }
-
-    private void incrementHeapSize(long increment) {
-      heapSize += increment;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SerialReplicationChecker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SerialReplicationChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SerialReplicationChecker.java
new file mode 100644
index 0000000..95f3868
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SerialReplicationChecker.java
@@ -0,0 +1,255 @@
+/**
+ * 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 java.util.Arrays;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.hadoop.conf.Configuration;
+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.MetaTableAccessor.ReplicationBarrierResult;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import org.apache.hbase.thirdparty.com.google.common.cache.Cache;
+import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
+import org.apache.hbase.thirdparty.com.google.common.cache.CacheLoader;
+import org.apache.hbase.thirdparty.com.google.common.cache.LoadingCache;
+
+/**
+ * <p>
+ * Helper class to determine whether we can push a given WAL entry without breaking the replication
+ * order. The class is designed to per {@link ReplicationSourceWALReader}, so not thread safe.
+ * </p>
+ * <p>
+ * We record all the open sequence number for a region in a special family in meta, which is called
+ * 'barrier', so there will be a sequence of open sequence number (b1, b2, b3, ...). We call [bn,
+ * bn+1) a range, and it is obvious that a region will always be on the same RS within a range.
+ * <p>
+ * When split and merge, we will also record the parent for the generated region(s) in the special
+ * family in meta. And also, we will write an extra 'open sequence number' for the parent region(s),
+ * which is the max sequence id of the region plus one.
+ * </p>
+ * </p>
+ * <p>
+ * For each peer, we record the last pushed sequence id for each region. It is managed by the
+ * replication storage.
+ * </p>
+ * <p>
+ * The algorithm works like this:
+ * <ol>
+ * <li>Locate the sequence id we want to push in the barriers</li>
+ * <li>If it is before the first barrier, we are safe to push. This usually because we enable serial
+ * replication for this table after we create the table and write data into the table.</li>
+ * <li>In general, if the previous range is finished, then we are safe to push. The way to determine
+ * whether a range is finish is straight-forward: check whether the last pushed sequence id is equal
+ * to the end barrier of the range minus 1. There are several exceptions:
+ * <ul>
+ * <li>If it is in the first range, we need to check whether there are parent regions. If so, we
+ * need to make sure that the data for parent regions have all been pushed.</li>
+ * <li>If it is in the last range, we need to check the region state. If state is OPENING, then we
+ * are not safe to push. This is because that, before we call reportRIT to master which update the
+ * open sequence number into meta table, we will write a open region event marker to WAL first, and
+ * its sequence id is greater than the newest open sequence number(which has not been updated to
+ * meta table yet so we do not know). For this scenario, the WAL entry for this open region event
+ * marker actually belongs to the range after the 'last' range, so we are not safe to push it.
+ * Otherwise the last pushed sequence id will be updated to this value and then we think the
+ * previous range has already been finished, but this is not true.</li>
+ * <li>Notice that the above two exceptions are not conflicts, since the first range can also be the
+ * last range if we only have one range.</li>
+ * </ul>
+ * </li>
+ * </ol>
+ * </p>
+ * <p>
+ * And for performance reason, we do not want to check meta for every WAL entry, so we introduce two
+ * in memory maps. The idea is simple:
+ * <ul>
+ * <li>If a range can be pushed, then put its end barrier into the {@code canPushUnder} map.</li>
+ * <li>Before accessing meta, first check the sequence id stored in the {@code canPushUnder} map. If
+ * the sequence id of WAL entry is less the one stored in {@code canPushUnder} map, then we are safe
+ * to push.</li>
+ * </ul>
+ * And for the last range, we do not have an end barrier, so we use the continuity of sequence id to
+ * determine whether we can push. The rule is:
+ * <ul>
+ * <li>When an entry is able to push, then put its sequence id into the {@code pushed} map.</li>
+ * <li>Check if the sequence id of WAL entry equals to the one stored in the {@code pushed} map plus
+ * one. If so, we are safe to push, and also update the {@code pushed} map with the sequence id of
+ * the WAL entry.</li>
+ * </ul>
+ * </p>
+ */
+@InterfaceAudience.Private
+class SerialReplicationChecker {
+
+  public static final String REPLICATION_SERIALLY_WAITING_KEY =
+    "hbase.serial.replication.waiting.ms";
+  public static final long REPLICATION_SERIALLY_WAITING_DEFAULT = 10000;
+
+  private final String peerId;
+
+  private final ReplicationQueueStorage storage;
+
+  private final Connection conn;
+
+  private final long waitTimeMs;
+
+  private final LoadingCache<String, MutableLong> pushed = CacheBuilder.newBuilder()
+    .expireAfterAccess(1, TimeUnit.DAYS).build(new CacheLoader<String, MutableLong>() {
+
+      @Override
+      public MutableLong load(String key) throws Exception {
+        return new MutableLong(HConstants.NO_SEQNUM);
+      }
+    });
+
+  // Use guava cache to set ttl for each key
+  private final Cache<String, Long> canPushUnder =
+    CacheBuilder.newBuilder().expireAfterAccess(1, TimeUnit.DAYS).build();
+
+  public SerialReplicationChecker(Configuration conf, ReplicationSource source) {
+    this.peerId = source.getPeerId();
+    this.storage = source.getQueueStorage();
+    this.conn = source.getServer().getConnection();
+    this.waitTimeMs =
+      conf.getLong(REPLICATION_SERIALLY_WAITING_KEY, REPLICATION_SERIALLY_WAITING_DEFAULT);
+  }
+
+  private boolean isRangeFinished(long endBarrier, String encodedRegionName) throws IOException {
+    long pushedSeqId;
+    try {
+      pushedSeqId = storage.getLastSequenceId(encodedRegionName, peerId);
+    } catch (ReplicationException e) {
+      throw new IOException(
+        "Failed to get pushed sequence id for " + encodedRegionName + ", peer " + peerId, e);
+    }
+    // endBarrier is the open sequence number. When opening a region, the open sequence number will
+    // be set to the old max sequence id plus one, so here we need to minus one.
+    return pushedSeqId >= endBarrier - 1;
+  }
+
+  private boolean isParentFinished(byte[] regionName) throws IOException {
+    long[] barriers = MetaTableAccessor.getReplicationBarrier(conn, regionName);
+    if (barriers.length == 0) {
+      return true;
+    }
+    return isRangeFinished(barriers[barriers.length - 1], RegionInfo.encodeRegionName(regionName));
+  }
+
+  // We may write a open region marker to WAL before we write the open sequence number to meta, so
+  // if a region is in OPENING state and we are in the last range, it is not safe to say we can push
+  // even if the previous range is finished.
+  private boolean isLastRangeAndOpening(ReplicationBarrierResult barrierResult, int index) {
+    return index == barrierResult.getBarriers().length &&
+      barrierResult.getState() == RegionState.State.OPENING;
+  }
+
+  private void recordCanPush(String encodedNameAsString, long seqId, long[] barriers, int index) {
+    if (barriers.length > index) {
+      canPushUnder.put(encodedNameAsString, barriers[index]);
+    }
+    pushed.getUnchecked(encodedNameAsString).setValue(seqId);
+  }
+
+  private boolean canPush(Entry entry, byte[] row) throws IOException {
+    String encodedNameAsString = Bytes.toString(entry.getKey().getEncodedRegionName());
+    long seqId = entry.getKey().getSequenceId();
+    ReplicationBarrierResult barrierResult = MetaTableAccessor.getReplicationBarrierResult(conn,
+      entry.getKey().getTableName(), row, entry.getKey().getEncodedRegionName());
+    long[] barriers = barrierResult.getBarriers();
+    int index = Arrays.binarySearch(barriers, seqId);
+    if (index == -1) {
+      // This means we are in the range before the first record openSeqNum, this usually because the
+      // wal is written before we enable serial replication for this table, just return true since
+      // we can not guarantee the order.
+      pushed.getUnchecked(encodedNameAsString).setValue(seqId);
+      return true;
+    }
+    // The sequence id range is left closed and right open, so either we decrease the missed insert
+    // point to make the index start from 0, or increase the hit insert point to make the index
+    // start from 1. Here we choose the latter one.
+    if (index < 0) {
+      index = -index - 1;
+    } else {
+      index++;
+    }
+    if (index == 1) {
+      // we are in the first range, check whether we have parents
+      for (byte[] regionName : barrierResult.getParentRegionNames()) {
+        if (!isParentFinished(regionName)) {
+          return false;
+        }
+      }
+      if (isLastRangeAndOpening(barrierResult, index)) {
+        return false;
+      }
+      recordCanPush(encodedNameAsString, seqId, barriers, 1);
+      return true;
+    }
+    // check whether the previous range is finished
+    if (!isRangeFinished(barriers[index - 1], encodedNameAsString)) {
+      return false;
+    }
+    if (isLastRangeAndOpening(barrierResult, index)) {
+      return false;
+    }
+    recordCanPush(encodedNameAsString, seqId, barriers, index);
+    return true;
+  }
+
+  public boolean canPush(Entry entry, Cell firstCellInEdit) throws IOException {
+    String encodedNameAsString = Bytes.toString(entry.getKey().getEncodedRegionName());
+    long seqId = entry.getKey().getSequenceId();
+    Long canReplicateUnderSeqId = canPushUnder.getIfPresent(encodedNameAsString);
+    if (canReplicateUnderSeqId != null) {
+      if (seqId < canReplicateUnderSeqId.longValue()) {
+        return true;
+      }
+      // we are already beyond the last safe point, remove
+      canPushUnder.invalidate(encodedNameAsString);
+    }
+    // This is for the case where the region is currently opened on us, if the sequence id is
+    // continuous then we are safe to replicate. If there is a breakpoint, then maybe the region
+    // has been moved to another RS and then back, so we need to check the barrier.
+    MutableLong previousPushedSeqId = pushed.getUnchecked(encodedNameAsString);
+    if (seqId == previousPushedSeqId.longValue() + 1) {
+      previousPushedSeqId.increment();
+      return true;
+    }
+    return canPush(entry, CellUtil.cloneRow(firstCellInEdit));
+  }
+
+  public void waitUntilCanPush(Entry entry, Cell firstCellInEdit)
+      throws IOException, InterruptedException {
+    byte[] row = CellUtil.cloneRow(firstCellInEdit);
+    while (!canPush(entry, row)) {
+      Thread.sleep(waitTimeMs);
+    }
+  }
+}


[11/22] hbase git commit: HBASE-20167 Optimize the implementation of ReplicationSourceWALReader

Posted by zh...@apache.org.
HBASE-20167 Optimize the implementation of ReplicationSourceWALReader


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

Branch: refs/heads/HBASE-20046-branch-2
Commit: 2d0d6a3ba1bdeac37e898d37d41eb6b079fc9a6d
Parents: cea5199
Author: zhangduo <zh...@apache.org>
Authored: Mon Mar 12 12:21:44 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Apr 9 15:18:44 2018 +0800

----------------------------------------------------------------------
 .../RecoveredReplicationSource.java             |  67 +++++------
 .../RecoveredReplicationSourceShipper.java      |  48 ++------
 .../RecoveredReplicationSourceWALReader.java    |  56 ----------
 .../regionserver/ReplicationSource.java         |  36 +++---
 .../regionserver/ReplicationSourceShipper.java  |  27 +++--
 .../ReplicationSourceWALReader.java             | 101 +++++------------
 .../SerialReplicationSourceWALReader.java       | 112 +++++++++++++++++++
 7 files changed, 218 insertions(+), 229 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2d0d6a3b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
index d9506c0..169b469 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
@@ -20,8 +20,8 @@ package org.apache.hadoop.hbase.replication.regionserver;
 import java.io.IOException;
 import java.util.List;
 import java.util.UUID;
+import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.PriorityBlockingQueue;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -59,31 +59,41 @@ public class RecoveredReplicationSource extends ReplicationSource {
   }
 
   @Override
-  protected void tryStartNewShipper(String walGroupId, PriorityBlockingQueue<Path> queue) {
-    final RecoveredReplicationSourceShipper worker =
-        new RecoveredReplicationSourceShipper(conf, walGroupId, queue, this,
-            this.queueStorage);
-    ReplicationSourceShipper extant = workerThreads.putIfAbsent(walGroupId, worker);
-    if (extant != null) {
-      LOG.debug("Someone has beat us to start a worker thread for wal group " + walGroupId);
-    } else {
-      LOG.debug("Starting up worker for wal group " + walGroupId);
-      worker.startup(this::uncaughtException);
-      worker.setWALReader(
-        startNewWALReader(worker.getName(), walGroupId, queue, worker.getStartPosition()));
-      workerThreads.put(walGroupId, worker);
-    }
+  protected RecoveredReplicationSourceShipper createNewShipper(String walGroupId,
+      PriorityBlockingQueue<Path> queue) {
+    return new RecoveredReplicationSourceShipper(conf, walGroupId, queue, this, queueStorage);
+  }
+
+  private void handleEmptyWALEntryBatch0(ReplicationSourceWALReader reader,
+      BlockingQueue<WALEntryBatch> entryBatchQueue, Path currentPath) throws InterruptedException {
+    LOG.trace("Didn't read any new entries from WAL");
+    // we're done with queue recovery, shut ourself down
+    reader.setReaderRunning(false);
+    // shuts down shipper thread immediately
+    entryBatchQueue.put(new WALEntryBatch(0, currentPath));
   }
 
   @Override
-  protected ReplicationSourceWALReader startNewWALReader(String threadName, String walGroupId,
+  protected ReplicationSourceWALReader createNewWALReader(String walGroupId,
       PriorityBlockingQueue<Path> queue, long startPosition) {
-    ReplicationSourceWALReader walReader =
-      new RecoveredReplicationSourceWALReader(fs, conf, queue, startPosition, walEntryFilter, this);
-    Threads.setDaemonThreadRunning(walReader,
-      threadName + ".replicationSource.replicationWALReaderThread." + walGroupId + "," + queueId,
-      this::uncaughtException);
-    return walReader;
+    if (replicationPeer.getPeerConfig().isSerial()) {
+      return new SerialReplicationSourceWALReader(fs, conf, queue, startPosition, walEntryFilter,
+        this) {
+
+        @Override
+        protected void handleEmptyWALEntryBatch(Path currentPath) throws InterruptedException {
+          handleEmptyWALEntryBatch0(this, entryBatchQueue, currentPath);
+        }
+      };
+    } else {
+      return new ReplicationSourceWALReader(fs, conf, queue, startPosition, walEntryFilter, this) {
+
+        @Override
+        protected void handleEmptyWALEntryBatch(Path currentPath) throws InterruptedException {
+          handleEmptyWALEntryBatch0(this, entryBatchQueue, currentPath);
+        }
+      };
+    }
   }
 
   public void locateRecoveredPaths(PriorityBlockingQueue<Path> queue) throws IOException {
@@ -166,21 +176,14 @@ public class RecoveredReplicationSource extends ReplicationSource {
     return path;
   }
 
-  public void tryFinish() {
+  void tryFinish() {
     // use synchronize to make sure one last thread will clean the queue
     synchronized (workerThreads) {
       Threads.sleep(100);// wait a short while for other worker thread to fully exit
-      boolean allTasksDone = true;
-      for (ReplicationSourceShipper worker : workerThreads.values()) {
-        if (!worker.isFinished()) {
-          allTasksDone = false;
-          break;
-        }
-      }
+      boolean allTasksDone = workerThreads.values().stream().allMatch(w -> w.isFinished());
       if (allTasksDone) {
         manager.removeRecoveredSource(this);
-        LOG.info("Finished recovering queue " + queueId + " with the following stats: "
-            + getStats());
+        LOG.info("Finished recovering queue {} with the following stats: {}", queueId, getStats());
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d0d6a3b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
index 9c36497..1ae5cb9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
@@ -48,46 +48,18 @@ public class RecoveredReplicationSourceShipper extends ReplicationSourceShipper
   }
 
   @Override
-  public void run() {
-    setWorkerState(WorkerState.RUNNING);
-    // Loop until we close down
-    while (isActive()) {
-      int sleepMultiplier = 1;
-      // Sleep until replication is enabled again
-      if (!source.isPeerEnabled()) {
-        if (source.sleepForRetries("Replication is disabled", sleepMultiplier)) {
-          sleepMultiplier++;
-        }
-        continue;
-      }
-
-      while (entryReader == null) {
-        if (source.sleepForRetries("Replication WAL entry reader thread not initialized",
-          sleepMultiplier)) {
-          sleepMultiplier++;
-        }
-      }
-
-      try {
-        WALEntryBatch entryBatch = entryReader.take();
-        shipEdits(entryBatch);
-        if (entryBatch.getWalEntries().isEmpty()) {
-          LOG.debug("Finished recovering queue for group " + walGroupId + " of peer "
-              + source.getQueueId());
-          source.getSourceMetrics().incrCompletedRecoveryQueue();
-          setWorkerState(WorkerState.FINISHED);
-          continue;
-        }
-      } catch (InterruptedException e) {
-        LOG.trace("Interrupted while waiting for next replication entry batch", e);
-        Thread.currentThread().interrupt();
-      }
+  protected void postShipEdits(WALEntryBatch entryBatch) {
+    if (entryBatch.getWalEntries().isEmpty()) {
+      LOG.debug("Finished recovering queue for group " + walGroupId + " of peer "
+          + source.getQueueId());
+      source.getSourceMetrics().incrCompletedRecoveryQueue();
+      setWorkerState(WorkerState.FINISHED);
     }
+  }
+
+  @Override
+  protected void postFinish() {
     source.tryFinish();
-    // If the worker exits run loop without finishing its task, mark it as stopped.
-    if (!isFinished()) {
-      setWorkerState(WorkerState.STOPPED);
-    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d0d6a3b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceWALReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceWALReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceWALReader.java
deleted file mode 100644
index 114f139..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceWALReader.java
+++ /dev/null
@@ -1,56 +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.concurrent.PriorityBlockingQueue;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.replication.WALEntryFilter;
-
-/**
- * Used by a {@link RecoveredReplicationSourceShipper}.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class RecoveredReplicationSourceWALReader extends ReplicationSourceWALReader {
-
-  private static final Logger LOG =
-    LoggerFactory.getLogger(RecoveredReplicationSourceWALReader.class);
-
-  public RecoveredReplicationSourceWALReader(FileSystem fs, Configuration conf,
-      PriorityBlockingQueue<Path> logQueue, long startPosition, WALEntryFilter filter,
-      ReplicationSource source) {
-    super(fs, conf, logQueue, startPosition, filter, source);
-  }
-
-  @Override
-  protected void handleEmptyWALEntryBatch(Path currentPath) throws InterruptedException {
-    LOG.trace("Didn't read any new entries from WAL");
-    // we're done with queue recovery, shut ourself down
-    setReaderRunning(false);
-    // shuts down shipper thread immediately
-    entryBatchQueue.put(new WALEntryBatch(replicationBatchCountCapacity, currentPath));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d0d6a3b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index f5e4185..3480919 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -85,7 +85,7 @@ public class ReplicationSource implements ReplicationSourceInterface {
   // per group queue size, keep no more than this number of logs in each wal group
   protected int queueSizePerGroup;
   protected ReplicationQueueStorage queueStorage;
-  private ReplicationPeer replicationPeer;
+  protected ReplicationPeer replicationPeer;
 
   protected Configuration conf;
   protected ReplicationQueueInfo replicationQueueInfo;
@@ -294,26 +294,32 @@ public class ReplicationSource implements ReplicationSourceInterface {
     this.walEntryFilter = new ChainWALEntryFilter(filters);
   }
 
-  protected void tryStartNewShipper(String walGroupId, PriorityBlockingQueue<Path> queue) {
-    ReplicationSourceShipper worker = new ReplicationSourceShipper(conf, walGroupId, queue, this);
+  private void tryStartNewShipper(String walGroupId, PriorityBlockingQueue<Path> queue) {
+    ReplicationSourceShipper worker = createNewShipper(walGroupId, queue);
     ReplicationSourceShipper extant = workerThreads.putIfAbsent(walGroupId, worker);
     if (extant != null) {
-      LOG.debug("Someone has beat us to start a worker thread for wal group " + walGroupId);
+      LOG.debug("Someone has beat us to start a worker thread for wal group {}", walGroupId);
     } else {
-      LOG.debug("Starting up worker for wal group " + walGroupId);
+      LOG.debug("Starting up worker for wal group {}", walGroupId);
+      ReplicationSourceWALReader walReader =
+        createNewWALReader(walGroupId, queue, worker.getStartPosition());
+      Threads.setDaemonThreadRunning(walReader, Thread.currentThread().getName() +
+        ".replicationSource.wal-reader." + walGroupId + "," + queueId, this::uncaughtException);
+      worker.setWALReader(walReader);
       worker.startup(this::uncaughtException);
-      worker.setWALReader(
-        startNewWALReader(worker.getName(), walGroupId, queue, worker.getStartPosition()));
     }
   }
 
-  protected ReplicationSourceWALReader startNewWALReader(String threadName, String walGroupId,
+  protected ReplicationSourceShipper createNewShipper(String walGroupId,
+      PriorityBlockingQueue<Path> queue) {
+    return new ReplicationSourceShipper(conf, walGroupId, queue, this);
+  }
+
+  protected ReplicationSourceWALReader createNewWALReader(String walGroupId,
       PriorityBlockingQueue<Path> queue, long startPosition) {
-    ReplicationSourceWALReader walReader =
-      new ReplicationSourceWALReader(fs, conf, queue, startPosition, walEntryFilter, this);
-    return (ReplicationSourceWALReader) Threads.setDaemonThreadRunning(walReader,
-      threadName + ".replicationSource.wal-reader." + walGroupId + "," + queueId,
-      this::uncaughtException);
+    return replicationPeer.getPeerConfig().isSerial()
+      ? new SerialReplicationSourceWALReader(fs, conf, queue, startPosition, walEntryFilter, this)
+      : new ReplicationSourceWALReader(fs, conf, queue, startPosition, walEntryFilter, this);
   }
 
   protected final void uncaughtException(Thread t, Throwable e) {
@@ -392,10 +398,6 @@ public class ReplicationSource implements ReplicationSourceInterface {
     return replicationPeer.isPeerEnabled();
   }
 
-  public boolean isSerial() {
-    return replicationPeer.getPeerConfig().isSerial();
-  }
-
   private void initialize() {
     int sleepMultiplier = 1;
     while (this.isSourceActive()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d0d6a3b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
index 50aaf95..aa5251e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
@@ -83,7 +83,7 @@ public class ReplicationSourceShipper extends Thread {
   }
 
   @Override
-  public void run() {
+  public final void run() {
     setWorkerState(WorkerState.RUNNING);
     // Loop until we close down
     while (isActive()) {
@@ -95,28 +95,31 @@ public class ReplicationSourceShipper extends Thread {
         }
         continue;
       }
-
-      while (entryReader == null) {
-        if (sleepForRetries("Replication WAL entry reader thread not initialized",
-          sleepMultiplier)) {
-          sleepMultiplier++;
-        }
-      }
-
       try {
         WALEntryBatch entryBatch = entryReader.take();
         shipEdits(entryBatch);
+        postShipEdits(entryBatch);
       } catch (InterruptedException e) {
         LOG.trace("Interrupted while waiting for next replication entry batch", e);
         Thread.currentThread().interrupt();
       }
     }
     // If the worker exits run loop without finishing its task, mark it as stopped.
-    if (state != WorkerState.FINISHED) {
+    if (!isFinished()) {
       setWorkerState(WorkerState.STOPPED);
+    } else {
+      postFinish();
     }
   }
 
+  // To be implemented by recovered shipper
+  protected void postShipEdits(WALEntryBatch entryBatch) {
+  }
+
+  // To be implemented by recovered shipper
+  protected void postFinish() {
+  }
+
   /**
    * Do the shipping logic
    */
@@ -229,8 +232,8 @@ public class ReplicationSourceShipper extends Thread {
 
   public void startup(UncaughtExceptionHandler handler) {
     String name = Thread.currentThread().getName();
-    Threads.setDaemonThreadRunning(this, name + ".replicationSource." + walGroupId + ","
-        + source.getQueueId(), handler);
+    Threads.setDaemonThreadRunning(this,
+      name + ".replicationSource.shipper" + walGroupId + "," + source.getQueueId(), handler);
   }
 
   public PriorityBlockingQueue<Path> getLogQueue() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d0d6a3b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
index da92a09..b125133 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
@@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.replication.WALEntryFilter;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
@@ -51,7 +50,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescript
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
-public class ReplicationSourceWALReader extends Thread {
+class ReplicationSourceWALReader extends Thread {
   private static final Logger LOG = LoggerFactory.getLogger(ReplicationSourceWALReader.class);
 
   private final PriorityBlockingQueue<Path> logQueue;
@@ -64,28 +63,19 @@ public class ReplicationSourceWALReader extends Thread {
   // max (heap) size of each batch - multiply by number of batches in queue to get total
   private final long replicationBatchSizeCapacity;
   // max count of each batch - multiply by number of batches in queue to get total
-  protected final int replicationBatchCountCapacity;
+  private final int replicationBatchCountCapacity;
   // position in the WAL to start reading at
   private long currentPosition;
   private final long sleepForRetries;
   private final int maxRetriesMultiplier;
   private final boolean eofAutoRecovery;
 
-  // used to store the first cell in an entry before filtering. This is because that if serial
-  // replication is enabled, we may find out that an entry can not be pushed after filtering. And
-  // when we try the next time, the cells maybe null since the entry has already been filtered,
-  // especially for region event wal entries. And this can also used to determine whether we can
-  // skip filtering.
-  private Cell firstCellInEntryBeforeFiltering;
-
   //Indicates whether this particular worker is running
   private boolean isReaderRunning = true;
 
   private AtomicLong totalBufferUsed;
   private long totalBufferQuota;
 
-  private final SerialReplicationChecker serialReplicationChecker;
-
   /**
    * Creates a reader worker for a given WAL queue. Reads WAL entries off a given queue, batches the
    * entries, and puts them on a batch queue.
@@ -120,7 +110,6 @@ public class ReplicationSourceWALReader extends Thread {
         this.conf.getInt("replication.source.maxretriesmultiplier", 300); // 5 minutes @ 1 sec per
     this.eofAutoRecovery = conf.getBoolean("replication.source.eof.autorecovery", false);
     this.entryBatchQueue = new LinkedBlockingQueue<>(batchCount);
-    this.serialReplicationChecker = new SerialReplicationChecker(conf, source);
     LOG.info("peerClusterZnode=" + source.getQueueId()
         + ", ReplicationSourceWALReaderThread : " + source.getPeerId()
         + " inited, replicationBatchSizeCapacity=" + replicationBatchSizeCapacity
@@ -169,75 +158,35 @@ public class ReplicationSourceWALReader extends Thread {
     }
   }
 
-  private void removeEntryFromStream(WALEntryStream entryStream, WALEntryBatch batch)
-      throws IOException {
-    entryStream.next();
-    firstCellInEntryBeforeFiltering = null;
-    batch.setLastWalPosition(entryStream.getPosition());
+  // returns true if we reach the size limit for batch, i.e, we need to finish the batch and return.
+  protected final boolean addEntryToBatch(WALEntryBatch batch, Entry entry) {
+    WALEdit edit = entry.getEdit();
+    if (edit == null || edit.isEmpty()) {
+      return false;
+    }
+    long entrySize = getEntrySize(entry);
+    batch.addEntry(entry);
+    updateBatchStats(batch, entry, entrySize);
+    boolean totalBufferTooLarge = acquireBufferQuota(entrySize);
+    // Stop if too many entries or too big
+    return totalBufferTooLarge || batch.getHeapSize() >= replicationBatchSizeCapacity ||
+      batch.getNbEntries() >= replicationBatchCountCapacity;
   }
 
-  private WALEntryBatch readWALEntries(WALEntryStream entryStream)
+  protected WALEntryBatch readWALEntries(WALEntryStream entryStream)
       throws IOException, InterruptedException {
     if (!entryStream.hasNext()) {
       return null;
     }
-    long positionBefore = entryStream.getPosition();
-    WALEntryBatch batch =
-      new WALEntryBatch(replicationBatchCountCapacity, entryStream.getCurrentPath());
+    WALEntryBatch batch = createBatch(entryStream);
     do {
-      Entry entry = entryStream.peek();
-      boolean isSerial = source.isSerial();
-      boolean doFiltering = true;
-      if (isSerial) {
-        if (firstCellInEntryBeforeFiltering == null) {
-          assert !entry.getEdit().isEmpty() : "should not write empty edits";
-          // Used to locate the region record in meta table. In WAL we only have the table name and
-          // encoded region name which can not be mapping to region name without scanning all the
-          // records for a table, so we need a start key, just like what we have done at client side
-          // when locating a region. For the markers, we will use the start key of the region as the
-          // row key for the edit. And we need to do this before filtering since all the cells may
-          // be filtered out, especially that for the markers.
-          firstCellInEntryBeforeFiltering = entry.getEdit().getCells().get(0);
-        } else {
-          // if this is not null then we know that the entry has already been filtered.
-          doFiltering = false;
-        }
-      }
-
-      if (doFiltering) {
-        entry = filterEntry(entry);
-      }
+      Entry entry = entryStream.next();
+      batch.setLastWalPosition(entryStream.getPosition());
+      entry = filterEntry(entry);
       if (entry != null) {
-        if (isSerial) {
-          if (!serialReplicationChecker.canPush(entry, firstCellInEntryBeforeFiltering)) {
-            if (batch.getLastWalPosition() > positionBefore) {
-              // we have something that can push, break
-              break;
-            } else {
-              serialReplicationChecker.waitUntilCanPush(entry, firstCellInEntryBeforeFiltering);
-            }
-          }
-          // arrive here means we can push the entry, record the last sequence id
-          batch.setLastSeqId(Bytes.toString(entry.getKey().getEncodedRegionName()),
-            entry.getKey().getSequenceId());
+        if (addEntryToBatch(batch, entry)) {
+          break;
         }
-        // actually remove the entry.
-        removeEntryFromStream(entryStream, batch);
-        WALEdit edit = entry.getEdit();
-        if (edit != null && !edit.isEmpty()) {
-          long entrySize = getEntrySize(entry);
-          batch.addEntry(entry);
-          updateBatchStats(batch, entry, entrySize);
-          boolean totalBufferTooLarge = acquireBufferQuota(entrySize);
-          // Stop if too many entries or too big
-          if (totalBufferTooLarge || batch.getHeapSize() >= replicationBatchSizeCapacity ||
-            batch.getNbEntries() >= replicationBatchCountCapacity) {
-            break;
-          }
-        }
-      } else {
-        // actually remove the entry.
-        removeEntryFromStream(entryStream, batch);
       }
     } while (entryStream.hasNext());
     return batch;
@@ -286,7 +235,11 @@ public class ReplicationSourceWALReader extends Thread {
     return true;
   }
 
-  private Entry filterEntry(Entry entry) {
+  protected final WALEntryBatch createBatch(WALEntryStream entryStream) {
+    return new WALEntryBatch(replicationBatchCountCapacity, entryStream.getCurrentPath());
+  }
+
+  protected final Entry filterEntry(Entry entry) {
     Entry filtered = filter.filter(entry);
     if (entry != null && filtered == null) {
       source.getSourceMetrics().incrLogEditsFiltered();

http://git-wip-us.apache.org/repos/asf/hbase/blob/2d0d6a3b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SerialReplicationSourceWALReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SerialReplicationSourceWALReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SerialReplicationSourceWALReader.java
new file mode 100644
index 0000000..5e9a9f6
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SerialReplicationSourceWALReader.java
@@ -0,0 +1,112 @@
+/**
+ * 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 java.util.concurrent.PriorityBlockingQueue;
+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.replication.WALEntryFilter;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * WAL reader for a serial replication peer.
+ */
+@InterfaceAudience.Private
+public class SerialReplicationSourceWALReader extends ReplicationSourceWALReader {
+
+  // used to store the first cell in an entry before filtering. This is because that if serial
+  // replication is enabled, we may find out that an entry can not be pushed after filtering. And
+  // when we try the next time, the cells maybe null since the entry has already been filtered,
+  // especially for region event wal entries. And this can also used to determine whether we can
+  // skip filtering.
+  private Cell firstCellInEntryBeforeFiltering;
+
+  private final SerialReplicationChecker checker;
+
+  public SerialReplicationSourceWALReader(FileSystem fs, Configuration conf,
+      PriorityBlockingQueue<Path> logQueue, long startPosition, WALEntryFilter filter,
+      ReplicationSource source) {
+    super(fs, conf, logQueue, startPosition, filter, source);
+    checker = new SerialReplicationChecker(conf, source);
+  }
+
+  @Override
+  protected WALEntryBatch readWALEntries(WALEntryStream entryStream)
+      throws IOException, InterruptedException {
+    if (!entryStream.hasNext()) {
+      return null;
+    }
+    long positionBefore = entryStream.getPosition();
+    WALEntryBatch batch = createBatch(entryStream);
+    do {
+      Entry entry = entryStream.peek();
+      boolean doFiltering = true;
+      if (firstCellInEntryBeforeFiltering == null) {
+        assert !entry.getEdit().isEmpty() : "should not write empty edits";
+        // Used to locate the region record in meta table. In WAL we only have the table name and
+        // encoded region name which can not be mapping to region name without scanning all the
+        // records for a table, so we need a start key, just like what we have done at client side
+        // when locating a region. For the markers, we will use the start key of the region as the
+        // row key for the edit. And we need to do this before filtering since all the cells may
+        // be filtered out, especially that for the markers.
+        firstCellInEntryBeforeFiltering = entry.getEdit().getCells().get(0);
+      } else {
+        // if this is not null then we know that the entry has already been filtered.
+        doFiltering = false;
+      }
+
+      if (doFiltering) {
+        entry = filterEntry(entry);
+      }
+      if (entry != null) {
+        if (!checker.canPush(entry, firstCellInEntryBeforeFiltering)) {
+          if (batch.getLastWalPosition() > positionBefore) {
+            // we have something that can push, break
+            break;
+          } else {
+            checker.waitUntilCanPush(entry, firstCellInEntryBeforeFiltering);
+          }
+        }
+        // arrive here means we can push the entry, record the last sequence id
+        batch.setLastSeqId(Bytes.toString(entry.getKey().getEncodedRegionName()),
+          entry.getKey().getSequenceId());
+        // actually remove the entry.
+        removeEntryFromStream(entryStream, batch);
+        if (addEntryToBatch(batch, entry)) {
+          break;
+        }
+      } else {
+        // actually remove the entry.
+        removeEntryFromStream(entryStream, batch);
+      }
+    } while (entryStream.hasNext());
+    return batch;
+  }
+
+  private void removeEntryFromStream(WALEntryStream entryStream, WALEntryBatch batch)
+      throws IOException {
+    entryStream.next();
+    firstCellInEntryBeforeFiltering = null;
+    batch.setLastWalPosition(entryStream.getPosition());
+  }
+}


[13/22] hbase git commit: HBASE-20165 Shell command to make a normal peer to be a serial replication peer

Posted by zh...@apache.org.
HBASE-20165 Shell command to make a normal peer to be a serial replication peer


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

Branch: refs/heads/HBASE-20046-branch-2
Commit: fedf3ca923ede4c870faceb33c89a5a7b635eff3
Parents: 2d0d6a3
Author: openinx <op...@gmail.com>
Authored: Sat Mar 10 19:36:43 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Apr 9 15:18:44 2018 +0800

----------------------------------------------------------------------
 .../src/main/ruby/hbase/replication_admin.rb    | 11 ++++-
 hbase-shell/src/main/ruby/shell.rb              |  1 +
 .../src/main/ruby/shell/commands/list_peers.rb  |  5 +-
 .../main/ruby/shell/commands/set_peer_serial.rb | 49 ++++++++++++++++++++
 .../test/ruby/hbase/replication_admin_test.rb   | 23 +++++++++
 5 files changed, 86 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fedf3ca9/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..5b87595 100644
--- a/hbase-shell/src/main/ruby/hbase/replication_admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/replication_admin.rb
@@ -284,6 +284,15 @@ module Hbase
       @admin.updateReplicationPeerConfig(id, rpc)
     end
 
+    def set_peer_serial(id, peer_serial)
+      rpc = get_peer_config(id)
+      return if rpc.nil?
+      rpc_builder = org.apache.hadoop.hbase.replication.ReplicationPeerConfig
+                       .newBuilder(rpc)
+      new_rpc = rpc_builder.setSerial(peer_serial).build
+      @admin.updateReplicationPeerConfig(id, new_rpc)
+    end
+
     # Set exclude namespaces config for the specified peer
     def set_peer_exclude_namespaces(id, exclude_namespaces)
       return if exclude_namespaces.nil?
@@ -362,7 +371,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)
+                   .newBuilder(replication_peer_config)
       unless config.nil?
         builder.putAllConfiguration(config)
       end

http://git-wip-us.apache.org/repos/asf/hbase/blob/fedf3ca9/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 3efe7e9..2e228f5 100644
--- a/hbase-shell/src/main/ruby/shell.rb
+++ b/hbase-shell/src/main/ruby/shell.rb
@@ -373,6 +373,7 @@ Shell.load_command_group(
     enable_peer
     disable_peer
     set_peer_replicate_all
+    set_peer_serial
     set_peer_namespaces
     append_peer_namespaces
     remove_peer_namespaces

http://git-wip-us.apache.org/repos/asf/hbase/blob/fedf3ca9/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..eefcc42 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])
+                            STATE REPLICATE_ALL NAMESPACES TABLE_CFS BANDWIDTH
+                            SERIAL])
 
         peers.each do |peer|
           id = peer.getPeerId
@@ -55,7 +56,7 @@ EOF
           formatter.row([id, config.getClusterKey,
                          config.getReplicationEndpointImpl, state,
                          config.replicateAllUserTables, namespaces, tableCFs,
-                         config.getBandwidth])
+                         config.getBandwidth, config.isSerial])
         end
 
         formatter.footer

http://git-wip-us.apache.org/repos/asf/hbase/blob/fedf3ca9/hbase-shell/src/main/ruby/shell/commands/set_peer_serial.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/shell/commands/set_peer_serial.rb b/hbase-shell/src/main/ruby/shell/commands/set_peer_serial.rb
new file mode 100644
index 0000000..d556077
--- /dev/null
+++ b/hbase-shell/src/main/ruby/shell/commands/set_peer_serial.rb
@@ -0,0 +1,49 @@
+#
+# 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 SetPeerSerial < Command
+      def help
+        <<-EOF
+  Set the serial flag to true or false for the specified peer.
+
+  If serial flag is true, then all logs of user tables (REPLICATION_SCOPE != 0) will be
+  replicated to peer cluster serially, which means that each segment of log for replicated
+  table will be pushed to peer cluster in order of their log sequence id.
+
+  If serial flag is false, then the source cluster won't ensure that the logs of replicated
+  table will be pushed to peer cluster serially.
+
+  Examples:
+
+    # set serial flag to true
+    hbase> set_peer_serial '1', true
+    # set serial flag to false
+    hbase> set_peer_serial '1', false
+  EOF
+      end
+
+      def command(id, peer_serial)
+        replication_admin.set_peer_serial(id, peer_serial)
+      end
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/hbase/blob/fedf3ca9/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..29de710 100644
--- a/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
+++ b/hbase-shell/src/test/ruby/hbase/replication_admin_test.rb
@@ -459,6 +459,29 @@ module Hbase
       replication_admin.remove_peer(@peer_id)
     end
 
+    define_test 'set_peer_serial' do
+      cluster_key = 'zk4,zk5,zk6:11000:/hbase-test'
+
+      args = { CLUSTER_KEY => cluster_key }
+      command(:add_peer, @peer_id, args)
+
+      assert_equal(1, command(:list_peers).length)
+      peer_config = command(:list_peers).get(0).getPeerConfig
+      assert_equal(false, peer_config.isSerial)
+
+      command(:set_peer_serial, @peer_id, true)
+      peer_config = command(:list_peers).get(0).getPeerConfig
+      assert_equal(true, peer_config.isSerial)
+
+      command(:set_peer_serial, @peer_id, false)
+      peer_config = command(:list_peers).get(0).getPeerConfig
+      assert_equal(false, peer_config.isSerial)
+
+      # cleanup for future tests
+      replication_admin.remove_peer(@peer_id)
+      assert_equal(0, command(:list_peers).length)
+    end
+
     define_test "set_peer_bandwidth: works with peer bandwidth upper limit" do
       cluster_key = "localhost:2181:/hbase-test"
       args = { CLUSTER_KEY => cluster_key }


[07/22] hbase git commit: HBASE-20050 Reimplement updateReplicationPositions logic in serial replication based on the newly introduced replication storage layer

Posted by zh...@apache.org.
HBASE-20050 Reimplement updateReplicationPositions logic in serial replication based on the newly introduced replication storage layer

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

Branch: refs/heads/HBASE-20046-branch-2
Commit: 1d11cdb26cf3c713a4f0306e05baa0c5865501dd
Parents: 39c1ddc
Author: huzheng <op...@gmail.com>
Authored: Wed Feb 28 16:25:24 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Apr 9 15:18:44 2018 +0800

----------------------------------------------------------------------
 .../replication/ReplicationQueueStorage.java    | 15 +++-
 .../replication/ZKReplicationQueueStorage.java  | 88 ++++++++++++++++++--
 .../replication/TestReplicationStateBasic.java  | 48 ++++++++++-
 .../TestZKReplicationQueueStorage.java          |  7 +-
 .../regionserver/ReplicationSourceManager.java  |  4 +-
 5 files changed, 146 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1d11cdb2/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
index e774148..4c93da6 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.replication;
 
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.SortedSet;
 
@@ -63,9 +64,19 @@ public interface ReplicationQueueStorage {
    * @param queueId a String that identifies the queue
    * @param fileName name of the WAL
    * @param position the current position in the file
+   * @param lastSeqIds map with {encodedRegionName, sequenceId} pairs for serial replication.
    */
-  void setWALPosition(ServerName serverName, String queueId, String fileName, long position)
-      throws ReplicationException;
+  void setWALPosition(ServerName serverName, String queueId, String fileName, long position,
+      Map<String, Long> lastSeqIds) throws ReplicationException;
+
+  /**
+   * Read the max sequence id of the specific region for a given peer. For serial replication, we
+   * need the max sequenced id to decide whether we can push the next entries.
+   * @param encodedRegionName the encoded region name
+   * @param peerId peer id
+   * @return the max sequence id of the specific region for a given peer.
+   */
+  long getLastSequenceId(String encodedRegionName, String peerId) throws ReplicationException;
 
   /**
    * Get the current position for a specific WAL in a given queue for a given regionserver.

http://git-wip-us.apache.org/repos/asf/hbase/blob/1d11cdb2/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
index da96c65..adbf259 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
@@ -23,6 +23,8 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
@@ -85,6 +87,10 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
       "zookeeper.znode.replication.hfile.refs";
   public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT = "hfile-refs";
 
+  public static final String ZOOKEEPER_ZNODE_REPLICATION_REGIONS_KEY =
+      "zookeeper.znode.replication.regions";
+  public static final String ZOOKEEPER_ZNODE_REPLICATION_REGIONS_DEFAULT = "regions";
+
   /**
    * The name of the znode that contains all replication queues
    */
@@ -95,6 +101,8 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
    */
   private final String hfileRefsZNode;
 
+  private final String regionsZNode;
+
   public ZKReplicationQueueStorage(ZKWatcher zookeeper, Configuration conf) {
     super(zookeeper, conf);
 
@@ -103,6 +111,8 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
       ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT);
     this.queuesZNode = ZNodePaths.joinZNode(replicationZNode, queuesZNodeName);
     this.hfileRefsZNode = ZNodePaths.joinZNode(replicationZNode, hfileRefsZNodeName);
+    this.regionsZNode = ZNodePaths.joinZNode(replicationZNode, conf
+        .get(ZOOKEEPER_ZNODE_REPLICATION_REGIONS_KEY, ZOOKEEPER_ZNODE_REPLICATION_REGIONS_DEFAULT));
   }
 
   private String getRsNode(ServerName serverName) {
@@ -121,6 +131,28 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
     return getFileNode(getQueueNode(serverName, queueId), fileName);
   }
 
+  /**
+   * Put all regions under /hbase/replication/regions znode will lead to too many children because
+   * of the huge number of regions in real production environment. So here we use hash of encoded
+   * region name to distribute the znode into multiple znodes. <br>
+   * So the final znode path will be format like this:
+   *
+   * <pre>
+   * /hbase/replication/regions/254/dd04e76a6966d4ffa908ed0586764767-100
+   * </pre>
+   *
+   * The 254 indicate the hash of encoded region name, the 100 indicate the peer id.
+   * @param encodedRegionName the encoded region name.
+   * @param peerId peer id for replication.
+   * @return ZNode path to persist the max sequence id that we've pushed for the given region and
+   *         peer.
+   */
+  private String getSerialReplicationRegionPeerNode(String encodedRegionName, String peerId) {
+    int hash = encodedRegionName.hashCode() & 0x0000FFFF;
+    String hashPath = ZNodePaths.joinZNode(regionsZNode, String.valueOf(hash));
+    return ZNodePaths.joinZNode(hashPath, String.format("%s-%s", encodedRegionName, peerId));
+  }
+
   @Override
   public void removeQueue(ServerName serverName, String queueId) throws ReplicationException {
     try {
@@ -137,8 +169,8 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
     try {
       ZKUtil.createWithParents(zookeeper, getFileNode(serverName, queueId, fileName));
     } catch (KeeperException e) {
-      throw new ReplicationException("Failed to add wal to queue (serverName=" + serverName +
-        ", queueId=" + queueId + ", fileName=" + fileName + ")", e);
+      throw new ReplicationException("Failed to add wal to queue (serverName=" + serverName
+          + ", queueId=" + queueId + ", fileName=" + fileName + ")", e);
     }
   }
 
@@ -157,15 +189,55 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
   }
 
   @Override
-  public void setWALPosition(ServerName serverName, String queueId, String fileName, long position)
-      throws ReplicationException {
+  public void setWALPosition(ServerName serverName, String queueId, String fileName, long position,
+      Map<String, Long> lastSeqIds) throws ReplicationException {
     try {
-      ZKUtil.setData(zookeeper, getFileNode(serverName, queueId, fileName),
-        ZKUtil.positionToByteArray(position));
+      List<ZKUtilOp> listOfOps = new ArrayList<>();
+      listOfOps.add(ZKUtilOp.setData(getFileNode(serverName, queueId, fileName),
+        ZKUtil.positionToByteArray(position)));
+      // Persist the max sequence id(s) of regions for serial replication atomically.
+      if (lastSeqIds != null && lastSeqIds.size() > 0) {
+        for (Entry<String, Long> lastSeqEntry : lastSeqIds.entrySet()) {
+          String peerId = new ReplicationQueueInfo(queueId).getPeerId();
+          String path = getSerialReplicationRegionPeerNode(lastSeqEntry.getKey(), peerId);
+          /*
+           * Make sure the existence of path
+           * /hbase/replication/regions/<hash>/<encoded-region-name>-<peer-id>. As the javadoc in
+           * multiOrSequential() method said, if received a NodeExistsException, all operations will
+           * fail. So create the path here, and in fact, no need to add this operation to listOfOps,
+           * because only need to make sure that update file position and sequence id atomically.
+           */
+          ZKUtil.createWithParents(zookeeper, path);
+          // Persist the max sequence id of region to zookeeper.
+          listOfOps
+              .add(ZKUtilOp.setData(path, ZKUtil.positionToByteArray(lastSeqEntry.getValue())));
+        }
+      }
+      ZKUtil.multiOrSequential(zookeeper, listOfOps, false);
     } catch (KeeperException e) {
-      throw new ReplicationException("Failed to set log position (serverName=" + serverName +
-        ", queueId=" + queueId + ", fileName=" + fileName + ", position=" + position + ")", e);
+      throw new ReplicationException("Failed to set log position (serverName=" + serverName
+          + ", queueId=" + queueId + ", fileName=" + fileName + ", position=" + position + ")", e);
+    }
+  }
+
+  @Override
+  public long getLastSequenceId(String encodedRegionName, String peerId)
+      throws ReplicationException {
+    byte[] data;
+    try {
+      data =
+          ZKUtil.getData(zookeeper, getSerialReplicationRegionPeerNode(encodedRegionName, peerId));
+    } catch (KeeperException | InterruptedException e) {
+      throw new ReplicationException("Failed to get the last sequence id(region="
+          + encodedRegionName + ", peerId=" + peerId + ")");
+    }
+    try {
+      return ZKUtil.parseWALPositionFrom(data);
+    } catch (DeserializationException de) {
+      LOG.warn("Failed to parse log position (region=" + encodedRegionName + ", peerId=" + peerId
+          + "), data=" + Bytes.toStringBinary(data));
     }
+    return HConstants.NO_SEQNUM;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/1d11cdb2/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..5999c1f 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
@@ -17,8 +17,10 @@
  */
 package org.apache.hadoop.hbase.replication;
 
+import static org.hamcrest.CoreMatchers.hasItems;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -26,6 +28,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
 import org.apache.hadoop.hbase.util.Pair;
@@ -35,6 +38,8 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
+
 /**
  * White box testing for replication state interfaces. Implementations should extend this class, and
  * initialize the interfaces properly.
@@ -122,7 +127,7 @@ public abstract class TestReplicationStateBasic {
     assertEquals(0, rqs.getWALsInQueue(server2, "qId1").size());
     assertEquals(5, rqs.getWALsInQueue(server3, "qId5").size());
     assertEquals(0, rqs.getWALPosition(server3, "qId1", "filename0"));
-    rqs.setWALPosition(server3, "qId5", "filename4", 354L);
+    rqs.setWALPosition(server3, "qId5", "filename4", 354L, null);
     assertEquals(354L, rqs.getWALPosition(server3, "qId5", "filename4"));
 
     assertEquals(5, rqs.getWALsInQueue(server3, "qId5").size());
@@ -270,6 +275,47 @@ public abstract class TestReplicationStateBasic {
     assertNumberOfPeers(2);
   }
 
+  private String getFileName(String base, int i) {
+    return String.format(base + "-%04d", i);
+  }
+
+  @Test
+  public void testPersistLogPositionAndSeqIdAtomically() throws Exception {
+    ServerName serverName1 = ServerName.valueOf("127.0.0.1", 8000, 10000);
+    assertTrue(rqs.getAllQueues(serverName1).isEmpty());
+    String queue1 = "1";
+    String region0 = "region0", region1 = "region1";
+    for (int i = 0; i < 10; i++) {
+      rqs.addWAL(serverName1, queue1, getFileName("file1", i));
+    }
+    List<String> queueIds = rqs.getAllQueues(serverName1);
+    assertEquals(1, queueIds.size());
+    assertThat(queueIds, hasItems("1"));
+
+    List<String> wals1 = rqs.getWALsInQueue(serverName1, queue1);
+    assertEquals(10, wals1.size());
+    for (int i = 0; i < 10; i++) {
+      assertThat(wals1, hasItems(getFileName("file1", i)));
+    }
+
+    for (int i = 0; i < 10; i++) {
+      assertEquals(0, rqs.getWALPosition(serverName1, queue1, getFileName("file1", i)));
+    }
+    assertEquals(HConstants.NO_SEQNUM, rqs.getLastSequenceId(region0, queue1));
+    assertEquals(HConstants.NO_SEQNUM, rqs.getLastSequenceId(region1, queue1));
+
+    for (int i = 0; i < 10; i++) {
+      rqs.setWALPosition(serverName1, queue1, getFileName("file1", i), (i + 1) * 100,
+        ImmutableMap.of(region0, i * 100L, region1, (i + 1) * 100L));
+    }
+
+    for (int i = 0; i < 10; i++) {
+      assertEquals((i + 1) * 100, rqs.getWALPosition(serverName1, queue1, getFileName("file1", i)));
+    }
+    assertEquals(900L, rqs.getLastSequenceId(region0, queue1));
+    assertEquals(1000L, rqs.getLastSequenceId(region1, queue1));
+  }
+
   protected void assertConnectedPeerStatus(boolean status, String peerId) throws Exception {
     // we can first check if the value was changed in the store, if it wasn't then fail right away
     if (status != rp.getPeerStorage().isPeerEnabled(peerId)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/1d11cdb2/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
index 2c01a26..8ff52f3 100644
--- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
@@ -127,7 +127,7 @@ public class TestZKReplicationQueueStorage {
     List<String> wals1 = STORAGE.getWALsInQueue(serverName1, queue1);
     List<String> wals2 = STORAGE.getWALsInQueue(serverName1, queue2);
     assertEquals(10, wals1.size());
-    assertEquals(10, wals1.size());
+    assertEquals(10, wals2.size());
     for (int i = 0; i < 10; i++) {
       assertThat(wals1, hasItems(getFileName("file1", i)));
       assertThat(wals2, hasItems(getFileName("file2", i)));
@@ -136,8 +136,9 @@ public class TestZKReplicationQueueStorage {
     for (int i = 0; i < 10; i++) {
       assertEquals(0, STORAGE.getWALPosition(serverName1, queue1, getFileName("file1", i)));
       assertEquals(0, STORAGE.getWALPosition(serverName1, queue2, getFileName("file2", i)));
-      STORAGE.setWALPosition(serverName1, queue1, getFileName("file1", i), (i + 1) * 100);
-      STORAGE.setWALPosition(serverName1, queue2, getFileName("file2", i), (i + 1) * 100 + 10);
+      STORAGE.setWALPosition(serverName1, queue1, getFileName("file1", i), (i + 1) * 100, null);
+      STORAGE.setWALPosition(serverName1, queue2, getFileName("file2", i), (i + 1) * 100 + 10,
+        null);
     }
 
     for (int i = 0; i < 10; i++) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/1d11cdb2/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 d11dc8e..eb9dba2 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
@@ -482,8 +482,8 @@ public class ReplicationSourceManager implements ReplicationListener {
   public void logPositionAndCleanOldLogs(Path log, String queueId, long position,
       boolean queueRecovered) {
     String fileName = log.getName();
-    abortWhenFail(
-      () -> this.queueStorage.setWALPosition(server.getServerName(), queueId, fileName, position));
+    abortWhenFail(() -> this.queueStorage.setWALPosition(server.getServerName(), queueId, fileName,
+      position, null));
     cleanOldLogs(fileName, queueId, queueRecovered);
   }
 


[08/22] hbase git commit: HBASE-20115 Reimplement serial replication based on the new replication storage layer

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryBatch.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryBatch.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryBatch.java
new file mode 100644
index 0000000..31c3ac7
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryBatch.java
@@ -0,0 +1,138 @@
+/**
+ * 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.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Holds a batch of WAL entries to replicate, along with some statistics
+ */
+@InterfaceAudience.Private
+class WALEntryBatch {
+  private List<Entry> walEntries;
+  // last WAL that was read
+  private Path lastWalPath;
+  // position in WAL of last entry in this batch
+  private long lastWalPosition = 0;
+  // number of distinct row keys in this batch
+  private int nbRowKeys = 0;
+  // number of HFiles
+  private int nbHFiles = 0;
+  // heap size of data we need to replicate
+  private long heapSize = 0;
+  // save the last sequenceid for each region if the table has serial-replication scope
+  private Map<String, Long> lastSeqIds = new HashMap<>();
+
+  /**
+   * @param lastWalPath Path of the WAL the last entry in this batch was read from
+   */
+  WALEntryBatch(int maxNbEntries, Path lastWalPath) {
+    this.walEntries = new ArrayList<>(maxNbEntries);
+    this.lastWalPath = lastWalPath;
+  }
+
+  public void addEntry(Entry entry) {
+    walEntries.add(entry);
+  }
+
+  /**
+   * @return the WAL Entries.
+   */
+  public List<Entry> getWalEntries() {
+    return walEntries;
+  }
+
+  /**
+   * @return the path of the last WAL that was read.
+   */
+  public Path getLastWalPath() {
+    return lastWalPath;
+  }
+
+  /**
+   * @return the position in the last WAL that was read.
+   */
+  public long getLastWalPosition() {
+    return lastWalPosition;
+  }
+
+  public void setLastWalPosition(long lastWalPosition) {
+    this.lastWalPosition = lastWalPosition;
+  }
+
+  public int getNbEntries() {
+    return walEntries.size();
+  }
+
+  /**
+   * @return the number of distinct row keys in this batch
+   */
+  public int getNbRowKeys() {
+    return nbRowKeys;
+  }
+
+  /**
+   * @return the number of HFiles in this batch
+   */
+  public int getNbHFiles() {
+    return nbHFiles;
+  }
+
+  /**
+   * @return total number of operations in this batch
+   */
+  public int getNbOperations() {
+    return getNbRowKeys() + getNbHFiles();
+  }
+
+  /**
+   * @return the heap size of this batch
+   */
+  public long getHeapSize() {
+    return heapSize;
+  }
+
+  /**
+   * @return the last sequenceid for each region if the table has serial-replication scope
+   */
+  public Map<String, Long> getLastSeqIds() {
+    return lastSeqIds;
+  }
+
+  public void incrementNbRowKeys(int increment) {
+    nbRowKeys += increment;
+  }
+
+  public void incrementNbHFiles(int increment) {
+    nbHFiles += increment;
+  }
+
+  public void incrementHeapSize(long increment) {
+    heapSize += increment;
+  }
+
+  public void setLastSeqId(String region, long sequenceId) {
+    lastSeqIds.put(region, sequenceId);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
index 7c83c0c..bcab9b4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
@@ -21,29 +21,26 @@ package org.apache.hadoop.hbase.replication.regionserver;
 import java.io.Closeable;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.util.NoSuchElementException;
 import java.util.OptionalLong;
 import java.util.concurrent.PriorityBlockingQueue;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
 import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.LeaseNotRecoveredException;
-import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.hbase.wal.WAL.Reader;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Streaming access to WAL entries. This class is given a queue of WAL {@link Path}, and continually
@@ -102,16 +99,18 @@ class WALEntryStream implements Closeable {
   }
 
   /**
-   * @return the next WAL entry in this stream
-   * @throws IOException
-   * @throws NoSuchElementException if no more entries in the stream.
+   * Returns the next WAL entry in this stream but does not advance.
+   */
+  public Entry peek() throws IOException {
+    return hasNext() ? currentEntry: null;
+  }
+
+  /**
+   * Returns the next WAL entry in this stream and advance the stream.
    */
   public Entry next() throws IOException {
-    if (!hasNext()) {
-      throw new NoSuchElementException();
-    }
-    Entry save = currentEntry;
-    currentEntry = null; // gets reloaded by hasNext()
+    Entry save = peek();
+    currentEntry = null;
     return save;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/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 a67bca1..85292f8 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
@@ -170,6 +170,14 @@ public class FSTableDescriptors implements TableDescriptors {
         // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
         .setBloomFilterType(BloomType.NONE)
         .build())
+      .setColumnFamily(ColumnFamilyDescriptorBuilder
+        .newBuilder(HConstants.REPLICATION_BARRIER_FAMILY)
+        .setMaxVersions(HConstants.ALL_VERSIONS)
+        .setInMemory(true)
+        .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+        // Disable blooms for meta.  Needs work.  Seems to mess w/ getClosestOrBefore.
+        .setBloomFilterType(BloomType.NONE)
+        .build())
       .setCoprocessor(CoprocessorDescriptorBuilder.newBuilder(
         MultiRowMutationEndpoint.class.getName())
         .setPriority(Coprocessor.PRIORITY_SYSTEM)

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
index c0b72aa..b106a31 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
@@ -18,13 +18,7 @@
  */
 package org.apache.hadoop.hbase.util;
 
-import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
-import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
-import org.apache.hbase.thirdparty.com.google.common.primitives.Ints;
-
 import edu.umd.cs.findbugs.annotations.CheckForNull;
-
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.EOFException;
@@ -54,7 +48,6 @@ import java.util.concurrent.FutureTask;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.regex.Pattern;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -71,9 +64,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.io.HFileLink;
@@ -81,8 +72,6 @@ import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos;
 import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSHedgedReadMetrics;
@@ -94,6 +83,17 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
+import org.apache.hbase.thirdparty.com.google.common.primitives.Ints;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.FSProtos;
 
 /**
  * Utility methods for interacting with the underlying file system.
@@ -1028,6 +1028,10 @@ public abstract class FSUtils extends CommonFSUtils {
     return regionDirs;
   }
 
+  public static Path getRegionDir(Path tableDir, RegionInfo region) {
+    return new Path(tableDir, ServerRegionReplicaUtil.getRegionInfoForFs(region).getEncodedName());
+  }
+
   /**
    * Filter for all dirs that are legal column family names.  This is generally used for colfam
    * dirs &lt;hbase.rootdir&gt;/&lt;tabledir&gt;/&lt;regiondir&gt;/&lt;colfamdir&gt;.

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/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 c1a77ee..ac23d1d 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
@@ -415,10 +415,16 @@ public class WALKeyImpl implements WALKey {
     this.replicationScope = replicationScope;
   }
 
-  public void serializeReplicationScope(boolean serialize) {
-    if (!serialize) {
-      setReplicationScope(null);
+  public void clearReplicationScope() {
+    setReplicationScope(null);
+  }
+
+  public boolean hasSerialReplicationScope() {
+    if (replicationScope == null || replicationScope.isEmpty()) {
+      return false;
     }
+    return replicationScope.values().stream()
+      .anyMatch(scope -> scope.intValue() == HConstants.REPLICATION_SCOPE_SERIAL);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
index ec93207..9161e25 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
@@ -494,7 +494,7 @@ public class TestMetaTableAccessor {
       List<RegionInfo> regionInfos = Lists.newArrayList(parent);
       MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3);
 
-      MetaTableAccessor.splitRegion(connection, parent, splitA, splitB, serverName0, 3);
+      MetaTableAccessor.splitRegion(connection, parent, -1L, splitA, splitB, serverName0, 3);
 
       assertEmptyMetaLocation(meta, splitA.getRegionName(), 1);
       assertEmptyMetaLocation(meta, splitA.getRegionName(), 2);
@@ -535,7 +535,8 @@ public class TestMetaTableAccessor {
       List<RegionInfo> regionInfos = Lists.newArrayList(parentA, parentB);
       MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3);
 
-      MetaTableAccessor.mergeRegions(connection, merged, parentA, parentB, serverName0, 3);
+      MetaTableAccessor.mergeRegions(connection, merged, parentA, -1L, parentB, -1L, serverName0,
+        3);
 
       assertEmptyMetaLocation(meta, merged.getRegionName(), 1);
       assertEmptyMetaLocation(meta, merged.getRegionName(), 2);
@@ -682,8 +683,8 @@ public class TestMetaTableAccessor {
       EnvironmentEdgeManager.injectEdge(edge);
       try {
         // now merge the regions, effectively deleting the rows for region a and b.
-        MetaTableAccessor.mergeRegions(connection, mergedRegionInfo, regionInfoA, regionInfoB, sn,
-          1);
+        MetaTableAccessor.mergeRegions(connection, mergedRegionInfo, regionInfoA, -1L, regionInfoB,
+          -1L, sn, 1);
       } finally {
         EnvironmentEdgeManager.reset();
       }
@@ -776,7 +777,8 @@ public class TestMetaTableAccessor {
       }
       SpyingRpcScheduler scheduler = (SpyingRpcScheduler) rs.getRpcServer().getScheduler();
       long prevCalls = scheduler.numPriorityCalls;
-      MetaTableAccessor.splitRegion(connection, parent, splitA, splitB, loc.getServerName(), 1);
+      MetaTableAccessor.splitRegion(connection, parent, -1L, splitA, splitB, loc.getServerName(),
+        1);
 
       assertTrue(prevCalls < scheduler.numPriorityCalls);
     }
@@ -813,7 +815,7 @@ public class TestMetaTableAccessor {
       List<RegionInfo> regionInfos = Lists.newArrayList(parent);
       MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3);
 
-      MetaTableAccessor.splitRegion(connection, parent, splitA, splitB, serverName0, 3);
+      MetaTableAccessor.splitRegion(connection, parent, -1L, splitA, splitB, serverName0, 3);
       Get get1 = new Get(splitA.getRegionName());
       Result resultA = meta.get(get1);
       Cell serverCellA = resultA.getColumnLatestCell(HConstants.CATALOG_FAMILY,

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java
index a4e8e19..e00f072 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionFileSystem.java
@@ -194,7 +194,7 @@ public class TestHRegionFileSystem {
 
   @Test
   public void testOnDiskRegionCreation() throws IOException {
-    Path rootDir = TEST_UTIL.getDataTestDirOnTestFS("testOnDiskRegionCreation");
+    Path rootDir = TEST_UTIL.getDataTestDirOnTestFS(name.getMethodName());
     FileSystem fs = TEST_UTIL.getTestFileSystem();
     Configuration conf = TEST_UTIL.getConfiguration();
 
@@ -226,7 +226,7 @@ public class TestHRegionFileSystem {
 
   @Test
   public void testNonIdempotentOpsWithRetries() throws IOException {
-    Path rootDir = TEST_UTIL.getDataTestDirOnTestFS("testOnDiskRegionCreation");
+    Path rootDir = TEST_UTIL.getDataTestDirOnTestFS(name.getMethodName());
     FileSystem fs = TEST_UTIL.getTestFileSystem();
     Configuration conf = TEST_UTIL.getConfiguration();
 
@@ -235,19 +235,15 @@ public class TestHRegionFileSystem {
     HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(conf, fs, rootDir, hri);
     assertTrue(fs.exists(regionFs.getRegionDir()));
 
-    regionFs = new HRegionFileSystem(conf, new MockFileSystemForCreate(),
-        null, null);
-    // HRegionFileSystem.createRegionOnFileSystem(conf, new MockFileSystemForCreate(), rootDir,
-    // hri);
+    regionFs = new HRegionFileSystem(conf, new MockFileSystemForCreate(), rootDir, hri);
     boolean result = regionFs.createDir(new Path("/foo/bar"));
     assertTrue("Couldn't create the directory", result);
 
-
-    regionFs = new HRegionFileSystem(conf, new MockFileSystem(), null, null);
+    regionFs = new HRegionFileSystem(conf, new MockFileSystem(), rootDir, hri);
     result = regionFs.rename(new Path("/foo/bar"), new Path("/foo/bar2"));
     assertTrue("Couldn't rename the directory", result);
 
-    regionFs = new HRegionFileSystem(conf, new MockFileSystem(), null, null);
+    regionFs = new HRegionFileSystem(conf, new MockFileSystem(), rootDir, hri);
     result = regionFs.deleteDir(new Path("/foo/bar"));
     assertTrue("Couldn't delete the directory", result);
     fs.delete(rootDir, true);

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
index 50dffd5..fab6512 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
@@ -343,12 +343,12 @@ public class TestRegionServerMetrics {
 
   @Test
   public void testStoreCount() throws Exception {
-    //Force a hfile.
+    // Force a hfile.
     doNPuts(1, false);
     TEST_UTIL.getAdmin().flush(tableName);
 
     metricsRegionServer.getRegionServerWrapper().forceRecompute();
-    assertGauge("storeCount", TABLES_ON_MASTER? 1: 4);
+    assertGauge("storeCount", TABLES_ON_MASTER ? 1 : 5);
     assertGauge("storeFileCount", 1);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDroppedTables.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDroppedTables.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDroppedTables.java
index ffa03a2..e9e92b8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDroppedTables.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationDroppedTables.java
@@ -20,8 +20,6 @@ package org.apache.hadoop.hbase.replication;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.fail;
 
-import java.util.ArrayList;
-import java.util.List;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -38,6 +36,7 @@ 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.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.junit.Before;
@@ -47,7 +46,7 @@ import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-@Category(LargeTests.class)
+@Category({ ReplicationTests.class, LargeTests.class })
 public class TestReplicationDroppedTables extends TestReplicationBase {
 
   @ClassRule
@@ -56,9 +55,6 @@ public class TestReplicationDroppedTables extends TestReplicationBase {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestReplicationDroppedTables.class);
 
-  /**
-   * @throws java.lang.Exception
-   */
   @Before
   public void setUp() throws Exception {
     // Starting and stopping replication can make us miss new logs,

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
new file mode 100644
index 0000000..1408cf0
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.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.replication;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.UUID;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+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.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALProvider;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+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;
+
+@Category({ ReplicationTests.class, LargeTests.class })
+public class TestSerialReplication {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestSerialReplication.class);
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static String PEER_ID = "1";
+
+  private static byte[] CF = Bytes.toBytes("CF");
+
+  private static byte[] CQ = Bytes.toBytes("CQ");
+
+  private static FileSystem FS;
+
+  private static Path LOG_DIR;
+
+  private static WALProvider.Writer WRITER;
+
+  public static final class LocalReplicationEndpoint extends BaseReplicationEndpoint {
+
+    private static final UUID PEER_UUID = UUID.randomUUID();
+
+    @Override
+    public UUID getPeerUUID() {
+      return PEER_UUID;
+    }
+
+    @Override
+    public boolean replicate(ReplicateContext replicateContext) {
+      synchronized (WRITER) {
+        try {
+          for (Entry entry : replicateContext.getEntries()) {
+            WRITER.append(entry);
+          }
+          WRITER.sync(false);
+        } catch (IOException e) {
+          throw new UncheckedIOException(e);
+        }
+      }
+      return true;
+    }
+
+    @Override
+    public void start() {
+      startAsync();
+    }
+
+    @Override
+    public void stop() {
+      stopAsync();
+    }
+
+    @Override
+    protected void doStart() {
+      notifyStarted();
+    }
+
+    @Override
+    protected void doStop() {
+      notifyStopped();
+    }
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    UTIL.getConfiguration().setInt("replication.source.nb.capacity", 10);
+    UTIL.startMiniCluster(3);
+    LOG_DIR = UTIL.getDataTestDirOnTestFS("replicated");
+    FS = UTIL.getTestFileSystem();
+    FS.mkdirs(LOG_DIR);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Rule
+  public final TestName name = new TestName();
+
+  private Path logPath;
+
+  @Before
+  public void setUp() throws IOException, StreamLacksCapabilityException {
+    UTIL.ensureSomeRegionServersAvailable(3);
+    logPath = new Path(LOG_DIR, name.getMethodName());
+    WRITER = WALFactory.createWALWriter(FS, logPath, UTIL.getConfiguration());
+    // add in disable state, so later when enabling it all sources will start push together.
+    UTIL.getAdmin().addReplicationPeer(PEER_ID,
+      ReplicationPeerConfig.newBuilder().setClusterKey("127.0.0.1:2181:/hbase")
+        .setReplicationEndpointImpl(LocalReplicationEndpoint.class.getName()).build(),
+      false);
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    UTIL.getAdmin().removeReplicationPeer(PEER_ID);
+    if (WRITER != null) {
+      WRITER.close();
+      WRITER = null;
+    }
+  }
+
+  @Test
+  public void testRegionMove() throws Exception {
+    TableName tableName = TableName.valueOf(name.getMethodName());
+    UTIL.getAdmin().createTable(
+      TableDescriptorBuilder.newBuilder(tableName).addColumnFamily(ColumnFamilyDescriptorBuilder
+        .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_SERIAL).build()).build());
+    UTIL.waitTableAvailable(tableName);
+    try (Table table = UTIL.getConnection().getTable(tableName)) {
+      for (int i = 0; i < 100; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+    RegionInfo region = UTIL.getAdmin().getRegions(tableName).get(0);
+    HRegionServer rs = UTIL.getOtherRegionServer(UTIL.getRSForFirstRegionInTable(tableName));
+    UTIL.getAdmin().move(region.getEncodedNameAsBytes(),
+      Bytes.toBytes(rs.getServerName().getServerName()));
+    UTIL.waitFor(30000, new ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        return !rs.getRegions(tableName).isEmpty();
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        return region + " is still not on " + rs;
+      }
+    });
+    try (Table table = UTIL.getConnection().getTable(tableName)) {
+      for (int i = 100; i < 200; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+    UTIL.getAdmin().enableReplicationPeer(PEER_ID);
+    UTIL.waitFor(30000, new ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        try (WAL.Reader reader = WALFactory.createReader(FS, logPath, UTIL.getConfiguration())) {
+          int count = 0;
+          while (reader.next() != null) {
+            count++;
+          }
+          return count >= 200;
+        } catch (IOException e) {
+          return false;
+        }
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        return "Not enough entries replicated";
+      }
+    });
+    try (WAL.Reader reader =
+      WALFactory.createReader(UTIL.getTestFileSystem(), logPath, UTIL.getConfiguration())) {
+      long seqId = -1L;
+      int count = 0;
+      for (Entry entry;;) {
+        entry = reader.next();
+        if (entry == null) {
+          break;
+        }
+        assertTrue(
+          "Sequence id go backwards from " + seqId + " to " + entry.getKey().getSequenceId(),
+          entry.getKey().getSequenceId() >= seqId);
+        count++;
+      }
+      assertEquals(200, count);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/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..6d75fec 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
@@ -321,7 +321,7 @@ public abstract class TestReplicationSourceManager {
     wal.rollWriter();
 
     manager.logPositionAndCleanOldLogs(manager.getSources().get(0).getCurrentPath(),
-        "1", 0, false);
+        "1", 0, null, false);
 
     wal.append(hri,
         new WALKeyImpl(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc, scopes),

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java
new file mode 100644
index 0000000..c8387c5
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java
@@ -0,0 +1,176 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication.regionserver;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.CellBuilderFactory;
+import org.apache.hadoop.hbase.CellBuilderType;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WALKeyImpl;
+import org.junit.AfterClass;
+import org.junit.Before;
+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.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
+
+@Category({ ReplicationTests.class, MediumTests.class })
+public class TestSerialReplicationChecker {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestSerialReplicationChecker.class);
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static String PEER_ID = "1";
+
+  private static ReplicationQueueStorage QUEUE_STORAGE;
+
+  private static String WAL_FILE_NAME = "test.wal";
+
+  private SerialReplicationChecker checker;
+
+  @Rule
+  public final TestName name = new TestName();
+
+  private TableName tableName;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    UTIL.startMiniCluster(1);
+    QUEUE_STORAGE = ReplicationStorageFactory.getReplicationQueueStorage(UTIL.getZooKeeperWatcher(),
+      UTIL.getConfiguration());
+    QUEUE_STORAGE.addWAL(UTIL.getMiniHBaseCluster().getRegionServer(0).getServerName(), PEER_ID,
+      WAL_FILE_NAME);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Before
+  public void setUp() throws IOException {
+    ReplicationSource source = mock(ReplicationSource.class);
+    when(source.getPeerId()).thenReturn(PEER_ID);
+    when(source.getQueueStorage()).thenReturn(QUEUE_STORAGE);
+    Server server = mock(Server.class);
+    when(server.getConnection()).thenReturn(UTIL.getConnection());
+    when(source.getServer()).thenReturn(server);
+    checker = new SerialReplicationChecker(UTIL.getConfiguration(), source);
+    tableName = TableName.valueOf(name.getMethodName());
+  }
+
+  private Entry createEntry(RegionInfo region, long seqId) {
+    WALKeyImpl key = mock(WALKeyImpl.class);
+    when(key.getTableName()).thenReturn(tableName);
+    when(key.getEncodedRegionName()).thenReturn(region.getEncodedNameAsBytes());
+    when(key.getSequenceId()).thenReturn(seqId);
+    Entry entry = mock(Entry.class);
+    when(entry.getKey()).thenReturn(key);
+    return entry;
+  }
+
+  private Cell createCell(RegionInfo region) {
+    return CellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(region.getStartKey())
+      .setType(Type.Put).build();
+  }
+
+  @Test
+  public void testNoBarrierCanPush() throws IOException {
+    RegionInfo region = RegionInfoBuilder.newBuilder(tableName).build();
+    assertTrue(checker.canPush(createEntry(region, 100), createCell(region)));
+  }
+
+  private void addStateAndBarrier(RegionInfo region, RegionState.State state, long... barriers)
+      throws IOException {
+    Put put = new Put(region.getRegionName(), EnvironmentEdgeManager.currentTime());
+    put.addColumn(HConstants.CATALOG_FAMILY, HConstants.STATE_QUALIFIER,
+      Bytes.toBytes(state.name()));
+    for (int i = 0; i < barriers.length; i++) {
+      put.addColumn(HConstants.REPLICATION_BARRIER_FAMILY, HConstants.SEQNUM_QUALIFIER,
+        put.getTimeStamp() - i, Bytes.toBytes(barriers[i]));
+    }
+    try (Table table = UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) {
+      table.put(put);
+    }
+  }
+
+  private void setState(RegionInfo region, RegionState.State state) throws IOException {
+    Put put = new Put(region.getRegionName(), EnvironmentEdgeManager.currentTime());
+    put.addColumn(HConstants.CATALOG_FAMILY, HConstants.STATE_QUALIFIER,
+      Bytes.toBytes(state.name()));
+    try (Table table = UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) {
+      table.put(put);
+    }
+  }
+
+  private void updatePushedSeqId(RegionInfo region, long seqId) throws ReplicationException {
+    QUEUE_STORAGE.setWALPosition(UTIL.getMiniHBaseCluster().getRegionServer(0).getServerName(),
+      PEER_ID, WAL_FILE_NAME, 10, ImmutableMap.of(region.getEncodedName(), seqId));
+  }
+
+  @Test
+  public void testLastRegionAndOpeningCanNotPush() throws IOException, ReplicationException {
+    RegionInfo region = RegionInfoBuilder.newBuilder(tableName).build();
+    addStateAndBarrier(region, RegionState.State.OPEN, 10);
+    Cell cell = createCell(region);
+    // can push since we are in the first range
+    assertTrue(checker.canPush(createEntry(region, 100), cell));
+    setState(region, RegionState.State.OPENING);
+    // can not push since we are in the last range and the state is OPENING
+    assertFalse(checker.canPush(createEntry(region, 102), cell));
+    addStateAndBarrier(region, RegionState.State.OPEN, 50);
+    // can not push since the previous range has not been finished yet
+    assertFalse(checker.canPush(createEntry(region, 102), cell));
+    updatePushedSeqId(region, 49);
+    // can push since the previous range has been finished
+    assertTrue(checker.canPush(createEntry(region, 102), cell));
+    setState(region, RegionState.State.OPENING);
+    assertFalse(checker.canPush(createEntry(region, 104), cell));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/f29bf1d7/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
index 2146e47..eb7d5a0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
@@ -21,13 +21,13 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.util.NavigableMap;
-import java.util.NoSuchElementException;
 import java.util.OptionalLong;
 import java.util.TreeMap;
 import java.util.concurrent.PriorityBlockingQueue;
@@ -40,13 +40,13 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Server;
 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.regionserver.wal.WALActionsListener;
 import org.apache.hadoop.hbase.replication.WALEntryFilter;
-import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.WALEntryBatch;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -180,15 +180,12 @@ public class TestWALEntryStream {
         new WALEntryStream(walQueue, fs, conf, 0, log, null, new MetricsSource("1"))) {
       // There's one edit in the log, read it. Reading past it needs to throw exception
       assertTrue(entryStream.hasNext());
-      WAL.Entry entry = entryStream.next();
+      WAL.Entry entry = entryStream.peek();
+      assertSame(entry, entryStream.next());
       assertNotNull(entry);
       assertFalse(entryStream.hasNext());
-      try {
-        entry = entryStream.next();
-        fail();
-      } catch (NoSuchElementException e) {
-        // expected
-      }
+      assertNull(entryStream.peek());
+      assertNull(entryStream.next());
       oldPos = entryStream.getPosition();
     }
 
@@ -346,10 +343,12 @@ public class TestWALEntryStream {
     // start up a batcher
     ReplicationSourceManager mockSourceManager = Mockito.mock(ReplicationSourceManager.class);
     when(mockSourceManager.getTotalBufferUsed()).thenReturn(new AtomicLong(0));
+    Server mockServer= Mockito.mock(Server.class);
     ReplicationSource source = Mockito.mock(ReplicationSource.class);
     when(source.getSourceManager()).thenReturn(mockSourceManager);
     when(source.getSourceMetrics()).thenReturn(new MetricsSource("1"));
     when(source.getWALFileLengthProvider()).thenReturn(log);
+    when(source.getServer()).thenReturn(mockServer);
     ReplicationSourceWALReader batcher = new ReplicationSourceWALReader(fs, conf,
         walQueue, 0, getDummyFilter(), source);
     Path walPath = walQueue.peek();


[04/22] hbase git commit: HBASE-20147 Serial replication will be stuck if we create a table with serial replication but add it to a peer after there are region moves

Posted by zh...@apache.org.
HBASE-20147 Serial replication will be stuck if we create a table with serial replication but add it to a peer after there are region moves


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

Branch: refs/heads/HBASE-20046-branch-2
Commit: 9369cf26eb404c91831efe9926b6d5a8146e80b6
Parents: 40ad5ae
Author: zhangduo <zh...@apache.org>
Authored: Wed Mar 21 21:03:14 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Apr 9 15:18:44 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/AsyncMetaTableAccessor.java    |  50 ++--
 .../apache/hadoop/hbase/MetaTableAccessor.java  |  26 ++-
 .../src/main/protobuf/MasterProcedure.proto     |   7 +-
 .../replication/ReplicationQueueStorage.java    |   8 +
 .../hbase/replication/ReplicationUtils.java     |   6 +-
 .../replication/ZKReplicationQueueStorage.java  |  50 ++--
 .../master/replication/AddPeerProcedure.java    |  21 +-
 .../master/replication/ModifyPeerProcedure.java | 166 +++++++++++++-
 .../replication/ReplicationPeerManager.java     |  32 +--
 .../replication/UpdatePeerConfigProcedure.java  |  59 ++++-
 .../regionserver/PeerProcedureHandlerImpl.java  |  17 +-
 .../regionserver/ReplicationSourceManager.java  |   4 +-
 .../replication/regionserver/WALEntryBatch.java |   8 +
 .../replication/SerialReplicationTestBase.java  | 229 +++++++++++++++++++
 .../TestAddToSerialReplicationPeer.java         | 215 +++++++++++++++++
 .../replication/TestSerialReplication.java      | 191 +---------------
 16 files changed, 825 insertions(+), 264 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/9369cf26/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
index 05e60d4..13245d3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
@@ -489,16 +489,17 @@ public class AsyncMetaTableAccessor {
       QueryType type) {
     return tableName.map((table) -> {
       switch (type) {
-      case REGION:
-        byte[] startRow = new byte[table.getName().length + 2];
-        System.arraycopy(table.getName(), 0, startRow, 0, table.getName().length);
-        startRow[startRow.length - 2] = HConstants.DELIMITER;
-        startRow[startRow.length - 1] = HConstants.DELIMITER;
-        return startRow;
-      case ALL:
-      case TABLE:
-      default:
-        return table.getName();
+        case REGION:
+        case REPLICATION:
+          byte[] startRow = new byte[table.getName().length + 2];
+          System.arraycopy(table.getName(), 0, startRow, 0, table.getName().length);
+          startRow[startRow.length - 2] = HConstants.DELIMITER;
+          startRow[startRow.length - 1] = HConstants.DELIMITER;
+          return startRow;
+        case ALL:
+        case TABLE:
+        default:
+          return table.getName();
       }
     });
   }
@@ -512,20 +513,21 @@ public class AsyncMetaTableAccessor {
     return tableName.map((table) -> {
       final byte[] stopRow;
       switch (type) {
-      case REGION:
-        stopRow = new byte[table.getName().length + 3];
-        System.arraycopy(table.getName(), 0, stopRow, 0, table.getName().length);
-        stopRow[stopRow.length - 3] = ' ';
-        stopRow[stopRow.length - 2] = HConstants.DELIMITER;
-        stopRow[stopRow.length - 1] = HConstants.DELIMITER;
-        break;
-      case ALL:
-      case TABLE:
-      default:
-        stopRow = new byte[table.getName().length + 1];
-        System.arraycopy(table.getName(), 0, stopRow, 0, table.getName().length);
-        stopRow[stopRow.length - 1] = ' ';
-        break;
+        case REGION:
+        case REPLICATION:
+          stopRow = new byte[table.getName().length + 3];
+          System.arraycopy(table.getName(), 0, stopRow, 0, table.getName().length);
+          stopRow[stopRow.length - 3] = ' ';
+          stopRow[stopRow.length - 2] = HConstants.DELIMITER;
+          stopRow[stopRow.length - 1] = HConstants.DELIMITER;
+          break;
+        case ALL:
+        case TABLE:
+        default:
+          stopRow = new byte[table.getName().length + 1];
+          System.arraycopy(table.getName(), 0, stopRow, 0, table.getName().length);
+          stopRow[stopRow.length - 1] = ' ';
+          break;
       }
       return stopRow;
     });

http://git-wip-us.apache.org/repos/asf/hbase/blob/9369cf26/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 a800c1c..4cc46c8 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
@@ -192,7 +192,8 @@ public class MetaTableAccessor {
   public enum QueryType {
     ALL(HConstants.TABLE_FAMILY, HConstants.CATALOG_FAMILY),
     REGION(HConstants.CATALOG_FAMILY),
-    TABLE(HConstants.TABLE_FAMILY);
+    TABLE(HConstants.TABLE_FAMILY),
+    REPLICATION(HConstants.REPLICATION_BARRIER_FAMILY);
 
     private final byte[][] families;
 
@@ -1168,8 +1169,9 @@ public class MetaTableAccessor {
     final List<T> results = new ArrayList<>();
     @Override
     public boolean visit(Result r) throws IOException {
-      if (r ==  null || r.isEmpty()) return true;
-      add(r);
+      if (r != null && !r.isEmpty()) {
+        add(r);
+      }
       return true;
     }
 
@@ -2108,6 +2110,24 @@ public class MetaTableAccessor {
     }
   }
 
+  public static List<Pair<String, Long>> getTableEncodedRegionNameAndLastBarrier(Connection conn,
+      TableName tableName) throws IOException {
+    List<Pair<String, Long>> list = new ArrayList<>();
+    scanMeta(conn, getTableStartRowForMeta(tableName, QueryType.REPLICATION),
+      getTableStopRowForMeta(tableName, QueryType.REPLICATION), QueryType.REPLICATION, r -> {
+        byte[] value =
+          r.getValue(HConstants.REPLICATION_BARRIER_FAMILY, HConstants.SEQNUM_QUALIFIER);
+        if (value == null) {
+          return true;
+        }
+        long lastBarrier = Bytes.toLong(value);
+        String encodedRegionName = RegionInfo.encodeRegionName(r.getRow());
+        list.add(Pair.newPair(encodedRegionName, lastBarrier));
+        return true;
+      });
+    return list;
+  }
+
   private static void debugLogMutations(List<? extends Mutation> mutations) throws IOException {
     if (!METALOG.isDebugEnabled()) {
       return;

http://git-wip-us.apache.org/repos/asf/hbase/blob/9369cf26/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 fa6fa75..f710759 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -377,7 +377,11 @@ enum PeerModificationState {
   PRE_PEER_MODIFICATION = 1;
   UPDATE_PEER_STORAGE = 2;
   REFRESH_PEER_ON_RS = 3;
-  POST_PEER_MODIFICATION = 4;
+  SERIAL_PEER_REOPEN_REGIONS = 4;
+  SERIAL_PEER_UPDATE_LAST_PUSHED_SEQ_ID = 5;
+  SERIAL_PEER_SET_PEER_ENABLED = 6;
+  SERIAL_PEER_ENABLE_PEER_REFRESH_PEER_ON_RS = 7;
+  POST_PEER_MODIFICATION = 8;
 }
 
 message PeerModificationStateData {
@@ -415,4 +419,5 @@ message AddPeerStateData {
 
 message UpdatePeerConfigStateData {
   required ReplicationPeer peer_config = 1;
+  optional ReplicationPeer old_peer_config = 2;
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/9369cf26/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
index cfe9c9c..99a1e97 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
@@ -79,6 +79,14 @@ public interface ReplicationQueueStorage {
   long getLastSequenceId(String encodedRegionName, String peerId) throws ReplicationException;
 
   /**
+   * Set the max sequence id of a bunch of regions for a given peer. Will be called when setting up
+   * a serial replication peer.
+   * @param peerId peer id
+   * @param lastSeqIds map with {encodedRegionName, sequenceId} pairs for serial replication.
+   */
+  void setLastSequenceIds(String peerId, Map<String, Long> lastSeqIds) throws ReplicationException;
+
+  /**
    * Get the current position for a specific WAL in a given queue for a given regionserver.
    * @param serverName the name of the regionserver
    * @param queueId a String that identifies the queue

http://git-wip-us.apache.org/repos/asf/hbase/blob/9369cf26/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 e2479e0..1c42de4 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
@@ -111,13 +111,11 @@ public final class ReplicationUtils {
     return true;
   }
 
-  public static boolean isKeyConfigEqual(ReplicationPeerConfig rpc1, ReplicationPeerConfig rpc2) {
+  public static boolean isNamespacesAndTableCFsEqual(ReplicationPeerConfig rpc1,
+      ReplicationPeerConfig rpc2) {
     if (rpc1.replicateAllUserTables() != rpc2.replicateAllUserTables()) {
       return false;
     }
-    if (rpc1.isSerial() != rpc2.isSerial()) {
-      return false;
-    }
     if (rpc1.replicateAllUserTables()) {
       return isNamespacesEqual(rpc1.getExcludeNamespaces(), rpc2.getExcludeNamespaces()) &&
         isTableCFsEqual(rpc1.getExcludeTableCFsMap(), rpc2.getExcludeTableCFsMap());

http://git-wip-us.apache.org/repos/asf/hbase/blob/9369cf26/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
index 1a5749e..2ab08ae 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
@@ -202,6 +202,24 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
     }
   }
 
+  private void addLastSeqIdsToOps(String queueId, Map<String, Long> lastSeqIds,
+      List<ZKUtilOp> listOfOps) throws KeeperException {
+    for (Entry<String, Long> lastSeqEntry : lastSeqIds.entrySet()) {
+      String peerId = new ReplicationQueueInfo(queueId).getPeerId();
+      String path = getSerialReplicationRegionPeerNode(lastSeqEntry.getKey(), peerId);
+      /*
+       * Make sure the existence of path
+       * /hbase/replication/regions/<hash>/<encoded-region-name>-<peer-id>. As the javadoc in
+       * multiOrSequential() method said, if received a NodeExistsException, all operations will
+       * fail. So create the path here, and in fact, no need to add this operation to listOfOps,
+       * because only need to make sure that update file position and sequence id atomically.
+       */
+      ZKUtil.createWithParents(zookeeper, path);
+      // Persist the max sequence id of region to zookeeper.
+      listOfOps.add(ZKUtilOp.setData(path, ZKUtil.positionToByteArray(lastSeqEntry.getValue())));
+    }
+  }
+
   @Override
   public void setWALPosition(ServerName serverName, String queueId, String fileName, long position,
       Map<String, Long> lastSeqIds) throws ReplicationException {
@@ -212,23 +230,8 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
           ZKUtil.positionToByteArray(position)));
       }
       // Persist the max sequence id(s) of regions for serial replication atomically.
-      for (Entry<String, Long> lastSeqEntry : lastSeqIds.entrySet()) {
-        String peerId = new ReplicationQueueInfo(queueId).getPeerId();
-        String path = getSerialReplicationRegionPeerNode(lastSeqEntry.getKey(), peerId);
-        /*
-         * Make sure the existence of path
-         * /hbase/replication/regions/<hash>/<encoded-region-name>-<peer-id>. As the javadoc in
-         * multiOrSequential() method said, if received a NodeExistsException, all operations will
-         * fail. So create the path here, and in fact, no need to add this operation to listOfOps,
-         * because only need to make sure that update file position and sequence id atomically.
-         */
-        ZKUtil.createWithParents(zookeeper, path);
-        // Persist the max sequence id of region to zookeeper.
-        listOfOps.add(ZKUtilOp.setData(path, ZKUtil.positionToByteArray(lastSeqEntry.getValue())));
-      }
-      if (!listOfOps.isEmpty()) {
-        ZKUtil.multiOrSequential(zookeeper, listOfOps, false);
-      }
+      addLastSeqIdsToOps(queueId, lastSeqIds, listOfOps);
+      ZKUtil.multiOrSequential(zookeeper, listOfOps, false);
     } catch (KeeperException e) {
       throw new ReplicationException("Failed to set log position (serverName=" + serverName
           + ", queueId=" + queueId + ", fileName=" + fileName + ", position=" + position + ")", e);
@@ -256,6 +259,19 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
   }
 
   @Override
+  public void setLastSequenceIds(String peerId, Map<String, Long> lastSeqIds)
+      throws ReplicationException {
+    try {
+      List<ZKUtilOp> listOfOps = new ArrayList<>();
+      addLastSeqIdsToOps(peerId, lastSeqIds, listOfOps);
+      ZKUtil.multiOrSequential(zookeeper, listOfOps, false);
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to set last sequence ids, peerId=" + peerId +
+        ", lastSeqIds.size=" + lastSeqIds.size(), e);
+    }
+  }
+
+  @Override
   public long getWALPosition(ServerName serverName, String queueId, String fileName)
       throws ReplicationException {
     byte[] bytes;

http://git-wip-us.apache.org/repos/asf/hbase/blob/9369cf26/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
index f0f7704..72228f6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
@@ -57,6 +57,21 @@ public class AddPeerProcedure extends ModifyPeerProcedure {
   }
 
   @Override
+  protected boolean reopenRegionsAfterRefresh() {
+    return true;
+  }
+
+  @Override
+  protected boolean enablePeerBeforeFinish() {
+    return enabled;
+  }
+
+  @Override
+  protected ReplicationPeerConfig getNewPeerConfig() {
+    return peerConfig;
+  }
+
+  @Override
   protected void prePeerModification(MasterProcedureEnv env)
       throws IOException, ReplicationException {
     MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
@@ -68,11 +83,13 @@ public class AddPeerProcedure extends ModifyPeerProcedure {
 
   @Override
   protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
-    env.getReplicationPeerManager().addPeer(peerId, peerConfig, enabled);
+    env.getReplicationPeerManager().addPeer(peerId, peerConfig,
+      peerConfig.isSerial() ? false : enabled);
   }
 
   @Override
-  protected void postPeerModification(MasterProcedureEnv env) throws IOException {
+  protected void postPeerModification(MasterProcedureEnv env)
+      throws IOException, ReplicationException {
     LOG.info("Successfully added {} peer {}, config {}", enabled ? "ENABLED" : "DISABLED", peerId,
       peerConfig);
     MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();

http://git-wip-us.apache.org/repos/asf/hbase/blob/9369cf26/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
index 83c5134..2b76487 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
@@ -18,11 +18,28 @@
 package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Stream;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.master.TableStateManager;
+import org.apache.hadoop.hbase.master.TableStateManager.TableStateNotFoundException;
+import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
 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.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.replication.ReplicationUtils;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.wal.WALSplitter;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -38,6 +55,8 @@ public abstract class ModifyPeerProcedure extends AbstractPeerProcedure<PeerModi
 
   private static final Logger LOG = LoggerFactory.getLogger(ModifyPeerProcedure.class);
 
+  private static final int SET_LAST_SEQ_ID_BATCH_SIZE = 1000;
+
   protected ModifyPeerProcedure() {
   }
 
@@ -73,6 +92,114 @@ public abstract class ModifyPeerProcedure extends AbstractPeerProcedure<PeerModi
     ProcedurePrepareLatch.releaseLatch(latch, this);
   }
 
+  /**
+   * Implementation class can override this method. The default return value is false which means we
+   * will jump to POST_PEER_MODIFICATION and finish the procedure. If returns true, we will jump to
+   * SERIAL_PEER_REOPEN_REGIONS.
+   */
+  protected boolean reopenRegionsAfterRefresh() {
+    return false;
+  }
+
+  /**
+   * The implementation class should override this method if the procedure may enter the serial
+   * related states.
+   */
+  protected boolean enablePeerBeforeFinish() {
+    throw new UnsupportedOperationException();
+  }
+
+  private void refreshPeer(MasterProcedureEnv env, PeerOperationType type) {
+    addChildProcedure(env.getMasterServices().getServerManager().getOnlineServersList().stream()
+      .map(sn -> new RefreshPeerProcedure(peerId, type, sn))
+      .toArray(RefreshPeerProcedure[]::new));
+  }
+
+  protected ReplicationPeerConfig getOldPeerConfig() {
+    return null;
+  }
+
+  protected ReplicationPeerConfig getNewPeerConfig() {
+    throw new UnsupportedOperationException();
+  }
+
+  private Stream<TableDescriptor> getTables(MasterProcedureEnv env) throws IOException {
+    ReplicationPeerConfig peerConfig = getNewPeerConfig();
+    Stream<TableDescriptor> stream = env.getMasterServices().getTableDescriptors().getAll().values()
+      .stream().filter(TableDescriptor::hasGlobalReplicationScope)
+      .filter(td -> ReplicationUtils.contains(peerConfig, td.getTableName()));
+    ReplicationPeerConfig oldPeerConfig = getOldPeerConfig();
+    if (oldPeerConfig != null && oldPeerConfig.isSerial()) {
+      stream = stream.filter(td -> !ReplicationUtils.contains(oldPeerConfig, td.getTableName()));
+    }
+    return stream;
+  }
+
+  private void reopenRegions(MasterProcedureEnv env) throws IOException {
+    Stream<TableDescriptor> stream = getTables(env);
+    TableStateManager tsm = env.getMasterServices().getTableStateManager();
+    stream.filter(td -> {
+      try {
+        return tsm.getTableState(td.getTableName()).isEnabled();
+      } catch (TableStateNotFoundException e) {
+        return false;
+      } catch (IOException e) {
+        throw new UncheckedIOException(e);
+      }
+    }).forEach(td -> {
+      try {
+        addChildProcedure(env.getAssignmentManager().createReopenProcedures(
+          env.getAssignmentManager().getRegionStates().getRegionsOfTable(td.getTableName())));
+      } catch (IOException e) {
+        throw new UncheckedIOException(e);
+      }
+    });
+  }
+
+  private void addToMap(Map<String, Long> lastSeqIds, String encodedRegionName, long barrier,
+      ReplicationQueueStorage queueStorage) throws ReplicationException {
+    if (barrier >= 0) {
+      lastSeqIds.put(encodedRegionName, barrier);
+      if (lastSeqIds.size() >= SET_LAST_SEQ_ID_BATCH_SIZE) {
+        queueStorage.setLastSequenceIds(peerId, lastSeqIds);
+        lastSeqIds.clear();
+      }
+    }
+  }
+
+  private void setLastSequenceIdForSerialPeer(MasterProcedureEnv env)
+      throws IOException, ReplicationException {
+    Stream<TableDescriptor> stream = getTables(env);
+    TableStateManager tsm = env.getMasterServices().getTableStateManager();
+    ReplicationQueueStorage queueStorage = env.getReplicationPeerManager().getQueueStorage();
+    Connection conn = env.getMasterServices().getConnection();
+    RegionStates regionStates = env.getAssignmentManager().getRegionStates();
+    MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
+    Map<String, Long> lastSeqIds = new HashMap<String, Long>();
+    stream.forEach(td -> {
+      try {
+        if (tsm.getTableState(td.getTableName()).isEnabled()) {
+          for (Pair<String, Long> name2Barrier : MetaTableAccessor
+            .getTableEncodedRegionNameAndLastBarrier(conn, td.getTableName())) {
+            addToMap(lastSeqIds, name2Barrier.getFirst(), name2Barrier.getSecond().longValue() - 1,
+              queueStorage);
+          }
+        } else {
+          for (RegionInfo region : regionStates.getRegionsOfTable(td.getTableName(), true)) {
+            long maxSequenceId =
+              WALSplitter.getMaxRegionSequenceId(mfs.getFileSystem(), mfs.getRegionDir(region));
+            addToMap(lastSeqIds, region.getEncodedName(), maxSequenceId, queueStorage);
+          }
+        }
+      } catch (IOException | ReplicationException e) {
+        throw new RuntimeException(e);
+      }
+    });
+    if (!lastSeqIds.isEmpty()) {
+      queueStorage.setLastSequenceIds(peerId, lastSeqIds);
+    }
+  }
+
   @Override
   protected Flow executeFromState(MasterProcedureEnv env, PeerModificationState state)
       throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
@@ -104,9 +231,42 @@ public abstract class ModifyPeerProcedure extends AbstractPeerProcedure<PeerModi
         setNextState(PeerModificationState.REFRESH_PEER_ON_RS);
         return Flow.HAS_MORE_STATE;
       case REFRESH_PEER_ON_RS:
-        addChildProcedure(env.getMasterServices().getServerManager().getOnlineServersList().stream()
-            .map(sn -> new RefreshPeerProcedure(peerId, getPeerOperationType(), sn))
-            .toArray(RefreshPeerProcedure[]::new));
+        refreshPeer(env, getPeerOperationType());
+        setNextState(reopenRegionsAfterRefresh() ? PeerModificationState.SERIAL_PEER_REOPEN_REGIONS
+          : PeerModificationState.POST_PEER_MODIFICATION);
+        return Flow.HAS_MORE_STATE;
+      case SERIAL_PEER_REOPEN_REGIONS:
+        try {
+          reopenRegions(env);
+        } catch (Exception e) {
+          LOG.warn("{} reopen regions for peer {} failed, retry", getClass().getName(), peerId, e);
+          throw new ProcedureYieldException();
+        }
+        setNextState(PeerModificationState.SERIAL_PEER_UPDATE_LAST_PUSHED_SEQ_ID);
+        return Flow.HAS_MORE_STATE;
+      case SERIAL_PEER_UPDATE_LAST_PUSHED_SEQ_ID:
+        try {
+          setLastSequenceIdForSerialPeer(env);
+        } catch (Exception e) {
+          LOG.warn("{} set last sequence id for peer {} failed, retry", getClass().getName(),
+            peerId, e);
+          throw new ProcedureYieldException();
+        }
+        setNextState(enablePeerBeforeFinish() ? PeerModificationState.SERIAL_PEER_SET_PEER_ENABLED
+          : PeerModificationState.POST_PEER_MODIFICATION);
+        return Flow.HAS_MORE_STATE;
+      case SERIAL_PEER_SET_PEER_ENABLED:
+        try {
+          env.getReplicationPeerManager().enablePeer(peerId);
+        } catch (ReplicationException e) {
+          LOG.warn("{} enable peer before finish for peer {} failed, retry", getClass().getName(),
+            peerId, e);
+          throw new ProcedureYieldException();
+        }
+        setNextState(PeerModificationState.SERIAL_PEER_ENABLE_PEER_REFRESH_PEER_ON_RS);
+        return Flow.HAS_MORE_STATE;
+      case SERIAL_PEER_ENABLE_PEER_REFRESH_PEER_ON_RS:
+        refreshPeer(env, PeerOperationType.ENABLE);
         setNextState(PeerModificationState.POST_PEER_MODIFICATION);
         return Flow.HAS_MORE_STATE;
       case POST_PEER_MODIFICATION:

http://git-wip-us.apache.org/repos/asf/hbase/blob/9369cf26/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 1e93373..a0e01e0 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
@@ -85,7 +85,7 @@ public class ReplicationPeerManager {
     }
   }
 
-  public void preAddPeer(String peerId, ReplicationPeerConfig peerConfig)
+  void preAddPeer(String peerId, ReplicationPeerConfig peerConfig)
       throws DoNotRetryIOException, ReplicationException {
     if (peerId.contains("-")) {
       throw new DoNotRetryIOException("Found invalid peer name: " + peerId);
@@ -109,43 +109,47 @@ public class ReplicationPeerManager {
     return desc;
   }
 
-  public void preRemovePeer(String peerId) throws DoNotRetryIOException {
+  void preRemovePeer(String peerId) throws DoNotRetryIOException {
     checkPeerExists(peerId);
   }
 
-  public void preEnablePeer(String peerId) throws DoNotRetryIOException {
+  void preEnablePeer(String peerId) throws DoNotRetryIOException {
     ReplicationPeerDescription desc = checkPeerExists(peerId);
     if (desc.isEnabled()) {
       throw new DoNotRetryIOException("Replication peer " + peerId + " has already been enabled");
     }
   }
 
-  public void preDisablePeer(String peerId) throws DoNotRetryIOException {
+  void preDisablePeer(String peerId) throws DoNotRetryIOException {
     ReplicationPeerDescription desc = checkPeerExists(peerId);
     if (!desc.isEnabled()) {
       throw new DoNotRetryIOException("Replication peer " + peerId + " has already been disabled");
     }
   }
 
-  public void preUpdatePeerConfig(String peerId, ReplicationPeerConfig peerConfig)
+  /**
+   * Return the old peer description. Can never be null.
+   */
+  ReplicationPeerDescription preUpdatePeerConfig(String peerId, ReplicationPeerConfig peerConfig)
       throws DoNotRetryIOException {
     checkPeerConfig(peerConfig);
     ReplicationPeerDescription desc = checkPeerExists(peerId);
     ReplicationPeerConfig oldPeerConfig = desc.getPeerConfig();
     if (!isStringEquals(peerConfig.getClusterKey(), oldPeerConfig.getClusterKey())) {
       throw new DoNotRetryIOException(
-          "Changing the cluster key on an existing peer is not allowed. Existing key '" +
-              oldPeerConfig.getClusterKey() + "' for peer " + peerId + " does not match new key '" +
-              peerConfig.getClusterKey() + "'");
+        "Changing the cluster key on an existing peer is not allowed. Existing key '" +
+          oldPeerConfig.getClusterKey() + "' for peer " + peerId + " does not match new key '" +
+          peerConfig.getClusterKey() + "'");
     }
 
     if (!isStringEquals(peerConfig.getReplicationEndpointImpl(),
       oldPeerConfig.getReplicationEndpointImpl())) {
       throw new DoNotRetryIOException("Changing the replication endpoint implementation class " +
-          "on an existing peer is not allowed. Existing class '" +
-          oldPeerConfig.getReplicationEndpointImpl() + "' for peer " + peerId +
-          " does not match new class '" + peerConfig.getReplicationEndpointImpl() + "'");
+        "on an existing peer is not allowed. Existing class '" +
+        oldPeerConfig.getReplicationEndpointImpl() + "' for peer " + peerId +
+        " does not match new class '" + peerConfig.getReplicationEndpointImpl() + "'");
     }
+    return desc;
   }
 
   public void addPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
@@ -216,7 +220,7 @@ public class ReplicationPeerManager {
     return desc != null ? Optional.of(desc.getPeerConfig()) : Optional.empty();
   }
 
-  public void removeAllQueuesAndHFileRefs(String peerId) throws ReplicationException {
+  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
     // scanned claims the queue of a RS which has not been scanned yet, we will miss that queue in
@@ -340,7 +344,7 @@ public class ReplicationPeerManager {
   public static ReplicationPeerManager create(ZKWatcher zk, Configuration conf)
       throws ReplicationException {
     ReplicationPeerStorage peerStorage =
-        ReplicationStorageFactory.getReplicationPeerStorage(zk, conf);
+      ReplicationStorageFactory.getReplicationPeerStorage(zk, conf);
     ConcurrentMap<String, ReplicationPeerDescription> peers = new ConcurrentHashMap<>();
     for (String peerId : peerStorage.listPeerIds()) {
       ReplicationPeerConfig peerConfig = peerStorage.getPeerConfig(peerId);
@@ -348,7 +352,7 @@ public class ReplicationPeerManager {
       peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, peerConfig));
     }
     return new ReplicationPeerManager(peerStorage,
-        ReplicationStorageFactory.getReplicationQueueStorage(zk, conf), peers);
+      ReplicationStorageFactory.getReplicationQueueStorage(zk, conf), peers);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/9369cf26/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
index 3497447..b7e670a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
@@ -24,6 +24,8 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
+import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -40,6 +42,10 @@ public class UpdatePeerConfigProcedure extends ModifyPeerProcedure {
 
   private ReplicationPeerConfig peerConfig;
 
+  private ReplicationPeerConfig oldPeerConfig;
+
+  private boolean enabled;
+
   public UpdatePeerConfigProcedure() {
   }
 
@@ -54,21 +60,53 @@ public class UpdatePeerConfigProcedure extends ModifyPeerProcedure {
   }
 
   @Override
+  protected boolean reopenRegionsAfterRefresh() {
+    // If we remove some tables from the peer config then we do not need to enter the extra states
+    // for serial replication. Could try to optimize later since it is not easy to determine this...
+    return peerConfig.isSerial() && (!oldPeerConfig.isSerial() ||
+      !ReplicationUtils.isNamespacesAndTableCFsEqual(peerConfig, oldPeerConfig));
+  }
+
+  @Override
+  protected boolean enablePeerBeforeFinish() {
+    // do not need to test reopenRegionsAfterRefresh since we can only enter here if
+    // reopenRegionsAfterRefresh returns true.
+    return enabled;
+  }
+
+  @Override
+  protected ReplicationPeerConfig getOldPeerConfig() {
+    return oldPeerConfig;
+  }
+
+  @Override
+  protected ReplicationPeerConfig getNewPeerConfig() {
+    return peerConfig;
+  }
+
+  @Override
   protected void prePeerModification(MasterProcedureEnv env) throws IOException {
     MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
     if (cpHost != null) {
       cpHost.preUpdateReplicationPeerConfig(peerId, peerConfig);
     }
-    env.getReplicationPeerManager().preUpdatePeerConfig(peerId, peerConfig);
+    ReplicationPeerDescription desc =
+      env.getReplicationPeerManager().preUpdatePeerConfig(peerId, peerConfig);
+    oldPeerConfig = desc.getPeerConfig();
+    enabled = desc.isEnabled();
   }
 
   @Override
   protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
     env.getReplicationPeerManager().updatePeerConfig(peerId, peerConfig);
+    if (enabled && reopenRegionsAfterRefresh()) {
+      env.getReplicationPeerManager().disablePeer(peerId);
+    }
   }
 
   @Override
-  protected void postPeerModification(MasterProcedureEnv env) throws IOException {
+  protected void postPeerModification(MasterProcedureEnv env)
+      throws IOException, ReplicationException {
     LOG.info("Successfully updated peer config of {} to {}", peerId, peerConfig);
     MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
     if (cpHost != null) {
@@ -79,14 +117,23 @@ public class UpdatePeerConfigProcedure extends ModifyPeerProcedure {
   @Override
   protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
     super.serializeStateData(serializer);
-    serializer.serialize(UpdatePeerConfigStateData.newBuilder()
-        .setPeerConfig(ReplicationPeerConfigUtil.convert(peerConfig)).build());
+    UpdatePeerConfigStateData.Builder builder = UpdatePeerConfigStateData.newBuilder()
+      .setPeerConfig(ReplicationPeerConfigUtil.convert(peerConfig));
+    if (oldPeerConfig != null) {
+      builder.setOldPeerConfig(ReplicationPeerConfigUtil.convert(oldPeerConfig));
+    }
+    serializer.serialize(builder.build());
   }
 
   @Override
   protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
     super.deserializeStateData(serializer);
-    peerConfig = ReplicationPeerConfigUtil
-        .convert(serializer.deserialize(UpdatePeerConfigStateData.class).getPeerConfig());
+    UpdatePeerConfigStateData data = serializer.deserialize(UpdatePeerConfigStateData.class);
+    peerConfig = ReplicationPeerConfigUtil.convert(data.getPeerConfig());
+    if (data.hasOldPeerConfig()) {
+      oldPeerConfig = ReplicationPeerConfigUtil.convert(data.getOldPeerConfig());
+    } else {
+      oldPeerConfig = null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/9369cf26/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
index a02d181..78c1977 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/PeerProcedureHandlerImpl.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerImpl;
+import org.apache.hadoop.hbase.replication.ReplicationPeers;
 import org.apache.hadoop.hbase.replication.ReplicationUtils;
 import org.apache.hadoop.hbase.util.KeyLocker;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -99,19 +100,26 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
   @Override
   public void updatePeerConfig(String peerId) throws ReplicationException, IOException {
     Lock peerLock = peersLock.acquireLock(peerId);
+    ReplicationPeers peers = replicationSourceManager.getReplicationPeers();
     ReplicationPeerImpl peer = null;
     ReplicationPeerConfig oldConfig = null;
+    PeerState oldState = null;
     boolean success = false;
     try {
-      peer = replicationSourceManager.getReplicationPeers().getPeer(peerId);
+      peer = peers.getPeer(peerId);
       if (peer == null) {
         throw new ReplicationException("Peer with id=" + peerId + " is not cached.");
       }
       oldConfig = peer.getPeerConfig();
-      ReplicationPeerConfig newConfig =
-          replicationSourceManager.getReplicationPeers().refreshPeerConfig(peerId);
+      oldState = peer.getPeerState();
+      ReplicationPeerConfig newConfig = peers.refreshPeerConfig(peerId);
+      // also need to refresh peer state here. When updating a serial replication peer we may
+      // disable it first and then enable it.
+      PeerState newState = peers.refreshPeerState(peerId);
       // RS need to start work with the new replication config change
-      if (!ReplicationUtils.isKeyConfigEqual(oldConfig, newConfig)) {
+      if (!ReplicationUtils.isNamespacesAndTableCFsEqual(oldConfig, newConfig) ||
+        oldConfig.isSerial() != newConfig.isSerial() ||
+        (oldState.equals(PeerState.ENABLED) && newState.equals(PeerState.DISABLED))) {
         replicationSourceManager.refreshSources(peerId);
       }
       success = true;
@@ -119,6 +127,7 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler {
       if (!success && peer != null) {
         // Reset peer config if refresh source failed
         peer.setPeerConfig(oldConfig);
+        peer.setPeerState(oldState.equals(PeerState.ENABLED));
       }
       peerLock.unlock();
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/9369cf26/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 23e1115..3ecc50a 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
@@ -510,7 +510,7 @@ public class ReplicationSourceManager implements ReplicationListener {
       // synchronized on walsById to avoid race with preLogRoll
       synchronized (this.walsById) {
         NavigableSet<String> wals = walsById.get(queueId).get(logPrefix);
-        if (wals != null && !wals.first().equals(log)) {
+        if (wals != null) {
           cleanOldLogs(wals, log, inclusive, queueId);
         }
       }
@@ -755,7 +755,7 @@ public class ReplicationSourceManager implements ReplicationListener {
    * @return a sorted set of wal names
    */
   @VisibleForTesting
-  Map<String, Map<String, NavigableSet<String>>> getWALs() {
+  public Map<String, Map<String, NavigableSet<String>>> getWALs() {
     return Collections.unmodifiableMap(walsById);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/9369cf26/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryBatch.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryBatch.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryBatch.java
index 960d473..22b2de7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryBatch.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryBatch.java
@@ -157,4 +157,12 @@ class WALEntryBatch {
   public void setLastSeqId(String region, long sequenceId) {
     lastSeqIds.put(region, sequenceId);
   }
+
+  @Override
+  public String toString() {
+    return "WALEntryBatch [walEntries=" + walEntries + ", lastWalPath=" + lastWalPath +
+      ", lastWalPosition=" + lastWalPosition + ", nbRowKeys=" + nbRowKeys + ", nbHFiles=" +
+      nbHFiles + ", heapSize=" + heapSize + ", lastSeqIds=" + lastSeqIds + ", endOfFile=" +
+      endOfFile + "]";
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/9369cf26/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SerialReplicationTestBase.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SerialReplicationTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SerialReplicationTestBase.java
new file mode 100644
index 0000000..83afd81
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/SerialReplicationTestBase.java
@@ -0,0 +1,229 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.UUID;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALProvider;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.rules.TestName;
+
+/**
+ * Base class for testing serial replication.
+ */
+public class SerialReplicationTestBase {
+
+  protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  protected static String PEER_ID = "1";
+
+  protected static byte[] CF = Bytes.toBytes("CF");
+
+  protected static byte[] CQ = Bytes.toBytes("CQ");
+
+  protected static FileSystem FS;
+
+  protected static Path LOG_DIR;
+
+  protected static WALProvider.Writer WRITER;
+
+  @Rule
+  public final TestName name = new TestName();
+
+  protected Path logPath;
+
+  public static final class LocalReplicationEndpoint extends BaseReplicationEndpoint {
+
+    private static final UUID PEER_UUID = UUID.randomUUID();
+
+    @Override
+    public UUID getPeerUUID() {
+      return PEER_UUID;
+    }
+
+    @Override
+    public boolean replicate(ReplicateContext replicateContext) {
+      synchronized (WRITER) {
+        try {
+          for (Entry entry : replicateContext.getEntries()) {
+            WRITER.append(entry);
+          }
+          WRITER.sync(false);
+        } catch (IOException e) {
+          throw new UncheckedIOException(e);
+        }
+      }
+      return true;
+    }
+
+    @Override
+    public void start() {
+      startAsync();
+    }
+
+    @Override
+    public void stop() {
+      stopAsync();
+    }
+
+    @Override
+    protected void doStart() {
+      notifyStarted();
+    }
+
+    @Override
+    protected void doStop() {
+      notifyStopped();
+    }
+  }
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    UTIL.getConfiguration().setInt("replication.source.nb.capacity", 10);
+    UTIL.startMiniCluster(3);
+    // disable balancer
+    UTIL.getAdmin().balancerSwitch(false, true);
+    LOG_DIR = UTIL.getDataTestDirOnTestFS("replicated");
+    FS = UTIL.getTestFileSystem();
+    FS.mkdirs(LOG_DIR);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    UTIL.getAdmin().removeReplicationPeer(PEER_ID);
+    rollAllWALs();
+    if (WRITER != null) {
+      WRITER.close();
+      WRITER = null;
+    }
+  }
+
+  protected static void moveRegion(RegionInfo region, HRegionServer rs) throws Exception {
+    UTIL.getAdmin().move(region.getEncodedNameAsBytes(),
+      Bytes.toBytes(rs.getServerName().getServerName()));
+    UTIL.waitFor(30000, new ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        return rs.getRegion(region.getEncodedName()) != null;
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        return region + " is still not on " + rs;
+      }
+    });
+  }
+
+  protected static void rollAllWALs() throws Exception {
+    for (RegionServerThread t : UTIL.getMiniHBaseCluster().getLiveRegionServerThreads()) {
+      t.getRegionServer().getWalRoller().requestRollAll();
+    }
+    UTIL.waitFor(30000, new ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        return UTIL.getMiniHBaseCluster().getLiveRegionServerThreads().stream()
+          .map(t -> t.getRegionServer()).allMatch(HRegionServer::walRollRequestFinished);
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        return "Log roll has not finished yet";
+      }
+    });
+  }
+
+  protected final void setupWALWriter() throws IOException {
+    logPath = new Path(LOG_DIR, name.getMethodName());
+    WRITER = WALFactory.createWALWriter(FS, logPath, UTIL.getConfiguration());
+  }
+
+  protected final void waitUntilReplicationDone(int expectedEntries) throws Exception {
+    UTIL.waitFor(30000, new ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        try (WAL.Reader reader = WALFactory.createReader(FS, logPath, UTIL.getConfiguration())) {
+          int count = 0;
+          while (reader.next() != null) {
+            count++;
+          }
+          return count >= expectedEntries;
+        } catch (IOException e) {
+          return false;
+        }
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        return "Not enough entries replicated";
+      }
+    });
+  }
+
+  protected final void addPeer(boolean enabled) throws IOException {
+    UTIL.getAdmin().addReplicationPeer(PEER_ID,
+      ReplicationPeerConfig.newBuilder().setClusterKey("127.0.0.1:2181:/hbase")
+        .setReplicationEndpointImpl(LocalReplicationEndpoint.class.getName()).setSerial(true)
+        .build(),
+      enabled);
+  }
+
+  protected final void checkOrder(int expectedEntries) throws IOException {
+    try (WAL.Reader reader =
+      WALFactory.createReader(UTIL.getTestFileSystem(), logPath, UTIL.getConfiguration())) {
+      long seqId = -1L;
+      int count = 0;
+      for (Entry entry;;) {
+        entry = reader.next();
+        if (entry == null) {
+          break;
+        }
+        assertTrue(
+          "Sequence id go backwards from " + seqId + " to " + entry.getKey().getSequenceId(),
+          entry.getKey().getSequenceId() >= seqId);
+        count++;
+      }
+      assertEquals(expectedEntries, count);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/9369cf26/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestAddToSerialReplicationPeer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestAddToSerialReplicationPeer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestAddToSerialReplicationPeer.java
new file mode 100644
index 0000000..64b5bb1
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestAddToSerialReplicationPeer.java
@@ -0,0 +1,215 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import java.io.IOException;
+import java.util.Collections;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.wal.AbstractFSWAL;
+import org.apache.hadoop.hbase.replication.regionserver.Replication;
+import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException;
+import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
+
+/**
+ * Testcase for HBASE-20147.
+ */
+@Category({ ReplicationTests.class, MediumTests.class })
+public class TestAddToSerialReplicationPeer extends SerialReplicationTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestAddToSerialReplicationPeer.class);
+
+  @Before
+  public void setUp() throws IOException, StreamLacksCapabilityException {
+    setupWALWriter();
+  }
+
+  // make sure that we will start replication for the sequence id after move, that's what we want to
+  // test here.
+  private void moveRegionAndArchiveOldWals(RegionInfo region, HRegionServer rs) throws Exception {
+    moveRegion(region, rs);
+    rollAllWALs();
+  }
+
+  private void waitUntilReplicatedToTheCurrentWALFile(HRegionServer rs) throws Exception {
+    Path path = ((AbstractFSWAL<?>) rs.getWAL(null)).getCurrentFileName();
+    String logPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(path.getName());
+    UTIL.waitFor(30000, new ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        ReplicationSourceManager manager =
+          ((Replication) rs.getReplicationSourceService()).getReplicationManager();
+        return manager.getWALs().get(PEER_ID).get(logPrefix).size() == 1;
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        return "Still not replicated to the current WAL file yet";
+      }
+    });
+  }
+
+  @Test
+  public void testAddPeer() throws Exception {
+    TableName tableName = TableName.valueOf(name.getMethodName());
+    UTIL.getAdmin().createTable(
+      TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(ColumnFamilyDescriptorBuilder
+        .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build());
+    UTIL.waitTableAvailable(tableName);
+    try (Table table = UTIL.getConnection().getTable(tableName)) {
+      for (int i = 0; i < 100; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+    RegionInfo region = UTIL.getAdmin().getRegions(tableName).get(0);
+    HRegionServer rs = UTIL.getOtherRegionServer(UTIL.getRSForFirstRegionInTable(tableName));
+    moveRegionAndArchiveOldWals(region, rs);
+    addPeer(true);
+    try (Table table = UTIL.getConnection().getTable(tableName)) {
+      for (int i = 0; i < 100; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+    waitUntilReplicationDone(100);
+    checkOrder(100);
+  }
+
+  @Test
+  public void testChangeToSerial() throws Exception {
+    ReplicationPeerConfig peerConfig =
+      ReplicationPeerConfig.newBuilder().setClusterKey("127.0.0.1:2181:/hbase")
+        .setReplicationEndpointImpl(LocalReplicationEndpoint.class.getName()).build();
+    UTIL.getAdmin().addReplicationPeer(PEER_ID, peerConfig, true);
+
+    TableName tableName = TableName.valueOf(name.getMethodName());
+
+    UTIL.getAdmin().createTable(
+      TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(ColumnFamilyDescriptorBuilder
+        .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build());
+    UTIL.waitTableAvailable(tableName);
+    try (Table table = UTIL.getConnection().getTable(tableName)) {
+      for (int i = 0; i < 100; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+
+    RegionInfo region = UTIL.getAdmin().getRegions(tableName).get(0);
+    HRegionServer srcRs = UTIL.getRSForFirstRegionInTable(tableName);
+    HRegionServer rs = UTIL.getOtherRegionServer(srcRs);
+    moveRegionAndArchiveOldWals(region, rs);
+    waitUntilReplicationDone(100);
+    waitUntilReplicatedToTheCurrentWALFile(srcRs);
+
+    UTIL.getAdmin().disableReplicationPeer(PEER_ID);
+    UTIL.getAdmin().updateReplicationPeerConfig(PEER_ID,
+      ReplicationPeerConfig.newBuilder(peerConfig).setSerial(true).build());
+    UTIL.getAdmin().enableReplicationPeer(PEER_ID);
+
+    try (Table table = UTIL.getConnection().getTable(tableName)) {
+      for (int i = 0; i < 100; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+    waitUntilReplicationDone(200);
+    checkOrder(200);
+  }
+
+  @Test
+  public void testAddToSerialPeer() throws Exception {
+    ReplicationPeerConfig peerConfig =
+      ReplicationPeerConfig.newBuilder().setClusterKey("127.0.0.1:2181:/hbase")
+        .setReplicationEndpointImpl(LocalReplicationEndpoint.class.getName())
+        .setReplicateAllUserTables(false).setSerial(true).build();
+    UTIL.getAdmin().addReplicationPeer(PEER_ID, peerConfig, true);
+
+    TableName tableName = TableName.valueOf(name.getMethodName());
+    UTIL.getAdmin().createTable(
+      TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(ColumnFamilyDescriptorBuilder
+        .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build());
+    UTIL.waitTableAvailable(tableName);
+    try (Table table = UTIL.getConnection().getTable(tableName)) {
+      for (int i = 0; i < 100; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+    RegionInfo region = UTIL.getAdmin().getRegions(tableName).get(0);
+    HRegionServer srcRs = UTIL.getRSForFirstRegionInTable(tableName);
+    HRegionServer rs = UTIL.getOtherRegionServer(srcRs);
+    moveRegionAndArchiveOldWals(region, rs);
+    waitUntilReplicatedToTheCurrentWALFile(rs);
+    UTIL.getAdmin().disableReplicationPeer(PEER_ID);
+    UTIL.getAdmin().updateReplicationPeerConfig(PEER_ID,
+      ReplicationPeerConfig.newBuilder(peerConfig)
+        .setTableCFsMap(ImmutableMap.of(tableName, Collections.emptyList())).build());
+    UTIL.getAdmin().enableReplicationPeer(PEER_ID);
+    try (Table table = UTIL.getConnection().getTable(tableName)) {
+      for (int i = 0; i < 100; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+    waitUntilReplicationDone(100);
+    checkOrder(100);
+  }
+
+  @Test
+  public void testDisabledTable() throws Exception {
+    TableName tableName = TableName.valueOf(name.getMethodName());
+    UTIL.getAdmin().createTable(
+      TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(ColumnFamilyDescriptorBuilder
+        .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build());
+    UTIL.waitTableAvailable(tableName);
+    try (Table table = UTIL.getConnection().getTable(tableName)) {
+      for (int i = 0; i < 100; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+    UTIL.getAdmin().disableTable(tableName);
+    rollAllWALs();
+    addPeer(true);
+    UTIL.getAdmin().enableTable(tableName);
+    try (Table table = UTIL.getConnection().getTable(tableName)) {
+      for (int i = 0; i < 100; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+    waitUntilReplicationDone(100);
+    checkOrder(100);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/9369cf26/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
index 37f11f6..94b79d9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java
@@ -23,211 +23,49 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
-import java.io.UncheckedIOException;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 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.Waiter.ExplainingPredicate;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException;
-import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.wal.WAL;
 import org.apache.hadoop.hbase.wal.WAL.Entry;
 import org.apache.hadoop.hbase.wal.WALFactory;
-import org.apache.hadoop.hbase.wal.WALProvider;
-import org.junit.After;
-import org.junit.AfterClass;
 import org.junit.Before;
-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;
 
-@Category({ ReplicationTests.class, LargeTests.class })
-public class TestSerialReplication {
+@Category({ ReplicationTests.class, MediumTests.class })
+public class TestSerialReplication extends SerialReplicationTestBase {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
     HBaseClassTestRule.forClass(TestSerialReplication.class);
 
-  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
-
-  private static String PEER_ID = "1";
-
-  private static byte[] CF = Bytes.toBytes("CF");
-
-  private static byte[] CQ = Bytes.toBytes("CQ");
-
-  private static FileSystem FS;
-
-  private static Path LOG_DIR;
-
-  private static WALProvider.Writer WRITER;
-
-  public static final class LocalReplicationEndpoint extends BaseReplicationEndpoint {
-
-    private static final UUID PEER_UUID = UUID.randomUUID();
-
-    @Override
-    public UUID getPeerUUID() {
-      return PEER_UUID;
-    }
-
-    @Override
-    public boolean replicate(ReplicateContext replicateContext) {
-      synchronized (WRITER) {
-        try {
-          for (Entry entry : replicateContext.getEntries()) {
-            WRITER.append(entry);
-          }
-          WRITER.sync(false);
-        } catch (IOException e) {
-          throw new UncheckedIOException(e);
-        }
-      }
-      return true;
-    }
-
-    @Override
-    public void start() {
-      startAsync();
-    }
-
-    @Override
-    public void stop() {
-      stopAsync();
-    }
-
-    @Override
-    protected void doStart() {
-      notifyStarted();
-    }
-
-    @Override
-    protected void doStop() {
-      notifyStopped();
-    }
-  }
-
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    UTIL.getConfiguration().setInt("replication.source.nb.capacity", 10);
-    UTIL.startMiniCluster(3);
-    // disable balancer
-    UTIL.getAdmin().balancerSwitch(false, true);
-    LOG_DIR = UTIL.getDataTestDirOnTestFS("replicated");
-    FS = UTIL.getTestFileSystem();
-    FS.mkdirs(LOG_DIR);
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    UTIL.shutdownMiniCluster();
-  }
-
-  @Rule
-  public final TestName name = new TestName();
-
-  private Path logPath;
-
   @Before
   public void setUp() throws IOException, StreamLacksCapabilityException {
-    logPath = new Path(LOG_DIR, name.getMethodName());
-    WRITER = WALFactory.createWALWriter(FS, logPath, UTIL.getConfiguration());
+    setupWALWriter();
     // add in disable state, so later when enabling it all sources will start push together.
-    UTIL.getAdmin().addReplicationPeer(PEER_ID,
-      ReplicationPeerConfig.newBuilder().setClusterKey("127.0.0.1:2181:/hbase")
-        .setReplicationEndpointImpl(LocalReplicationEndpoint.class.getName()).setSerial(true)
-        .build(),
-      false);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    UTIL.getAdmin().removeReplicationPeer(PEER_ID);
-    for (RegionServerThread t : UTIL.getMiniHBaseCluster().getLiveRegionServerThreads()) {
-      t.getRegionServer().getWalRoller().requestRollAll();
-    }
-    UTIL.waitFor(30000, new ExplainingPredicate<Exception>() {
-
-      @Override
-      public boolean evaluate() throws Exception {
-        return UTIL.getMiniHBaseCluster().getLiveRegionServerThreads().stream()
-          .map(t -> t.getRegionServer()).allMatch(HRegionServer::walRollRequestFinished);
-      }
-
-      @Override
-      public String explainFailure() throws Exception {
-        return "Log roll has not finished yet";
-      }
-    });
-    for (RegionServerThread t : UTIL.getMiniHBaseCluster().getLiveRegionServerThreads()) {
-      t.getRegionServer().getWalRoller().requestRollAll();
-    }
-    if (WRITER != null) {
-      WRITER.close();
-      WRITER = null;
-    }
-  }
-
-  private void moveRegion(RegionInfo region, HRegionServer rs) throws Exception {
-    UTIL.getAdmin().move(region.getEncodedNameAsBytes(),
-      Bytes.toBytes(rs.getServerName().getServerName()));
-    UTIL.waitFor(30000, new ExplainingPredicate<Exception>() {
-
-      @Override
-      public boolean evaluate() throws Exception {
-        return rs.getRegion(region.getEncodedName()) != null;
-      }
-
-      @Override
-      public String explainFailure() throws Exception {
-        return region + " is still not on " + rs;
-      }
-    });
+    addPeer(false);
   }
 
   private void enablePeerAndWaitUntilReplicationDone(int expectedEntries) throws Exception {
     UTIL.getAdmin().enableReplicationPeer(PEER_ID);
-    UTIL.waitFor(30000, new ExplainingPredicate<Exception>() {
-
-      @Override
-      public boolean evaluate() throws Exception {
-        try (WAL.Reader reader = WALFactory.createReader(FS, logPath, UTIL.getConfiguration())) {
-          int count = 0;
-          while (reader.next() != null) {
-            count++;
-          }
-          return count >= expectedEntries;
-        } catch (IOException e) {
-          return false;
-        }
-      }
-
-      @Override
-      public String explainFailure() throws Exception {
-        return "Not enough entries replicated";
-      }
-    });
+    waitUntilReplicationDone(expectedEntries);
   }
 
   @Test
@@ -251,22 +89,7 @@ public class TestSerialReplication {
       }
     }
     enablePeerAndWaitUntilReplicationDone(200);
-    try (WAL.Reader reader =
-      WALFactory.createReader(UTIL.getTestFileSystem(), logPath, UTIL.getConfiguration())) {
-      long seqId = -1L;
-      int count = 0;
-      for (Entry entry;;) {
-        entry = reader.next();
-        if (entry == null) {
-          break;
-        }
-        assertTrue(
-          "Sequence id go backwards from " + seqId + " to " + entry.getKey().getSequenceId(),
-          entry.getKey().getSequenceId() >= seqId);
-        count++;
-      }
-      assertEquals(200, count);
-    }
+    checkOrder(200);
   }
 
   @Test


[21/22] hbase git commit: HBASE-20296 Remove last pushed sequence ids when removing tables from a peer

Posted by zh...@apache.org.
HBASE-20296 Remove last pushed sequence ids when removing tables from a peer


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

Branch: refs/heads/HBASE-20046-branch-2
Commit: 74ab10c353968cfcfe9b7ab07101bd3bfab74044
Parents: ead569c
Author: zhangduo <zh...@apache.org>
Authored: Sat Mar 31 20:25:13 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Apr 9 15:18:44 2018 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/MetaTableAccessor.java  |  72 +++++-----
 .../replication/ReplicationQueueStorage.java    |   9 ++
 .../replication/ZKReplicationQueueStorage.java  |  15 +++
 .../master/replication/AddPeerProcedure.java    |  14 +-
 .../master/replication/ModifyPeerProcedure.java | 134 ++++++++++---------
 .../replication/UpdatePeerConfigProcedure.java  |  96 ++++++++++++-
 .../hadoop/hbase/client/TestEnableTable.java    |   4 +-
 .../TestRemoveFromSerialReplicationPeer.java    | 120 +++++++++++++++++
 8 files changed, 363 insertions(+), 101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/74ab10c3/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 4cc46c8..0f5ef09 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
@@ -57,6 +57,8 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.master.RegionState.State;
@@ -682,20 +684,19 @@ public class MetaTableAccessor {
     scanMeta(connection, null, null, QueryType.ALL, v);
   }
 
-  public static void scanMetaForTableRegions(Connection connection,
-      Visitor visitor, TableName tableName) throws IOException {
+  public static void scanMetaForTableRegions(Connection connection, Visitor visitor,
+      TableName tableName) throws IOException {
     scanMeta(connection, tableName, QueryType.REGION, Integer.MAX_VALUE, visitor);
   }
 
-  public static void scanMeta(Connection connection, TableName table,
-      QueryType type, int maxRows, final Visitor visitor) throws IOException {
+  public static void scanMeta(Connection connection, TableName table, QueryType type, int maxRows,
+      final Visitor visitor) throws IOException {
     scanMeta(connection, getTableStartRowForMeta(table, type), getTableStopRowForMeta(table, type),
-        type, maxRows, visitor);
+      type, maxRows, visitor);
   }
 
-  public static void scanMeta(Connection connection,
-      @Nullable final byte[] startRow, @Nullable final byte[] stopRow,
-      QueryType type, final Visitor visitor) throws IOException {
+  public static void scanMeta(Connection connection, @Nullable final byte[] startRow,
+      @Nullable final byte[] stopRow, QueryType type, final Visitor visitor) throws IOException {
     scanMeta(connection, startRow, stopRow, type, Integer.MAX_VALUE, visitor);
   }
 
@@ -708,26 +709,19 @@ public class MetaTableAccessor {
    * @param tableName  table withing we scan
    * @param row        start scan from this row
    * @param rowLimit   max number of rows to return
-   * @throws IOException
    */
-  public static void scanMeta(Connection connection,
-      final Visitor visitor, final TableName tableName,
-      final byte[] row, final int rowLimit)
-      throws IOException {
-
+  public static void scanMeta(Connection connection, final Visitor visitor,
+      final TableName tableName, final byte[] row, final int rowLimit) throws IOException {
     byte[] startRow = null;
     byte[] stopRow = null;
     if (tableName != null) {
-      startRow =
-          getTableStartRowForMeta(tableName, QueryType.REGION);
+      startRow = getTableStartRowForMeta(tableName, QueryType.REGION);
       if (row != null) {
-        RegionInfo closestRi =
-            getClosestRegionInfo(connection, tableName, row);
-        startRow = RegionInfo
-            .createRegionName(tableName, closestRi.getStartKey(), HConstants.ZEROES, false);
+        RegionInfo closestRi = getClosestRegionInfo(connection, tableName, row);
+        startRow =
+          RegionInfo.createRegionName(tableName, closestRi.getStartKey(), HConstants.ZEROES, false);
       }
-      stopRow =
-          getTableStopRowForMeta(tableName, QueryType.REGION);
+      stopRow = getTableStopRowForMeta(tableName, QueryType.REGION);
     }
     scanMeta(connection, startRow, stopRow, QueryType.REGION, rowLimit, visitor);
   }
@@ -743,11 +737,16 @@ public class MetaTableAccessor {
    * @param type scanned part of meta
    * @param maxRows maximum rows to return
    * @param visitor Visitor invoked against each row.
-   * @throws IOException
    */
   public static void scanMeta(Connection connection, @Nullable final byte[] startRow,
       @Nullable final byte[] stopRow, QueryType type, int maxRows, final Visitor visitor)
       throws IOException {
+    scanMeta(connection, startRow, stopRow, type, null, maxRows, visitor);
+  }
+
+  private static void scanMeta(Connection connection, @Nullable final byte[] startRow,
+      @Nullable final byte[] stopRow, QueryType type, @Nullable Filter filter, int maxRows,
+      final Visitor visitor) throws IOException {
     int rowUpperLimit = maxRows > 0 ? maxRows : Integer.MAX_VALUE;
     Scan scan = getMetaScan(connection, rowUpperLimit);
 
@@ -760,13 +759,14 @@ public class MetaTableAccessor {
     if (stopRow != null) {
       scan.withStopRow(stopRow);
     }
+    if (filter != null) {
+      scan.setFilter(filter);
+    }
 
     if (LOG.isTraceEnabled()) {
-      LOG.trace("Scanning META"
-          + " starting at row=" + Bytes.toStringBinary(startRow)
-          + " stopping at row=" + Bytes.toStringBinary(stopRow)
-          + " for max=" + rowUpperLimit
-          + " with caching=" + scan.getCaching());
+      LOG.trace("Scanning META" + " starting at row=" + Bytes.toStringBinary(startRow) +
+        " stopping at row=" + Bytes.toStringBinary(stopRow) + " for max=" + rowUpperLimit +
+        " with caching=" + scan.getCaching());
     }
 
     int currentRow = 0;
@@ -1973,7 +1973,7 @@ public class MetaTableAccessor {
     byte[] value = getParentsBytes(parents);
     put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
       .setFamily(HConstants.REPLICATION_BARRIER_FAMILY).setQualifier(REPLICATION_PARENT_QUALIFIER)
-      .setTimestamp(put.getTimeStamp()).setType(Type.Put).setValue(value).build());
+      .setTimestamp(put.getTimestamp()).setType(Type.Put).setValue(value).build());
   }
 
   private static Put makePutForReplicationBarrier(RegionInfo regionInfo, long openSeqNum, long ts)
@@ -1988,7 +1988,7 @@ public class MetaTableAccessor {
       .setRow(put.getRow())
       .setFamily(HConstants.REPLICATION_BARRIER_FAMILY)
       .setQualifier(HConstants.SEQNUM_QUALIFIER)
-      .setTimestamp(put.getTimeStamp())
+      .setTimestamp(put.getTimestamp())
       .setType(Type.Put)
       .setValue(Bytes.toBytes(openSeqNum))
       .build());
@@ -2128,6 +2128,18 @@ public class MetaTableAccessor {
     return list;
   }
 
+  public static List<String> getTableEncodedRegionNamesForSerialReplication(Connection conn,
+      TableName tableName) throws IOException {
+    List<String> list = new ArrayList<>();
+    scanMeta(conn, getTableStartRowForMeta(tableName, QueryType.REPLICATION),
+      getTableStopRowForMeta(tableName, QueryType.REPLICATION), QueryType.REPLICATION,
+      new FirstKeyOnlyFilter(), Integer.MAX_VALUE, r -> {
+        list.add(RegionInfo.encodeRegionName(r.getRow()));
+        return true;
+      });
+    return list;
+  }
+
   private static void debugLogMutations(List<? extends Mutation> mutations) throws IOException {
     if (!METALOG.isDebugEnabled()) {
       return;

http://git-wip-us.apache.org/repos/asf/hbase/blob/74ab10c3/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
index cd37ac2..84653ad 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
@@ -91,6 +91,15 @@ public interface ReplicationQueueStorage {
    * @param peerId peer id
    */
   void removeLastSequenceIds(String peerId) throws ReplicationException;
+
+  /**
+   * Remove the max sequence id record for the given peer and regions.
+   * @param peerId peer id
+   * @param encodedRegionNames the encoded region names
+   */
+  void removeLastSequenceIds(String peerId, List<String> encodedRegionNames)
+      throws ReplicationException;
+
   /**
    * Get the current position for a specific WAL in a given queue for a given regionserver.
    * @param serverName the name of the regionserver

http://git-wip-us.apache.org/repos/asf/hbase/blob/74ab10c3/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
index 96b0b91..6d72128 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
@@ -28,6 +28,7 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
+import java.util.stream.Collectors;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -347,6 +348,20 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
   }
 
   @Override
+  public void removeLastSequenceIds(String peerId, List<String> encodedRegionNames)
+      throws ReplicationException {
+    try {
+      List<ZKUtilOp> listOfOps =
+        encodedRegionNames.stream().map(n -> getSerialReplicationRegionPeerNode(n, peerId))
+          .map(ZKUtilOp::deleteNodeFailSilent).collect(Collectors.toList());
+      ZKUtil.multiOrSequential(zookeeper, listOfOps, true);
+    } catch (KeeperException e) {
+      throw new ReplicationException("Failed to remove last sequence ids, peerId=" + peerId +
+        ", encodedRegionNames.size=" + encodedRegionNames.size(), e);
+    }
+  }
+
+  @Override
   public long getWALPosition(ServerName serverName, String queueId, String fileName)
       throws ReplicationException {
     byte[] bytes;

http://git-wip-us.apache.org/repos/asf/hbase/blob/74ab10c3/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
index 72228f6..2f2d5a5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java
@@ -29,6 +29,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddPeerStateData;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerModificationState;
 
 /**
  * The procedure for adding a new replication peer.
@@ -57,8 +58,15 @@ public class AddPeerProcedure extends ModifyPeerProcedure {
   }
 
   @Override
-  protected boolean reopenRegionsAfterRefresh() {
-    return true;
+  protected PeerModificationState nextStateAfterRefresh() {
+    return peerConfig.isSerial() ? PeerModificationState.SERIAL_PEER_REOPEN_REGIONS
+      : super.nextStateAfterRefresh();
+  }
+
+  @Override
+  protected void updateLastPushedSequenceIdForSerialPeer(MasterProcedureEnv env)
+      throws IOException, ReplicationException {
+    setLastPushedSequenceId(env, peerConfig);
   }
 
   @Override
@@ -102,7 +110,7 @@ public class AddPeerProcedure extends ModifyPeerProcedure {
   protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
     super.serializeStateData(serializer);
     serializer.serialize(AddPeerStateData.newBuilder()
-        .setPeerConfig(ReplicationPeerConfigUtil.convert(peerConfig)).setEnabled(enabled).build());
+      .setPeerConfig(ReplicationPeerConfigUtil.convert(peerConfig)).setEnabled(enabled).build());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/74ab10c3/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
index 2b76487..8bedeff 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java
@@ -18,11 +18,10 @@
 package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
-import java.io.UncheckedIOException;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.stream.Stream;
 import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.TableDescriptor;
@@ -55,7 +54,7 @@ public abstract class ModifyPeerProcedure extends AbstractPeerProcedure<PeerModi
 
   private static final Logger LOG = LoggerFactory.getLogger(ModifyPeerProcedure.class);
 
-  private static final int SET_LAST_SEQ_ID_BATCH_SIZE = 1000;
+  protected static final int UPDATE_LAST_SEQ_ID_BATCH_SIZE = 1000;
 
   protected ModifyPeerProcedure() {
   }
@@ -93,12 +92,11 @@ public abstract class ModifyPeerProcedure extends AbstractPeerProcedure<PeerModi
   }
 
   /**
-   * Implementation class can override this method. The default return value is false which means we
-   * will jump to POST_PEER_MODIFICATION and finish the procedure. If returns true, we will jump to
-   * SERIAL_PEER_REOPEN_REGIONS.
+   * Implementation class can override this method. By default we will jump to
+   * POST_PEER_MODIFICATION and finish the procedure.
    */
-  protected boolean reopenRegionsAfterRefresh() {
-    return false;
+  protected PeerModificationState nextStateAfterRefresh() {
+    return PeerModificationState.POST_PEER_MODIFICATION;
   }
 
   /**
@@ -123,80 +121,97 @@ public abstract class ModifyPeerProcedure extends AbstractPeerProcedure<PeerModi
     throw new UnsupportedOperationException();
   }
 
-  private Stream<TableDescriptor> getTables(MasterProcedureEnv env) throws IOException {
-    ReplicationPeerConfig peerConfig = getNewPeerConfig();
-    Stream<TableDescriptor> stream = env.getMasterServices().getTableDescriptors().getAll().values()
-      .stream().filter(TableDescriptor::hasGlobalReplicationScope)
-      .filter(td -> ReplicationUtils.contains(peerConfig, td.getTableName()));
-    ReplicationPeerConfig oldPeerConfig = getOldPeerConfig();
-    if (oldPeerConfig != null && oldPeerConfig.isSerial()) {
-      stream = stream.filter(td -> !ReplicationUtils.contains(oldPeerConfig, td.getTableName()));
-    }
-    return stream;
+  protected void updateLastPushedSequenceIdForSerialPeer(MasterProcedureEnv env)
+      throws IOException, ReplicationException {
+    throw new UnsupportedOperationException();
   }
 
   private void reopenRegions(MasterProcedureEnv env) throws IOException {
-    Stream<TableDescriptor> stream = getTables(env);
+    ReplicationPeerConfig peerConfig = getNewPeerConfig();
+    ReplicationPeerConfig oldPeerConfig = getOldPeerConfig();
     TableStateManager tsm = env.getMasterServices().getTableStateManager();
-    stream.filter(td -> {
-      try {
-        return tsm.getTableState(td.getTableName()).isEnabled();
-      } catch (TableStateNotFoundException e) {
-        return false;
-      } catch (IOException e) {
-        throw new UncheckedIOException(e);
+    for (TableDescriptor td : env.getMasterServices().getTableDescriptors().getAll().values()) {
+      if (!td.hasGlobalReplicationScope()) {
+        continue;
+      }
+      TableName tn = td.getTableName();
+      if (!ReplicationUtils.contains(peerConfig, tn)) {
+        continue;
+      }
+      if (oldPeerConfig != null && oldPeerConfig.isSerial() &&
+        ReplicationUtils.contains(oldPeerConfig, tn)) {
+        continue;
       }
-    }).forEach(td -> {
       try {
-        addChildProcedure(env.getAssignmentManager().createReopenProcedures(
-          env.getAssignmentManager().getRegionStates().getRegionsOfTable(td.getTableName())));
-      } catch (IOException e) {
-        throw new UncheckedIOException(e);
+        if (!tsm.getTableState(tn).isEnabled()) {
+          continue;
+        }
+      } catch (TableStateNotFoundException e) {
+        continue;
       }
-    });
+      addChildProcedure(env.getAssignmentManager().createReopenProcedures(
+        env.getAssignmentManager().getRegionStates().getRegionsOfTable(tn)));
+    }
   }
 
   private void addToMap(Map<String, Long> lastSeqIds, String encodedRegionName, long barrier,
       ReplicationQueueStorage queueStorage) throws ReplicationException {
     if (barrier >= 0) {
       lastSeqIds.put(encodedRegionName, barrier);
-      if (lastSeqIds.size() >= SET_LAST_SEQ_ID_BATCH_SIZE) {
+      if (lastSeqIds.size() >= UPDATE_LAST_SEQ_ID_BATCH_SIZE) {
         queueStorage.setLastSequenceIds(peerId, lastSeqIds);
         lastSeqIds.clear();
       }
     }
   }
 
-  private void setLastSequenceIdForSerialPeer(MasterProcedureEnv env)
-      throws IOException, ReplicationException {
-    Stream<TableDescriptor> stream = getTables(env);
+  protected final void setLastPushedSequenceId(MasterProcedureEnv env,
+      ReplicationPeerConfig peerConfig) throws IOException, ReplicationException {
+    Map<String, Long> lastSeqIds = new HashMap<String, Long>();
+    for (TableDescriptor td : env.getMasterServices().getTableDescriptors().getAll().values()) {
+      if (!td.hasGlobalReplicationScope()) {
+        continue;
+      }
+      TableName tn = td.getTableName();
+      if (!ReplicationUtils.contains(peerConfig, tn)) {
+        continue;
+      }
+      setLastPushedSequenceIdForTable(env, tn, lastSeqIds);
+    }
+    if (!lastSeqIds.isEmpty()) {
+      env.getReplicationPeerManager().getQueueStorage().setLastSequenceIds(peerId, lastSeqIds);
+    }
+  }
+
+  // Will put the encodedRegionName->lastPushedSeqId pair into the map passed in, if the map is
+  // large enough we will call queueStorage.setLastSequenceIds and clear the map. So the caller
+  // should not forget to check whether the map is empty at last, if not you should call
+  // queueStorage.setLastSequenceIds to write out the remaining entries in the map.
+  protected final void setLastPushedSequenceIdForTable(MasterProcedureEnv env, TableName tableName,
+      Map<String, Long> lastSeqIds) throws IOException, ReplicationException {
     TableStateManager tsm = env.getMasterServices().getTableStateManager();
     ReplicationQueueStorage queueStorage = env.getReplicationPeerManager().getQueueStorage();
     Connection conn = env.getMasterServices().getConnection();
     RegionStates regionStates = env.getAssignmentManager().getRegionStates();
     MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
-    Map<String, Long> lastSeqIds = new HashMap<String, Long>();
-    stream.forEach(td -> {
-      try {
-        if (tsm.getTableState(td.getTableName()).isEnabled()) {
-          for (Pair<String, Long> name2Barrier : MetaTableAccessor
-            .getTableEncodedRegionNameAndLastBarrier(conn, td.getTableName())) {
-            addToMap(lastSeqIds, name2Barrier.getFirst(), name2Barrier.getSecond().longValue() - 1,
-              queueStorage);
-          }
-        } else {
-          for (RegionInfo region : regionStates.getRegionsOfTable(td.getTableName(), true)) {
-            long maxSequenceId =
-              WALSplitter.getMaxRegionSequenceId(mfs.getFileSystem(), mfs.getRegionDir(region));
-            addToMap(lastSeqIds, region.getEncodedName(), maxSequenceId, queueStorage);
-          }
-        }
-      } catch (IOException | ReplicationException e) {
-        throw new RuntimeException(e);
+    boolean isTableEnabled;
+    try {
+      isTableEnabled = tsm.getTableState(tableName).isEnabled();
+    } catch (TableStateNotFoundException e) {
+      return;
+    }
+    if (isTableEnabled) {
+      for (Pair<String, Long> name2Barrier : MetaTableAccessor
+        .getTableEncodedRegionNameAndLastBarrier(conn, tableName)) {
+        addToMap(lastSeqIds, name2Barrier.getFirst(), name2Barrier.getSecond().longValue() - 1,
+          queueStorage);
+      }
+    } else {
+      for (RegionInfo region : regionStates.getRegionsOfTable(tableName, true)) {
+        long maxSequenceId =
+          WALSplitter.getMaxRegionSequenceId(mfs.getFileSystem(), mfs.getRegionDir(region));
+        addToMap(lastSeqIds, region.getEncodedName(), maxSequenceId, queueStorage);
       }
-    });
-    if (!lastSeqIds.isEmpty()) {
-      queueStorage.setLastSequenceIds(peerId, lastSeqIds);
     }
   }
 
@@ -232,8 +247,7 @@ public abstract class ModifyPeerProcedure extends AbstractPeerProcedure<PeerModi
         return Flow.HAS_MORE_STATE;
       case REFRESH_PEER_ON_RS:
         refreshPeer(env, getPeerOperationType());
-        setNextState(reopenRegionsAfterRefresh() ? PeerModificationState.SERIAL_PEER_REOPEN_REGIONS
-          : PeerModificationState.POST_PEER_MODIFICATION);
+        setNextState(nextStateAfterRefresh());
         return Flow.HAS_MORE_STATE;
       case SERIAL_PEER_REOPEN_REGIONS:
         try {
@@ -246,7 +260,7 @@ public abstract class ModifyPeerProcedure extends AbstractPeerProcedure<PeerModi
         return Flow.HAS_MORE_STATE;
       case SERIAL_PEER_UPDATE_LAST_PUSHED_SEQ_ID:
         try {
-          setLastSequenceIdForSerialPeer(env);
+          updateLastPushedSequenceIdForSerialPeer(env);
         } catch (Exception e) {
           LOG.warn("{} set last sequence id for peer {} failed, retry", getClass().getName(),
             peerId, e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/74ab10c3/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
index ccfd4a0..39c8fa9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/UpdatePeerConfigProcedure.java
@@ -18,6 +18,14 @@
 package org.apache.hadoop.hbase.master.replication;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
@@ -25,11 +33,13 @@ import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
 import org.apache.hadoop.hbase.replication.ReplicationUtils;
 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.UpdatePeerConfigStateData;
 
 /**
@@ -59,12 +69,84 @@ public class UpdatePeerConfigProcedure extends ModifyPeerProcedure {
     return PeerOperationType.UPDATE_CONFIG;
   }
 
+  private void addToList(List<String> encodedRegionNames, String encodedRegionName,
+      ReplicationQueueStorage queueStorage) throws ReplicationException {
+    encodedRegionNames.add(encodedRegionName);
+    if (encodedRegionNames.size() >= UPDATE_LAST_SEQ_ID_BATCH_SIZE) {
+      queueStorage.removeLastSequenceIds(peerId, encodedRegionNames);
+      encodedRegionNames.clear();
+    }
+  }
+
+  @Override
+  protected PeerModificationState nextStateAfterRefresh() {
+    if (peerConfig.isSerial()) {
+      if (oldPeerConfig.isSerial()) {
+        // both serial, then if the ns/table-cfs configs are not changed, just go with the normal
+        // way, otherwise we need to reopen the regions for the newly added tables.
+        return ReplicationUtils.isNamespacesAndTableCFsEqual(peerConfig, oldPeerConfig)
+          ? super.nextStateAfterRefresh()
+          : PeerModificationState.SERIAL_PEER_REOPEN_REGIONS;
+      } else {
+        // we change the peer to serial, need to reopen all regions
+        return PeerModificationState.SERIAL_PEER_REOPEN_REGIONS;
+      }
+    } else {
+      if (oldPeerConfig.isSerial()) {
+        // we remove the serial flag for peer, then we do not need to reopen all regions, but we
+        // need to remove the last pushed sequence ids.
+        return PeerModificationState.SERIAL_PEER_UPDATE_LAST_PUSHED_SEQ_ID;
+      } else {
+        // not serial for both, just go with the normal way.
+        return super.nextStateAfterRefresh();
+      }
+    }
+  }
+
   @Override
-  protected boolean reopenRegionsAfterRefresh() {
-    // If we remove some tables from the peer config then we do not need to enter the extra states
-    // for serial replication. Could try to optimize later since it is not easy to determine this...
-    return peerConfig.isSerial() && (!oldPeerConfig.isSerial() ||
-      !ReplicationUtils.isNamespacesAndTableCFsEqual(peerConfig, oldPeerConfig));
+  protected void updateLastPushedSequenceIdForSerialPeer(MasterProcedureEnv env)
+      throws IOException, ReplicationException {
+    if (!oldPeerConfig.isSerial()) {
+      assert peerConfig.isSerial();
+      // change to serial
+      setLastPushedSequenceId(env, peerConfig);
+      return;
+    }
+    if (!peerConfig.isSerial()) {
+      // remove the serial flag
+      env.getReplicationPeerManager().removeAllLastPushedSeqIds(peerId);
+      return;
+    }
+    // enter here means peerConfig and oldPeerConfig are both serial, let's find out the diffs and
+    // process them
+    ReplicationQueueStorage queueStorage = env.getReplicationPeerManager().getQueueStorage();
+    Connection conn = env.getMasterServices().getConnection();
+    Map<String, Long> lastSeqIds = new HashMap<String, Long>();
+    List<String> encodedRegionNames = new ArrayList<>();
+    for (TableDescriptor td : env.getMasterServices().getTableDescriptors().getAll().values()) {
+      if (!td.hasGlobalReplicationScope()) {
+        continue;
+      }
+      TableName tn = td.getTableName();
+      if (ReplicationUtils.contains(oldPeerConfig, tn)) {
+        if (!ReplicationUtils.contains(peerConfig, tn)) {
+          // removed from peer config
+          for (String encodedRegionName : MetaTableAccessor
+            .getTableEncodedRegionNamesForSerialReplication(conn, tn)) {
+            addToList(encodedRegionNames, encodedRegionName, queueStorage);
+          }
+        }
+      } else if (ReplicationUtils.contains(peerConfig, tn)) {
+        // newly added to peer config
+        setLastPushedSequenceIdForTable(env, tn, lastSeqIds);
+      }
+    }
+    if (!encodedRegionNames.isEmpty()) {
+      queueStorage.removeLastSequenceIds(peerId, encodedRegionNames);
+    }
+    if (!lastSeqIds.isEmpty()) {
+      queueStorage.setLastSequenceIds(peerId, lastSeqIds);
+    }
   }
 
   @Override
@@ -99,7 +181,9 @@ public class UpdatePeerConfigProcedure extends ModifyPeerProcedure {
   @Override
   protected void updatePeerStorage(MasterProcedureEnv env) throws ReplicationException {
     env.getReplicationPeerManager().updatePeerConfig(peerId, peerConfig);
-    if (enabled && reopenRegionsAfterRefresh()) {
+    // if we need to jump to the special states for serial peers, then we need to disable the peer
+    // first if it is not disabled yet.
+    if (enabled && nextStateAfterRefresh() != super.nextStateAfterRefresh()) {
       env.getReplicationPeerManager().disablePeer(peerId);
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/74ab10c3/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java
index 3b807aa..7a1bc55 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java
@@ -186,8 +186,8 @@ public class TestEnableTable {
         fail("Got an exception while deleting " + tableName);
       }
       int rowCount = 0;
-      try (ResultScanner scanner =
-          metaTable.getScanner(MetaTableAccessor.getScanForTableName(TEST_UTIL.getConnection(), tableName))) {
+      try (ResultScanner scanner = metaTable
+        .getScanner(MetaTableAccessor.getScanForTableName(TEST_UTIL.getConnection(), tableName))) {
         for (Result result : scanner) {
           LOG.info("Found when none expected: " + result);
           rowCount++;

http://git-wip-us.apache.org/repos/asf/hbase/blob/74ab10c3/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestRemoveFromSerialReplicationPeer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestRemoveFromSerialReplicationPeer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestRemoveFromSerialReplicationPeer.java
new file mode 100644
index 0000000..eda15d8
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestRemoveFromSerialReplicationPeer.java
@@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.replication;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.Collections;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.ReplicationTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils.StreamLacksCapabilityException;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
+
+/**
+ * Testcase for HBASE-20296.
+ */
+@Category({ ReplicationTests.class, MediumTests.class })
+public class TestRemoveFromSerialReplicationPeer extends SerialReplicationTestBase {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestRemoveFromSerialReplicationPeer.class);
+
+  @Before
+  public void setUp() throws IOException, StreamLacksCapabilityException {
+    setupWALWriter();
+  }
+
+  private void waitUntilHasLastPushedSequenceId(RegionInfo region) throws Exception {
+    ReplicationQueueStorage queueStorage =
+      UTIL.getMiniHBaseCluster().getMaster().getReplicationPeerManager().getQueueStorage();
+    UTIL.waitFor(30000, new ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        return queueStorage.getLastSequenceId(region.getEncodedName(), PEER_ID) > 0;
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        return "Still no last pushed sequence id for " + region;
+      }
+    });
+  }
+
+  @Test
+  public void testRemoveTable() throws Exception {
+    TableName tableName = createTable();
+    ReplicationPeerConfig peerConfig = ReplicationPeerConfig.newBuilder()
+      .setClusterKey("127.0.0.1:2181:/hbase")
+      .setReplicationEndpointImpl(LocalReplicationEndpoint.class.getName())
+      .setReplicateAllUserTables(false)
+      .setTableCFsMap(ImmutableMap.of(tableName, Collections.emptyList())).setSerial(true).build();
+    UTIL.getAdmin().addReplicationPeer(PEER_ID, peerConfig, true);
+    try (Table table = UTIL.getConnection().getTable(tableName)) {
+      for (int i = 0; i < 100; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+    RegionInfo region = UTIL.getMiniHBaseCluster().getRegions(tableName).get(0).getRegionInfo();
+    waitUntilHasLastPushedSequenceId(region);
+
+    UTIL.getAdmin().updateReplicationPeerConfig(PEER_ID,
+      ReplicationPeerConfig.newBuilder(peerConfig).setTableCFsMap(Collections.emptyMap()).build());
+
+    ReplicationQueueStorage queueStorage =
+      UTIL.getMiniHBaseCluster().getMaster().getReplicationPeerManager().getQueueStorage();
+    assertEquals(HConstants.NO_SEQNUM,
+      queueStorage.getLastSequenceId(region.getEncodedName(), PEER_ID));
+  }
+
+  @Test
+  public void testRemoveSerialFlag() throws Exception {
+    TableName tableName = createTable();
+    addPeer(true);
+    try (Table table = UTIL.getConnection().getTable(tableName)) {
+      for (int i = 0; i < 100; i++) {
+        table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i)));
+      }
+    }
+    RegionInfo region = UTIL.getMiniHBaseCluster().getRegions(tableName).get(0).getRegionInfo();
+    waitUntilHasLastPushedSequenceId(region);
+    UTIL.getAdmin().updateReplicationPeerConfig(PEER_ID, ReplicationPeerConfig
+      .newBuilder(UTIL.getAdmin().getReplicationPeerConfig(PEER_ID)).setSerial(false).build());
+    waitUntilReplicationDone(100);
+
+    ReplicationQueueStorage queueStorage =
+      UTIL.getMiniHBaseCluster().getMaster().getReplicationPeerManager().getQueueStorage();
+    assertEquals(HConstants.NO_SEQNUM,
+      queueStorage.getLastSequenceId(region.getEncodedName(), PEER_ID));
+  }
+}


[02/22] hbase git commit: HBASE-20362 TestMasterShutdown.testMasterShutdownBeforeStartingAnyRegionServer is flaky

Posted by zh...@apache.org.
HBASE-20362 TestMasterShutdown.testMasterShutdownBeforeStartingAnyRegionServer is flaky


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

Branch: refs/heads/HBASE-20046-branch-2
Commit: 39c1ddc6e33fcdb4086ea3e6e7a671dd55bb469a
Parents: 1ec708b
Author: zhangduo <zh...@apache.org>
Authored: Sun Apr 8 15:31:02 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Apr 9 14:55:42 2018 +0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/client/Admin.java   | 15 ++++++++++++---
 .../hadoop/hbase/master/TestMasterShutdown.java      | 12 ++++++------
 2 files changed, 18 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/39c1ddc6/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..a6a4224 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
@@ -1,5 +1,4 @@
 /**
- *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -1415,15 +1414,25 @@ public interface Admin extends Abortable, Closeable {
       throws IOException;
 
   /**
+   * <p>
    * Shuts down the HBase cluster.
-   *
+   * </p>
+   * <p>
+   * Notice that, a success shutdown call may ends with an error since the remote server has already
+   * been shutdown.
+   * </p>
    * @throws IOException if a remote or network exception occurs
    */
   void shutdown() throws IOException;
 
   /**
+   * <p>
    * Shuts down the current HBase master only. Does not shutdown the cluster.
-   *
+   * </p>
+   * <p>
+   * Notice that, a success stopMaster call may ends with an error since the remote server has
+   * already been shutdown.
+   * </p>
    * @throws IOException if a remote or network exception occurs
    * @see #shutdown()
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/39c1ddc6/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
index 4c51026..63cfe1f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterShutdown.java
@@ -55,7 +55,6 @@ public class TestMasterShutdown {
    * <p>
    * Starts with three masters.  Tells the active master to shutdown the cluster.
    * Verifies that all masters are properly shutdown.
-   * @throws Exception
    */
   @Test
   public void testMasterShutdown() throws Exception {
@@ -130,17 +129,18 @@ public class TestMasterShutdown {
       public void run() {
         LOG.info("Before call to shutdown master");
         try {
-          try (Connection connection =
-              ConnectionFactory.createConnection(util.getConfiguration())) {
+          try (
+            Connection connection = ConnectionFactory.createConnection(util.getConfiguration())) {
             try (Admin admin = connection.getAdmin()) {
               admin.shutdown();
             }
           }
-          LOG.info("After call to shutdown master");
-          cluster.waitOnMaster(MASTER_INDEX);
         } catch (Exception e) {
+          LOG.info("Error while calling Admin.shutdown, which is expected: " + e.getMessage());
         }
-      };
+        LOG.info("After call to shutdown master");
+        cluster.waitOnMaster(MASTER_INDEX);
+      }
     };
     shutdownThread.start();
     LOG.info("Called master join on " + master.getName());


[06/22] hbase git commit: HBASE-20116 Optimize the region last pushed sequence id layout on zk

Posted by zh...@apache.org.
HBASE-20116 Optimize the region last pushed sequence id layout on zk


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

Branch: refs/heads/HBASE-20046-branch-2
Commit: 2b9fed863f19510007f4bce08082ee42fb867eac
Parents: 16a4dd6
Author: huzheng <op...@gmail.com>
Authored: Tue Mar 20 10:13:15 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Apr 9 15:18:44 2018 +0800

----------------------------------------------------------------------
 .../replication/ZKReplicationQueueStorage.java  | 22 ++++++++++++++------
 .../replication/TestReplicationStateBasic.java  |  3 ++-
 .../TestZKReplicationQueueStorage.java          | 10 +++++++++
 3 files changed, 28 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2b9fed86/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
index 63f43e8..6c9752a 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -138,19 +139,28 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
    * So the final znode path will be format like this:
    *
    * <pre>
-   * /hbase/replication/regions/254/dd04e76a6966d4ffa908ed0586764767-100
+   * /hbase/replication/regions/e1/ff/dd04e76a6966d4ffa908ed0586764767-100
    * </pre>
    *
-   * The 254 indicate the hash of encoded region name, the 100 indicate the peer id.
+   * The e1 indicate the first level hash of encoded region name, and the ff indicate the second
+   * level hash of encoded region name, the 100 indicate the peer id. <br>
+   * Note that here we use two-level hash because if only one-level hash (such as mod 65535), it
+   * will still lead to too many children under the /hbase/replication/regions znode.
    * @param encodedRegionName the encoded region name.
    * @param peerId peer id for replication.
    * @return ZNode path to persist the max sequence id that we've pushed for the given region and
    *         peer.
    */
-  private String getSerialReplicationRegionPeerNode(String encodedRegionName, String peerId) {
-    int hash = encodedRegionName.hashCode() & 0x0000FFFF;
-    String hashPath = ZNodePaths.joinZNode(regionsZNode, String.valueOf(hash));
-    return ZNodePaths.joinZNode(hashPath, String.format("%s-%s", encodedRegionName, peerId));
+  @VisibleForTesting
+  public String getSerialReplicationRegionPeerNode(String encodedRegionName, String peerId) {
+    if (encodedRegionName == null || encodedRegionName.length() != RegionInfo.MD5_HEX_LENGTH) {
+      throw new IllegalArgumentException(
+          "Invalid encoded region name: " + encodedRegionName + ", length should be 32.");
+    }
+    return new StringBuilder(regionsZNode).append(ZNodePaths.ZNODE_PATH_SEPARATOR)
+        .append(encodedRegionName.substring(0, 2)).append(ZNodePaths.ZNODE_PATH_SEPARATOR)
+        .append(encodedRegionName.substring(2, 4)).append(ZNodePaths.ZNODE_PATH_SEPARATOR)
+        .append(encodedRegionName).append("-").append(peerId).toString();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/2b9fed86/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 21b09aa..3ed4121 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
@@ -285,7 +285,8 @@ public abstract class TestReplicationStateBasic {
     ServerName serverName1 = ServerName.valueOf("127.0.0.1", 8000, 10000);
     assertTrue(rqs.getAllQueues(serverName1).isEmpty());
     String queue1 = "1";
-    String region0 = "region0", region1 = "region1";
+    String region0 = "6b2c8f8555335cc9af74455b94516cbe",
+        region1 = "6ecd2e9e010499f8ddef97ee8f70834f";
     for (int i = 0; i < 10; i++) {
       rqs.addWAL(serverName1, queue1, getFileName("file1", i));
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2b9fed86/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
index c813870..28cdff1 100644
--- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -251,4 +252,13 @@ public class TestZKReplicationQueueStorage {
     assertEquals(1, allHFileRefs.size());
     assertThat(allHFileRefs, hasItems("test"));
   }
+
+  @Test
+  public void testRegionsZNodeLayout() throws Exception {
+    String peerId = "1";
+    String encodedRegionName = "31d9792f4435b99d9fb1016f6fbc8dc7";
+    String expectedPath = "/hbase/replication/regions/31/d9/" + encodedRegionName + "-" + peerId;
+    String path = STORAGE.getSerialReplicationRegionPeerNode(encodedRegionName, peerId);
+    Assert.assertEquals(expectedPath, path);
+  }
 }


[19/22] hbase git commit: HBASE-20116 addendum fix javadoc and also a simple optimization

Posted by zh...@apache.org.
HBASE-20116 addendum fix javadoc and also a simple optimization


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

Branch: refs/heads/HBASE-20046-branch-2
Commit: 40ad5aeafe5019abf59c3d149df42d5f4afcc1a2
Parents: aec43bb
Author: zhangduo <zh...@apache.org>
Authored: Thu Mar 22 08:31:20 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Apr 9 15:18:44 2018 +0800

----------------------------------------------------------------------
 .../replication/ZKReplicationQueueStorage.java  | 22 ++++++++++++--------
 .../TestZKReplicationQueueStorage.java          |  2 +-
 2 files changed, 14 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/40ad5aea/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
index 6c9752a..1a5749e 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
@@ -133,26 +133,30 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
   }
 
   /**
+   * <p>
    * Put all regions under /hbase/replication/regions znode will lead to too many children because
-   * of the huge number of regions in real production environment. So here we use hash of encoded
-   * region name to distribute the znode into multiple znodes. <br>
+   * of the huge number of regions in real production environment. So here we will distribute the
+   * znodes to multiple directories.
+   * </p>
+   * <p>
    * So the final znode path will be format like this:
    *
    * <pre>
-   * /hbase/replication/regions/e1/ff/dd04e76a6966d4ffa908ed0586764767-100
+   * /hbase/replication/regions/dd/04/e76a6966d4ffa908ed0586764767-100
    * </pre>
    *
-   * The e1 indicate the first level hash of encoded region name, and the ff indicate the second
-   * level hash of encoded region name, the 100 indicate the peer id. <br>
-   * Note that here we use two-level hash because if only one-level hash (such as mod 65535), it
-   * will still lead to too many children under the /hbase/replication/regions znode.
+   * Here the full encoded region name is dd04e76a6966d4ffa908ed0586764767, and we use the first two
+   * characters 'dd' as the first level directory name, and use the next two characters '04' as the
+   * second level directory name, and the rest part as the prefix of the znode, and the suffix '100'
+   * is the peer id.
+   * </p>
    * @param encodedRegionName the encoded region name.
    * @param peerId peer id for replication.
    * @return ZNode path to persist the max sequence id that we've pushed for the given region and
    *         peer.
    */
   @VisibleForTesting
-  public String getSerialReplicationRegionPeerNode(String encodedRegionName, String peerId) {
+  String getSerialReplicationRegionPeerNode(String encodedRegionName, String peerId) {
     if (encodedRegionName == null || encodedRegionName.length() != RegionInfo.MD5_HEX_LENGTH) {
       throw new IllegalArgumentException(
           "Invalid encoded region name: " + encodedRegionName + ", length should be 32.");
@@ -160,7 +164,7 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
     return new StringBuilder(regionsZNode).append(ZNodePaths.ZNODE_PATH_SEPARATOR)
         .append(encodedRegionName.substring(0, 2)).append(ZNodePaths.ZNODE_PATH_SEPARATOR)
         .append(encodedRegionName.substring(2, 4)).append(ZNodePaths.ZNODE_PATH_SEPARATOR)
-        .append(encodedRegionName).append("-").append(peerId).toString();
+        .append(encodedRegionName.substring(4)).append("-").append(peerId).toString();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/40ad5aea/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
index 28cdff1..ca86a05 100644
--- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
@@ -257,7 +257,7 @@ public class TestZKReplicationQueueStorage {
   public void testRegionsZNodeLayout() throws Exception {
     String peerId = "1";
     String encodedRegionName = "31d9792f4435b99d9fb1016f6fbc8dc7";
-    String expectedPath = "/hbase/replication/regions/31/d9/" + encodedRegionName + "-" + peerId;
+    String expectedPath = "/hbase/replication/regions/31/d9/792f4435b99d9fb1016f6fbc8dc7-" + peerId;
     String path = STORAGE.getSerialReplicationRegionPeerNode(encodedRegionName, peerId);
     Assert.assertEquals(expectedPath, path);
   }


[14/22] hbase git commit: HBASE-20117 Cleanup the unused replication barriers in meta table

Posted by zh...@apache.org.
HBASE-20117 Cleanup the unused replication barriers in meta table


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

Branch: refs/heads/HBASE-20046-branch-2
Commit: 644bfe36b297b2787bf07a46eb6f5085322edfa9
Parents: fedf3ca
Author: zhangduo <zh...@apache.org>
Authored: Tue Mar 13 21:36:06 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Apr 9 15:18:44 2018 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/MetaTableAccessor.java  |   2 +-
 .../hbase/replication/ReplicationUtils.java     |  56 +++-
 .../org/apache/hadoop/hbase/master/HMaster.java |  91 +++---
 .../cleaner/ReplicationBarrierCleaner.java      | 162 ++++++++++
 .../replication/ReplicationPeerManager.java     |  10 +
 .../NamespaceTableCfWALEntryFilter.java         |  39 +--
 .../cleaner/TestReplicationBarrierCleaner.java  | 293 +++++++++++++++++++
 .../TestSerialReplicationChecker.java           |   2 +-
 8 files changed, 565 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/644bfe36/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 2a88b56..a800c1c 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
@@ -2053,7 +2053,7 @@ public class MetaTableAccessor {
     return Bytes.toLong(c.getValueArray(), c.getValueOffset(), c.getValueLength());
   }
 
-  private static long[] getReplicationBarriers(Result result) {
+  public static long[] getReplicationBarriers(Result result) {
     return result.getColumnCells(HConstants.REPLICATION_BARRIER_FAMILY, HConstants.SEQNUM_QUALIFIER)
       .stream().mapToLong(MetaTableAccessor::getReplicationBarrier).sorted().distinct().toArray();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/644bfe36/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 857b385..e2479e0 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
@@ -39,15 +39,6 @@ public final class ReplicationUtils {
   private ReplicationUtils() {
   }
 
-  /**
-   * @param c Configuration to look at
-   * @return True if replication for bulk load data is enabled.
-   */
-  public static boolean isReplicationForBulkLoadDataEnabled(final Configuration c) {
-    return c.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
-      HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
-  }
-
   public static Configuration getPeerClusterConfiguration(ReplicationPeerConfig peerConfig,
       Configuration baseConf) throws ReplicationException {
     Configuration otherConf;
@@ -135,4 +126,51 @@ public final class ReplicationUtils {
         isTableCFsEqual(rpc1.getTableCFsMap(), rpc2.getTableCFsMap());
     }
   }
+
+  /**
+   * @param c Configuration to look at
+   * @return True if replication for bulk load data is enabled.
+   */
+  public static boolean isReplicationForBulkLoadDataEnabled(final Configuration c) {
+    return c.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
+      HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
+  }
+
+  /**
+   * Returns whether we should replicate the given table.
+   */
+  public static boolean contains(ReplicationPeerConfig peerConfig, TableName tableName) {
+    String namespace = tableName.getNamespaceAsString();
+    if (peerConfig.replicateAllUserTables()) {
+      // replicate all user tables, but filter by exclude namespaces and table-cfs config
+      Set<String> excludeNamespaces = peerConfig.getExcludeNamespaces();
+      if (excludeNamespaces != null && excludeNamespaces.contains(namespace)) {
+        return false;
+      }
+      Map<TableName, List<String>> excludedTableCFs = peerConfig.getTableCFsMap();
+      // trap here, must check existence first since HashMap allows null value.
+      if (excludedTableCFs == null || !excludedTableCFs.containsKey(tableName)) {
+        return true;
+      }
+      List<String> cfs = excludedTableCFs.get(tableName);
+      // if cfs is null or empty then we can make sure that we do not need to replicate this table,
+      // otherwise, we may still need to replicate the table but filter out some families.
+      return cfs != null && !cfs.isEmpty();
+    } else {
+      // Not replicate all user tables, so filter by namespaces and table-cfs config
+      Set<String> namespaces = peerConfig.getNamespaces();
+      Map<TableName, List<String>> tableCFs = peerConfig.getTableCFsMap();
+
+      if (namespaces == null && tableCFs == null) {
+        return false;
+      }
+
+      // First filter by namespaces config
+      // If table's namespace in peer config, all the tables data are applicable for replication
+      if (namespaces != null && namespaces.contains(namespace)) {
+        return true;
+      }
+      return tableCFs != null && tableCFs.containsKey(tableName);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/644bfe36/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 7d751fb..6d0b58b 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
@@ -71,6 +71,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.PleaseHoldException;
 import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
+import org.apache.hadoop.hbase.ScheduledChore;
 import org.apache.hadoop.hbase.ServerMetricsBuilder;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableDescriptors;
@@ -109,6 +110,7 @@ import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
 import org.apache.hadoop.hbase.master.cleaner.CleanerChore;
 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
 import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
+import org.apache.hadoop.hbase.master.cleaner.ReplicationBarrierCleaner;
 import org.apache.hadoop.hbase.master.locking.LockManager;
 import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan;
 import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
@@ -371,6 +373,7 @@ public class HMaster extends HRegionServer implements MasterServices {
   CatalogJanitor catalogJanitorChore;
   private LogCleaner logCleaner;
   private HFileCleaner hfileCleaner;
+  private ReplicationBarrierCleaner replicationBarrierCleaner;
   private ExpiredMobFileCleanerChore expiredMobFileCleanerChore;
   private MobCompactionChore mobCompactChore;
   private MasterMobCompactionThread mobCompactThread;
@@ -1179,19 +1182,30 @@ public class HMaster extends HRegionServer implements MasterServices {
          getMasterWalManager().getOldLogDir());
     getChoreService().scheduleChore(logCleaner);
 
-   //start the hfile archive cleaner thread
+    // start the hfile archive cleaner thread
     Path archiveDir = HFileArchiveUtil.getArchivePath(conf);
     Map<String, Object> params = new HashMap<>();
     params.put(MASTER, this);
     this.hfileCleaner = new HFileCleaner(cleanerInterval, this, conf, getMasterFileSystem()
         .getFileSystem(), archiveDir, params);
     getChoreService().scheduleChore(hfileCleaner);
+
+    replicationBarrierCleaner =
+      new ReplicationBarrierCleaner(conf, this, getConnection(), replicationPeerManager);
+    getChoreService().scheduleChore(replicationBarrierCleaner);
+
     serviceStarted = true;
     if (LOG.isTraceEnabled()) {
       LOG.trace("Started service threads");
     }
   }
 
+  private void cancelChore(ScheduledChore chore) {
+    if (chore != null) {
+      chore.cancel();
+    }
+  }
+
   @Override
   protected void stopServiceThreads() {
     if (masterJettyServer != null) {
@@ -1205,24 +1219,33 @@ public class HMaster extends HRegionServer implements MasterServices {
     super.stopServiceThreads();
     stopChores();
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Stopping service threads");
-    }
+    LOG.debug("Stopping service threads");
 
-    // Clean up and close up shop
-    if (this.logCleaner != null) this.logCleaner.cancel(true);
-    if (this.hfileCleaner != null) this.hfileCleaner.cancel(true);
-    if (this.quotaManager != null) this.quotaManager.stop();
+    if (this.quotaManager != null) {
+      this.quotaManager.stop();
+    }
 
-    if (this.activeMasterManager != null) this.activeMasterManager.stop();
-    if (this.serverManager != null) this.serverManager.stop();
-    if (this.assignmentManager != null) this.assignmentManager.stop();
+    if (this.activeMasterManager != null) {
+      this.activeMasterManager.stop();
+    }
+    if (this.serverManager != null) {
+      this.serverManager.stop();
+    }
+    if (this.assignmentManager != null) {
+      this.assignmentManager.stop();
+    }
 
     stopProcedureExecutor();
 
-    if (this.walManager != null) this.walManager.stop();
-    if (this.fileSystemManager != null) this.fileSystemManager.stop();
-    if (this.mpmHost != null) this.mpmHost.stop("server shutting down.");
+    if (this.walManager != null) {
+      this.walManager.stop();
+    }
+    if (this.fileSystemManager != null) {
+      this.fileSystemManager.stop();
+    }
+    if (this.mpmHost != null) {
+      this.mpmHost.stop("server shutting down.");
+    }
   }
 
   private void startProcedureExecutor() throws IOException {
@@ -1261,37 +1284,21 @@ public class HMaster extends HRegionServer implements MasterServices {
   }
 
   private void stopChores() {
-    if (this.expiredMobFileCleanerChore != null) {
-      this.expiredMobFileCleanerChore.cancel(true);
-    }
-    if (this.mobCompactChore != null) {
-      this.mobCompactChore.cancel(true);
-    }
-    if (this.balancerChore != null) {
-      this.balancerChore.cancel(true);
-    }
-    if (this.normalizerChore != null) {
-      this.normalizerChore.cancel(true);
-    }
-    if (this.clusterStatusChore != null) {
-      this.clusterStatusChore.cancel(true);
-    }
-    if (this.catalogJanitorChore != null) {
-      this.catalogJanitorChore.cancel(true);
-    }
-    if (this.clusterStatusPublisherChore != null){
-      clusterStatusPublisherChore.cancel(true);
-    }
+    cancelChore(this.expiredMobFileCleanerChore);
+    cancelChore(this.mobCompactChore);
+    cancelChore(this.balancerChore);
+    cancelChore(this.normalizerChore);
+    cancelChore(this.clusterStatusChore);
+    cancelChore(this.catalogJanitorChore);
+    cancelChore(this.clusterStatusPublisherChore);
     if (this.mobCompactThread != null) {
       this.mobCompactThread.close();
     }
-
-    if (this.quotaObserverChore != null) {
-      quotaObserverChore.cancel();
-    }
-    if (this.snapshotQuotaChore != null) {
-      snapshotQuotaChore.cancel();
-    }
+    cancelChore(this.clusterStatusPublisherChore);
+    cancelChore(this.snapshotQuotaChore);
+    cancelChore(this.logCleaner);
+    cancelChore(this.hfileCleaner);
+    cancelChore(this.replicationBarrierCleaner);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/644bfe36/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationBarrierCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationBarrierCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationBarrierCleaner.java
new file mode 100644
index 0000000..16b8fc5
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationBarrierCleaner.java
@@ -0,0 +1,162 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.cleaner;
+
+import java.io.IOException;
+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.HConstants;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.ScheduledChore;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.RegionInfo;
+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.master.replication.ReplicationPeerManager;
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Used to clean the useless barriers in {@link HConstants#REPLICATION_BARRIER_FAMILY_STR} family in
+ * meta table.
+ */
+@InterfaceAudience.Private
+public class ReplicationBarrierCleaner extends ScheduledChore {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ReplicationBarrierCleaner.class);
+
+  private static final String REPLICATION_BARRIER_CLEANER_INTERVAL =
+    "hbase.master.cleaner.replication.barrier.interval";
+
+  // 12 hour. Usually regions will not be moved so the barrier are rarely updated. Use a large
+  // interval.
+  private static final int DEFAULT_REPLICATION_BARRIER_CLEANER_INTERVAL = 12 * 60 * 60 * 1000;
+
+  private final Connection conn;
+
+  private final ReplicationPeerManager peerManager;
+
+  public ReplicationBarrierCleaner(Configuration conf, Stoppable stopper, Connection conn,
+      ReplicationPeerManager peerManager) {
+    super("ReplicationBarrierCleaner", stopper, conf.getInt(REPLICATION_BARRIER_CLEANER_INTERVAL,
+      DEFAULT_REPLICATION_BARRIER_CLEANER_INTERVAL));
+    this.conn = conn;
+    this.peerManager = peerManager;
+  }
+
+  @Override
+  protected void chore() {
+    long totalRows = 0;
+    long cleanedRows = 0;
+    long deletedRows = 0;
+    long deletedBarriers = 0;
+    TableName tableName = null;
+    List<String> peerIds = null;
+    try (Table metaTable = conn.getTable(TableName.META_TABLE_NAME);
+        ResultScanner scanner = metaTable.getScanner(
+          new Scan().addFamily(HConstants.REPLICATION_BARRIER_FAMILY).readAllVersions())) {
+      for (;;) {
+        Result result = scanner.next();
+        if (result == null) {
+          break;
+        }
+        totalRows++;
+        long[] barriers = MetaTableAccessor.getReplicationBarriers(result);
+        if (barriers.length == 0) {
+          continue;
+        }
+        byte[] regionName = result.getRow();
+        TableName tn = RegionInfo.getTable(regionName);
+        if (!tn.equals(tableName)) {
+          tableName = tn;
+          peerIds = peerManager.getSerialPeerIdsBelongsTo(tableName);
+        }
+        if (peerIds.isEmpty()) {
+          // no serial replication, only keep the newest barrier
+          Cell cell = result.getColumnLatestCell(HConstants.REPLICATION_BARRIER_FAMILY,
+            HConstants.SEQNUM_QUALIFIER);
+          metaTable.delete(new Delete(regionName).addFamily(HConstants.REPLICATION_BARRIER_FAMILY,
+            cell.getTimestamp() - 1));
+          cleanedRows++;
+          deletedBarriers += barriers.length - 1;
+          continue;
+        }
+        String encodedRegionName = RegionInfo.encodeRegionName(regionName);
+        long pushedSeqId = Long.MAX_VALUE;
+        for (String peerId : peerIds) {
+          pushedSeqId = Math.min(pushedSeqId,
+            peerManager.getQueueStorage().getLastSequenceId(encodedRegionName, peerId));
+        }
+        int index = Arrays.binarySearch(barriers, pushedSeqId);
+        if (index == -1) {
+          // beyond the first barrier, usually this should not happen but anyway let's add a check
+          // for it.
+          continue;
+        }
+        if (index < 0) {
+          index = -index - 1;
+        } else {
+          index++;
+        }
+        // A special case for merged/split region, where we are in the last closed range and the
+        // pushedSeqId is the last barrier minus 1.
+        if (index == barriers.length - 1 && pushedSeqId == barriers[barriers.length - 1] - 1) {
+          // check if the region has already been removed, i.e, no catalog family
+          if (!metaTable.exists(new Get(regionName).addFamily(HConstants.CATALOG_FAMILY))) {
+            metaTable
+              .delete(new Delete(regionName).addFamily(HConstants.REPLICATION_BARRIER_FAMILY));
+            deletedRows++;
+            deletedBarriers += barriers.length;
+            continue;
+          }
+        }
+        // the barrier before 'index - 1'(exclusive) can be safely removed. See the algorithm in
+        // SerialReplicationChecker for more details.
+        if (index - 1 > 0) {
+          List<Cell> cells = result.getColumnCells(HConstants.REPLICATION_BARRIER_FAMILY,
+            HConstants.SEQNUM_QUALIFIER);
+          // All barriers before this cell(exclusive) can be removed
+          Cell cell = cells.get(cells.size() - index);
+          metaTable.delete(new Delete(regionName).addFamily(HConstants.REPLICATION_BARRIER_FAMILY,
+            cell.getTimestamp() - 1));
+          cleanedRows++;
+          deletedBarriers += index - 1;
+        }
+      }
+    } catch (ReplicationException | IOException e) {
+      LOG.warn("Failed to clean up replication barrier", e);
+    }
+    if (totalRows > 0) {
+      LOG.info(
+        "Cleanup replication barriers: " +
+          "totalRows {}, cleanedRows {}, deletedRows {}, deletedBarriers {}",
+        totalRows, cleanedRows, deletedRows, deletedBarriers);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/644bfe36/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 19cd89d..1e93373 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
@@ -327,6 +327,16 @@ public class ReplicationPeerManager {
     }
   }
 
+  public List<String> getSerialPeerIdsBelongsTo(TableName tableName) {
+    return peers.values().stream().filter(p -> p.getPeerConfig().isSerial())
+      .filter(p -> ReplicationUtils.contains(p.getPeerConfig(), tableName)).map(p -> p.getPeerId())
+      .collect(Collectors.toList());
+  }
+
+  public ReplicationQueueStorage getQueueStorage() {
+    return queueStorage;
+  }
+
   public static ReplicationPeerManager create(ZKWatcher zk, Configuration conf)
       throws ReplicationException {
     ReplicationPeerStorage peerStorage =

http://git-wip-us.apache.org/repos/asf/hbase/blob/644bfe36/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 08c9f37..3a3200a 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
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.replication;
 
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.TableName;
@@ -53,44 +52,10 @@ public class NamespaceTableCfWALEntryFilter implements WALEntryFilter, WALCellFi
 
   @Override
   public Entry filter(Entry entry) {
-    TableName tabName = entry.getKey().getTableName();
-    String namespace = tabName.getNamespaceAsString();
-    ReplicationPeerConfig peerConfig = this.peer.getPeerConfig();
-
-    if (peerConfig.replicateAllUserTables()) {
-      // replicate all user tables, but filter by exclude namespaces config
-      Set<String> excludeNamespaces = peerConfig.getExcludeNamespaces();
-
-      // return null(prevent replicating) if logKey's table is in this peer's
-      // exclude namespaces list
-      if (excludeNamespaces != null && excludeNamespaces.contains(namespace)) {
-        return null;
-      }
-
+    if (ReplicationUtils.contains(this.peer.getPeerConfig(), entry.getKey().getTableName())) {
       return entry;
     } else {
-      // Not replicate all user tables, so filter by namespaces and table-cfs config
-      Set<String> namespaces = peerConfig.getNamespaces();
-      Map<TableName, List<String>> tableCFs = peerConfig.getTableCFsMap();
-
-      if (namespaces == null && tableCFs == null) {
-        return null;
-      }
-
-      // First filter by namespaces config
-      // If table's namespace in peer config, all the tables data are applicable for replication
-      if (namespaces != null && namespaces.contains(namespace)) {
-        return entry;
-      }
-
-      // Then filter by table-cfs config
-      // return null(prevent replicating) if logKey's table isn't in this peer's
-      // replicable tables list
-      if (tableCFs == null || !tableCFs.containsKey(tabName)) {
-        return null;
-      }
-
-      return entry;
+      return null;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/644bfe36/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationBarrierCleaner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationBarrierCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationBarrierCleaner.java
new file mode 100644
index 0000000..671bc22
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationBarrierCleaner.java
@@ -0,0 +1,293 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.cleaner;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertFalse;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.Stoppable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
+import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
+import org.apache.hadoop.hbase.replication.ReplicationException;
+import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+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;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
+
+@Category({ MasterTests.class, MediumTests.class })
+public class TestReplicationBarrierCleaner {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestReplicationBarrierCleaner.class);
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestHFileCleaner.class);
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  @Rule
+  public final TestName name = new TestName();
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    UTIL.startMiniCluster(1);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    try (Table table = UTIL.getConnection().getTable(TableName.META_TABLE_NAME);
+        ResultScanner scanner = table.getScanner(new Scan().addFamily(HConstants.CATALOG_FAMILY)
+          .addFamily(HConstants.REPLICATION_BARRIER_FAMILY).setFilter(new FirstKeyOnlyFilter()))) {
+      for (;;) {
+        Result result = scanner.next();
+        if (result == null) {
+          break;
+        }
+        TableName tableName = RegionInfo.getTable(result.getRow());
+        if (!tableName.isSystemTable()) {
+          table.delete(new Delete(result.getRow()));
+        }
+      }
+    }
+  }
+
+  private ReplicationPeerManager create(ReplicationQueueStorage queueStorage,
+      List<String> firstPeerIds, @SuppressWarnings("unchecked") List<String>... peerIds) {
+    ReplicationPeerManager peerManager = mock(ReplicationPeerManager.class);
+    if (queueStorage != null) {
+      when(peerManager.getQueueStorage()).thenReturn(queueStorage);
+    }
+    if (peerIds.length == 0) {
+      when(peerManager.getSerialPeerIdsBelongsTo(any(TableName.class))).thenReturn(firstPeerIds);
+    } else {
+      when(peerManager.getSerialPeerIdsBelongsTo(any(TableName.class))).thenReturn(firstPeerIds,
+        peerIds);
+    }
+    return peerManager;
+  }
+
+  private ReplicationQueueStorage create(Long lastPushedSeqId, Long... lastPushedSeqIds)
+      throws ReplicationException {
+    ReplicationQueueStorage queueStorage = mock(ReplicationQueueStorage.class);
+    if (lastPushedSeqIds.length == 0) {
+      when(queueStorage.getLastSequenceId(anyString(), anyString())).thenReturn(lastPushedSeqId);
+    } else {
+      when(queueStorage.getLastSequenceId(anyString(), anyString())).thenReturn(lastPushedSeqId,
+        lastPushedSeqIds);
+    }
+    return queueStorage;
+  }
+
+  private ReplicationBarrierCleaner create(ReplicationPeerManager peerManager) throws IOException {
+    return new ReplicationBarrierCleaner(UTIL.getConfiguration(), new WarnOnlyStoppable(),
+      UTIL.getConnection(), peerManager);
+  }
+
+  private void addBarrier(RegionInfo region, long... barriers) throws IOException {
+    Put put = new Put(region.getRegionName(), EnvironmentEdgeManager.currentTime());
+    for (int i = 0; i < barriers.length; i++) {
+      put.addColumn(HConstants.REPLICATION_BARRIER_FAMILY, HConstants.SEQNUM_QUALIFIER,
+        put.getTimeStamp() - barriers.length + i, Bytes.toBytes(barriers[i]));
+    }
+    try (Table table = UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) {
+      table.put(put);
+    }
+  }
+
+  private void fillCatalogFamily(RegionInfo region) throws IOException {
+    try (Table table = UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) {
+      table.put(new Put(region.getRegionName()).addColumn(HConstants.CATALOG_FAMILY,
+        Bytes.toBytes("whatever"), Bytes.toBytes("whatever")));
+    }
+  }
+
+  private void clearCatalogFamily(RegionInfo region) throws IOException {
+    try (Table table = UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) {
+      table.delete(new Delete(region.getRegionName()).addFamily(HConstants.CATALOG_FAMILY));
+    }
+  }
+
+  @Test
+  public void testNothing() throws IOException {
+    ReplicationPeerManager peerManager = mock(ReplicationPeerManager.class);
+    ReplicationBarrierCleaner cleaner = create(peerManager);
+    cleaner.chore();
+    verify(peerManager, never()).getSerialPeerIdsBelongsTo(any(TableName.class));
+    verify(peerManager, never()).getQueueStorage();
+  }
+
+  @Test
+  public void testCleanNoPeers() throws IOException {
+    TableName tableName1 = TableName.valueOf(name.getMethodName() + "_1");
+    RegionInfo region11 =
+      RegionInfoBuilder.newBuilder(tableName1).setEndKey(Bytes.toBytes(1)).build();
+    addBarrier(region11, 10, 20, 30, 40, 50, 60);
+    RegionInfo region12 =
+      RegionInfoBuilder.newBuilder(tableName1).setStartKey(Bytes.toBytes(1)).build();
+    addBarrier(region12, 20, 30, 40, 50, 60, 70);
+
+    TableName tableName2 = TableName.valueOf(name.getMethodName() + "_2");
+    RegionInfo region21 =
+      RegionInfoBuilder.newBuilder(tableName2).setEndKey(Bytes.toBytes(1)).build();
+    addBarrier(region21, 100, 200, 300, 400);
+    RegionInfo region22 =
+      RegionInfoBuilder.newBuilder(tableName2).setStartKey(Bytes.toBytes(1)).build();
+    addBarrier(region22, 200, 300, 400, 500, 600);
+
+    @SuppressWarnings("unchecked")
+    ReplicationPeerManager peerManager =
+      create(null, Collections.emptyList(), Collections.emptyList());
+    ReplicationBarrierCleaner cleaner = create(peerManager);
+    cleaner.chore();
+
+    // should never call this method
+    verify(peerManager, never()).getQueueStorage();
+    // should only be called twice although we have 4 regions to clean
+    verify(peerManager, times(2)).getSerialPeerIdsBelongsTo(any(TableName.class));
+
+    assertArrayEquals(new long[] { 60 },
+      MetaTableAccessor.getReplicationBarrier(UTIL.getConnection(), region11.getRegionName()));
+    assertArrayEquals(new long[] { 70 },
+      MetaTableAccessor.getReplicationBarrier(UTIL.getConnection(), region12.getRegionName()));
+
+    assertArrayEquals(new long[] { 400 },
+      MetaTableAccessor.getReplicationBarrier(UTIL.getConnection(), region21.getRegionName()));
+    assertArrayEquals(new long[] { 600 },
+      MetaTableAccessor.getReplicationBarrier(UTIL.getConnection(), region22.getRegionName()));
+  }
+
+  @Test
+  public void testDeleteBarriers() throws IOException, ReplicationException {
+    TableName tableName = TableName.valueOf(name.getMethodName());
+    RegionInfo region = RegionInfoBuilder.newBuilder(tableName).build();
+    addBarrier(region, 10, 20, 30, 40, 50, 60);
+    // two peers
+    ReplicationQueueStorage queueStorage = create(-1L, 2L, 15L, 25L, 20L, 25L, 65L, 55L, 70L, 70L);
+    List<String> peerIds = Lists.newArrayList("1", "2");
+
+    @SuppressWarnings("unchecked")
+    ReplicationPeerManager peerManager =
+      create(queueStorage, peerIds, peerIds, peerIds, peerIds, peerIds);
+    ReplicationBarrierCleaner cleaner = create(peerManager);
+
+    // beyond the first barrier, no deletion
+    cleaner.chore();
+    assertArrayEquals(new long[] { 10, 20, 30, 40, 50, 60 },
+      MetaTableAccessor.getReplicationBarrier(UTIL.getConnection(), region.getRegionName()));
+
+    // in the first range, still no deletion
+    cleaner.chore();
+    assertArrayEquals(new long[] { 10, 20, 30, 40, 50, 60 },
+      MetaTableAccessor.getReplicationBarrier(UTIL.getConnection(), region.getRegionName()));
+
+    // in the second range, 10 is deleted
+    cleaner.chore();
+    assertArrayEquals(new long[] { 20, 30, 40, 50, 60 },
+      MetaTableAccessor.getReplicationBarrier(UTIL.getConnection(), region.getRegionName()));
+
+    // between 50 and 60, so the barriers before 50 will be deleted
+    cleaner.chore();
+    assertArrayEquals(new long[] { 50, 60 },
+      MetaTableAccessor.getReplicationBarrier(UTIL.getConnection(), region.getRegionName()));
+
+    // in the last open range, 50 is deleted
+    cleaner.chore();
+    assertArrayEquals(new long[] { 60 },
+      MetaTableAccessor.getReplicationBarrier(UTIL.getConnection(), region.getRegionName()));
+  }
+
+  @Test
+  public void testDeleteRowForDeletedRegion() throws IOException, ReplicationException {
+    TableName tableName = TableName.valueOf(name.getMethodName());
+    RegionInfo region = RegionInfoBuilder.newBuilder(tableName).build();
+    addBarrier(region, 40, 50, 60);
+    fillCatalogFamily(region);
+
+    ReplicationQueueStorage queueStorage = create(59L);
+    @SuppressWarnings("unchecked")
+    ReplicationPeerManager peerManager = create(queueStorage, Lists.newArrayList("1"));
+    ReplicationBarrierCleaner cleaner = create(peerManager);
+
+    // we have something in catalog family, so only delete 40
+    cleaner.chore();
+    assertArrayEquals(new long[] { 50, 60 },
+      MetaTableAccessor.getReplicationBarrier(UTIL.getConnection(), region.getRegionName()));
+
+    // No catalog family, then we should remove the whole row
+    clearCatalogFamily(region);
+    cleaner.chore();
+    try (Table table = UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) {
+      assertFalse(table
+        .exists(new Get(region.getRegionName()).addFamily(HConstants.REPLICATION_BARRIER_FAMILY)));
+    }
+  }
+
+  private static class WarnOnlyStoppable implements Stoppable {
+    @Override
+    public void stop(String why) {
+      LOG.warn("TestReplicationBarrierCleaner received stop, ignoring. Reason: " + why);
+    }
+
+    @Override
+    public boolean isStopped() {
+      return false;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/644bfe36/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java
index 58e9543..29749bd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java
@@ -157,7 +157,7 @@ public class TestSerialReplicationChecker {
     }
     for (int i = 0; i < barriers.length; i++) {
       put.addColumn(HConstants.REPLICATION_BARRIER_FAMILY, HConstants.SEQNUM_QUALIFIER,
-        put.getTimeStamp() - i, Bytes.toBytes(barriers[i]));
+        put.getTimeStamp() - barriers.length + i, Bytes.toBytes(barriers[i]));
     }
     try (Table table = UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) {
       table.put(put);


[20/22] hbase git commit: HBASE-20206 WALEntryStream should not switch WAL file silently

Posted by zh...@apache.org.
HBASE-20206 WALEntryStream should not switch WAL file silently


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

Branch: refs/heads/HBASE-20046-branch-2
Commit: 16a4dd6b8f98cb1116007764cb86f6835a7ca84f
Parents: 644bfe3
Author: zhangduo <zh...@apache.org>
Authored: Sun Mar 18 18:09:45 2018 +0800
Committer: zhangduo <zh...@apache.org>
Committed: Mon Apr 9 15:18:44 2018 +0800

----------------------------------------------------------------------
 .../replication/ReplicationQueueStorage.java    |   2 +-
 .../replication/ZKReplicationQueueStorage.java  |  39 ++--
 .../replication/TestReplicationStateBasic.java  |   3 +-
 .../TestZKReplicationQueueStorage.java          |   6 +-
 .../RecoveredReplicationSource.java             |  33 ----
 .../RecoveredReplicationSourceShipper.java      |  13 +-
 .../regionserver/ReplicationSource.java         |   2 +-
 .../regionserver/ReplicationSourceManager.java  | 100 +++++-----
 .../regionserver/ReplicationSourceShipper.java  |  96 +++++-----
 .../ReplicationSourceWALReader.java             |  50 ++++-
 .../SerialReplicationSourceWALReader.java       |  29 ++-
 .../replication/regionserver/WALEntryBatch.java |  22 +++
 .../regionserver/WALEntryStream.java            |   5 +-
 .../TestReplicationSourceManager.java           |  17 +-
 .../regionserver/TestWALEntryStream.java        | 188 ++++++++++++++-----
 15 files changed, 384 insertions(+), 221 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/16a4dd6b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
index 4c93da6..cfe9c9c 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueStorage.java
@@ -63,7 +63,7 @@ public interface ReplicationQueueStorage {
    * @param serverName the name of the regionserver
    * @param queueId a String that identifies the queue
    * @param fileName name of the WAL
-   * @param position the current position in the file
+   * @param position the current position in the file. Will ignore if less than or equal to 0.
    * @param lastSeqIds map with {encodedRegionName, sequenceId} pairs for serial replication.
    */
   void setWALPosition(ServerName serverName, String queueId, String fileName, long position,

http://git-wip-us.apache.org/repos/asf/hbase/blob/16a4dd6b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
index adbf259..63f43e8 100644
--- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
+++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ZKReplicationQueueStorage.java
@@ -193,27 +193,28 @@ class ZKReplicationQueueStorage extends ZKReplicationStorageBase
       Map<String, Long> lastSeqIds) throws ReplicationException {
     try {
       List<ZKUtilOp> listOfOps = new ArrayList<>();
-      listOfOps.add(ZKUtilOp.setData(getFileNode(serverName, queueId, fileName),
-        ZKUtil.positionToByteArray(position)));
+      if (position > 0) {
+        listOfOps.add(ZKUtilOp.setData(getFileNode(serverName, queueId, fileName),
+          ZKUtil.positionToByteArray(position)));
+      }
       // Persist the max sequence id(s) of regions for serial replication atomically.
-      if (lastSeqIds != null && lastSeqIds.size() > 0) {
-        for (Entry<String, Long> lastSeqEntry : lastSeqIds.entrySet()) {
-          String peerId = new ReplicationQueueInfo(queueId).getPeerId();
-          String path = getSerialReplicationRegionPeerNode(lastSeqEntry.getKey(), peerId);
-          /*
-           * Make sure the existence of path
-           * /hbase/replication/regions/<hash>/<encoded-region-name>-<peer-id>. As the javadoc in
-           * multiOrSequential() method said, if received a NodeExistsException, all operations will
-           * fail. So create the path here, and in fact, no need to add this operation to listOfOps,
-           * because only need to make sure that update file position and sequence id atomically.
-           */
-          ZKUtil.createWithParents(zookeeper, path);
-          // Persist the max sequence id of region to zookeeper.
-          listOfOps
-              .add(ZKUtilOp.setData(path, ZKUtil.positionToByteArray(lastSeqEntry.getValue())));
-        }
+      for (Entry<String, Long> lastSeqEntry : lastSeqIds.entrySet()) {
+        String peerId = new ReplicationQueueInfo(queueId).getPeerId();
+        String path = getSerialReplicationRegionPeerNode(lastSeqEntry.getKey(), peerId);
+        /*
+         * Make sure the existence of path
+         * /hbase/replication/regions/<hash>/<encoded-region-name>-<peer-id>. As the javadoc in
+         * multiOrSequential() method said, if received a NodeExistsException, all operations will
+         * fail. So create the path here, and in fact, no need to add this operation to listOfOps,
+         * because only need to make sure that update file position and sequence id atomically.
+         */
+        ZKUtil.createWithParents(zookeeper, path);
+        // Persist the max sequence id of region to zookeeper.
+        listOfOps.add(ZKUtilOp.setData(path, ZKUtil.positionToByteArray(lastSeqEntry.getValue())));
+      }
+      if (!listOfOps.isEmpty()) {
+        ZKUtil.multiOrSequential(zookeeper, listOfOps, false);
       }
-      ZKUtil.multiOrSequential(zookeeper, listOfOps, false);
     } catch (KeeperException e) {
       throw new ReplicationException("Failed to set log position (serverName=" + serverName
           + ", queueId=" + queueId + ", fileName=" + fileName + ", position=" + position + ")", e);

http://git-wip-us.apache.org/repos/asf/hbase/blob/16a4dd6b/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 5999c1f..21b09aa 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
@@ -25,6 +25,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.fs.Path;
@@ -127,7 +128,7 @@ public abstract class TestReplicationStateBasic {
     assertEquals(0, rqs.getWALsInQueue(server2, "qId1").size());
     assertEquals(5, rqs.getWALsInQueue(server3, "qId5").size());
     assertEquals(0, rqs.getWALPosition(server3, "qId1", "filename0"));
-    rqs.setWALPosition(server3, "qId5", "filename4", 354L, null);
+    rqs.setWALPosition(server3, "qId5", "filename4", 354L, Collections.emptyMap());
     assertEquals(354L, rqs.getWALPosition(server3, "qId5", "filename4"));
 
     assertEquals(5, rqs.getWALsInQueue(server3, "qId5").size());

http://git-wip-us.apache.org/repos/asf/hbase/blob/16a4dd6b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
----------------------------------------------------------------------
diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
index 8ff52f3..c813870 100644
--- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
+++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationQueueStorage.java
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 import java.util.SortedSet;
@@ -136,9 +137,10 @@ public class TestZKReplicationQueueStorage {
     for (int i = 0; i < 10; i++) {
       assertEquals(0, STORAGE.getWALPosition(serverName1, queue1, getFileName("file1", i)));
       assertEquals(0, STORAGE.getWALPosition(serverName1, queue2, getFileName("file2", i)));
-      STORAGE.setWALPosition(serverName1, queue1, getFileName("file1", i), (i + 1) * 100, null);
+      STORAGE.setWALPosition(serverName1, queue1, getFileName("file1", i), (i + 1) * 100,
+        Collections.emptyMap());
       STORAGE.setWALPosition(serverName1, queue2, getFileName("file2", i), (i + 1) * 100 + 10,
-        null);
+        Collections.emptyMap());
     }
 
     for (int i = 0; i < 10; i++) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/16a4dd6b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
index 169b469..f1ad99d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSource.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.replication.regionserver;
 import java.io.IOException;
 import java.util.List;
 import java.util.UUID;
-import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.PriorityBlockingQueue;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -64,38 +63,6 @@ public class RecoveredReplicationSource extends ReplicationSource {
     return new RecoveredReplicationSourceShipper(conf, walGroupId, queue, this, queueStorage);
   }
 
-  private void handleEmptyWALEntryBatch0(ReplicationSourceWALReader reader,
-      BlockingQueue<WALEntryBatch> entryBatchQueue, Path currentPath) throws InterruptedException {
-    LOG.trace("Didn't read any new entries from WAL");
-    // we're done with queue recovery, shut ourself down
-    reader.setReaderRunning(false);
-    // shuts down shipper thread immediately
-    entryBatchQueue.put(new WALEntryBatch(0, currentPath));
-  }
-
-  @Override
-  protected ReplicationSourceWALReader createNewWALReader(String walGroupId,
-      PriorityBlockingQueue<Path> queue, long startPosition) {
-    if (replicationPeer.getPeerConfig().isSerial()) {
-      return new SerialReplicationSourceWALReader(fs, conf, queue, startPosition, walEntryFilter,
-        this) {
-
-        @Override
-        protected void handleEmptyWALEntryBatch(Path currentPath) throws InterruptedException {
-          handleEmptyWALEntryBatch0(this, entryBatchQueue, currentPath);
-        }
-      };
-    } else {
-      return new ReplicationSourceWALReader(fs, conf, queue, startPosition, walEntryFilter, this) {
-
-        @Override
-        protected void handleEmptyWALEntryBatch(Path currentPath) throws InterruptedException {
-          handleEmptyWALEntryBatch0(this, entryBatchQueue, currentPath);
-        }
-      };
-    }
-  }
-
   public void locateRecoveredPaths(PriorityBlockingQueue<Path> queue) throws IOException {
     boolean hasPathChanged = false;
     PriorityBlockingQueue<Path> newPaths =

http://git-wip-us.apache.org/repos/asf/hbase/blob/16a4dd6b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
index 1ae5cb9..d74211e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RecoveredReplicationSourceShipper.java
@@ -48,13 +48,10 @@ public class RecoveredReplicationSourceShipper extends ReplicationSourceShipper
   }
 
   @Override
-  protected void postShipEdits(WALEntryBatch entryBatch) {
-    if (entryBatch.getWalEntries().isEmpty()) {
-      LOG.debug("Finished recovering queue for group " + walGroupId + " of peer "
-          + source.getQueueId());
-      source.getSourceMetrics().incrCompletedRecoveryQueue();
-      setWorkerState(WorkerState.FINISHED);
-    }
+  protected void noMoreData() {
+    LOG.debug("Finished recovering queue for group {} of peer {}", walGroupId, source.getQueueId());
+    source.getSourceMetrics().incrCompletedRecoveryQueue();
+    setWorkerState(WorkerState.FINISHED);
   }
 
   @Override
@@ -63,7 +60,7 @@ public class RecoveredReplicationSourceShipper extends ReplicationSourceShipper
   }
 
   @Override
-  public long getStartPosition() {
+  long getStartPosition() {
     long startPosition = getRecoveredQueueStartPos();
     int numRetries = 0;
     while (numRetries <= maxRetriesMultiplier) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/16a4dd6b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
index 3480919..236c575 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java
@@ -315,7 +315,7 @@ public class ReplicationSource implements ReplicationSourceInterface {
     return new ReplicationSourceShipper(conf, walGroupId, queue, this);
   }
 
-  protected ReplicationSourceWALReader createNewWALReader(String walGroupId,
+  private ReplicationSourceWALReader createNewWALReader(String walGroupId,
       PriorityBlockingQueue<Path> queue, long startPosition) {
     return replicationPeer.getPeerConfig().isSerial()
       ? new SerialReplicationSourceWALReader(fs, conf, queue, startPosition, walEntryFilter, this)

http://git-wip-us.apache.org/repos/asf/hbase/blob/16a4dd6b/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 06fe977..23e1115 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
@@ -25,6 +25,7 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.NavigableSet;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
@@ -82,25 +83,28 @@ import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFacto
  * operations.</li>
  * <li>Need synchronized on {@link #walsById}. There are four methods which modify it,
  * {@link #addPeer(String)}, {@link #removePeer(String)},
- * {@link #cleanOldLogs(SortedSet, String, String)} and {@link #preLogRoll(Path)}. {@link #walsById}
- * is a ConcurrentHashMap and there is a Lock for peer id in {@link PeerProcedureHandlerImpl}. So
- * there is no race between {@link #addPeer(String)} and {@link #removePeer(String)}.
- * {@link #cleanOldLogs(SortedSet, String, String)} is called by {@link ReplicationSourceInterface}.
- * So no race with {@link #addPeer(String)}. {@link #removePeer(String)} will terminate the
- * {@link ReplicationSourceInterface} firstly, then remove the wals from {@link #walsById}. So no
- * race with {@link #removePeer(String)}. The only case need synchronized is
- * {@link #cleanOldLogs(SortedSet, String, String)} and {@link #preLogRoll(Path)}.</li>
+ * {@link #cleanOldLogs(NavigableSet, String, boolean, String)} and {@link #preLogRoll(Path)}.
+ * {@link #walsById} is a ConcurrentHashMap and there is a Lock for peer id in
+ * {@link PeerProcedureHandlerImpl}. So there is no race between {@link #addPeer(String)} and
+ * {@link #removePeer(String)}. {@link #cleanOldLogs(NavigableSet, String, boolean, String)} is
+ * called by {@link ReplicationSourceInterface}. So no race with {@link #addPeer(String)}.
+ * {@link #removePeer(String)} will terminate the {@link ReplicationSourceInterface} firstly, then
+ * remove the wals from {@link #walsById}. So no race with {@link #removePeer(String)}. The only
+ * case need synchronized is {@link #cleanOldLogs(NavigableSet, String, boolean, String)} and
+ * {@link #preLogRoll(Path)}.</li>
  * <li>No need synchronized on {@link #walsByIdRecoveredQueues}. There are three methods which
- * modify it, {@link #removePeer(String)} , {@link #cleanOldLogs(SortedSet, String, String)} and
+ * modify it, {@link #removePeer(String)} ,
+ * {@link #cleanOldLogs(NavigableSet, String, boolean, String)} and
  * {@link ReplicationSourceManager.NodeFailoverWorker#run()}.
- * {@link #cleanOldLogs(SortedSet, String, String)} is called by {@link ReplicationSourceInterface}.
- * {@link #removePeer(String)} will terminate the {@link ReplicationSourceInterface} firstly, then
- * remove the wals from {@link #walsByIdRecoveredQueues}. And
- * {@link ReplicationSourceManager.NodeFailoverWorker#run()} will add the wals to
- * {@link #walsByIdRecoveredQueues} firstly, then start up a {@link ReplicationSourceInterface}. So
- * there is no race here. For {@link ReplicationSourceManager.NodeFailoverWorker#run()} and
- * {@link #removePeer(String)}, there is already synchronized on {@link #oldsources}. So no need
- * synchronized on {@link #walsByIdRecoveredQueues}.</li>
+ * {@link #cleanOldLogs(NavigableSet, String, boolean, String)} is called by
+ * {@link ReplicationSourceInterface}. {@link #removePeer(String)} will terminate the
+ * {@link ReplicationSourceInterface} firstly, then remove the wals from
+ * {@link #walsByIdRecoveredQueues}. And {@link ReplicationSourceManager.NodeFailoverWorker#run()}
+ * will add the wals to {@link #walsByIdRecoveredQueues} firstly, then start up a
+ * {@link ReplicationSourceInterface}. So there is no race here. For
+ * {@link ReplicationSourceManager.NodeFailoverWorker#run()} and {@link #removePeer(String)}, there
+ * is already synchronized on {@link #oldsources}. So no need synchronized on
+ * {@link #walsByIdRecoveredQueues}.</li>
  * <li>Need synchronized on {@link #latestPaths} to avoid the new open source miss new log.</li>
  * <li>Need synchronized on {@link #oldsources} to avoid adding recovered source for the
  * to-be-removed peer.</li>
@@ -124,11 +128,11 @@ public class ReplicationSourceManager implements ReplicationListener {
   // All logs we are currently tracking
   // Index structure of the map is: queue_id->logPrefix/logGroup->logs
   // For normal replication source, the peer id is same with the queue id
-  private final ConcurrentMap<String, Map<String, SortedSet<String>>> walsById;
+  private final ConcurrentMap<String, Map<String, NavigableSet<String>>> walsById;
   // Logs for recovered sources we are currently tracking
   // the map is: queue_id->logPrefix/logGroup->logs
   // For recovered source, the queue id's format is peer_id-servername-*
-  private final ConcurrentMap<String, Map<String, SortedSet<String>>> walsByIdRecoveredQueues;
+  private final ConcurrentMap<String, Map<String, NavigableSet<String>>> walsByIdRecoveredQueues;
 
   private final Configuration conf;
   private final FileSystem fs;
@@ -335,14 +339,14 @@ public class ReplicationSourceManager implements ReplicationListener {
     // synchronized on latestPaths to avoid missing the new log
     synchronized (this.latestPaths) {
       this.sources.put(peerId, src);
-      Map<String, SortedSet<String>> walsByGroup = new HashMap<>();
+      Map<String, NavigableSet<String>> walsByGroup = new HashMap<>();
       this.walsById.put(peerId, walsByGroup);
       // Add the latest wal to that source's queue
       if (this.latestPaths.size() > 0) {
         for (Path logPath : latestPaths) {
           String name = logPath.getName();
           String walPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(name);
-          SortedSet<String> logs = new TreeSet<>();
+          NavigableSet<String> logs = new TreeSet<>();
           logs.add(name);
           walsByGroup.put(walPrefix, logs);
           // Abort RS and throw exception to make add peer failed
@@ -474,50 +478,51 @@ public class ReplicationSourceManager implements ReplicationListener {
   /**
    * This method will log the current position to storage. And also clean old logs from the
    * replication queue.
-   * @param log Path to the log currently being replicated
    * @param queueId id of the replication queue
-   * @param position current location in the log
    * @param queueRecovered indicates if this queue comes from another region server
+   * @param entryBatch the wal entry batch we just shipped
    */
-  public void logPositionAndCleanOldLogs(Path log, String queueId, long position,
-      Map<String, Long> lastSeqIds, boolean queueRecovered) {
-    String fileName = log.getName();
+  public void logPositionAndCleanOldLogs(String queueId, boolean queueRecovered,
+      WALEntryBatch entryBatch) {
+    String fileName = entryBatch.getLastWalPath().getName();
     abortWhenFail(() -> this.queueStorage.setWALPosition(server.getServerName(), queueId, fileName,
-      position, lastSeqIds));
-    cleanOldLogs(fileName, queueId, queueRecovered);
+      entryBatch.getLastWalPosition(), entryBatch.getLastSeqIds()));
+    cleanOldLogs(fileName, entryBatch.isEndOfFile(), queueId, queueRecovered);
   }
 
   /**
    * Cleans a log file and all older logs from replication queue. Called when we are sure that a log
    * file is closed and has no more entries.
    * @param log Path to the log
+   * @param inclusive whether we should also remove the given log file
    * @param queueId id of the replication queue
    * @param queueRecovered Whether this is a recovered queue
    */
   @VisibleForTesting
-  void cleanOldLogs(String log, String queueId, boolean queueRecovered) {
+  void cleanOldLogs(String log, boolean inclusive, String queueId, boolean queueRecovered) {
     String logPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(log);
     if (queueRecovered) {
-      SortedSet<String> wals = walsByIdRecoveredQueues.get(queueId).get(logPrefix);
-      if (wals != null && !wals.first().equals(log)) {
-        cleanOldLogs(wals, log, queueId);
+      NavigableSet<String> wals = walsByIdRecoveredQueues.get(queueId).get(logPrefix);
+      if (wals != null) {
+        cleanOldLogs(wals, log, inclusive, queueId);
       }
     } else {
       // synchronized on walsById to avoid race with preLogRoll
       synchronized (this.walsById) {
-        SortedSet<String> wals = walsById.get(queueId).get(logPrefix);
+        NavigableSet<String> wals = walsById.get(queueId).get(logPrefix);
         if (wals != null && !wals.first().equals(log)) {
-          cleanOldLogs(wals, log, queueId);
+          cleanOldLogs(wals, log, inclusive, queueId);
         }
       }
     }
   }
 
-  private void cleanOldLogs(SortedSet<String> wals, String key, String id) {
-    SortedSet<String> walSet = wals.headSet(key);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Removing " + walSet.size() + " logs in the list: " + walSet);
+  private void cleanOldLogs(NavigableSet<String> wals, String key, boolean inclusive, String id) {
+    NavigableSet<String> walSet = wals.headSet(key, inclusive);
+    if (walSet.isEmpty()) {
+      return;
     }
+    LOG.debug("Removing {} logs in the list: {}", walSet.size(), walSet);
     for (String wal : walSet) {
       abortWhenFail(() -> this.queueStorage.removeWAL(server.getServerName(), id, wal));
     }
@@ -542,11 +547,12 @@ public class ReplicationSourceManager implements ReplicationListener {
       // synchronized on walsById to avoid race with cleanOldLogs
       synchronized (this.walsById) {
         // Update walsById map
-        for (Map.Entry<String, Map<String, SortedSet<String>>> entry : this.walsById.entrySet()) {
+        for (Map.Entry<String, Map<String, NavigableSet<String>>> entry : this.walsById
+          .entrySet()) {
           String peerId = entry.getKey();
-          Map<String, SortedSet<String>> walsByPrefix = entry.getValue();
+          Map<String, NavigableSet<String>> walsByPrefix = entry.getValue();
           boolean existingPrefix = false;
-          for (Map.Entry<String, SortedSet<String>> walsEntry : walsByPrefix.entrySet()) {
+          for (Map.Entry<String, NavigableSet<String>> walsEntry : walsByPrefix.entrySet()) {
             SortedSet<String> wals = walsEntry.getValue();
             if (this.sources.isEmpty()) {
               // If there's no slaves, don't need to keep the old wals since
@@ -560,8 +566,8 @@ public class ReplicationSourceManager implements ReplicationListener {
           }
           if (!existingPrefix) {
             // The new log belongs to a new group, add it into this peer
-            LOG.debug("Start tracking logs for wal group " + logPrefix + " for peer " + peerId);
-            SortedSet<String> wals = new TreeSet<>();
+            LOG.debug("Start tracking logs for wal group {} for peer {}", logPrefix, peerId);
+            NavigableSet<String> wals = new TreeSet<>();
             wals.add(logName);
             walsByPrefix.put(logPrefix, wals);
           }
@@ -700,11 +706,11 @@ public class ReplicationSourceManager implements ReplicationListener {
             continue;
           }
           // track sources in walsByIdRecoveredQueues
-          Map<String, SortedSet<String>> walsByGroup = new HashMap<>();
+          Map<String, NavigableSet<String>> walsByGroup = new HashMap<>();
           walsByIdRecoveredQueues.put(queueId, walsByGroup);
           for (String wal : walsSet) {
             String walPrefix = AbstractFSWALProvider.getWALPrefixFromWALName(wal);
-            SortedSet<String> wals = walsByGroup.get(walPrefix);
+            NavigableSet<String> wals = walsByGroup.get(walPrefix);
             if (wals == null) {
               wals = new TreeSet<>();
               walsByGroup.put(walPrefix, wals);
@@ -749,7 +755,7 @@ public class ReplicationSourceManager implements ReplicationListener {
    * @return a sorted set of wal names
    */
   @VisibleForTesting
-  Map<String, Map<String, SortedSet<String>>> getWALs() {
+  Map<String, Map<String, NavigableSet<String>>> getWALs() {
     return Collections.unmodifiableMap(walsById);
   }
 
@@ -758,7 +764,7 @@ public class ReplicationSourceManager implements ReplicationListener {
    * @return a sorted set of wal names
    */
   @VisibleForTesting
-  Map<String, Map<String, SortedSet<String>>> getWalsByIdRecoveredQueues() {
+  Map<String, Map<String, NavigableSet<String>>> getWalsByIdRecoveredQueues() {
     return Collections.unmodifiableMap(walsByIdRecoveredQueues);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/16a4dd6b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
index aa5251e..2097d00 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceShipper.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.replication.regionserver;
 
 import java.io.IOException;
 import java.util.List;
-import java.util.Map;
 import java.util.concurrent.PriorityBlockingQueue;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
@@ -52,17 +51,18 @@ public class ReplicationSourceShipper extends Thread {
     FINISHED,  // The worker is done processing a recovered queue
   }
 
-  protected final Configuration conf;
+  private final Configuration conf;
   protected final String walGroupId;
   protected final PriorityBlockingQueue<Path> queue;
-  protected final ReplicationSourceInterface source;
+  private final ReplicationSourceInterface source;
 
   // Last position in the log that we sent to ZooKeeper
-  protected long lastLoggedPosition = -1;
+  // It will be accessed by the stats thread so make it volatile
+  private volatile long currentPosition = -1;
   // Path of the current log
-  protected volatile Path currentPath;
+  private Path currentPath;
   // Current state of the worker thread
-  private WorkerState state;
+  private volatile WorkerState state;
   protected ReplicationSourceWALReader entryReader;
 
   // How long should we sleep for each retry
@@ -97,8 +97,12 @@ public class ReplicationSourceShipper extends Thread {
       }
       try {
         WALEntryBatch entryBatch = entryReader.take();
-        shipEdits(entryBatch);
-        postShipEdits(entryBatch);
+        // the NO_MORE_DATA instance has no path so do not all shipEdits
+        if (entryBatch == WALEntryBatch.NO_MORE_DATA) {
+          noMoreData();
+        } else {
+          shipEdits(entryBatch);
+        }
       } catch (InterruptedException e) {
         LOG.trace("Interrupted while waiting for next replication entry batch", e);
         Thread.currentThread().interrupt();
@@ -113,7 +117,7 @@ public class ReplicationSourceShipper extends Thread {
   }
 
   // To be implemented by recovered shipper
-  protected void postShipEdits(WALEntryBatch entryBatch) {
+  protected void noMoreData() {
   }
 
   // To be implemented by recovered shipper
@@ -123,14 +127,11 @@ public class ReplicationSourceShipper extends Thread {
   /**
    * Do the shipping logic
    */
-  protected final void shipEdits(WALEntryBatch entryBatch) {
+  private void shipEdits(WALEntryBatch entryBatch) {
     List<Entry> entries = entryBatch.getWalEntries();
-    long lastReadPosition = entryBatch.getLastWalPosition();
-    currentPath = entryBatch.getLastWalPath();
     int sleepMultiplier = 0;
     if (entries.isEmpty()) {
-      if (lastLoggedPosition != lastReadPosition) {
-        updateLogPosition(lastReadPosition, entryBatch.getLastSeqIds());
+      if (updateLogPosition(entryBatch)) {
         // if there was nothing to ship and it's not an error
         // set "ageOfLastShippedOp" to <now> to indicate that we're current
         source.getSourceMetrics().setAgeOfLastShippedOp(EnvironmentEdgeManager.currentTime(),
@@ -168,16 +169,12 @@ public class ReplicationSourceShipper extends Thread {
         } else {
           sleepMultiplier = Math.max(sleepMultiplier - 1, 0);
         }
-
-        if (this.lastLoggedPosition != lastReadPosition) {
-          // Clean up hfile references
-          int size = entries.size();
-          for (int i = 0; i < size; i++) {
-            cleanUpHFileRefs(entries.get(i).getEdit());
-          }
-          // Log and clean up WAL logs
-          updateLogPosition(lastReadPosition, entryBatch.getLastSeqIds());
+        // Clean up hfile references
+        for (Entry entry : entries) {
+          cleanUpHFileRefs(entry.getEdit());
         }
+        // Log and clean up WAL logs
+        updateLogPosition(entryBatch);
 
         source.postShipEdits(entries, currentSize);
         // FIXME check relationship between wal group and overall
@@ -224,10 +221,29 @@ public class ReplicationSourceShipper extends Thread {
     }
   }
 
-  private void updateLogPosition(long lastReadPosition, Map<String, Long> lastSeqIds) {
-    source.getSourceManager().logPositionAndCleanOldLogs(currentPath, source.getQueueId(),
-      lastReadPosition, lastSeqIds, source.isRecovered());
-    lastLoggedPosition = lastReadPosition;
+  private boolean updateLogPosition(WALEntryBatch batch) {
+    boolean updated = false;
+    // if end of file is true, then the logPositionAndCleanOldLogs method will remove the file
+    // record on zk, so let's call it. The last wal position maybe zero if end of file is true and
+    // there is no entry in the batch. It is OK because that the queue storage will ignore the zero
+    // position and the file will be removed soon in cleanOldLogs.
+    if (batch.isEndOfFile() || !batch.getLastWalPath().equals(currentPath) ||
+      batch.getLastWalPosition() != currentPosition) {
+      source.getSourceManager().logPositionAndCleanOldLogs(source.getQueueId(),
+        source.isRecovered(), batch);
+      updated = true;
+    }
+    // if end of file is true, then we can just skip to the next file in queue.
+    // the only exception is for recovered queue, if we reach the end of the queue, then there will
+    // no more files so here the currentPath may be null.
+    if (batch.isEndOfFile()) {
+      currentPath = entryReader.getCurrentPath();
+      currentPosition = 0L;
+    } else {
+      currentPath = batch.getLastWalPath();
+      currentPosition = batch.getLastWalPosition();
+    }
+    return updated;
   }
 
   public void startup(UncaughtExceptionHandler handler) {
@@ -236,39 +252,31 @@ public class ReplicationSourceShipper extends Thread {
       name + ".replicationSource.shipper" + walGroupId + "," + source.getQueueId(), handler);
   }
 
-  public PriorityBlockingQueue<Path> getLogQueue() {
-    return this.queue;
-  }
-
-  public Path getCurrentPath() {
-    return this.entryReader.getCurrentPath();
+  Path getCurrentPath() {
+    return entryReader.getCurrentPath();
   }
 
-  public long getCurrentPosition() {
-    return this.lastLoggedPosition;
+  long getCurrentPosition() {
+    return currentPosition;
   }
 
-  public void setWALReader(ReplicationSourceWALReader entryReader) {
+  void setWALReader(ReplicationSourceWALReader entryReader) {
     this.entryReader = entryReader;
   }
 
-  public long getStartPosition() {
+  long getStartPosition() {
     return 0;
   }
 
-  protected final boolean isActive() {
+  private boolean isActive() {
     return source.isSourceActive() && state == WorkerState.RUNNING && !isInterrupted();
   }
 
-  public void setWorkerState(WorkerState state) {
+  protected final void setWorkerState(WorkerState state) {
     this.state = state;
   }
 
-  public WorkerState getWorkerState() {
-    return state;
-  }
-
-  public void stopWorker() {
+  void stopWorker() {
     setWorkerState(WorkerState.STOPPED);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/16a4dd6b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
index b125133..2154856 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceWALReader.java
@@ -59,7 +59,7 @@ class ReplicationSourceWALReader extends Thread {
   private final WALEntryFilter filter;
   private final ReplicationSource source;
 
-  protected final BlockingQueue<WALEntryBatch> entryBatchQueue;
+  private final BlockingQueue<WALEntryBatch> entryBatchQueue;
   // max (heap) size of each batch - multiply by number of batches in queue to get total
   private final long replicationBatchSizeCapacity;
   // max count of each batch - multiply by number of batches in queue to get total
@@ -130,6 +130,7 @@ class ReplicationSourceWALReader extends Thread {
             continue;
           }
           WALEntryBatch batch = readWALEntries(entryStream);
+          currentPosition = entryStream.getPosition();
           if (batch != null) {
             // need to propagate the batch even it has no entries since it may carry the last
             // sequence id information for serial replication.
@@ -138,9 +139,8 @@ class ReplicationSourceWALReader extends Thread {
             sleepMultiplier = 1;
           } else { // got no entries and didn't advance position in WAL
             handleEmptyWALEntryBatch(entryStream.getCurrentPath());
+            entryStream.reset(); // reuse stream
           }
-          currentPosition = entryStream.getPosition();
-          entryStream.reset(); // reuse stream
         }
       } catch (IOException e) { // stream related
         if (sleepMultiplier < maxRetriesMultiplier) {
@@ -173,13 +173,31 @@ class ReplicationSourceWALReader extends Thread {
       batch.getNbEntries() >= replicationBatchCountCapacity;
   }
 
+  protected static final boolean switched(WALEntryStream entryStream, Path path) {
+    return !path.equals(entryStream.getCurrentPath());
+  }
+
   protected WALEntryBatch readWALEntries(WALEntryStream entryStream)
       throws IOException, InterruptedException {
+    Path currentPath = entryStream.getCurrentPath();
     if (!entryStream.hasNext()) {
-      return null;
+      // check whether we have switched a file
+      if (currentPath != null && switched(entryStream, currentPath)) {
+        return WALEntryBatch.endOfFile(currentPath);
+      } else {
+        return null;
+      }
+    }
+    if (currentPath != null) {
+      if (switched(entryStream, currentPath)) {
+        return WALEntryBatch.endOfFile(currentPath);
+      }
+    } else {
+      // when reading from the entry stream first time we will enter here
+      currentPath = entryStream.getCurrentPath();
     }
     WALEntryBatch batch = createBatch(entryStream);
-    do {
+    for (;;) {
       Entry entry = entryStream.next();
       batch.setLastWalPosition(entryStream.getPosition());
       entry = filterEntry(entry);
@@ -188,13 +206,29 @@ class ReplicationSourceWALReader extends Thread {
           break;
         }
       }
-    } while (entryStream.hasNext());
+      boolean hasNext = entryStream.hasNext();
+      // always return if we have switched to a new file
+      if (switched(entryStream, currentPath)) {
+        batch.setEndOfFile(true);
+        break;
+      }
+      if (!hasNext) {
+        break;
+      }
+    }
     return batch;
   }
 
-  protected void handleEmptyWALEntryBatch(Path currentPath) throws InterruptedException {
+  private void handleEmptyWALEntryBatch(Path currentPath) throws InterruptedException {
     LOG.trace("Didn't read any new entries from WAL");
-    Thread.sleep(sleepForRetries);
+    if (source.isRecovered()) {
+      // we're done with queue recovery, shut ourself down
+      setReaderRunning(false);
+      // shuts down shipper thread immediately
+      entryBatchQueue.put(WALEntryBatch.NO_MORE_DATA);
+    } else {
+      Thread.sleep(sleepForRetries);
+    }
   }
 
   // if we get an EOF due to a zero-length log, and there are other logs in queue

http://git-wip-us.apache.org/repos/asf/hbase/blob/16a4dd6b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SerialReplicationSourceWALReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SerialReplicationSourceWALReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SerialReplicationSourceWALReader.java
index 5e9a9f6..9edcc8a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SerialReplicationSourceWALReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/SerialReplicationSourceWALReader.java
@@ -53,12 +53,26 @@ public class SerialReplicationSourceWALReader extends ReplicationSourceWALReader
   @Override
   protected WALEntryBatch readWALEntries(WALEntryStream entryStream)
       throws IOException, InterruptedException {
+    Path currentPath = entryStream.getCurrentPath();
     if (!entryStream.hasNext()) {
-      return null;
+      // check whether we have switched a file
+      if (currentPath != null && switched(entryStream, currentPath)) {
+        return WALEntryBatch.endOfFile(currentPath);
+      } else {
+        return null;
+      }
+    }
+    if (currentPath != null) {
+      if (switched(entryStream, currentPath)) {
+        return WALEntryBatch.endOfFile(currentPath);
+      }
+    } else {
+      // when reading from the entry stream first time we will enter here
+      currentPath = entryStream.getCurrentPath();
     }
     long positionBefore = entryStream.getPosition();
     WALEntryBatch batch = createBatch(entryStream);
-    do {
+    for (;;) {
       Entry entry = entryStream.peek();
       boolean doFiltering = true;
       if (firstCellInEntryBeforeFiltering == null) {
@@ -99,7 +113,16 @@ public class SerialReplicationSourceWALReader extends ReplicationSourceWALReader
         // actually remove the entry.
         removeEntryFromStream(entryStream, batch);
       }
-    } while (entryStream.hasNext());
+      boolean hasNext = entryStream.hasNext();
+      // always return if we have switched to a new file.
+      if (switched(entryStream, currentPath)) {
+        batch.setEndOfFile(true);
+        break;
+      }
+      if (!hasNext) {
+        break;
+      }
+    }
     return batch;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/16a4dd6b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryBatch.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryBatch.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryBatch.java
index 31c3ac7..960d473 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryBatch.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryBatch.java
@@ -30,6 +30,10 @@ import org.apache.yetus.audience.InterfaceAudience;
  */
 @InterfaceAudience.Private
 class WALEntryBatch {
+
+  // used by recovered replication queue to indicate that all the entries have been read.
+  public static final WALEntryBatch NO_MORE_DATA = new WALEntryBatch(0, null);
+
   private List<Entry> walEntries;
   // last WAL that was read
   private Path lastWalPath;
@@ -43,6 +47,8 @@ class WALEntryBatch {
   private long heapSize = 0;
   // save the last sequenceid for each region if the table has serial-replication scope
   private Map<String, Long> lastSeqIds = new HashMap<>();
+  // indicate that this is the end of the current file
+  private boolean endOfFile;
 
   /**
    * @param lastWalPath Path of the WAL the last entry in this batch was read from
@@ -52,6 +58,14 @@ class WALEntryBatch {
     this.lastWalPath = lastWalPath;
   }
 
+
+  static WALEntryBatch endOfFile(Path lastWalPath) {
+    WALEntryBatch batch = new WALEntryBatch(0, lastWalPath);
+    batch.setLastWalPosition(-1L);
+    batch.setEndOfFile(true);
+    return batch;
+  }
+
   public void addEntry(Entry entry) {
     walEntries.add(entry);
   }
@@ -120,6 +134,14 @@ class WALEntryBatch {
     return lastSeqIds;
   }
 
+  public boolean isEndOfFile() {
+    return endOfFile;
+  }
+
+  public void setEndOfFile(boolean endOfFile) {
+    this.endOfFile = endOfFile;
+  }
+
   public void incrementNbRowKeys(int increment) {
     nbRowKeys += increment;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/16a4dd6b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
index c639a48..b2c199e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/WALEntryStream.java
@@ -155,7 +155,6 @@ class WALEntryStream implements Closeable {
   /**
    * Should be called if the stream is to be reused (i.e. used again after hasNext() has returned
    * false)
-   * @throws IOException
    */
   public void reset() throws IOException {
     if (reader != null && currentPath != null) {
@@ -304,6 +303,9 @@ class WALEntryStream implements Closeable {
       if (reader != null) {
         return true;
       }
+    } else {
+      // no more files in queue, this could only happen for recovered queue.
+      setCurrentPath(null);
     }
     return false;
   }
@@ -394,6 +396,7 @@ class WALEntryStream implements Closeable {
 
   private void resetReader() throws IOException {
     try {
+      currentEntry = null;
       reader.reset();
       seek();
     } catch (FileNotFoundException fnfe) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/16a4dd6b/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 6d75fec..eb46cd7 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
@@ -32,6 +32,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
+import java.util.NavigableSet;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeMap;
@@ -308,25 +309,25 @@ public abstract class TestReplicationSourceManager {
     for (int i = 0; i < 3; i++) {
       wal.append(hri,
         new WALKeyImpl(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc, scopes),
-          edit, true);
+        edit, true);
     }
     wal.sync();
 
     int logNumber = 0;
-    for (Map.Entry<String, SortedSet<String>> entry : manager.getWALs().get(slaveId).entrySet()) {
+    for (Map.Entry<String, NavigableSet<String>> entry : manager.getWALs().get(slaveId)
+      .entrySet()) {
       logNumber += entry.getValue().size();
     }
     assertEquals(6, logNumber);
 
     wal.rollWriter();
 
-    manager.logPositionAndCleanOldLogs(manager.getSources().get(0).getCurrentPath(),
-        "1", 0, null, false);
+    manager.logPositionAndCleanOldLogs("1", false,
+      new WALEntryBatch(0, manager.getSources().get(0).getCurrentPath()));
 
     wal.append(hri,
-        new WALKeyImpl(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc, scopes),
-        edit,
-        true);
+      new WALKeyImpl(hri.getEncodedNameAsBytes(), test, System.currentTimeMillis(), mvcc, scopes),
+      edit, true);
     wal.sync();
 
     assertEquals(1, manager.getWALs().size());
@@ -396,7 +397,7 @@ public abstract class TestReplicationSourceManager {
     assertEquals(1, manager.getWalsByIdRecoveredQueues().size());
     String id = "1-" + server.getServerName().getServerName();
     assertEquals(files, manager.getWalsByIdRecoveredQueues().get(id).get(group));
-    manager.cleanOldLogs(file2, id, true);
+    manager.cleanOldLogs(file2, false, id, true);
     // log1 should be deleted
     assertEquals(Sets.newHashSet(file2), manager.getWalsByIdRecoveredQueues().get(id).get(group));
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/16a4dd6b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
index eb7d5a0..2670756 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntryStream.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Server;
 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;
@@ -75,7 +76,7 @@ public class TestWALEntryStream {
       HBaseClassTestRule.forClass(TestWALEntryStream.class);
 
   private static HBaseTestingUtility TEST_UTIL;
-  private static Configuration conf;
+  private static Configuration CONF;
   private static FileSystem fs;
   private static MiniDFSCluster cluster;
   private static final TableName tableName = TableName.valueOf("tablename");
@@ -102,7 +103,7 @@ public class TestWALEntryStream {
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     TEST_UTIL = new HBaseTestingUtility();
-    conf = TEST_UTIL.getConfiguration();
+    CONF = TEST_UTIL.getConfiguration();
     TEST_UTIL.startMiniDFSCluster(3);
 
     cluster = TEST_UTIL.getDFSCluster();
@@ -118,7 +119,7 @@ public class TestWALEntryStream {
   public void setUp() throws Exception {
     walQueue = new PriorityBlockingQueue<>();
     pathWatcher = new PathWatcher();
-    final WALFactory wals = new WALFactory(conf, tn.getMethodName());
+    final WALFactory wals = new WALFactory(CONF, tn.getMethodName());
     wals.getWALProvider().addWALActionsListener(pathWatcher);
     log = wals.getWAL(info);
   }
@@ -144,13 +145,13 @@ public class TestWALEntryStream {
           mvcc.advanceTo(1);
 
           for (int i = 0; i < nbRows; i++) {
-            appendToLogPlus(walEditKVs);
+            appendToLogAndSync(walEditKVs);
           }
 
           log.rollWriter();
 
           try (WALEntryStream entryStream =
-              new WALEntryStream(walQueue, fs, conf, 0, log, null, new MetricsSource("1"))) {
+              new WALEntryStream(walQueue, fs, CONF, 0, log, null, new MetricsSource("1"))) {
             int i = 0;
             while (entryStream.hasNext()) {
               assertNotNull(entryStream.next());
@@ -174,10 +175,10 @@ public class TestWALEntryStream {
    */
   @Test
   public void testAppendsWithRolls() throws Exception {
-    appendToLog();
+    appendToLogAndSync();
     long oldPos;
     try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, conf, 0, log, null, new MetricsSource("1"))) {
+        new WALEntryStream(walQueue, fs, CONF, 0, log, null, new MetricsSource("1"))) {
       // There's one edit in the log, read it. Reading past it needs to throw exception
       assertTrue(entryStream.hasNext());
       WAL.Entry entry = entryStream.peek();
@@ -189,9 +190,9 @@ public class TestWALEntryStream {
       oldPos = entryStream.getPosition();
     }
 
-    appendToLog();
+    appendToLogAndSync();
 
-    try (WALEntryStream entryStream = new WALEntryStream(walQueue, fs, conf, oldPos,
+    try (WALEntryStream entryStream = new WALEntryStream(walQueue, fs, CONF, oldPos,
         log, null, new MetricsSource("1"))) {
       // Read the newly added entry, make sure we made progress
       WAL.Entry entry = entryStream.next();
@@ -201,11 +202,11 @@ public class TestWALEntryStream {
     }
 
     // We rolled but we still should see the end of the first log and get that item
-    appendToLog();
+    appendToLogAndSync();
     log.rollWriter();
-    appendToLog();
+    appendToLogAndSync();
 
-    try (WALEntryStream entryStream = new WALEntryStream(walQueue, fs, conf, oldPos,
+    try (WALEntryStream entryStream = new WALEntryStream(walQueue, fs, CONF, oldPos,
         log, null, new MetricsSource("1"))) {
       WAL.Entry entry = entryStream.next();
       assertNotEquals(oldPos, entryStream.getPosition());
@@ -231,7 +232,7 @@ public class TestWALEntryStream {
     appendToLog("1");
     appendToLog("2");// 2
     try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, conf, 0, log, null, new MetricsSource("1"))) {
+        new WALEntryStream(walQueue, fs, CONF, 0, log, null, new MetricsSource("1"))) {
       assertEquals("1", getRow(entryStream.next()));
 
       appendToLog("3"); // 3 - comes in after reader opened
@@ -256,7 +257,7 @@ public class TestWALEntryStream {
   public void testNewEntriesWhileStreaming() throws Exception {
     appendToLog("1");
     try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, conf, 0, log, null, new MetricsSource("1"))) {
+        new WALEntryStream(walQueue, fs, CONF, 0, log, null, new MetricsSource("1"))) {
       entryStream.next(); // we've hit the end of the stream at this point
 
       // some new entries come in while we're streaming
@@ -279,7 +280,7 @@ public class TestWALEntryStream {
     long lastPosition = 0;
     appendToLog("1");
     try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, conf, 0, log, null, new MetricsSource("1"))) {
+        new WALEntryStream(walQueue, fs, CONF, 0, log, null, new MetricsSource("1"))) {
       entryStream.next(); // we've hit the end of the stream at this point
       appendToLog("2");
       appendToLog("3");
@@ -287,7 +288,7 @@ public class TestWALEntryStream {
     }
     // next stream should picks up where we left off
     try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, conf, lastPosition, log, null, new MetricsSource("1"))) {
+        new WALEntryStream(walQueue, fs, CONF, lastPosition, log, null, new MetricsSource("1"))) {
       assertEquals("2", getRow(entryStream.next()));
       assertEquals("3", getRow(entryStream.next()));
       assertFalse(entryStream.hasNext()); // done
@@ -302,16 +303,16 @@ public class TestWALEntryStream {
   @Test
   public void testPosition() throws Exception {
     long lastPosition = 0;
-    appendEntriesToLog(3);
+    appendEntriesToLogAndSync(3);
     // read only one element
-    try (WALEntryStream entryStream = new WALEntryStream(walQueue, fs, conf, lastPosition,
+    try (WALEntryStream entryStream = new WALEntryStream(walQueue, fs, CONF, lastPosition,
         log, null, new MetricsSource("1"))) {
       entryStream.next();
       lastPosition = entryStream.getPosition();
     }
     // there should still be two more entries from where we left off
     try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, conf, lastPosition, log, null, new MetricsSource("1"))) {
+        new WALEntryStream(walQueue, fs, CONF, lastPosition, log, null, new MetricsSource("1"))) {
       assertNotNull(entryStream.next());
       assertNotNull(entryStream.next());
       assertFalse(entryStream.hasNext());
@@ -322,38 +323,44 @@ public class TestWALEntryStream {
   @Test
   public void testEmptyStream() throws Exception {
     try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, conf, 0, log, null, new MetricsSource("1"))) {
+        new WALEntryStream(walQueue, fs, CONF, 0, log, null, new MetricsSource("1"))) {
       assertFalse(entryStream.hasNext());
     }
   }
 
+  private ReplicationSourceWALReader createReader(boolean recovered, Configuration conf) {
+    ReplicationSourceManager mockSourceManager = Mockito.mock(ReplicationSourceManager.class);
+    when(mockSourceManager.getTotalBufferUsed()).thenReturn(new AtomicLong(0));
+    Server mockServer = Mockito.mock(Server.class);
+    ReplicationSource source = Mockito.mock(ReplicationSource.class);
+    when(source.getSourceManager()).thenReturn(mockSourceManager);
+    when(source.getSourceMetrics()).thenReturn(new MetricsSource("1"));
+    when(source.getWALFileLengthProvider()).thenReturn(log);
+    when(source.getServer()).thenReturn(mockServer);
+    when(source.isRecovered()).thenReturn(recovered);
+    ReplicationSourceWALReader reader =
+      new ReplicationSourceWALReader(fs, conf, walQueue, 0, getDummyFilter(), source);
+    reader.start();
+    return reader;
+  }
+
   @Test
-  public void testReplicationSourceWALReaderThread() throws Exception {
-    appendEntriesToLog(3);
+  public void testReplicationSourceWALReader() throws Exception {
+    appendEntriesToLogAndSync(3);
     // get ending position
     long position;
     try (WALEntryStream entryStream =
-        new WALEntryStream(walQueue, fs, conf, 0, log, null, new MetricsSource("1"))) {
+        new WALEntryStream(walQueue, fs, CONF, 0, log, null, new MetricsSource("1"))) {
       entryStream.next();
       entryStream.next();
       entryStream.next();
       position = entryStream.getPosition();
     }
 
-    // start up a batcher
-    ReplicationSourceManager mockSourceManager = Mockito.mock(ReplicationSourceManager.class);
-    when(mockSourceManager.getTotalBufferUsed()).thenReturn(new AtomicLong(0));
-    Server mockServer= Mockito.mock(Server.class);
-    ReplicationSource source = Mockito.mock(ReplicationSource.class);
-    when(source.getSourceManager()).thenReturn(mockSourceManager);
-    when(source.getSourceMetrics()).thenReturn(new MetricsSource("1"));
-    when(source.getWALFileLengthProvider()).thenReturn(log);
-    when(source.getServer()).thenReturn(mockServer);
-    ReplicationSourceWALReader batcher = new ReplicationSourceWALReader(fs, conf,
-        walQueue, 0, getDummyFilter(), source);
+    // start up a reader
     Path walPath = walQueue.peek();
-    batcher.start();
-    WALEntryBatch entryBatch = batcher.take();
+    ReplicationSourceWALReader reader = createReader(false, CONF);
+    WALEntryBatch entryBatch = reader.take();
 
     // should've batched up our entries
     assertNotNull(entryBatch);
@@ -363,11 +370,96 @@ public class TestWALEntryStream {
     assertEquals(3, entryBatch.getNbRowKeys());
 
     appendToLog("foo");
-    entryBatch = batcher.take();
+    entryBatch = reader.take();
     assertEquals(1, entryBatch.getNbEntries());
     assertEquals("foo", getRow(entryBatch.getWalEntries().get(0)));
   }
 
+  @Test
+  public void testReplicationSourceWALReaderRecovered() throws Exception {
+    appendEntriesToLogAndSync(10);
+    Path walPath = walQueue.peek();
+    log.rollWriter();
+    appendEntriesToLogAndSync(5);
+    log.shutdown();
+
+    Configuration conf = new Configuration(CONF);
+    conf.setInt("replication.source.nb.capacity", 10);
+
+    ReplicationSourceWALReader reader = createReader(true, conf);
+
+    WALEntryBatch batch = reader.take();
+    assertEquals(walPath, batch.getLastWalPath());
+    assertEquals(10, batch.getNbEntries());
+    assertFalse(batch.isEndOfFile());
+
+    batch = reader.take();
+    assertEquals(walPath, batch.getLastWalPath());
+    assertEquals(0, batch.getNbEntries());
+    assertTrue(batch.isEndOfFile());
+
+    walPath = walQueue.peek();
+    batch = reader.take();
+    assertEquals(walPath, batch.getLastWalPath());
+    assertEquals(5, batch.getNbEntries());
+    // Actually this should be true but we haven't handled this yet since for a normal queue the
+    // last one is always open... Not a big deal for now.
+    assertFalse(batch.isEndOfFile());
+
+    assertSame(WALEntryBatch.NO_MORE_DATA, reader.take());
+  }
+
+  // Testcase for HBASE-20206
+  @Test
+  public void testReplicationSourceWALReaderWrongPosition() throws Exception {
+    appendEntriesToLogAndSync(1);
+    Path walPath = walQueue.peek();
+    log.rollWriter();
+    appendEntriesToLogAndSync(20);
+    TEST_UTIL.waitFor(5000, new ExplainingPredicate<Exception>() {
+
+      @Override
+      public boolean evaluate() throws Exception {
+        return fs.getFileStatus(walPath).getLen() > 0;
+      }
+
+      @Override
+      public String explainFailure() throws Exception {
+        return walPath + " has not been closed yet";
+      }
+
+    });
+    long walLength = fs.getFileStatus(walPath).getLen();
+
+    ReplicationSourceWALReader reader = createReader(false, CONF);
+
+    WALEntryBatch entryBatch = reader.take();
+    assertEquals(walPath, entryBatch.getLastWalPath());
+    assertTrue("Position " + entryBatch.getLastWalPosition() + " is out of range, file length is " +
+      walLength, entryBatch.getLastWalPosition() <= walLength);
+    assertEquals(1, entryBatch.getNbEntries());
+    assertTrue(entryBatch.isEndOfFile());
+
+    Path walPath2 = walQueue.peek();
+    entryBatch = reader.take();
+    assertEquals(walPath2, entryBatch.getLastWalPath());
+    assertEquals(20, entryBatch.getNbEntries());
+    assertFalse(entryBatch.isEndOfFile());
+
+    log.rollWriter();
+    appendEntriesToLogAndSync(10);
+    entryBatch = reader.take();
+    assertEquals(walPath2, entryBatch.getLastWalPath());
+    assertEquals(0, entryBatch.getNbEntries());
+    assertTrue(entryBatch.isEndOfFile());
+
+    Path walPath3 = walQueue.peek();
+    entryBatch = reader.take();
+    assertEquals(walPath3, entryBatch.getLastWalPath());
+    assertEquals(10, entryBatch.getNbEntries());
+    assertFalse(entryBatch.isEndOfFile());
+  }
+
   private String getRow(WAL.Entry entry) {
     Cell cell = entry.getEdit().getCells().get(0);
     return Bytes.toString(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
@@ -380,22 +472,28 @@ public class TestWALEntryStream {
     log.sync(txid);
   }
 
-  private void appendEntriesToLog(int count) throws IOException {
+  private void appendEntriesToLogAndSync(int count) throws IOException {
+    long txid = -1L;
     for (int i = 0; i < count; i++) {
-      appendToLog();
+      txid = appendToLog(1);
     }
+    log.sync(txid);
   }
 
-  private void appendToLog() throws IOException {
-    appendToLogPlus(1);
+  private void appendToLogAndSync() throws IOException {
+    appendToLogAndSync(1);
   }
 
-  private void appendToLogPlus(int count) throws IOException {
-    final long txid = log.append(info, new WALKeyImpl(info.getEncodedNameAsBytes(), tableName,
-            System.currentTimeMillis(), mvcc, scopes), getWALEdits(count), true);
+  private void appendToLogAndSync(int count) throws IOException {
+    long txid = appendToLog(count);
     log.sync(txid);
   }
 
+  private long appendToLog(int count) throws IOException {
+    return log.append(info, new WALKeyImpl(info.getEncodedNameAsBytes(), tableName,
+      System.currentTimeMillis(), mvcc, scopes), getWALEdits(count), true);
+  }
+
   private WALEdit getWALEdits(int count) {
     WALEdit edit = new WALEdit();
     for (int i = 0; i < count; i++) {
@@ -439,7 +537,7 @@ public class TestWALEntryStream {
     appendToLog("2");
     long size = log.getLogFileSizeIfBeingWritten(walQueue.peek()).getAsLong();
     AtomicLong fileLength = new AtomicLong(size - 1);
-    try (WALEntryStream entryStream = new WALEntryStream(walQueue, fs, conf, 0,
+    try (WALEntryStream entryStream = new WALEntryStream(walQueue, fs, CONF, 0,
         p -> OptionalLong.of(fileLength.get()), null, new MetricsSource("1"))) {
       assertTrue(entryStream.hasNext());
       assertNotNull(entryStream.next());