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 <hbase.rootdir>/<tabledir>/<regiondir>/<colfamdir>.
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());