[20/50] [abbrv] hbase git commit: HBASE-19543 Abstract a replication storage interface to extract the zk specific code
HBASE-19543 Abstract a replication storage interface to extract the zk specific code Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/9b97636c Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/9b97636c Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/9b97636c Branch: refs/heads/HBASE-19397-branch-2 Commit: 9b97636c6e8098994a4e55227354fa919ec0bb55 Parents: f68d3b1 Author: zhangduoAuthored: Fri Dec 22 14:37:28 2017 +0800 Committer: zhangduo Committed: Thu Feb 22 15:18:19 2018 +0800 -- .../hadoop/hbase/util/CollectionUtils.java | 3 + hbase-replication/pom.xml | 12 + .../replication/ReplicationPeerStorage.java | 74 .../replication/ReplicationQueueStorage.java| 164 +++ .../replication/ReplicationStateZKBase.java | 1 - .../replication/ReplicationStorageFactory.java | 49 +++ .../replication/ZKReplicationPeerStorage.java | 164 +++ .../replication/ZKReplicationQueueStorage.java | 425 +++ .../replication/ZKReplicationStorageBase.java | 75 .../TestZKReplicationPeerStorage.java | 171 .../TestZKReplicationQueueStorage.java | 171 .../org/apache/hadoop/hbase/master/HMaster.java | 35 +- .../hadoop/hbase/master/MasterServices.java | 6 +- .../master/procedure/MasterProcedureEnv.java| 24 +- .../master/replication/AddPeerProcedure.java| 6 +- .../replication/DisablePeerProcedure.java | 7 +- .../master/replication/EnablePeerProcedure.java | 6 +- .../master/replication/ModifyPeerProcedure.java | 41 +- .../master/replication/RemovePeerProcedure.java | 6 +- .../master/replication/ReplicationManager.java | 199 - .../replication/ReplicationPeerManager.java | 331 +++ .../replication/UpdatePeerConfigProcedure.java | 7 +- .../replication/TestReplicationAdmin.java | 64 ++- .../hbase/master/MockNoopMasterServices.java| 13 +- .../hbase/master/TestMasterNoCluster.java | 3 +- .../TestReplicationDisableInactivePeer.java | 6 +- 26 files changed, 1749 insertions(+), 314 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/9b97636c/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java -- diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java index 875b124..8bbb6f1 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/CollectionUtils.java @@ -107,6 +107,9 @@ public class CollectionUtils { return list.get(list.size() - 1); } + public static List nullToEmpty(List list) { +return list != null ? list : Collections.emptyList(); + } /** * In HBASE-16648 we found that ConcurrentHashMap.get is much faster than computeIfAbsent if the * value already exists. Notice that the implementation does not guarantee that the supplier will http://git-wip-us.apache.org/repos/asf/hbase/blob/9b97636c/hbase-replication/pom.xml -- diff --git a/hbase-replication/pom.xml b/hbase-replication/pom.xml index edce309..77517cc 100644 --- a/hbase-replication/pom.xml +++ b/hbase-replication/pom.xml @@ -104,6 +104,18 @@ org.apache.hbase hbase-zookeeper + + org.apache.hbase + hbase-common + test-jar + test + + + org.apache.hbase + hbase-zookeeper + test-jar + test + org.apache.commons http://git-wip-us.apache.org/repos/asf/hbase/blob/9b97636c/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java -- diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java new file mode 100644 index 000..e00cd0d --- /dev/null +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerStorage.java @@ -0,0 +1,74 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You
[48/50] [abbrv] hbase git commit: HBASE-19923 Reset peer state and config when refresh replication source failed
HBASE-19923 Reset peer state and config when refresh replication source failed Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/9cd0f701 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/9cd0f701 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/9cd0f701 Branch: refs/heads/HBASE-19397-branch-2 Commit: 9cd0f70144a0913e1bb791914908045987b36f7d Parents: e0717d8 Author: Guanghao ZhangAuthored: Tue Feb 6 14:58:39 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 15:23:52 2018 +0800 -- .../hbase/replication/ReplicationPeerImpl.java | 4 ++-- .../regionserver/PeerProcedureHandlerImpl.java | 24 2 files changed, 22 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/9cd0f701/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java -- diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java index 604e0bb..d656466 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java @@ -54,11 +54,11 @@ public class ReplicationPeerImpl implements ReplicationPeer { this.peerConfigListeners = new ArrayList<>(); } - void setPeerState(boolean enabled) { + public void setPeerState(boolean enabled) { this.peerState = enabled ? PeerState.ENABLED : PeerState.DISABLED; } - void setPeerConfig(ReplicationPeerConfig peerConfig) { + public void setPeerConfig(ReplicationPeerConfig peerConfig) { this.peerConfig = peerConfig; peerConfigListeners.forEach(listener -> listener.peerConfigUpdated(peerConfig)); } http://git-wip-us.apache.org/repos/asf/hbase/blob/9cd0f701/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 ce8fdae..a02d181 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 @@ -62,18 +62,26 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler { private void refreshPeerState(String peerId) throws ReplicationException, IOException { PeerState newState; Lock peerLock = peersLock.acquireLock(peerId); +ReplicationPeerImpl peer = null; +PeerState oldState = null; +boolean success = false; try { - ReplicationPeerImpl peer = replicationSourceManager.getReplicationPeers().getPeer(peerId); + peer = replicationSourceManager.getReplicationPeers().getPeer(peerId); if (peer == null) { throw new ReplicationException("Peer with id=" + peerId + " is not cached."); } - PeerState oldState = peer.getPeerState(); + oldState = peer.getPeerState(); newState = replicationSourceManager.getReplicationPeers().refreshPeerState(peerId); // RS need to start work with the new replication state change if (oldState.equals(PeerState.ENABLED) && newState.equals(PeerState.DISABLED)) { replicationSourceManager.refreshSources(peerId); } + success = true; } finally { + if (!success && peer != null) { +// Reset peer state if refresh source failed +peer.setPeerState(oldState.equals(PeerState.ENABLED)); + } peerLock.unlock(); } } @@ -91,19 +99,27 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler { @Override public void updatePeerConfig(String peerId) throws ReplicationException, IOException { Lock peerLock = peersLock.acquireLock(peerId); +ReplicationPeerImpl peer = null; +ReplicationPeerConfig oldConfig = null; +boolean success = false; try { - ReplicationPeerImpl peer = replicationSourceManager.getReplicationPeers().getPeer(peerId); + peer = replicationSourceManager.getReplicationPeers().getPeer(peerId); if (peer == null) { throw new ReplicationException("Peer with id=" + peerId + " is not cached."); } - ReplicationPeerConfig oldConfig = peer.getPeerConfig(); + oldConfig = peer.getPeerConfig();
[15/50] [abbrv] hbase git commit: HBASE-19525 RS side changes for moving peer modification from zk watcher to procedure
HBASE-19525 RS side changes for moving peer modification from zk watcher to procedure Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/f68d3b15 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/f68d3b15 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/f68d3b15 Branch: refs/heads/HBASE-19397-branch-2 Commit: f68d3b1501dd1c9918b5426fe0707519bb3b02c5 Parents: 62a2a8c Author: huzhengAuthored: Wed Dec 20 10:47:18 2017 +0800 Committer: zhangduo Committed: Thu Feb 22 15:15:11 2018 +0800 -- .../hadoop/hbase/protobuf/ProtobufUtil.java | 11 +- .../hbase/shaded/protobuf/ProtobufUtil.java | 13 +- .../hbase/replication/ReplicationListener.java | 14 -- .../hbase/replication/ReplicationPeer.java | 28 ++- .../replication/ReplicationPeerZKImpl.java | 186 --- .../replication/ReplicationPeersZKImpl.java | 19 +- .../replication/ReplicationTrackerZKImpl.java | 73 +- .../regionserver/ReplicationSourceService.java | 6 + .../handler/RSProcedureHandler.java | 3 + .../replication/BaseReplicationEndpoint.java| 2 +- .../regionserver/PeerProcedureHandler.java | 38 .../regionserver/PeerProcedureHandlerImpl.java | 81 +++ .../regionserver/RefreshPeerCallable.java | 39 +++- .../replication/regionserver/Replication.java | 9 + .../regionserver/ReplicationSource.java | 8 +- .../regionserver/ReplicationSourceManager.java | 37 ++- .../TestReplicationAdminUsingProcedure.java | 226 +++ .../replication/DummyModifyPeerProcedure.java | 48 .../TestDummyModifyPeerProcedure.java | 80 --- .../TestReplicationTrackerZKImpl.java | 61 - .../TestReplicationSourceManager.java | 32 ++- 21 files changed, 532 insertions(+), 482 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/f68d3b15/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java index 3033da7..f11fcf6 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.hbase.protobuf; +import static org.apache.hadoop.hbase.protobuf.ProtobufMagic.PB_MAGIC; + import com.google.protobuf.ByteString; import com.google.protobuf.CodedInputStream; import com.google.protobuf.InvalidProtocolBufferException; @@ -191,7 +193,7 @@ public final class ProtobufUtil { * byte array that is bytes.length plus {@link ProtobufMagic#PB_MAGIC}.length. */ public static byte [] prependPBMagic(final byte [] bytes) { -return Bytes.add(ProtobufMagic.PB_MAGIC, bytes); +return Bytes.add(PB_MAGIC, bytes); } /** @@ -216,10 +218,11 @@ public final class ProtobufUtil { * @param bytes bytes to check * @throws DeserializationException if we are missing the pb magic prefix */ - public static void expectPBMagicPrefix(final byte [] bytes) throws DeserializationException { + public static void expectPBMagicPrefix(final byte[] bytes) throws DeserializationException { if (!isPBMagicPrefix(bytes)) { - throw new DeserializationException("Missing pb magic " + - Bytes.toString(ProtobufMagic.PB_MAGIC) + " prefix"); + String bytesPrefix = bytes == null ? "null" : Bytes.toStringBinary(bytes, 0, PB_MAGIC.length); + throw new DeserializationException( + "Missing pb magic " + Bytes.toString(PB_MAGIC) + " prefix, bytes: " + bytesPrefix); } } http://git-wip-us.apache.org/repos/asf/hbase/blob/f68d3b15/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java index 4472846..520a4cd 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.hbase.shaded.protobuf; +import static org.apache.hadoop.hbase.protobuf.ProtobufMagic.PB_MAGIC; + import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; @@ -277,7 +279,7 @@ public final class ProtobufUtil { * byte array that
[50/50] [abbrv] hbase git commit: HBASE-19707 Race in start and terminate of a replication source after we async start replicatione endpoint
HBASE-19707 Race in start and terminate of a replication source after we async start replicatione endpoint Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/41991d61 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/41991d61 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/41991d61 Branch: refs/heads/HBASE-19397-branch-2 Commit: 41991d6103a4144012a7d6050e6f3620b859d8d1 Parents: 2fa6823 Author: zhangduoAuthored: Fri Jan 5 18:28:44 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 15:23:52 2018 +0800 -- .../RecoveredReplicationSource.java | 16 +- .../regionserver/ReplicationSource.java | 203 ++- 2 files changed, 116 insertions(+), 103 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/41991d61/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 1be9a88..3cae0f2 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 @@ -68,7 +68,7 @@ public class RecoveredReplicationSource extends ReplicationSource { 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(getUncaughtExceptionHandler()); + worker.startup(this::uncaughtException); worker.setWALReader( startNewWALReader(worker.getName(), walGroupId, queue, worker.getStartPosition())); workerThreads.put(walGroupId, worker); @@ -76,13 +76,13 @@ public class RecoveredReplicationSource extends ReplicationSource { } @Override - protected ReplicationSourceWALReader startNewWALReader(String threadName, - String walGroupId, PriorityBlockingQueue queue, long startPosition) { -ReplicationSourceWALReader walReader = new RecoveredReplicationSourceWALReader(fs, -conf, queue, startPosition, walEntryFilter, this); -Threads.setDaemonThreadRunning(walReader, threadName -+ ".replicationSource.replicationWALReaderThread." + walGroupId + "," + queueId, - getUncaughtExceptionHandler()); + protected ReplicationSourceWALReader startNewWALReader(String threadName, String walGroupId, + PriorityBlockingQueue 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; } http://git-wip-us.apache.org/repos/asf/hbase/blob/41991d61/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 0092251..09b6cc1 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 @@ -75,7 +75,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists; * */ @InterfaceAudience.Private -public class ReplicationSource extends Thread implements ReplicationSourceInterface { +public class ReplicationSource implements ReplicationSourceInterface { private static final Logger LOG = LoggerFactory.getLogger(ReplicationSource.class); // Queues of logs to process, entry in format of walGroupId->queue, @@ -114,10 +114,8 @@ public class ReplicationSource extends Thread implements ReplicationSourceInterf private MetricsSource metrics; // WARN threshold for the number of queued logs, defaults to 2 private int logQueueWarnThreshold; - // whether the replication endpoint has been initialized - private volatile boolean endpointInitialized = false; // ReplicationEndpoint which will handle the actual replication - private ReplicationEndpoint replicationEndpoint; + private volatile
[08/50] [abbrv] hbase git commit: HBASE-2004 TestClientClusterStatus is flakey
HBASE-2004 TestClientClusterStatus is flakey Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/92bb4db9 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/92bb4db9 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/92bb4db9 Branch: refs/heads/HBASE-19397-branch-2 Commit: 92bb4db9ef367039029017aafe61c51f2c0181f9 Parents: baec532 Author: Michael StackAuthored: Wed Feb 21 14:52:10 2018 -0800 Committer: Michael Stack Committed: Wed Feb 21 14:52:10 2018 -0800 -- .../hadoop/hbase/TestClientClusterStatus.java | 16 +--- 1 file changed, 9 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/92bb4db9/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java index bfdae8a..392ff6e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java @@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; +import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -100,13 +101,14 @@ public class TestClientClusterStatus { @Test public void testNone() throws Exception { -ClusterStatus status0 - = new ClusterStatus(ADMIN.getClusterMetrics(EnumSet.allOf(Option.class))); -ClusterStatus status1 - = new ClusterStatus(ADMIN.getClusterMetrics(EnumSet.noneOf(Option.class))); -Assert.assertEquals(status0, status1); -checkPbObjectNotNull(status0); -checkPbObjectNotNull(status1); +ClusterMetrics status0 = ADMIN.getClusterMetrics(EnumSet.allOf(Option.class)); +ClusterMetrics status1 = ADMIN.getClusterMetrics(EnumSet.noneOf(Option.class)); +// Do a rough compare. More specific compares can fail because all regions not deployed yet +// or more requests than expected. +Assert.assertEquals(status0.getLiveServerMetrics().size(), +status1.getLiveServerMetrics().size()); +checkPbObjectNotNull(new ClusterStatus(status0)); +checkPbObjectNotNull(new ClusterStatus(status1)); } @Test
[23/50] [abbrv] hbase git commit: HBASE-19579 Add peer lock test for shell command list_locks
HBASE-19579 Add peer lock test for shell command list_locks Signed-off-by: zhangduoProject: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/253b1807 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/253b1807 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/253b1807 Branch: refs/heads/HBASE-19397-branch-2 Commit: 253b1807041e8815ac7723206492bd7c690af4aa Parents: 72db7a6 Author: Guanghao Zhang Authored: Sat Dec 23 21:04:27 2017 +0800 Committer: zhangduo Committed: Thu Feb 22 15:21:30 2018 +0800 -- .../src/main/protobuf/LockService.proto | 1 + .../src/test/ruby/shell/list_locks_test.rb | 19 +++ 2 files changed, 20 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/253b1807/hbase-protocol-shaded/src/main/protobuf/LockService.proto -- diff --git a/hbase-protocol-shaded/src/main/protobuf/LockService.proto b/hbase-protocol-shaded/src/main/protobuf/LockService.proto index b8d180c..0675070 100644 --- a/hbase-protocol-shaded/src/main/protobuf/LockService.proto +++ b/hbase-protocol-shaded/src/main/protobuf/LockService.proto @@ -77,6 +77,7 @@ enum LockedResourceType { NAMESPACE = 2; TABLE = 3; REGION = 4; + PEER = 5; } message LockedResource { http://git-wip-us.apache.org/repos/asf/hbase/blob/253b1807/hbase-shell/src/test/ruby/shell/list_locks_test.rb -- diff --git a/hbase-shell/src/test/ruby/shell/list_locks_test.rb b/hbase-shell/src/test/ruby/shell/list_locks_test.rb index f465a6b..ef1c0ce 100644 --- a/hbase-shell/src/test/ruby/shell/list_locks_test.rb +++ b/hbase-shell/src/test/ruby/shell/list_locks_test.rb @@ -67,6 +67,25 @@ module Hbase proc_id) end +define_test 'list peer locks' do + lock = create_exclusive_lock(0) + peer_id = '1' + + @scheduler.waitPeerExclusiveLock(lock, peer_id) + output = capture_stdout { @list_locks.command } + @scheduler.wakePeerExclusiveLock(lock, peer_id) + + assert_equal( +"PEER(1)\n" \ +"Lock type: EXCLUSIVE, procedure: {" \ + "\"className\"=>\"org.apache.hadoop.hbase.master.locking.LockProcedure\", " \ + "\"procId\"=>\"0\", \"submittedTime\"=>\"0\", \"state\"=>\"RUNNABLE\", " \ + "\"lastUpdate\"=>\"0\", " \ + "\"stateMessage\"=>[{\"lockType\"=>\"EXCLUSIVE\", \"description\"=>\"description\"}]" \ +"}\n\n", +output) +end + define_test 'list server locks' do lock = create_exclusive_lock(0)
[37/50] [abbrv] hbase git commit: HBASE-19633 Clean up the replication queues in the postPeerModification stage when removing a peer
HBASE-19633 Clean up the replication queues in the postPeerModification stage when removing a peer Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/bd71e06a Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/bd71e06a Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/bd71e06a Branch: refs/heads/HBASE-19397-branch-2 Commit: bd71e06a3cfc77551c574e4d68e8128be65e4427 Parents: 05aa1aa Author: zhangduoAuthored: Tue Jan 2 09:57:23 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 15:23:34 2018 +0800 -- .../replication/ReplicationPeerConfig.java | 2 +- .../replication/VerifyReplication.java | 34 ++ .../hbase/replication/ReplicationPeers.java | 32 +++-- .../replication/ZKReplicationQueueStorage.java | 3 +- .../replication/ZKReplicationStorageBase.java | 4 +-- .../replication/TestReplicationStateBasic.java | 10 ++ .../master/replication/AddPeerProcedure.java| 5 ++- .../replication/DisablePeerProcedure.java | 3 +- .../master/replication/EnablePeerProcedure.java | 3 +- .../master/replication/ModifyPeerProcedure.java | 34 ++ .../replication/RefreshPeerProcedure.java | 17 - .../master/replication/RemovePeerProcedure.java | 7 ++-- .../replication/ReplicationPeerManager.java | 31 +++- .../replication/UpdatePeerConfigProcedure.java | 3 +- .../RemoteProcedureResultReporter.java | 3 +- .../regionserver/RefreshPeerCallable.java | 5 +-- .../regionserver/ReplicationSourceManager.java | 38 +++- .../TestReplicationAdminUsingProcedure.java | 7 ++-- 18 files changed, 124 insertions(+), 117 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/bd71e06a/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 b80ee16..fdae288 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 @@ -27,8 +27,8 @@ import java.util.Set; import java.util.TreeMap; import org.apache.hadoop.hbase.TableName; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.yetus.audience.InterfaceAudience; /** * A configuration for the replication peer cluster. http://git-wip-us.apache.org/repos/asf/hbase/blob/bd71e06a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java -- diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java index f0070f0..fe45762 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java @@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.mapreduce.replication; import java.io.IOException; import java.util.Arrays; import java.util.UUID; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.fs.FileSystem; @@ -45,13 +44,14 @@ import org.apache.hadoop.hbase.filter.PrefixFilter; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.mapreduce.TableInputFormat; import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; -import org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat; import org.apache.hadoop.hbase.mapreduce.TableMapper; +import org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat; import org.apache.hadoop.hbase.mapreduce.TableSplit; import org.apache.hadoop.hbase.replication.ReplicationException; -import org.apache.hadoop.hbase.replication.ReplicationFactory; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; +import org.apache.hadoop.hbase.replication.ReplicationPeerStorage; import org.apache.hadoop.hbase.replication.ReplicationPeers; +import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; @@ -66,6 +66,7 @@ import org.apache.hadoop.util.Tool;
[10/50] [abbrv] hbase git commit: HBASE-19391 Calling HRegion#initializeRegionInternals from a region replica can still re-create a region directory
HBASE-19391 Calling HRegion#initializeRegionInternals from a region replica can still re-create a region directory Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/c06c4f30 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c06c4f30 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c06c4f30 Branch: refs/heads/HBASE-19397-branch-2 Commit: c06c4f30d1c7c437461bda0cf3776f90c220e840 Parents: c677c30 Author: Esteban GutierrezAuthored: Wed Feb 21 15:28:50 2018 -0800 Committer: huaxiangsun Committed: Wed Feb 21 16:34:53 2018 -0800 -- .../hbase/regionserver/HRegionFileSystem.java | 29 ++-- 1 file changed, 15 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/c06c4f30/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 6ad3f1a..3c2d392 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 @@ -969,28 +969,29 @@ public class HRegionFileSystem { public static HRegionFileSystem createRegionOnFileSystem(final Configuration conf, final FileSystem fs, final Path tableDir, final RegionInfo regionInfo) throws IOException { HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tableDir, regionInfo); -Path regionDir = regionFs.getRegionDir(); -if (fs.exists(regionDir)) { - LOG.warn("Trying to create a region that already exists on disk: " + regionDir); - throw new IOException("The specified region already exists on disk: " + regionDir); -} +// We only create a .regioninfo and the region directory if this is the default region replica +if (regionInfo.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) { + Path regionDir = regionFs.getRegionDir(); + if (fs.exists(regionDir)) { +LOG.warn("Trying to create a region that already exists on disk: " + regionDir); +throw new IOException("The specified region already exists on disk: " + regionDir); + } -// Create the region directory -if (!createDirOnFileSystem(fs, conf, regionDir)) { - LOG.warn("Unable to create the region directory: " + regionDir); - throw new IOException("Unable to create region directory: " + regionDir); -} + // Create the region directory + if (!createDirOnFileSystem(fs, conf, regionDir)) { +LOG.warn("Unable to create the region directory: " + regionDir); +throw new IOException("Unable to create region directory: " + regionDir); + } -// Write HRI to a file in case we need to recover hbase:meta -// Only primary replicas should write region info -if (regionInfo.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) { + // Write HRI to a file in case we need to recover hbase:meta regionFs.writeRegionInfoOnFilesystem(false); + return regionFs; } else { if (LOG.isDebugEnabled()) LOG.debug("Skipping creation of .regioninfo file for " + regionInfo); } -return regionFs; +return null; } /**
[21/50] [abbrv] hbase git commit: HBASE-19630 Add peer cluster key check when add new replication peer
HBASE-19630 Add peer cluster key check when add new replication peer Signed-off-by: zhangduoProject: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/ad4157d4 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/ad4157d4 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/ad4157d4 Branch: refs/heads/HBASE-19397-branch-2 Commit: ad4157d46f20518814bb1c59280b15b6bf4e51f3 Parents: 48d2587 Author: Guanghao Zhang Authored: Tue Dec 26 21:10:00 2017 +0800 Committer: zhangduo Committed: Thu Feb 22 15:21:30 2018 +0800 -- .../replication/ReplicationPeerManager.java | 54 .../replication/TestReplicationAdmin.java | 22 2 files changed, 54 insertions(+), 22 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/ad4157d4/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java index 84abfeb..b78cbce 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.master.replication; +import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.List; @@ -42,6 +43,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerStorage; import org.apache.hadoop.hbase.replication.ReplicationQueueInfo; import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; +import org.apache.hadoop.hbase.zookeeper.ZKConfig; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.yetus.audience.InterfaceAudience; @@ -216,36 +218,36 @@ public final class ReplicationPeerManager { return desc != null ? Optional.of(desc.getPeerConfig()) : Optional.empty(); } - /** - * If replicate_all flag is true, it means all user tables will be replicated to peer cluster. - * Then allow config exclude namespaces or exclude table-cfs which can't be replicated to peer - * cluster. - * - * If replicate_all flag is false, it means all user tables can't be replicated to peer cluster. - * Then allow to config namespaces or table-cfs which will be replicated to peer cluster. - */ - private static void checkPeerConfig(ReplicationPeerConfig peerConfig) - throws DoNotRetryIOException { + private void checkPeerConfig(ReplicationPeerConfig peerConfig) throws DoNotRetryIOException { +checkClusterKey(peerConfig.getClusterKey()); + if (peerConfig.replicateAllUserTables()) { - if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty()) || -(peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) { -throw new DoNotRetryIOException("Need clean namespaces or table-cfs config firstly " + - "when you want replicate all cluster"); + // If replicate_all flag is true, it means all user tables will be replicated to peer cluster. + // Then allow config exclude namespaces or exclude table-cfs which can't be replicated to peer + // cluster. + if ((peerConfig.getNamespaces() != null && !peerConfig.getNamespaces().isEmpty()) + || (peerConfig.getTableCFsMap() != null && !peerConfig.getTableCFsMap().isEmpty())) { +throw new DoNotRetryIOException("Need clean namespaces or table-cfs config firstly " ++ "when you want replicate all cluster"); } checkNamespacesAndTableCfsConfigConflict(peerConfig.getExcludeNamespaces(), peerConfig.getExcludeTableCFsMap()); } else { - if ((peerConfig.getExcludeNamespaces() != null && -!peerConfig.getExcludeNamespaces().isEmpty()) || -(peerConfig.getExcludeTableCFsMap() != null && - !peerConfig.getExcludeTableCFsMap().isEmpty())) { + // If replicate_all flag is false, it means all user tables can't be replicated to peer + // cluster. Then allow to config namespaces or table-cfs which will be replicated to peer + // cluster. + if ((peerConfig.getExcludeNamespaces() != null + && !peerConfig.getExcludeNamespaces().isEmpty()) + || (peerConfig.getExcludeTableCFsMap() != null + && !peerConfig.getExcludeTableCFsMap().isEmpty())) { throw new DoNotRetryIOException( -"Need
[25/50] [abbrv] hbase git commit: HBASE-19599 Remove ReplicationQueuesClient, use ReplicationQueueStorage directly
HBASE-19599 Remove ReplicationQueuesClient, use ReplicationQueueStorage directly Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/72db7a63 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/72db7a63 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/72db7a63 Branch: refs/heads/HBASE-19397-branch-2 Commit: 72db7a6346d8903f21cd496d821597ef6a883fe9 Parents: 9b97636 Author: zhangduoAuthored: Mon Dec 25 18:49:56 2017 +0800 Committer: zhangduo Committed: Thu Feb 22 15:21:30 2018 +0800 -- .../hbase/replication/ReplicationFactory.java | 19 +- .../replication/ReplicationPeersZKImpl.java | 21 +- .../replication/ReplicationQueueStorage.java| 26 +- .../replication/ReplicationQueuesClient.java| 93 - .../ReplicationQueuesClientArguments.java | 40 -- .../ReplicationQueuesClientZKImpl.java | 181 - .../replication/ZKReplicationQueueStorage.java | 90 - .../replication/TestReplicationStateBasic.java | 378 +++ .../replication/TestReplicationStateZKImpl.java | 153 .../TestZKReplicationQueueStorage.java | 74 .../cleaner/ReplicationZKNodeCleaner.java | 71 ++-- .../cleaner/ReplicationZKNodeCleanerChore.java | 5 +- .../replication/ReplicationPeerManager.java | 31 +- .../master/ReplicationHFileCleaner.java | 109 ++ .../master/ReplicationLogCleaner.java | 44 +-- .../regionserver/DumpReplicationQueues.java | 78 ++-- .../hbase/util/hbck/ReplicationChecker.java | 14 +- .../client/TestAsyncReplicationAdminApi.java| 31 +- .../replication/TestReplicationAdmin.java | 2 + .../hbase/master/cleaner/TestLogsCleaner.java | 81 ++-- .../cleaner/TestReplicationHFileCleaner.java| 29 -- .../cleaner/TestReplicationZKNodeCleaner.java | 12 +- .../replication/TestReplicationStateBasic.java | 378 --- .../replication/TestReplicationStateZKImpl.java | 232 .../TestReplicationSourceManagerZkImpl.java | 41 -- 25 files changed, 890 insertions(+), 1343 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/72db7a63/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java -- diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java index 9f4ad18..6c1c213 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java @@ -1,5 +1,4 @@ -/* - * +/** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -37,20 +36,14 @@ public class ReplicationFactory { args); } - public static ReplicationQueuesClient - getReplicationQueuesClient(ReplicationQueuesClientArguments args) throws Exception { -return (ReplicationQueuesClient) ConstructorUtils -.invokeConstructor(ReplicationQueuesClientZKImpl.class, args); - } - - public static ReplicationPeers getReplicationPeers(final ZKWatcher zk, Configuration conf, - Abortable abortable) { + public static ReplicationPeers getReplicationPeers(ZKWatcher zk, Configuration conf, + Abortable abortable) { return getReplicationPeers(zk, conf, null, abortable); } - public static ReplicationPeers getReplicationPeers(final ZKWatcher zk, Configuration conf, - final ReplicationQueuesClient queuesClient, Abortable abortable) { -return new ReplicationPeersZKImpl(zk, conf, queuesClient, abortable); + public static ReplicationPeers getReplicationPeers(ZKWatcher zk, Configuration conf, + ReplicationQueueStorage queueStorage, Abortable abortable) { +return new ReplicationPeersZKImpl(zk, conf, queueStorage, abortable); } public static ReplicationTracker getReplicationTracker(ZKWatcher zookeeper, http://git-wip-us.apache.org/repos/asf/hbase/blob/72db7a63/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java -- diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java index 419e289..4e5f757 100644 ---
[24/50] [abbrv] hbase git commit: HBASE-19599 Remove ReplicationQueuesClient, use ReplicationQueueStorage directly
http://git-wip-us.apache.org/repos/asf/hbase/blob/72db7a63/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java index 6e27a21..d8f9625 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/DumpReplicationQueues.java @@ -21,13 +21,13 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashSet; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Queue; import java.util.Set; import java.util.stream.Collectors; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.fs.FileStatus; @@ -48,17 +48,18 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; import org.apache.hadoop.hbase.replication.ReplicationPeers; import org.apache.hadoop.hbase.replication.ReplicationQueueInfo; +import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; import org.apache.hadoop.hbase.replication.ReplicationQueues; -import org.apache.hadoop.hbase.replication.ReplicationQueuesClient; -import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments; +import org.apache.hadoop.hbase.replication.ReplicationQueuesArguments; +import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; import org.apache.hadoop.hbase.replication.ReplicationTracker; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; -import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import org.apache.hbase.thirdparty.com.google.common.util.concurrent.AtomicLongMap; /** @@ -303,57 +304,53 @@ public class DumpReplicationQueues extends Configured implements Tool { } public String dumpQueues(ClusterConnection connection, ZKWatcher zkw, Set peerIds, - boolean hdfs) throws Exception { -ReplicationQueuesClient queuesClient; + boolean hdfs) throws Exception { +ReplicationQueueStorage queueStorage; ReplicationPeers replicationPeers; ReplicationQueues replicationQueues; ReplicationTracker replicationTracker; -ReplicationQueuesClientArguments replicationArgs = -new ReplicationQueuesClientArguments(getConf(), new WarnOnlyAbortable(), zkw); +ReplicationQueuesArguments replicationArgs = +new ReplicationQueuesArguments(getConf(), new WarnOnlyAbortable(), zkw); StringBuilder sb = new StringBuilder(); -queuesClient = ReplicationFactory.getReplicationQueuesClient(replicationArgs); -queuesClient.init(); +queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, getConf()); replicationQueues = ReplicationFactory.getReplicationQueues(replicationArgs); -replicationPeers = ReplicationFactory.getReplicationPeers(zkw, getConf(), queuesClient, connection); +replicationPeers = +ReplicationFactory.getReplicationPeers(zkw, getConf(), queueStorage, connection); replicationTracker = ReplicationFactory.getReplicationTracker(zkw, replicationPeers, getConf(), new WarnOnlyAbortable(), new WarnOnlyStoppable()); -List liveRegionServers = replicationTracker.getListOfRegionServers(); +Set liveRegionServers = new HashSet<>(replicationTracker.getListOfRegionServers()); // Loops each peer on each RS and dumps the queues -try { - List regionservers = queuesClient.getListOfReplicators(); - if (regionservers == null || regionservers.isEmpty()) { -return sb.toString(); +List regionservers = queueStorage.getListOfReplicators(); +if (regionservers == null || regionservers.isEmpty()) { + return sb.toString(); +} +for (ServerName regionserver : regionservers) { + List queueIds = queueStorage.getAllQueues(regionserver); + replicationQueues.init(regionserver.getServerName()); + if (!liveRegionServers.contains(regionserver.getServerName())) { +deadRegionServers.add(regionserver.getServerName()); } - for (String regionserver : regionservers) { -List queueIds = queuesClient.getAllQueues(regionserver); -replicationQueues.init(regionserver); -if (!liveRegionServers.contains(regionserver)) { - deadRegionServers.add(regionserver); -} -for (String
[31/50] [abbrv] hbase git commit: HBASE-19686 Use KeyLocker instead of ReentrantLock in PeerProcedureHandlerImpl
HBASE-19686 Use KeyLocker instead of ReentrantLock in PeerProcedureHandlerImpl Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/88228a25 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/88228a25 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/88228a25 Branch: refs/heads/HBASE-19397-branch-2 Commit: 88228a25c8feef9e37af485ed63e4d87c08d9f10 Parents: 9b0aa69 Author: zhangduoAuthored: Tue Jan 2 16:13:55 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 15:23:34 2018 +0800 -- .../regionserver/PeerProcedureHandlerImpl.java | 41 ++-- 1 file changed, 29 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/88228a25/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 1efe180..c09c6a0 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 @@ -19,10 +19,10 @@ package org.apache.hadoop.hbase.replication.regionserver; import java.io.IOException; -import java.util.concurrent.locks.ReentrantLock; - +import java.util.concurrent.locks.Lock; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState; +import org.apache.hadoop.hbase.util.KeyLocker; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,7 +32,7 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler { private static final Logger LOG = LoggerFactory.getLogger(PeerProcedureHandlerImpl.class); private final ReplicationSourceManager replicationSourceManager; - private final ReentrantLock peersLock = new ReentrantLock(); + private final KeyLocker peersLock = new KeyLocker<>(); public PeerProcedureHandlerImpl(ReplicationSourceManager replicationSourceManager) { this.replicationSourceManager = replicationSourceManager; @@ -40,40 +40,57 @@ public class PeerProcedureHandlerImpl implements PeerProcedureHandler { @Override public void addPeer(String peerId) throws ReplicationException, IOException { -peersLock.lock(); +Lock peerLock = peersLock.acquireLock(peerId); try { replicationSourceManager.addPeer(peerId); } finally { - peersLock.unlock(); + peerLock.unlock(); } } @Override public void removePeer(String peerId) throws ReplicationException, IOException { -peersLock.lock(); +Lock peerLock = peersLock.acquireLock(peerId); try { if (replicationSourceManager.getReplicationPeers().getPeer(peerId) != null) { replicationSourceManager.removePeer(peerId); } } finally { - peersLock.unlock(); + peerLock.unlock(); } } @Override public void disablePeer(String peerId) throws ReplicationException, IOException { -PeerState newState = replicationSourceManager.getReplicationPeers().refreshPeerState(peerId); -LOG.info("disable replication peer, id: " + peerId + ", new state: " + newState); +PeerState newState; +Lock peerLock = peersLock.acquireLock(peerId); +try { + newState = replicationSourceManager.getReplicationPeers().refreshPeerState(peerId); +} finally { + peerLock.unlock(); +} +LOG.info("disable replication peer, id: {}, new state: {}", peerId, newState); } @Override public void enablePeer(String peerId) throws ReplicationException, IOException { -PeerState newState = replicationSourceManager.getReplicationPeers().refreshPeerState(peerId); -LOG.info("enable replication peer, id: " + peerId + ", new state: " + newState); +PeerState newState; +Lock peerLock = peersLock.acquireLock(peerId); +try { + newState = replicationSourceManager.getReplicationPeers().refreshPeerState(peerId); +} finally { + peerLock.unlock(); +} +LOG.info("enable replication peer, id: {}, new state: {}", peerId, newState); } @Override public void updatePeerConfig(String peerId) throws ReplicationException, IOException { -replicationSourceManager.getReplicationPeers().refreshPeerConfig(peerId); +Lock peerLock = peersLock.acquireLock(peerId); +try { +
[11/50] [abbrv] hbase git commit: HBASE-20038 TestLockProcedure.testTimeout is flakey
HBASE-20038 TestLockProcedure.testTimeout is flakey Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/0432d49d Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/0432d49d Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/0432d49d Branch: refs/heads/HBASE-19397-branch-2 Commit: 0432d49d838c283004b7c6b5ab4c584d57c9 Parents: c06c4f3 Author: zhangduoAuthored: Wed Feb 21 21:20:24 2018 +0800 Committer: Michael Stack Committed: Wed Feb 21 17:23:49 2018 -0800 -- .../org/apache/hadoop/hbase/master/locking/TestLockProcedure.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/0432d49d/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java index 85b00d0..c985fa7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java @@ -284,7 +284,7 @@ public class TestLockProcedure { sendHeartbeatAndCheckLocked(procId, true); Thread.sleep(HEARTBEAT_TIMEOUT / 2); sendHeartbeatAndCheckLocked(procId, true); -Thread.sleep(2 * HEARTBEAT_TIMEOUT); +Thread.sleep(4 * HEARTBEAT_TIMEOUT); sendHeartbeatAndCheckLocked(procId, false); ProcedureTestingUtility.waitProcedure(procExec, procId); ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
[17/50] [abbrv] hbase git commit: HBASE-19524 Master side changes for moving peer modification from zk watcher to procedure
HBASE-19524 Master side changes for moving peer modification from zk watcher to procedure Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/1275b9f6 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/1275b9f6 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/1275b9f6 Branch: refs/heads/HBASE-19397-branch-2 Commit: 1275b9f6eb609a14f7cb6caba063edecf6a638db Parents: 92a370b Author: zhangduoAuthored: Mon Dec 18 15:22:36 2017 +0800 Committer: zhangduo Committed: Thu Feb 22 15:15:11 2018 +0800 -- .../procedure2/RemoteProcedureDispatcher.java | 3 +- .../src/main/protobuf/MasterProcedure.proto | 21 - .../src/main/protobuf/RegionServerStatus.proto | 3 +- .../src/main/protobuf/Replication.proto | 5 + .../replication/ReplicationPeersZKImpl.java | 4 +- .../org/apache/hadoop/hbase/master/HMaster.java | 95 --- .../hadoop/hbase/master/MasterRpcServices.java | 4 +- .../hadoop/hbase/master/MasterServices.java | 26 +++--- .../assignment/RegionTransitionProcedure.java | 13 +-- .../master/procedure/MasterProcedureEnv.java| 5 + .../master/procedure/ProcedurePrepareLatch.java | 2 +- .../master/replication/AddPeerProcedure.java| 97 .../replication/DisablePeerProcedure.java | 70 ++ .../master/replication/EnablePeerProcedure.java | 69 ++ .../master/replication/ModifyPeerProcedure.java | 97 +--- .../master/replication/RefreshPeerCallable.java | 67 -- .../replication/RefreshPeerProcedure.java | 28 -- .../master/replication/RemovePeerProcedure.java | 69 ++ .../master/replication/ReplicationManager.java | 76 --- .../replication/UpdatePeerConfigProcedure.java | 92 +++ .../hbase/regionserver/HRegionServer.java | 5 +- .../regionserver/RefreshPeerCallable.java | 70 ++ .../hbase/master/MockNoopMasterServices.java| 23 +++-- .../replication/DummyModifyPeerProcedure.java | 13 ++- 24 files changed, 733 insertions(+), 224 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/1275b9f6/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java -- diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java index dca0bec..a22a7ba 100644 --- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java +++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java @@ -247,9 +247,8 @@ public abstract class RemoteProcedureDispatcher
[44/50] [abbrv] hbase git commit: HBASE-19636 All rs should already start work with the new peer change when replication peer procedure is finished
HBASE-19636 All rs should already start work with the new peer change when replication peer procedure is finished Signed-off-by: zhangduoProject: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/2fa68238 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/2fa68238 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/2fa68238 Branch: refs/heads/HBASE-19397-branch-2 Commit: 2fa68238a2992c68d733777320921343a251571d Parents: cd48ac7 Author: Guanghao Zhang Authored: Thu Jan 4 16:58:01 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 15:23:52 2018 +0800 -- .../replication/ReplicationPeerConfig.java | 1 - .../hbase/replication/ReplicationPeerImpl.java | 4 +- .../hbase/replication/ReplicationQueueInfo.java | 23 +- .../hbase/replication/ReplicationUtils.java | 56 ++ .../replication/TestReplicationStateZKImpl.java | 21 - .../regionserver/ReplicationSourceService.java | 3 +- .../regionserver/PeerProcedureHandler.java | 3 + .../regionserver/PeerProcedureHandlerImpl.java | 50 +- .../RecoveredReplicationSource.java | 6 +- .../RecoveredReplicationSourceShipper.java | 8 +- .../replication/regionserver/Replication.java | 11 +- .../regionserver/ReplicationSource.java | 34 +- .../regionserver/ReplicationSourceFactory.java | 4 +- .../ReplicationSourceInterface.java | 8 +- .../regionserver/ReplicationSourceManager.java | 895 ++- .../regionserver/ReplicationSourceShipper.java | 6 +- .../ReplicationSourceWALReader.java | 2 +- .../replication/ReplicationSourceDummy.java | 2 +- .../replication/TestNamespaceReplication.java | 57 +- .../TestReplicationSourceManager.java | 5 +- 20 files changed, 654 insertions(+), 545 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/2fa68238/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 fdae288..bf8d030 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 @@ -25,7 +25,6 @@ import java.util.List; import java.util.Map; import java.util.Set; import java.util.TreeMap; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; http://git-wip-us.apache.org/repos/asf/hbase/blob/2fa68238/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java -- diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java index 3e17025..604e0bb 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java @@ -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 @@ -28,6 +27,7 @@ import org.apache.yetus.audience.InterfaceAudience; @InterfaceAudience.Private public class ReplicationPeerImpl implements ReplicationPeer { + private final Configuration conf; private final String id; http://git-wip-us.apache.org/repos/asf/hbase/blob/2fa68238/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueInfo.java -- diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueInfo.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueInfo.java index ecd888f..cd65f9b 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueInfo.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueueInfo.java @@ -29,7 +29,7 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.hbase.ServerName; /** - * This class is responsible for the parsing logic for a znode representing a queue. + * This class is responsible for the parsing logic for a queue id
[01/50] [abbrv] hbase git commit: HBASE-20037 Race when calling SequenceIdAccounting.resetHighest [Forced Update!]
Repository: hbase Updated Branches: refs/heads/HBASE-19397-branch-2 a98b04f49 -> 9cd0f7014 (forced update) HBASE-20037 Race when calling SequenceIdAccounting.resetHighest Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/30c2dcd8 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/30c2dcd8 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/30c2dcd8 Branch: refs/heads/HBASE-19397-branch-2 Commit: 30c2dcd8832d6148fc1c4a430f90e7660b1af3c1 Parents: 2b4df5e Author: zhangduoAuthored: Wed Feb 21 18:19:01 2018 +0800 Committer: Michael Stack Committed: Wed Feb 21 10:04:33 2018 -0800 -- .../hbase/regionserver/wal/AbstractFSWAL.java | 37 + .../hbase/regionserver/wal/AsyncFSWAL.java | 42 ++-- .../wal/AsyncProtobufLogWriter.java | 4 +- .../hadoop/hbase/regionserver/wal/FSHLog.java | 6 +-- .../regionserver/wal/ProtobufLogWriter.java | 11 +++-- .../regionserver/wal/SequenceIdAccounting.java | 40 ++- 6 files changed, 74 insertions(+), 66 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/30c2dcd8/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java index 14fbe10..ce8dafa 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java @@ -661,9 +661,26 @@ public abstract class AbstractFSWAL implements WAL { } } + protected final void logRollAndSetupWalProps(Path oldPath, Path newPath, long oldFileLen) { +int oldNumEntries = this.numEntries.getAndSet(0); +String newPathString = newPath != null ? CommonFSUtils.getPath(newPath) : null; +if (oldPath != null) { + this.walFile2Props.put(oldPath, +new WalProps(this.sequenceIdAccounting.resetHighest(), oldFileLen)); + this.totalLogSize.addAndGet(oldFileLen); + LOG.info("Rolled WAL {} with entries={}, filesize={}; new WAL {}", +CommonFSUtils.getPath(oldPath), oldNumEntries, StringUtils.byteDesc(oldFileLen), +newPathString); +} else { + LOG.info("New WAL {}", newPathString); +} + } + /** + * * Cleans up current writer closing it and then puts in place the passed in * nextWriter. + * * * * In the case of creating a new WAL, oldPath will be null. @@ -672,26 +689,17 @@ public abstract class AbstractFSWAL implements WAL { * In the case of closing out this FSHLog with no further use newPath and nextWriter will be * null. * + * * @param oldPath may be null * @param newPath may be null * @param nextWriter may be null * @return the passed in newPath * @throws IOException if there is a problem flushing or closing the underlying FS */ + @VisibleForTesting Path replaceWriter(Path oldPath, Path newPath, W nextWriter) throws IOException { try (TraceScope scope = TraceUtil.createTrace("FSHFile.replaceWriter")) { - long oldFileLen = doReplaceWriter(oldPath, newPath, nextWriter); - int oldNumEntries = this.numEntries.getAndSet(0); - final String newPathString = (null == newPath ? null : CommonFSUtils.getPath(newPath)); - if (oldPath != null) { -this.walFile2Props.put(oldPath, - new WalProps(this.sequenceIdAccounting.resetHighest(), oldFileLen)); -this.totalLogSize.addAndGet(oldFileLen); -LOG.info("Rolled WAL " + CommonFSUtils.getPath(oldPath) + " with entries=" + oldNumEntries + - ", filesize=" + StringUtils.byteDesc(oldFileLen) + "; new WAL " + newPathString); - } else { -LOG.info("New WAL " + newPathString); - } + doReplaceWriter(oldPath, newPath, nextWriter); return newPath; } } @@ -1021,10 +1029,7 @@ public abstract class AbstractFSWAL implements WAL { protected abstract W createWriterInstance(Path path) throws IOException, CommonFSUtils.StreamLacksCapabilityException; - /** - * @return old wal file size - */ - protected abstract long doReplaceWriter(Path oldPath, Path newPath, W nextWriter) + protected abstract void doReplaceWriter(Path oldPath, Path newPath, W nextWriter) throws IOException; protected abstract void doShutdown() throws IOException; http://git-wip-us.apache.org/repos/asf/hbase/blob/30c2dcd8/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
[43/50] [abbrv] hbase git commit: HBASE-19636 All rs should already start work with the new peer change when replication peer procedure is finished
http://git-wip-us.apache.org/repos/asf/hbase/blob/2fa68238/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 16d0d75..02010e7 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 @@ -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.replication.regionserver; import java.io.IOException; @@ -33,7 +31,7 @@ import java.util.SortedSet; import java.util.TreeSet; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.RejectedExecutionException; @@ -71,27 +69,53 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; /** - * This class is responsible to manage all the replication - * sources. There are two classes of sources: + * This class is responsible to manage all the replication sources. There are two classes of + * sources: * - * Normal sources are persistent and one per peer cluster - * Old sources are recovered from a failed region server and our - * only goal is to finish replicating the WAL queue it had up in ZK + * Normal sources are persistent and one per peer cluster + * Old sources are recovered from a failed region server and our only goal is to finish + * replicating the WAL queue it had + * + * + * When a region server dies, this class uses a watcher to get notified and it tries to grab a lock + * in order to transfer all the queues in a local old source. + * + * Synchronization specification: + * + * No need synchronized on {@link #sources}. {@link #sources} is a ConcurrentHashMap and there + * is a Lock for peer id in {@link PeerProcedureHandlerImpl}. So there is no race for peer + * operations. + * 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)}. + * No need synchronized on {@link #walsByIdRecoveredQueues}. There are three methods which + * modify it, {@link #removePeer(String)} , {@link #cleanOldLogs(SortedSet, String, 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}. + * Need synchronized on {@link #latestPaths} to avoid the new open source miss new log. + * Need synchronized on {@link #oldsources} to avoid adding recovered source for the + * to-be-removed peer. * - * - * When a region server dies, this class uses a watcher to get notified and it - * tries to grab a lock in order to transfer all the queues in a local - * old
[13/50] [abbrv] hbase git commit: HBASE-19564 Procedure id is missing in the response of peer related operations
HBASE-19564 Procedure id is missing in the response of peer related operations Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/e40463df Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/e40463df Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/e40463df Branch: refs/heads/HBASE-19397-branch-2 Commit: e40463dfb475117859900232cd9173fc283179f3 Parents: 092635f Author: zhangduoAuthored: Wed Dec 20 20:57:37 2017 +0800 Committer: zhangduo Committed: Thu Feb 22 15:15:11 2018 +0800 -- .../hadoop/hbase/master/MasterRpcServices.java | 24 ++-- .../master/replication/ModifyPeerProcedure.java | 4 +--- 2 files changed, 13 insertions(+), 15 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/e40463df/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java index 8025a51..72bf2d1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java @@ -1886,10 +1886,10 @@ public class MasterRpcServices extends RSRpcServices public AddReplicationPeerResponse addReplicationPeer(RpcController controller, AddReplicationPeerRequest request) throws ServiceException { try { - master.addReplicationPeer(request.getPeerId(), -ReplicationPeerConfigUtil.convert(request.getPeerConfig()), request.getPeerState() -.getState().equals(ReplicationState.State.ENABLED)); - return AddReplicationPeerResponse.newBuilder().build(); + long procId = master.addReplicationPeer(request.getPeerId(), +ReplicationPeerConfigUtil.convert(request.getPeerConfig()), + request.getPeerState().getState().equals(ReplicationState.State.ENABLED)); + return AddReplicationPeerResponse.newBuilder().setProcId(procId).build(); } catch (ReplicationException | IOException e) { throw new ServiceException(e); } @@ -1899,8 +1899,8 @@ public class MasterRpcServices extends RSRpcServices public RemoveReplicationPeerResponse removeReplicationPeer(RpcController controller, RemoveReplicationPeerRequest request) throws ServiceException { try { - master.removeReplicationPeer(request.getPeerId()); - return RemoveReplicationPeerResponse.newBuilder().build(); + long procId = master.removeReplicationPeer(request.getPeerId()); + return RemoveReplicationPeerResponse.newBuilder().setProcId(procId).build(); } catch (ReplicationException | IOException e) { throw new ServiceException(e); } @@ -1910,8 +1910,8 @@ public class MasterRpcServices extends RSRpcServices public EnableReplicationPeerResponse enableReplicationPeer(RpcController controller, EnableReplicationPeerRequest request) throws ServiceException { try { - master.enableReplicationPeer(request.getPeerId()); - return EnableReplicationPeerResponse.newBuilder().build(); + long procId = master.enableReplicationPeer(request.getPeerId()); + return EnableReplicationPeerResponse.newBuilder().setProcId(procId).build(); } catch (ReplicationException | IOException e) { throw new ServiceException(e); } @@ -1921,8 +1921,8 @@ public class MasterRpcServices extends RSRpcServices public DisableReplicationPeerResponse disableReplicationPeer(RpcController controller, DisableReplicationPeerRequest request) throws ServiceException { try { - master.disableReplicationPeer(request.getPeerId()); - return DisableReplicationPeerResponse.newBuilder().build(); + long procId = master.disableReplicationPeer(request.getPeerId()); + return DisableReplicationPeerResponse.newBuilder().setProcId(procId).build(); } catch (ReplicationException | IOException e) { throw new ServiceException(e); } @@ -1948,9 +1948,9 @@ public class MasterRpcServices extends RSRpcServices public UpdateReplicationPeerConfigResponse updateReplicationPeerConfig(RpcController controller, UpdateReplicationPeerConfigRequest request) throws ServiceException { try { - master.updateReplicationPeerConfig(request.getPeerId(), + long procId = master.updateReplicationPeerConfig(request.getPeerId(), ReplicationPeerConfigUtil.convert(request.getPeerConfig())); - return UpdateReplicationPeerConfigResponse.newBuilder().build(); + return UpdateReplicationPeerConfigResponse.newBuilder().setProcId(procId).build(); } catch
[16/50] [abbrv] hbase git commit: HBASE-19536 Client side changes for moving peer modification from zk watcher to procedure
HBASE-19536 Client side changes for moving peer modification from zk watcher to procedure Signed-off-by: zhangduoProject: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/092635fa Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/092635fa Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/092635fa Branch: refs/heads/HBASE-19397-branch-2 Commit: 092635fae48a6f659f1ecf7687f48ef0ff29c2bc Parents: 1275b9f Author: Guanghao Zhang Authored: Tue Dec 19 15:50:57 2017 +0800 Committer: zhangduo Committed: Thu Feb 22 15:15:11 2018 +0800 -- .../org/apache/hadoop/hbase/client/Admin.java | 87 ++- .../apache/hadoop/hbase/client/HBaseAdmin.java | 149 ++- .../hadoop/hbase/client/RawAsyncHBaseAdmin.java | 82 +- 3 files changed, 238 insertions(+), 80 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/092635fa/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 40dac2f..b8546fa 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 @@ -2473,7 +2473,7 @@ public interface Admin extends Abortable, Closeable { /** * Add a new replication peer for replicating data to slave cluster. * @param peerId a short name that identifies the peer - * @param peerConfig configuration for the replication slave cluster + * @param peerConfig configuration for the replication peer * @throws IOException if a remote or network exception occurs */ default void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig) @@ -2484,7 +2484,7 @@ public interface Admin extends Abortable, Closeable { /** * Add a new replication peer for replicating data to slave cluster. * @param peerId a short name that identifies the peer - * @param peerConfig configuration for the replication slave cluster + * @param peerConfig configuration for the replication peer * @param enabled peer state, true if ENABLED and false if DISABLED * @throws IOException if a remote or network exception occurs */ @@ -2492,6 +2492,37 @@ public interface Admin extends Abortable, Closeable { throws IOException; /** + * Add a new replication peer but does not block and wait for it. + * + * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw + * ExecutionException if there was an error while executing the operation or TimeoutException in + * case the wait timeout was not long enough to allow the operation to complete. + * @param peerId a short name that identifies the peer + * @param peerConfig configuration for the replication peer + * @return the result of the async operation + * @throws IOException IOException if a remote or network exception occurs + */ + default Future addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig) + throws IOException { +return addReplicationPeerAsync(peerId, peerConfig, true); + } + + /** + * Add a new replication peer but does not block and wait for it. + * + * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw + * ExecutionException if there was an error while executing the operation or TimeoutException in + * case the wait timeout was not long enough to allow the operation to complete. + * @param peerId a short name that identifies the peer + * @param peerConfig configuration for the replication peer + * @param enabled peer state, true if ENABLED and false if DISABLED + * @return the result of the async operation + * @throws IOException IOException if a remote or network exception occurs + */ + Future addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig, + boolean enabled) throws IOException; + + /** * Remove a peer and stop the replication. * @param peerId a short name that identifies the peer * @throws IOException if a remote or network exception occurs @@ -2499,6 +2530,18 @@ public interface Admin extends Abortable, Closeable { void removeReplicationPeer(String peerId) throws IOException; /** + * Remove a replication peer but does not block and wait for it. + * + * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw + * ExecutionException if there was an error while executing the operation or TimeoutException in + * case the wait timeout was not long enough to
[27/50] [abbrv] hbase git commit: HBASE-19642 Fix locking for peer modification procedure
HBASE-19642 Fix locking for peer modification procedure Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/07714b7a Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/07714b7a Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/07714b7a Branch: refs/heads/HBASE-19397-branch-2 Commit: 07714b7a095350acd0e71a164f75343641ed2a57 Parents: 71c8eab Author: zhangduoAuthored: Wed Dec 27 18:27:13 2017 +0800 Committer: zhangduo Committed: Thu Feb 22 15:22:31 2018 +0800 -- .../master/replication/ModifyPeerProcedure.java | 21 +--- 1 file changed, 18 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/07714b7a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java index 279fbc7..a682606 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ModifyPeerProcedure.java @@ -46,6 +46,8 @@ public abstract class ModifyPeerProcedure protected String peerId; + private volatile boolean locked; + // used to keep compatible with old client where we can only returns after updateStorage. protected ProcedurePrepareLatch latch; @@ -145,17 +147,30 @@ public abstract class ModifyPeerProcedure @Override protected LockState acquireLock(MasterProcedureEnv env) { -return env.getProcedureScheduler().waitPeerExclusiveLock(this, peerId) - ? LockState.LOCK_EVENT_WAIT - : LockState.LOCK_ACQUIRED; +if (env.getProcedureScheduler().waitPeerExclusiveLock(this, peerId)) { + return LockState.LOCK_EVENT_WAIT; +} +locked = true; +return LockState.LOCK_ACQUIRED; } @Override protected void releaseLock(MasterProcedureEnv env) { +locked = false; env.getProcedureScheduler().wakePeerExclusiveLock(this, peerId); } @Override + protected boolean holdLock(MasterProcedureEnv env) { +return true; + } + + @Override + protected boolean hasLock(MasterProcedureEnv env) { +return locked; + } + + @Override protected void rollbackState(MasterProcedureEnv env, PeerModificationState state) throws IOException, InterruptedException { if (state == PeerModificationState.PRE_PEER_MODIFICATION) {
[39/50] [abbrv] hbase git commit: HBASE-19783 Change replication peer cluster key/endpoint from a not-null value to null is not allowed
HBASE-19783 Change replication peer cluster key/endpoint from a not-null value to null is not allowed Signed-off-by: zhangduoProject: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/c445b312 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c445b312 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c445b312 Branch: refs/heads/HBASE-19397-branch-2 Commit: c445b312f79218442cb2d093a3b137fc5fd6a809 Parents: affbdb0 Author: Guanghao Zhang Authored: Fri Jan 12 22:04:38 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 15:23:52 2018 +0800 -- .../replication/ReplicationPeerManager.java | 28 +--- 1 file changed, 19 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/c445b312/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 696b2d7..19fc7f4 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 @@ -132,20 +132,19 @@ public class ReplicationPeerManager { checkPeerConfig(peerConfig); ReplicationPeerDescription desc = checkPeerExists(peerId); ReplicationPeerConfig oldPeerConfig = desc.getPeerConfig(); -if (!StringUtils.isBlank(peerConfig.getClusterKey()) && - !peerConfig.getClusterKey().equals(oldPeerConfig.getClusterKey())) { +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() + "'"); + oldPeerConfig.getClusterKey() + "' for peer " + peerId + " does not match new key '" + + peerConfig.getClusterKey() + "'"); } -if (!StringUtils.isBlank(peerConfig.getReplicationEndpointImpl()) && - !peerConfig.getReplicationEndpointImpl().equals(oldPeerConfig.getReplicationEndpointImpl())) { +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() + "'"); } } @@ -341,4 +340,15 @@ public class ReplicationPeerManager { return new ReplicationPeerManager(peerStorage, ReplicationStorageFactory.getReplicationQueueStorage(zk, conf), peers); } + + /** + * For replication peer cluster key or endpoint class, null and empty string is same. So here + * don't use {@link StringUtils#equals(CharSequence, CharSequence)} directly. + */ + private boolean isStringEquals(String s1, String s2) { +if (StringUtils.isBlank(s1)) { + return StringUtils.isBlank(s2); +} +return s1.equals(s2); + } }
[12/50] [abbrv] hbase git commit: HBASE-19520 Add UTs for the new lock type PEER
HBASE-19520 Add UTs for the new lock type PEER Signed-off-by: zhangduoProject: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/eb2e14d1 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/eb2e14d1 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/eb2e14d1 Branch: refs/heads/HBASE-19397-branch-2 Commit: eb2e14d1027b423532725325fd797d64c02f5283 Parents: e40463d Author: Guanghao Zhang Authored: Wed Dec 20 16:43:38 2017 +0800 Committer: zhangduo Committed: Thu Feb 22 15:15:11 2018 +0800 -- .../procedure/MasterProcedureScheduler.java | 9 +- .../procedure/TestMasterProcedureScheduler.java | 65 - ...TestMasterProcedureSchedulerConcurrency.java | 135 +++ 3 files changed, 201 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/eb2e14d1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java index 58f9ccd..b18dd6c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java @@ -277,6 +277,13 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { while (tableIter.hasNext()) { count += tableIter.next().size(); } + +// Peer queues +final AvlTreeIterator peerIter = new AvlTreeIterator<>(peerMap); +while (peerIter.hasNext()) { + count += peerIter.next().size(); +} + return count; } @@ -807,7 +814,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { * @see #wakePeerExclusiveLock(Procedure, String) * @param procedure the procedure trying to acquire the lock * @param peerId peer to lock - * @return true if the procedure has to wait for the per to be available + * @return true if the procedure has to wait for the peer to be available */ public boolean waitPeerExclusiveLock(Procedure procedure, String peerId) { schedLock(); http://git-wip-us.apache.org/repos/asf/hbase/blob/eb2e14d1/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java index 160f4d2..65757db 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterProcedureScheduler.java @@ -908,6 +908,27 @@ public class TestMasterProcedureScheduler { } } + public static class TestPeerProcedure extends TestProcedure implements PeerProcedureInterface { +private final String peerId; +private final PeerOperationType opType; + +public TestPeerProcedure(long procId, String peerId, PeerOperationType opType) { + super(procId); + this.peerId = peerId; + this.opType = opType; +} + +@Override +public String getPeerId() { + return peerId; +} + +@Override +public PeerOperationType getPeerOperationType() { + return opType; +} + } + private static LockProcedure createLockProcedure(LockType lockType, long procId) throws Exception { LockProcedure procedure = new LockProcedure(); @@ -930,22 +951,19 @@ public class TestMasterProcedureScheduler { return createLockProcedure(LockType.SHARED, procId); } - private static void assertLockResource(LockedResource resource, - LockedResourceType resourceType, String resourceName) - { + private static void assertLockResource(LockedResource resource, LockedResourceType resourceType, + String resourceName) { assertEquals(resourceType, resource.getResourceType()); assertEquals(resourceName, resource.getResourceName()); } - private static void assertExclusiveLock(LockedResource resource, Procedure procedure) - { + private static void assertExclusiveLock(LockedResource resource, Procedure procedure) { assertEquals(LockType.EXCLUSIVE, resource.getLockType()); assertEquals(procedure, resource.getExclusiveLockOwnerProcedure()); assertEquals(0, resource.getSharedLockCount()); } - private static
[29/50] [abbrv] hbase git commit: HBASE-19623 Create replication endpoint asynchronously when adding a replication source
HBASE-19623 Create replication endpoint asynchronously when adding a replication source Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/9b0aa69a Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/9b0aa69a Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/9b0aa69a Branch: refs/heads/HBASE-19397-branch-2 Commit: 9b0aa69a832976d073776a52087b8efc98fbcc72 Parents: bd71e06 Author: zhangduoAuthored: Tue Jan 2 13:25:58 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 15:23:34 2018 +0800 -- .../hbase/replication/ReplicationPeer.java | 8 ++ .../hbase/replication/ReplicationPeers.java | 18 +-- .../replication/ZKReplicationPeerStorage.java | 7 +- .../replication/TestReplicationStateBasic.java | 20 +--- .../TestZKReplicationPeerStorage.java | 14 +-- .../HBaseInterClusterReplicationEndpoint.java | 17 ++- .../RecoveredReplicationSource.java | 13 +-- .../regionserver/ReplicationSource.java | 110 +++ .../ReplicationSourceInterface.java | 8 +- .../regionserver/ReplicationSourceManager.java | 47 +--- .../client/TestAsyncReplicationAdminApi.java| 2 - .../replication/TestReplicationAdmin.java | 2 - .../replication/ReplicationSourceDummy.java | 7 +- .../replication/TestReplicationSource.java | 5 +- .../TestReplicationSourceManager.java | 8 +- 15 files changed, 116 insertions(+), 170 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/9b0aa69a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java -- diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java index 4846018..2da3cce 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java @@ -54,6 +54,14 @@ public interface ReplicationPeer { PeerState getPeerState(); /** + * Test whether the peer is enabled. + * @return {@code true} if enabled, otherwise {@code false}. + */ + default boolean isPeerEnabled() { +return getPeerState() == PeerState.ENABLED; + } + + /** * Get the peer config object * @return the ReplicationPeerConfig for this peer */ http://git-wip-us.apache.org/repos/asf/hbase/blob/9b0aa69a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java -- diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java index 422801b..45940a5 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.replication; import java.io.IOException; +import java.util.Collections; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -86,21 +87,6 @@ public class ReplicationPeers { } /** - * Get the peer state for the specified connected remote slave cluster. The value might be read - * from cache, so it is recommended to use {@link #peerStorage } to read storage directly if - * reading the state after enabling or disabling it. - * @param peerId a short that identifies the cluster - * @return true if replication is enabled, false otherwise. - */ - public boolean isPeerEnabled(String peerId) { -ReplicationPeer replicationPeer = this.peerCache.get(peerId); -if (replicationPeer == null) { - throw new IllegalArgumentException("Peer with id= " + peerId + " is not cached"); -} -return replicationPeer.getPeerState() == PeerState.ENABLED; - } - - /** * Returns the ReplicationPeerImpl for the specified cached peer. This ReplicationPeer will * continue to track changes to the Peer's state and config. This method returns null if no peer * has been cached with the given peerId. @@ -117,7 +103,7 @@ public class ReplicationPeers { * @return a Set of Strings for peerIds */ public Set getAllPeerIds() { -return peerCache.keySet(); +return Collections.unmodifiableSet(peerCache.keySet()); } public static Configuration getPeerClusterConfiguration(ReplicationPeerConfig peerConfig,
[04/50] [abbrv] hbase git commit: HBASE-20039 MR tests out to hbase-mapreduce mobile
HBASE-20039 MR tests out to hbase-mapreduce mobile Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/d8ec10e2 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/d8ec10e2 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/d8ec10e2 Branch: refs/heads/HBASE-19397-branch-2 Commit: d8ec10e20f2c30641b3c8d13a902c9c158b9dd3b Parents: 66ba8aa Author: Mike DrobAuthored: Wed Feb 21 10:53:42 2018 -0600 Committer: Mike Drob Committed: Wed Feb 21 14:53:30 2018 -0600 -- .../mapreduce/TestHBaseMRTestingUtility.java| 73 .../hadoop/hbase/TestHBaseTestingUtility.java | 32 - 2 files changed, 73 insertions(+), 32 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/d8ec10e2/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHBaseMRTestingUtility.java -- diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHBaseMRTestingUtility.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHBaseMRTestingUtility.java new file mode 100644 index 000..2467dca --- /dev/null +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHBaseMRTestingUtility.java @@ -0,0 +1,73 @@ +/** + * 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.mapreduce; + +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.MapReduceTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +@Category({MapReduceTests.class, LargeTests.class}) +public class TestHBaseMRTestingUtility { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHBaseMRTestingUtility.class); + + @Test + public void testMRYarnConfigsPopulation() throws IOException { +Map dummyProps = new HashMap<>(); +dummyProps.put("mapreduce.jobtracker.address", "dummyhost:11234"); +dummyProps.put("yarn.resourcemanager.address", "dummyhost:11235"); +dummyProps.put("mapreduce.jobhistory.address", "dummyhost:11236"); +dummyProps.put("yarn.resourcemanager.scheduler.address", "dummyhost:11237"); +dummyProps.put("mapreduce.jobhistory.webapp.address", "dummyhost:11238"); +dummyProps.put("yarn.resourcemanager.webapp.address", "dummyhost:11239"); + +HBaseTestingUtility hbt = new HBaseTestingUtility(); + +// populate the mr props to the Configuration instance +for (Map.Entry entry : dummyProps.entrySet()) { + hbt.getConfiguration().set(entry.getKey(), entry.getValue()); +} + +for (Map.Entry entry : dummyProps.entrySet()) { + assertTrue("The Configuration for key " + entry.getKey() +" and value: " + entry.getValue() + + " is not populated correctly", hbt.getConfiguration().get(entry.getKey()).equals(entry.getValue())); +} + +hbt.startMiniMapReduceCluster(); + +// Confirm that MiniMapReduceCluster overwrites the mr properties and updates the Configuration +for (Map.Entry entry : dummyProps.entrySet()) { + assertFalse("The MR prop: " + entry.getValue() + " is not overwritten when map reduce mini"+ + "cluster is started", hbt.getConfiguration().get(entry.getKey()).equals(entry.getValue())); +} + +hbt.shutdownMiniMapReduceCluster(); + } +} http://git-wip-us.apache.org/repos/asf/hbase/blob/d8ec10e2/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java -- diff --git
[33/50] [abbrv] hbase git commit: HBASE-19617 Remove ReplicationQueues, use ReplicationQueueStorage directly
HBASE-19617 Remove ReplicationQueues, use ReplicationQueueStorage directly Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/bef6d77b Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/bef6d77b Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/bef6d77b Branch: refs/heads/HBASE-19397-branch-2 Commit: bef6d77b2101eca8e702e4e3b9959c106c9b2d35 Parents: 07714b7 Author: zhangduoAuthored: Wed Dec 27 22:03:51 2017 +0800 Committer: zhangduo Committed: Thu Feb 22 15:23:34 2018 +0800 -- .../hbase/replication/ReplicationFactory.java | 9 +- .../hbase/replication/ReplicationQueues.java| 161 --- .../replication/ReplicationQueuesArguments.java | 70 --- .../replication/ReplicationQueuesZKImpl.java| 417 - .../hbase/replication/ReplicationTableBase.java | 442 --- .../replication/ReplicationTrackerZKImpl.java | 21 +- .../replication/ZKReplicationQueueStorage.java | 22 + .../replication/TestReplicationStateBasic.java | 131 +++--- .../replication/TestReplicationStateZKImpl.java | 41 +- .../regionserver/DumpReplicationQueues.java | 15 +- .../RecoveredReplicationSource.java | 17 +- .../RecoveredReplicationSourceShipper.java | 22 +- .../replication/regionserver/Replication.java | 20 +- .../regionserver/ReplicationSource.java | 16 +- .../ReplicationSourceInterface.java | 11 +- .../regionserver/ReplicationSourceManager.java | 274 ++-- .../regionserver/ReplicationSyncUp.java | 32 +- .../hbase/master/cleaner/TestLogsCleaner.java | 12 +- .../cleaner/TestReplicationHFileCleaner.java| 23 +- .../cleaner/TestReplicationZKNodeCleaner.java | 22 +- .../replication/ReplicationSourceDummy.java | 6 +- .../replication/TestReplicationSyncUpTool.java | 4 +- .../TestReplicationSourceManager.java | 97 ++-- .../TestReplicationSourceManagerZkImpl.java | 57 +-- 24 files changed, 365 insertions(+), 1577 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/bef6d77b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java -- diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java index 6c1c213..5e70e57 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java @@ -17,12 +17,11 @@ */ package org.apache.hadoop.hbase.replication; -import org.apache.commons.lang3.reflect.ConstructorUtils; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; +import org.apache.yetus.audience.InterfaceAudience; /** * A factory class for instantiating replication objects that deal with replication state. @@ -30,12 +29,6 @@ import org.apache.hadoop.hbase.zookeeper.ZKWatcher; @InterfaceAudience.Private public class ReplicationFactory { - public static ReplicationQueues getReplicationQueues(ReplicationQueuesArguments args) - throws Exception { -return (ReplicationQueues) ConstructorUtils.invokeConstructor(ReplicationQueuesZKImpl.class, - args); - } - public static ReplicationPeers getReplicationPeers(ZKWatcher zk, Configuration conf, Abortable abortable) { return getReplicationPeers(zk, conf, null, abortable); http://git-wip-us.apache.org/repos/asf/hbase/blob/bef6d77b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java -- diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java deleted file mode 100644 index a2d21f7..000 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationQueues.java +++ /dev/null @@ -1,161 +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
[22/50] [abbrv] hbase git commit: HBASE-19592 Add UTs to test retry on update zk failure
HBASE-19592 Add UTs to test retry on update zk failure Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/71c8eab9 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/71c8eab9 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/71c8eab9 Branch: refs/heads/HBASE-19397-branch-2 Commit: 71c8eab9e4fef49acfb7bf490772481b1810a07b Parents: ad4157d Author: zhangduoAuthored: Tue Dec 26 20:39:00 2017 +0800 Committer: zhangduo Committed: Thu Feb 22 15:21:30 2018 +0800 -- .../replication/ReplicationPeerManager.java | 5 +- .../TestReplicationProcedureRetry.java | 200 +++ 2 files changed, 202 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/71c8eab9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java index b78cbce..f4ccce8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationPeerManager.java @@ -53,7 +53,7 @@ import org.apache.yetus.audience.InterfaceAudience; * Used to add/remove a replication peer. */ @InterfaceAudience.Private -public final class ReplicationPeerManager { +public class ReplicationPeerManager { private final ReplicationPeerStorage peerStorage; @@ -61,8 +61,7 @@ public final class ReplicationPeerManager { private final ConcurrentMap peers; - private ReplicationPeerManager(ReplicationPeerStorage peerStorage, - ReplicationQueueStorage queueStorage, + ReplicationPeerManager(ReplicationPeerStorage peerStorage, ReplicationQueueStorage queueStorage, ConcurrentMap peers) { this.peerStorage = peerStorage; this.queueStorage = queueStorage; http://git-wip-us.apache.org/repos/asf/hbase/blob/71c8eab9/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java new file mode 100644 index 000..ab35b46 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java @@ -0,0 +1,200 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.replication; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager; +import org.apache.hadoop.hbase.testclassification.MediumTests; +import org.apache.hadoop.hbase.testclassification.ReplicationTests; +import org.apache.zookeeper.KeeperException; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.mockito.invocation.InvocationOnMock; +
[40/50] [abbrv] hbase git commit: HBASE-19697 Remove TestReplicationAdminUsingProcedure
HBASE-19697 Remove TestReplicationAdminUsingProcedure Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/b00237fd Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/b00237fd Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/b00237fd Branch: refs/heads/HBASE-19397-branch-2 Commit: b00237fd21f9e1239dc76e476e690194c84ba8af Parents: ff08447 Author: zhangduoAuthored: Wed Jan 3 21:13:57 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 15:23:52 2018 +0800 -- .../TestReplicationAdminUsingProcedure.java | 225 --- 1 file changed, 225 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/b00237fd/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java deleted file mode 100644 index 1300376..000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdminUsingProcedure.java +++ /dev/null @@ -1,225 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.client.replication; - -import java.io.IOException; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.ResultScanner; -import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; -import org.apache.hadoop.hbase.replication.TestReplicationBase; -import org.apache.hadoop.hbase.testclassification.ClientTests; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.log4j.Logger; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; -import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap; - -@Category({ MediumTests.class, ClientTests.class }) -public class TestReplicationAdminUsingProcedure extends TestReplicationBase { - - private static final String PEER_ID = "2"; - private static final Logger LOG = Logger.getLogger(TestReplicationAdminUsingProcedure.class); - - @BeforeClass - public static void setUpBeforeClass() throws Exception { -conf1.setInt("hbase.multihconnection.threads.max", 10); - -// Start the master & slave mini cluster. -TestReplicationBase.setUpBeforeClass(); - -// Remove the replication peer -hbaseAdmin.removeReplicationPeer(PEER_ID); - } - - private void loadData(int startRowKey, int endRowKey) throws IOException { -for (int i = startRowKey; i < endRowKey; i++) { - byte[] rowKey = Bytes.add(row, Bytes.toBytes(i)); - Put put = new Put(rowKey); - put.addColumn(famName, null, Bytes.toBytes(i)); - htable1.put(put); -} - } - - private void waitForReplication(int expectedRows, int retries) - throws IOException, InterruptedException { -Scan scan; -for (int i = 0; i < retries; i++) { - scan = new Scan(); - if (i == retries - 1) { -throw new IOException("Waited too much time for normal batch replication"); - } - try (ResultScanner scanner = htable2.getScanner(scan)) { -int count = 0; -for (Result res : scanner) { - count++; -} -if (count != expectedRows) { - LOG.info("Only got " + count + " rows, expected rows: " + expectedRows); - Thread.sleep(SLEEP_TIME); -} else { - return; -} - } -} - } - - @Before - public void setUp() throws IOException { -
[28/50] [abbrv] hbase git commit: HBASE-19544 Add UTs for testing concurrent modifications on replication peer
HBASE-19544 Add UTs for testing concurrent modifications on replication peer Signed-off-by: zhangduoProject: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/96000876 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/96000876 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/96000876 Branch: refs/heads/HBASE-19397-branch-2 Commit: 96000876f3b3223af8090f3c8342c3f664eaa3e8 Parents: 88228a2 Author: Guanghao Zhang Authored: Tue Jan 2 17:07:41 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 15:23:34 2018 +0800 -- .../replication/TestReplicationAdmin.java | 69 1 file changed, 69 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/96000876/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java index 772a9d6..a753d23 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/replication/TestReplicationAdmin.java @@ -31,6 +31,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; import java.util.regex.Pattern; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; @@ -58,6 +59,8 @@ 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; /** * Unit testing of ReplicationAdmin @@ -69,6 +72,8 @@ public class TestReplicationAdmin { public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestReplicationAdmin.class); + private static final Logger LOG = LoggerFactory.getLogger(TestReplicationAdmin.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @@ -118,6 +123,70 @@ public class TestReplicationAdmin { } @Test + public void testConcurrentPeerOperations() throws Exception { +int threadNum = 5; +AtomicLong successCount = new AtomicLong(0); + +// Test concurrent add peer operation +Thread[] addPeers = new Thread[threadNum]; +for (int i = 0; i < threadNum; i++) { + addPeers[i] = new Thread(() -> { +try { + hbaseAdmin.addReplicationPeer(ID_ONE, +ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build()); + successCount.incrementAndGet(); +} catch (Exception e) { + LOG.debug("Got exception when add replication peer", e); +} + }); + addPeers[i].start(); +} +for (Thread addPeer : addPeers) { + addPeer.join(); +} +assertEquals(1, successCount.get()); + +// Test concurrent remove peer operation +successCount.set(0); +Thread[] removePeers = new Thread[threadNum]; +for (int i = 0; i < threadNum; i++) { + removePeers[i] = new Thread(() -> { +try { + hbaseAdmin.removeReplicationPeer(ID_ONE); + successCount.incrementAndGet(); +} catch (Exception e) { + LOG.debug("Got exception when remove replication peer", e); +} + }); + removePeers[i].start(); +} +for (Thread removePeer : removePeers) { + removePeer.join(); +} +assertEquals(1, successCount.get()); + +// Test concurrent add peer operation again +successCount.set(0); +addPeers = new Thread[threadNum]; +for (int i = 0; i < threadNum; i++) { + addPeers[i] = new Thread(() -> { +try { + hbaseAdmin.addReplicationPeer(ID_ONE, +ReplicationPeerConfig.newBuilder().setClusterKey(KEY_ONE).build()); + successCount.incrementAndGet(); +} catch (Exception e) { + LOG.debug("Got exception when add replication peer", e); +} + }); + addPeers[i].start(); +} +for (Thread addPeer : addPeers) { + addPeer.join(); +} +assertEquals(1, successCount.get()); + } + + @Test public void testAddInvalidPeer() { ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder(); builder.setClusterKey(KEY_ONE);
[02/50] [abbrv] hbase git commit: HBASE-19953 Ensure post DDL hooks are only called after successful operations
HBASE-19953 Ensure post DDL hooks are only called after successful operations The 1.x functionality of Master DDL operations is that "post" observer hooks are only invoked when the DDL action was successful. With the async-ness of ProcV2, we find ourselves in a case where the post-hook may be invoked before the Procedure runs and fails. We need to introduce some blocking to wait and see if the Procedure is going to fail on a precondition before invoking the hook. Signed-off-by: Michael StackProject: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/e10db9d2 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/e10db9d2 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/e10db9d2 Branch: refs/heads/HBASE-19397-branch-2 Commit: e10db9d29ae68e023d8cde4a3dd476d0be4f2994 Parents: 30c2dcd Author: Josh Elser Authored: Thu Feb 15 18:00:09 2018 -0500 Committer: Josh Elser Committed: Wed Feb 21 13:53:09 2018 -0500 -- .../hadoop/hbase/master/ClusterSchema.java | 10 +- .../hbase/master/ClusterSchemaServiceImpl.java | 16 +- .../org/apache/hadoop/hbase/master/HMaster.java | 43 ++- .../hbase/master/TableNamespaceManager.java | 3 +- .../AbstractStateMachineNamespaceProcedure.java | 13 + .../procedure/CreateNamespaceProcedure.java | 22 +- .../procedure/DeleteNamespaceProcedure.java | 34 +- .../procedure/ModifyNamespaceProcedure.java | 30 +- .../master/procedure/ProcedurePrepareLatch.java | 14 + .../procedure/TestMasterObserverPostCalls.java | 368 +++ 10 files changed, 518 insertions(+), 35 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/e10db9d2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java index 746f9ad..56a1f33 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java @@ -22,6 +22,7 @@ import java.io.InterruptedIOException; import java.util.List; import org.apache.hadoop.hbase.NamespaceDescriptor; +import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.util.NonceKey; @@ -79,32 +80,35 @@ public interface ClusterSchema { * Create a new Namespace. * @param namespaceDescriptor descriptor for new Namespace * @param nonceKey A unique identifier for this operation from the client or process. + * @param latch A latch to block on for precondition validation * @return procedure id * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException} *as well as {@link IOException} */ - long createNamespace(NamespaceDescriptor namespaceDescriptor, NonceKey nonceKey) + long createNamespace(NamespaceDescriptor namespaceDescriptor, NonceKey nonceKey, ProcedurePrepareLatch latch) throws IOException; /** * Modify an existing Namespace. * @param nonceKey A unique identifier for this operation from the client or process. + * @param latch A latch to block on for precondition validation * @return procedure id * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException} *as well as {@link IOException} */ - long modifyNamespace(NamespaceDescriptor descriptor, NonceKey nonceKey) + long modifyNamespace(NamespaceDescriptor descriptor, NonceKey nonceKey, ProcedurePrepareLatch latch) throws IOException; /** * Delete an existing Namespace. * Only empty Namespaces (no tables) can be removed. * @param nonceKey A unique identifier for this operation from the client or process. + * @param latch A latch to block on for precondition validation * @return procedure id * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException} *as well as {@link IOException} */ - long deleteNamespace(String name, NonceKey nonceKey) + long deleteNamespace(String name, NonceKey nonceKey, ProcedurePrepareLatch latch) throws IOException; /** http://git-wip-us.apache.org/repos/asf/hbase/blob/e10db9d2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
[18/50] [abbrv] hbase git commit: HBASE-19216 Implement a general framework to execute remote procedure on RS
HBASE-19216 Implement a general framework to execute remote procedure on RS Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/92a370b4 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/92a370b4 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/92a370b4 Branch: refs/heads/HBASE-19397-branch-2 Commit: 92a370b43e87d76d50acb1567f58b11152c35131 Parents: 0432d49 Author: zhangduoAuthored: Fri Dec 15 21:06:44 2017 +0800 Committer: zhangduo Committed: Thu Feb 22 15:15:11 2018 +0800 -- .../hbase/procedure2/LockedResourceType.java| 4 +- .../procedure2/RemoteProcedureDispatcher.java | 23 ++- .../src/main/protobuf/Admin.proto | 9 +- .../src/main/protobuf/MasterProcedure.proto | 30 +++ .../src/main/protobuf/RegionServerStatus.proto | 15 ++ .../apache/hadoop/hbase/executor/EventType.java | 26 ++- .../hadoop/hbase/executor/ExecutorType.java | 3 +- .../org/apache/hadoop/hbase/master/HMaster.java | 30 ++- .../hadoop/hbase/master/MasterRpcServices.java | 13 ++ .../assignment/RegionTransitionProcedure.java | 18 +- .../procedure/MasterProcedureScheduler.java | 113 ++- .../procedure/PeerProcedureInterface.java | 34 .../hbase/master/procedure/PeerQueue.java | 54 + .../master/procedure/RSProcedureDispatcher.java | 101 ++ .../hbase/master/procedure/SchemaLocking.java | 5 + .../master/replication/ModifyPeerProcedure.java | 127 .../master/replication/RefreshPeerCallable.java | 67 +++ .../replication/RefreshPeerProcedure.java | 197 +++ .../hbase/procedure2/RSProcedureCallable.java | 43 .../hbase/regionserver/HRegionServer.java | 61 ++ .../hbase/regionserver/RSRpcServices.java | 58 -- .../handler/RSProcedureHandler.java | 51 + .../assignment/TestAssignmentManager.java | 20 +- .../replication/DummyModifyPeerProcedure.java | 41 .../TestDummyModifyPeerProcedure.java | 80 .../security/access/TestAccessController.java | 1 + 26 files changed, 1109 insertions(+), 115 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/92a370b4/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java -- diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java index c5fe62b..dc9b5d4 100644 --- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java +++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/LockedResourceType.java @@ -1,4 +1,4 @@ -/* +/** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -22,5 +22,5 @@ import org.apache.yetus.audience.InterfaceAudience; @InterfaceAudience.Private public enum LockedResourceType { - SERVER, NAMESPACE, TABLE, REGION + SERVER, NAMESPACE, TABLE, REGION, PEER } http://git-wip-us.apache.org/repos/asf/hbase/blob/92a370b4/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java -- diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java index 861e3b2..dca0bec 100644 --- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java +++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureDispatcher.java @@ -226,13 +226,30 @@ public abstract class RemoteProcedureDispatcher { +/** + * For building the remote operation. + */ RemoteOperation remoteCallBuild(TEnv env, TRemote remote); -void remoteCallCompleted(TEnv env, TRemote remote, RemoteOperation response); + +/** + * Called when the executeProcedure call is failed. + */ void remoteCallFailed(TEnv env, TRemote remote, IOException exception); + +/** + * Called when RS tells the remote procedure is succeeded through the + * {@code reportProcedureDone} method. + */ +void remoteOperationCompleted(TEnv env); + +/** + * Called when RS tells the remote procedure is failed through the
[47/50] [abbrv] hbase git commit: HBASE-19936 Introduce a new base class for replication peer procedure
HBASE-19936 Introduce a new base class for replication peer procedure Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/e0717d81 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/e0717d81 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/e0717d81 Branch: refs/heads/HBASE-19397-branch-2 Commit: e0717d81d90603a93f432c660b256165da64bab3 Parents: 1d883eb Author: zhangduoAuthored: Mon Feb 5 16:14:25 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 15:23:52 2018 +0800 -- .../src/main/protobuf/MasterProcedure.proto | 2 +- .../replication/AbstractPeerProcedure.java | 97 .../master/replication/ModifyPeerProcedure.java | 67 +- 3 files changed, 102 insertions(+), 64 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/e0717d81/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto -- diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto index ae676ea..83099c3 100644 --- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto +++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto @@ -398,7 +398,7 @@ message RefreshPeerParameter { required ServerName target_server = 3; } -message ModifyPeerStateData { +message PeerProcedureStateData { required string peer_id = 1; } http://git-wip-us.apache.org/repos/asf/hbase/blob/e0717d81/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java new file mode 100644 index 000..0ad8a63 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AbstractPeerProcedure.java @@ -0,0 +1,97 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.master.replication; + +import java.io.IOException; +import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; +import org.apache.hadoop.hbase.master.procedure.PeerProcedureInterface; +import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch; +import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; +import org.apache.hadoop.hbase.procedure2.StateMachineProcedure; +import org.apache.yetus.audience.InterfaceAudience; + +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.PeerProcedureStateData; + +/** + * The base class for all replication peer related procedure. + */ +@InterfaceAudience.Private +public abstract class AbstractPeerProcedure +extends StateMachineProcedure implements PeerProcedureInterface { + + protected String peerId; + + private volatile boolean locked; + + // used to keep compatible with old client where we can only returns after updateStorage. + protected ProcedurePrepareLatch latch; + + protected AbstractPeerProcedure() { + } + + protected AbstractPeerProcedure(String peerId) { +this.peerId = peerId; +this.latch = ProcedurePrepareLatch.createLatch(2, 0); + } + + public ProcedurePrepareLatch getLatch() { +return latch; + } + + @Override + public String getPeerId() { +return peerId; + } + + @Override + protected LockState acquireLock(MasterProcedureEnv env) { +if (env.getProcedureScheduler().waitPeerExclusiveLock(this, peerId)) { + return LockState.LOCK_EVENT_WAIT; +} +locked = true; +return LockState.LOCK_ACQUIRED; + } + + @Override + protected void releaseLock(MasterProcedureEnv env) { +locked = false; +env.getProcedureScheduler().wakePeerExclusiveLock(this, peerId); + } + + @Override + protected boolean
[46/50] [abbrv] hbase git commit: HBASE-19719 Fix checkstyle issues
HBASE-19719 Fix checkstyle issues Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/23e43032 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/23e43032 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/23e43032 Branch: refs/heads/HBASE-19397-branch-2 Commit: 23e430328d45efd3a17ef1e183d7fdcfb3dbe81a Parents: 5dc9906 Author: zhangduoAuthored: Sat Jan 6 08:30:55 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 15:23:52 2018 +0800 -- .../hbase/replication/ReplicationStorageFactory.java | 2 +- .../master/assignment/RegionTransitionProcedure.java | 4 ++-- .../hbase/master/procedure/RSProcedureDispatcher.java | 13 ++--- .../master/ReplicationPeerConfigUpgrader.java | 8 4 files changed, 13 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/23e43032/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java -- diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java index 60d0749..462cfed 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStorageFactory.java @@ -27,7 +27,7 @@ import org.apache.yetus.audience.InterfaceAudience; * For now we only have zk based implementation. */ @InterfaceAudience.Private -public class ReplicationStorageFactory { +public final class ReplicationStorageFactory { private ReplicationStorageFactory() { } http://git-wip-us.apache.org/repos/asf/hbase/blob/23e43032/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java index 1724a38..8277dbe 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionTransitionProcedure.java @@ -36,11 +36,11 @@ 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.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.RegionTransitionState; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - /** * Base class for the Assign and Unassign Procedure. * http://git-wip-us.apache.org/repos/asf/hbase/blob/23e43032/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java index 57a4535..6c78914 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/RSProcedureDispatcher.java @@ -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; @@ -36,6 +35,12 @@ import org.apache.hadoop.ipc.RemoteException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap; +import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback; +import org.apache.hbase.thirdparty.com.google.protobuf.RpcController; +import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService; @@ -47,12 +52,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.OpenRegionR
[38/50] [abbrv] hbase git commit: HBASE-19661 Replace ReplicationStateZKBase with ZKReplicationStorageBase
HBASE-19661 Replace ReplicationStateZKBase with ZKReplicationStorageBase Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/ff084473 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/ff084473 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/ff084473 Branch: refs/heads/HBASE-19397-branch-2 Commit: ff084473811ce47235095b775c0c0d5df36605f5 Parents: 43f7163 Author: huzhengAuthored: Fri Dec 29 15:55:28 2017 +0800 Committer: zhangduo Committed: Thu Feb 22 15:23:52 2018 +0800 -- .../hbase/replication/ReplicationFactory.java | 5 +- .../replication/ReplicationStateZKBase.java | 159 --- .../replication/ReplicationTrackerZKImpl.java | 21 +-- .../replication/ZKReplicationPeerStorage.java | 24 ++- .../replication/ZKReplicationStorageBase.java | 13 +- .../org/apache/hadoop/hbase/master/HMaster.java | 4 +- .../master/ReplicationPeerConfigUpgrader.java | 128 +++ .../regionserver/DumpReplicationQueues.java | 18 +-- .../replication/regionserver/Replication.java | 3 +- .../org/apache/hadoop/hbase/util/HBaseFsck.java | 3 +- .../TestReplicationTrackerZKImpl.java | 3 +- .../replication/master/TestTableCFsUpdater.java | 41 ++--- .../TestReplicationSourceManager.java | 6 +- 13 files changed, 136 insertions(+), 292 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/ff084473/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java -- diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java index 6c66aff..2a970ba 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationFactory.java @@ -33,9 +33,8 @@ public class ReplicationFactory { return new ReplicationPeers(zk, conf); } - public static ReplicationTracker getReplicationTracker(ZKWatcher zookeeper, - final ReplicationPeers replicationPeers, Configuration conf, Abortable abortable, + public static ReplicationTracker getReplicationTracker(ZKWatcher zookeeper, Abortable abortable, Stoppable stopper) { -return new ReplicationTrackerZKImpl(zookeeper, replicationPeers, conf, abortable, stopper); +return new ReplicationTrackerZKImpl(zookeeper, abortable, stopper); } } http://git-wip-us.apache.org/repos/asf/hbase/blob/ff084473/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java -- diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java deleted file mode 100644 index a48683e..000 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationStateZKBase.java +++ /dev/null @@ -1,159 +0,0 @@ -/* - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.replication; - -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.util.List; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Abortable; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.hbase.thirdparty.com.google.protobuf.CodedOutputStream; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos; -import org.apache.hadoop.hbase.zookeeper.ZKConfig; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; -import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import
[07/50] [abbrv] hbase git commit: HBASE-20042 TestRegionServerAbort flakey
HBASE-20042 TestRegionServerAbort flakey Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/baec532a Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/baec532a Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/baec532a Branch: refs/heads/HBASE-19397-branch-2 Commit: baec532aa24fec7e62a73ffc992f2a95ba5c9c69 Parents: 67b7ce3 Author: Michael StackAuthored: Wed Feb 21 13:24:32 2018 -0800 Committer: Michael Stack Committed: Wed Feb 21 13:24:39 2018 -0800 -- .../hbase/regionserver/TestRegionServerAbort.java | 12 1 file changed, 12 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/baec532a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java index fdb6887..0c778fb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import java.io.IOException; +import java.util.List; import java.util.Optional; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -52,6 +53,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; +import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hdfs.DFSConfigKeys; @@ -146,6 +148,16 @@ public class TestRegionServerAbort { // should have triggered an abort due to FileNotFoundException // verify that the regionserver is stopped +List regions = null; +do { + regions = cluster.findRegionsForTable(tableName); + if (regions != null && regions.size() > 0) { +break; + } + LOG.warn("Waiting on regions for {} to online"); + Threads.sleep(100); +} while(true); + HRegion firstRegion = cluster.findRegionsForTable(tableName).get(0); assertNotNull(firstRegion); assertNotNull(firstRegion.getRegionServerServices());
[03/50] [abbrv] hbase git commit: HBASE-20031 Unable to run integration test using mvn due to missing HBaseClassTestRule
HBASE-20031 Unable to run integration test using mvn due to missing HBaseClassTestRule Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/66ba8aa5 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/66ba8aa5 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/66ba8aa5 Branch: refs/heads/HBASE-19397-branch-2 Commit: 66ba8aa58cee2bcf74a59cd10238e21604607f40 Parents: e10db9d Author: tedyuAuthored: Wed Feb 21 12:04:17 2018 -0800 Committer: tedyu Committed: Wed Feb 21 12:04:17 2018 -0800 -- .../java/org/apache/hadoop/hbase/HBaseClassTestRule.java | 4 .../apache/hadoop/hbase/HBaseClassTestRuleChecker.java| 10 ++ src/main/asciidoc/_chapters/developer.adoc| 3 ++- 3 files changed, 16 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/66ba8aa5/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java -- diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java index a41e383..c3bef0f 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase; import java.util.concurrent.TimeUnit; +import org.apache.hadoop.hbase.testclassification.IntegrationTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -63,6 +64,9 @@ public final class HBaseClassTestRule implements TestRule { // All tests have a 10minute timeout. return TimeUnit.MINUTES.toSeconds(13); } + if (c == IntegrationTests.class) { +return TimeUnit.MINUTES.toSeconds(Long.MAX_VALUE); + } } throw new IllegalArgumentException( clazz.getName() + " does not have SmallTests/MediumTests/LargeTests in @Category"); http://git-wip-us.apache.org/repos/asf/hbase/blob/66ba8aa5/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRuleChecker.java -- diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRuleChecker.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRuleChecker.java index 97c657f..c374903 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRuleChecker.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRuleChecker.java @@ -22,8 +22,12 @@ import static org.junit.Assert.fail; import java.lang.reflect.Field; import java.lang.reflect.Modifier; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.hbase.testclassification.IntegrationTests; import org.apache.yetus.audience.InterfaceAudience; import org.junit.ClassRule; +import org.junit.experimental.categories.Category; import org.junit.runner.Description; import org.junit.runner.notification.RunListener; import org.junit.runner.notification.RunListener.ThreadSafe; @@ -37,6 +41,12 @@ public class HBaseClassTestRuleChecker extends RunListener { @Override public void testStarted(Description description) throws Exception { +Category[] categories = description.getTestClass().getAnnotationsByType(Category.class); +for (Class c : categories[0].value()) { + if (c == IntegrationTests.class) { +return; + } +} for (Field field : description.getTestClass().getFields()) { if (Modifier.isStatic(field.getModifiers()) && field.getType() == HBaseClassTestRule.class && field.isAnnotationPresent(ClassRule.class)) { http://git-wip-us.apache.org/repos/asf/hbase/blob/66ba8aa5/src/main/asciidoc/_chapters/developer.adoc -- diff --git a/src/main/asciidoc/_chapters/developer.adoc b/src/main/asciidoc/_chapters/developer.adoc index 0cd6ad9..ef9ae25 100644 --- a/src/main/asciidoc/_chapters/developer.adoc +++ b/src/main/asciidoc/_chapters/developer.adoc @@ -1015,7 +1015,7 @@ The first three categories, `small`, `medium`, and `large`, are for test cases w type `$ mvn test`. In other words, these three categorizations are for HBase unit tests. The `integration` category is not for unit tests, but for integration tests. -These are run when you invoke `$ mvn verify`. +These are normally run when you invoke `$ mvn verify`. Integration tests are described in < >. Keep reading to figure which
[30/50] [abbrv] hbase git commit: HBASE-19687 Move the logic in ReplicationZKNodeCleaner to ReplicationChecker and remove ReplicationZKNodeCleanerChore
HBASE-19687 Move the logic in ReplicationZKNodeCleaner to ReplicationChecker and remove ReplicationZKNodeCleanerChore Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/43f71632 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/43f71632 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/43f71632 Branch: refs/heads/HBASE-19397-branch-2 Commit: 43f7163211bc1fa1252bf7a585a3b1f6a017ecc4 Parents: 9600087 Author: zhangduoAuthored: Wed Jan 3 09:39:44 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 15:23:34 2018 +0800 -- .../replication/VerifyReplication.java | 6 +- .../hbase/replication/ReplicationPeers.java | 26 +-- .../hbase/replication/ReplicationUtils.java | 38 .../replication/TestReplicationStateBasic.java | 2 +- .../org/apache/hadoop/hbase/master/HMaster.java | 13 -- .../cleaner/ReplicationZKNodeCleaner.java | 192 --- .../cleaner/ReplicationZKNodeCleanerChore.java | 54 -- .../replication/ReplicationPeerManager.java | 18 +- .../org/apache/hadoop/hbase/util/HBaseFsck.java | 11 +- .../hbase/util/hbck/ReplicationChecker.java | 109 +++ .../cleaner/TestReplicationZKNodeCleaner.java | 115 --- .../hbase/util/TestHBaseFsckReplication.java| 101 ++ .../hadoop/hbase/util/hbck/HbckTestingUtil.java | 6 +- 13 files changed, 226 insertions(+), 465 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/43f71632/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java -- diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java index fe45762..fac4875 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java @@ -50,8 +50,8 @@ import org.apache.hadoop.hbase.mapreduce.TableSplit; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerStorage; -import org.apache.hadoop.hbase.replication.ReplicationPeers; import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; +import org.apache.hadoop.hbase.replication.ReplicationUtils; import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; @@ -345,10 +345,10 @@ public class VerifyReplication extends Configured implements Tool { } }); ReplicationPeerStorage storage = - ReplicationStorageFactory.getReplicationPeerStorage(localZKW, conf); +ReplicationStorageFactory.getReplicationPeerStorage(localZKW, conf); ReplicationPeerConfig peerConfig = storage.getPeerConfig(peerId); return Pair.newPair(peerConfig, -ReplicationPeers.getPeerClusterConfiguration(peerConfig, conf)); +ReplicationUtils.getPeerClusterConfiguration(peerConfig, conf)); } catch (ReplicationException e) { throw new IOException("An error occurred while trying to connect to the remove peer cluster", e); http://git-wip-us.apache.org/repos/asf/hbase/blob/43f71632/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java -- diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java index 45940a5..fcbc350 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeers.java @@ -17,14 +17,11 @@ */ package org.apache.hadoop.hbase.replication; -import java.io.IOException; import java.util.Collections; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CompoundConfiguration; -import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.yetus.audience.InterfaceAudience; @@ -106,25 +103,6 @@ public class ReplicationPeers { return
[49/50] [abbrv] hbase git commit: HBASE-19634 Add permission check for executeProcedures in AccessController
HBASE-19634 Add permission check for executeProcedures in AccessController Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/cd48ac73 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/cd48ac73 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/cd48ac73 Branch: refs/heads/HBASE-19397-branch-2 Commit: cd48ac73904d31d9fb28975c73c8b92c2424fd5b Parents: b00237f Author: zhangduoAuthored: Thu Jan 4 16:18:21 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 15:23:52 2018 +0800 -- .../hbase/coprocessor/RegionServerObserver.java | 14 ++ .../hbase/regionserver/RSRpcServices.java | 52 +++- .../RegionServerCoprocessorHost.java| 18 +++ .../hbase/security/access/AccessController.java | 30 ++- .../hadoop/hbase/TestJMXConnectorServer.java| 7 +++ .../security/access/TestAccessController.java | 18 +-- 6 files changed, 100 insertions(+), 39 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/cd48ac73/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java index c1af3fb..5b751df 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionServerObserver.java @@ -126,4 +126,18 @@ public interface RegionServerObserver { default void postClearCompactionQueues( final ObserverContext ctx) throws IOException {} + + /** + * This will be called before executing procedures + * @param ctx the environment to interact with the framework and region server. + */ + default void preExecuteProcedures(ObserverContext ctx) + throws IOException {} + + /** + * This will be called after executing procedures + * @param ctx the environment to interact with the framework and region server. + */ + default void postExecuteProcedures(ObserverContext ctx) + throws IOException {} } http://git-wip-us.apache.org/repos/asf/hbase/blob/cd48ac73/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index 2ee0c52..5b4e3b8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -3467,36 +3467,40 @@ public class RSRpcServices implements HBaseRPCErrorHandler, } @Override + @QosPriority(priority = HConstants.ADMIN_QOS) public ExecuteProceduresResponse executeProcedures(RpcController controller, ExecuteProceduresRequest request) throws ServiceException { -if (request.getOpenRegionCount() > 0) { - for (OpenRegionRequest req : request.getOpenRegionList()) { -openRegion(controller, req); +try { + checkOpen(); + regionServer.getRegionServerCoprocessorHost().preExecuteProcedures(); + if (request.getOpenRegionCount() > 0) { +for (OpenRegionRequest req : request.getOpenRegionList()) { + openRegion(controller, req); +} } -} -if (request.getCloseRegionCount() > 0) { - for (CloseRegionRequest req : request.getCloseRegionList()) { -closeRegion(controller, req); + if (request.getCloseRegionCount() > 0) { +for (CloseRegionRequest req : request.getCloseRegionList()) { + closeRegion(controller, req); +} } -} -if (request.getProcCount() > 0) { - for (RemoteProcedureRequest req : request.getProcList()) { -RSProcedureCallable callable; -try { - callable = - Class.forName(req.getProcClass()).asSubclass(RSProcedureCallable.class).newInstance(); -} catch (Exception e) { - // here we just ignore the error as this should not happen and we do not provide a general - // way to report errors for all types of remote procedure. The procedure will hang at - // master side but after you solve the problem and restart master it will be executed - // again and pass. - LOG.warn("create procedure of type " + req.getProcClass() + " failed, give up", e); - continue; + if (request.getProcCount() > 0) { +for
[36/50] [abbrv] hbase git commit: HBASE-19622 Reimplement ReplicationPeers with the new replication storage interface
HBASE-19622 Reimplement ReplicationPeers with the new replication storage interface Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/05aa1aac Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/05aa1aac Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/05aa1aac Branch: refs/heads/HBASE-19397-branch-2 Commit: 05aa1aac55fb9fc10370c5aeda5c0754eafda900 Parents: bba95c5 Author: huzhengAuthored: Tue Dec 26 16:46:10 2017 +0800 Committer: zhangduo Committed: Thu Feb 22 15:23:34 2018 +0800 -- .../replication/ReplicationPeerConfigUtil.java | 10 +- .../replication/VerifyReplication.java | 9 +- .../hbase/replication/ReplicationFactory.java | 10 +- .../hbase/replication/ReplicationPeerImpl.java | 60 +- .../replication/ReplicationPeerStorage.java | 3 +- .../hbase/replication/ReplicationPeers.java | 238 .../replication/ReplicationPeersZKImpl.java | 552 --- .../replication/ZKReplicationPeerStorage.java | 12 +- .../replication/ZKReplicationStorageBase.java | 3 +- .../replication/TestReplicationStateBasic.java | 125 ++--- .../replication/TestReplicationStateZKImpl.java | 2 +- .../TestZKReplicationPeerStorage.java | 12 +- .../cleaner/ReplicationZKNodeCleaner.java | 57 +- .../replication/ReplicationPeerManager.java | 6 +- .../regionserver/DumpReplicationQueues.java | 2 +- .../regionserver/PeerProcedureHandlerImpl.java | 49 +- .../replication/regionserver/Replication.java | 2 +- .../regionserver/ReplicationSource.java | 7 +- .../regionserver/ReplicationSourceManager.java | 44 +- .../cleaner/TestReplicationHFileCleaner.java| 7 +- .../replication/TestMultiSlaveReplication.java | 2 - .../TestReplicationTrackerZKImpl.java | 26 +- .../TestReplicationSourceManager.java | 17 +- .../hadoop/hbase/HBaseZKTestingUtility.java | 3 +- 24 files changed, 307 insertions(+), 951 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/05aa1aac/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 022bf64..a234a9b 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 @@ -247,22 +247,22 @@ public final class ReplicationPeerConfigUtil { public static ReplicationPeerConfig parsePeerFrom(final byte[] bytes) throws DeserializationException { if (ProtobufUtil.isPBMagicPrefix(bytes)) { - int pblen = ProtobufUtil.lengthOfPBMagic(); + int pbLen = ProtobufUtil.lengthOfPBMagic(); ReplicationProtos.ReplicationPeer.Builder builder = ReplicationProtos.ReplicationPeer.newBuilder(); ReplicationProtos.ReplicationPeer peer; try { -ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen); +ProtobufUtil.mergeFrom(builder, bytes, pbLen, bytes.length - pbLen); peer = builder.build(); } catch (IOException e) { throw new DeserializationException(e); } return convert(peer); } else { - if (bytes.length > 0) { -return ReplicationPeerConfig.newBuilder().setClusterKey(Bytes.toString(bytes)).build(); + if (bytes == null || bytes.length <= 0) { +throw new DeserializationException("Bytes to deserialize should not be empty."); } - return ReplicationPeerConfig.newBuilder().setClusterKey("").build(); + return ReplicationPeerConfig.newBuilder().setClusterKey(Bytes.toString(bytes)).build(); } } http://git-wip-us.apache.org/repos/asf/hbase/blob/05aa1aac/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java -- diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java index 09d4b4b..f0070f0 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java @@ -339,15 +339,10 @@ public class VerifyReplication extends Configured implements
[35/50] [abbrv] hbase git commit: HBASE-19622 Reimplement ReplicationPeers with the new replication storage interface
http://git-wip-us.apache.org/repos/asf/hbase/blob/05aa1aac/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 222ff6c..27e9d40 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 @@ -167,7 +167,6 @@ public class ReplicationSourceManager implements ReplicationListener { this.clusterId = clusterId; this.walFileLengthProvider = walFileLengthProvider; this.replicationTracker.registerListener(this); -this.replicationPeers.getAllPeerIds(); // It's preferable to failover 1 RS at a time, but with good zk servers // more could be processed at the same time. int nbWorkers = conf.getInt("replication.executor.workers", 1); @@ -271,8 +270,8 @@ public class ReplicationSourceManager implements ReplicationListener { } List otherRegionServers = replicationTracker.getListOfRegionServers().stream() .map(ServerName::valueOf).collect(Collectors.toList()); -LOG.info( - "Current list of replicators: " + currentReplicators + " other RSs: " + otherRegionServers); +LOG.info("Current list of replicators: " + currentReplicators + " other RSs: " ++ otherRegionServers); // Look if there's anything to process after a restart for (ServerName rs : currentReplicators) { @@ -289,7 +288,7 @@ public class ReplicationSourceManager implements ReplicationListener { * The returned future is for adoptAbandonedQueues task. */ Future init() throws IOException, ReplicationException { -for (String id : this.replicationPeers.getConnectedPeerIds()) { +for (String id : this.replicationPeers.getAllPeerIds()) { addSource(id); if (replicationForBulkLoadDataEnabled) { // Check if peer exists in hfile-refs queue, if not add it. This can happen in the case @@ -308,8 +307,8 @@ public class ReplicationSourceManager implements ReplicationListener { */ @VisibleForTesting ReplicationSourceInterface addSource(String id) throws IOException, ReplicationException { -ReplicationPeerConfig peerConfig = replicationPeers.getReplicationPeerConfig(id); -ReplicationPeer peer = replicationPeers.getConnectedPeer(id); +ReplicationPeerConfig peerConfig = replicationPeers.getPeerConfig(id); +ReplicationPeer peer = replicationPeers.getPeer(id); ReplicationSourceInterface src = getReplicationSource(id, peerConfig, peer); synchronized (this.walsById) { this.sources.add(src); @@ -355,7 +354,7 @@ public class ReplicationSourceManager implements ReplicationListener { public void deleteSource(String peerId, boolean closeConnection) { abortWhenFail(() -> this.queueStorage.removeQueue(server.getServerName(), peerId)); if (closeConnection) { - this.replicationPeers.peerDisconnected(peerId); + this.replicationPeers.removePeer(peerId); } } @@ -448,12 +447,12 @@ public class ReplicationSourceManager implements ReplicationListener { // update replication queues on ZK // synchronize on replicationPeers to avoid adding source for the to-be-removed peer synchronized (replicationPeers) { - for (String id : replicationPeers.getConnectedPeerIds()) { + for (String id : replicationPeers.getAllPeerIds()) { try { this.queueStorage.addWAL(server.getServerName(), id, logName); } catch (ReplicationException e) { - throw new IOException("Cannot add log to replication queue" + -" when creating a new source, queueId=" + id + ", filename=" + logName, e); + throw new IOException("Cannot add log to replication queue" + + " when creating a new source, queueId=" + id + ", filename=" + logName, e); } } } @@ -598,7 +597,7 @@ public class ReplicationSourceManager implements ReplicationListener { public void addPeer(String id) throws ReplicationException, IOException { LOG.info("Trying to add peer, peerId: " + id); -boolean added = this.replicationPeers.peerConnected(id); +boolean added = this.replicationPeers.addPeer(id); if (added) { LOG.info("Peer " + id + " connected success, trying to start the replication source thread."); addSource(id); @@ -734,16 +733,21 @@ public class ReplicationSourceManager implements ReplicationListener { // there is not an actual peer defined corresponding to peerId for the failover. ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId);
[26/50] [abbrv] hbase git commit: HBASE-19573 Rewrite ReplicationPeer with the new replication storage interface
HBASE-19573 Rewrite ReplicationPeer with the new replication storage interface Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/48d2587a Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/48d2587a Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/48d2587a Branch: refs/heads/HBASE-19397-branch-2 Commit: 48d2587a7ee291c32cb46056e37542003b541330 Parents: 253b180 Author: Guanghao ZhangAuthored: Tue Dec 26 11:39:34 2017 +0800 Committer: zhangduo Committed: Thu Feb 22 15:21:30 2018 +0800 -- .../replication/VerifyReplication.java | 5 - .../hbase/replication/ReplicationPeer.java | 42 ++-- .../hbase/replication/ReplicationPeerImpl.java | 169 ++ .../replication/ReplicationPeerZKImpl.java | 233 --- .../hbase/replication/ReplicationPeers.java | 4 +- .../replication/ReplicationPeersZKImpl.java | 23 +- .../replication/TestReplicationStateBasic.java | 7 +- .../regionserver/PeerProcedureHandlerImpl.java | 29 +-- 8 files changed, 216 insertions(+), 296 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/48d2587a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java -- diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java index 9065f4e..09d4b4b 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/replication/VerifyReplication.java @@ -51,7 +51,6 @@ import org.apache.hadoop.hbase.mapreduce.TableSplit; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationFactory; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; -import org.apache.hadoop.hbase.replication.ReplicationPeerZKImpl; import org.apache.hadoop.hbase.replication.ReplicationPeers; import org.apache.hadoop.hbase.snapshot.RestoreSnapshotHelper; import org.apache.hadoop.hbase.util.Bytes; @@ -333,7 +332,6 @@ public class VerifyReplication extends Configured implements Tool { private static Pair getPeerQuorumConfig( final Configuration conf, String peerId) throws IOException { ZKWatcher localZKW = null; -ReplicationPeerZKImpl peer = null; try { localZKW = new ZKWatcher(conf, "VerifyReplication", new Abortable() { @@ -354,9 +352,6 @@ public class VerifyReplication extends Configured implements Tool { throw new IOException( "An error occurred while trying to connect to the remove peer cluster", e); } finally { - if (peer != null) { -peer.close(); - } if (localZKW != null) { localZKW.close(); } http://git-wip-us.apache.org/repos/asf/hbase/blob/48d2587a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java -- diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java index b66d76d..4846018 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java @@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.TableName; import org.apache.yetus.audience.InterfaceAudience; - /** * ReplicationPeer manages enabled / disabled state for the peer. */ @@ -49,65 +48,52 @@ public interface ReplicationPeer { String getId(); /** - * Get the peer config object - * @return the ReplicationPeerConfig for this peer - */ - public ReplicationPeerConfig getPeerConfig(); - - /** - * Get the peer config object. if loadFromBackingStore is true, it will load from backing store - * directly and update its load peer config. otherwise, just return the local cached peer config. - * @return the ReplicationPeerConfig for this peer - */ - public ReplicationPeerConfig getPeerConfig(boolean loadFromBackingStore) - throws ReplicationException; - - /** * Returns the state of the peer by reading local cache. * @return the enabled state */ PeerState getPeerState(); /** - * Returns the state of peer, if loadFromBackingStore is true, it will load from backing
[34/50] [abbrv] hbase git commit: HBASE-19635 Introduce a thread at RS side to call reportProcedureDone
HBASE-19635 Introduce a thread at RS side to call reportProcedureDone Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/bba95c54 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/bba95c54 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/bba95c54 Branch: refs/heads/HBASE-19397-branch-2 Commit: bba95c54284f2860fe6deaa13423c1d8175e01ed Parents: bef6d77 Author: zhangduoAuthored: Wed Dec 27 20:13:42 2017 +0800 Committer: zhangduo Committed: Thu Feb 22 15:23:34 2018 +0800 -- .../src/main/protobuf/RegionServerStatus.proto | 5 +- .../hadoop/hbase/master/MasterRpcServices.java | 15 ++- .../hbase/regionserver/HRegionServer.java | 72 .../RemoteProcedureResultReporter.java | 111 +++ .../handler/RSProcedureHandler.java | 2 +- 5 files changed, 149 insertions(+), 56 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/bba95c54/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto -- diff --git a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto index 4f75941..3f836cd 100644 --- a/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto +++ b/hbase-protocol-shaded/src/main/protobuf/RegionServerStatus.proto @@ -146,7 +146,7 @@ message RegionSpaceUseReportRequest { message RegionSpaceUseReportResponse { } -message ReportProcedureDoneRequest { +message RemoteProcedureResult { required uint64 proc_id = 1; enum Status { SUCCESS = 1; @@ -155,6 +155,9 @@ message ReportProcedureDoneRequest { required Status status = 2; optional ForeignExceptionMessage error = 3; } +message ReportProcedureDoneRequest { + repeated RemoteProcedureResult result = 1; +} message ReportProcedureDoneResponse { } http://git-wip-us.apache.org/repos/asf/hbase/blob/bba95c54/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java index 72bf2d1..377a9c6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java @@ -265,6 +265,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RemoteProcedureResult; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest; @@ -2254,12 +2255,14 @@ public class MasterRpcServices extends RSRpcServices @Override public ReportProcedureDoneResponse reportProcedureDone(RpcController controller, ReportProcedureDoneRequest request) throws ServiceException { -if (request.getStatus() == ReportProcedureDoneRequest.Status.SUCCESS) { - master.remoteProcedureCompleted(request.getProcId()); -} else { - master.remoteProcedureFailed(request.getProcId(), -RemoteProcedureException.fromProto(request.getError())); -} +request.getResultList().forEach(result -> { + if (result.getStatus() == RemoteProcedureResult.Status.SUCCESS) { +master.remoteProcedureCompleted(result.getProcId()); + } else { +master.remoteProcedureFailed(result.getProcId(), + RemoteProcedureException.fromProto(result.getError())); + } +}); return ReportProcedureDoneResponse.getDefaultInstance(); } } http://git-wip-us.apache.org/repos/asf/hbase/blob/bba95c54/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index ae57469..e2aac03 100644 ---
[09/50] [abbrv] hbase git commit: HBASE-20042 TestRegionServerAbort flakey; ADDENDUM, RETRY
HBASE-20042 TestRegionServerAbort flakey; ADDENDUM, RETRY Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/c677c300 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c677c300 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c677c300 Branch: refs/heads/HBASE-19397-branch-2 Commit: c677c30020778efd4134729c31c294bcd718e5f5 Parents: 92bb4db Author: Michael StackAuthored: Wed Feb 21 15:26:21 2018 -0800 Committer: Michael Stack Committed: Wed Feb 21 15:26:59 2018 -0800 -- .../hbase/regionserver/TestRegionServerAbort.java | 17 +++-- 1 file changed, 3 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/c677c300/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java index 0c778fb..d5cbad0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java @@ -144,21 +144,10 @@ public class TestRegionServerAbort { put.addColumn(FAMILY_BYTES, Bytes.toBytes("c"), new byte[]{}); put.setAttribute(StopBlockingRegionObserver.DO_ABORT, new byte[]{1}); -table.put(put); -// should have triggered an abort due to FileNotFoundException - -// verify that the regionserver is stopped -List regions = null; -do { - regions = cluster.findRegionsForTable(tableName); - if (regions != null && regions.size() > 0) { -break; - } - LOG.warn("Waiting on regions for {} to online"); - Threads.sleep(100); -} while(true); - +List regions = cluster.findRegionsForTable(tableName); HRegion firstRegion = cluster.findRegionsForTable(tableName).get(0); +table.put(put); +// Verify that the regionserver is stopped assertNotNull(firstRegion); assertNotNull(firstRegion.getRegionServerServices()); LOG.info("isAborted = " + firstRegion.getRegionServerServices().isAborted());
[42/50] [abbrv] hbase git commit: HBASE-19711 TestReplicationAdmin.testConcurrentPeerOperations hangs
HBASE-19711 TestReplicationAdmin.testConcurrentPeerOperations hangs Signed-off-by: zhangduoProject: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/5dc9906a Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/5dc9906a Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/5dc9906a Branch: refs/heads/HBASE-19397-branch-2 Commit: 5dc9906a4dbb66fc407924d576c9d3e92be0bf2d Parents: 41991d6 Author: Guanghao Zhang Authored: Fri Jan 5 15:39:06 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 15:23:52 2018 +0800 -- .../procedure/MasterProcedureScheduler.java | 23 1 file changed, 19 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/5dc9906a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java index b18dd6c..eab06a2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java @@ -290,7 +290,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { @Override public void completionCleanup(final Procedure proc) { if (proc instanceof TableProcedureInterface) { - TableProcedureInterface iProcTable = (TableProcedureInterface)proc; + TableProcedureInterface iProcTable = (TableProcedureInterface) proc; boolean tableDeleted; if (proc.hasException()) { Exception procEx = proc.getException().unwrapRemoteException(); @@ -311,9 +311,7 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { } } else if (proc instanceof PeerProcedureInterface) { PeerProcedureInterface iProcPeer = (PeerProcedureInterface) proc; - if (iProcPeer.getPeerOperationType() == PeerOperationType.REMOVE) { -removePeerQueue(iProcPeer.getPeerId()); - } + tryCleanupPeerQueue(iProcPeer.getPeerId(), proc); } else { // No cleanup for ServerProcedureInterface types, yet. return; @@ -402,6 +400,23 @@ public class MasterProcedureScheduler extends AbstractProcedureScheduler { locking.removePeerLock(peerId); } + private void tryCleanupPeerQueue(String peerId, Procedure procedure) { +schedLock(); +try { + PeerQueue queue = AvlTree.get(peerMap, peerId, PEER_QUEUE_KEY_COMPARATOR); + if (queue == null) { +return; + } + + final LockAndQueue lock = locking.getPeerLock(peerId); + if (queue.isEmpty() && lock.tryExclusiveLock(procedure)) { +removeFromRunQueue(peerRunQueue, queue); +removePeerQueue(peerId); + } +} finally { + schedUnlock(); +} + } private static boolean isPeerProcedure(Procedure proc) { return proc instanceof PeerProcedureInterface;
[14/50] [abbrv] hbase git commit: HBASE-19580 Use slf4j instead of commons-logging in new, just-added Peer Procedure classes
HBASE-19580 Use slf4j instead of commons-logging in new, just-added Peer Procedure classes Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/62a2a8c9 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/62a2a8c9 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/62a2a8c9 Branch: refs/heads/HBASE-19397-branch-2 Commit: 62a2a8c99fe9e17274a4cf68b9a2a68555403db4 Parents: eb2e14d Author: zhangduoAuthored: Thu Dec 21 21:59:46 2017 +0800 Committer: zhangduo Committed: Thu Feb 22 15:15:11 2018 +0800 -- .../hadoop/hbase/master/replication/AddPeerProcedure.java | 6 +++--- .../hadoop/hbase/master/replication/DisablePeerProcedure.java | 6 +++--- .../hadoop/hbase/master/replication/EnablePeerProcedure.java | 6 +++--- .../hadoop/hbase/master/replication/ModifyPeerProcedure.java | 6 +++--- .../hadoop/hbase/master/replication/RefreshPeerProcedure.java | 6 +++--- .../hadoop/hbase/master/replication/RemovePeerProcedure.java | 6 +++--- .../hbase/master/replication/UpdatePeerConfigProcedure.java| 6 +++--- 7 files changed, 21 insertions(+), 21 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/62a2a8c9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java index c3862d8..066c3e7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/AddPeerProcedure.java @@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.master.replication; import java.io.IOException; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; @@ -28,6 +26,8 @@ import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddPeerStateData; @@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.A @InterfaceAudience.Private public class AddPeerProcedure extends ModifyPeerProcedure { - private static final Log LOG = LogFactory.getLog(AddPeerProcedure.class); + private static final Logger LOG = LoggerFactory.getLogger(AddPeerProcedure.class); private ReplicationPeerConfig peerConfig; http://git-wip-us.apache.org/repos/asf/hbase/blob/62a2a8c9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java index 0b32db9..9a28de6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/DisablePeerProcedure.java @@ -19,11 +19,11 @@ package org.apache.hadoop.hbase.master.replication; import java.io.IOException; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * The procedure for disabling a replication peer. @@ -31,7 +31,7 @@ import org.apache.yetus.audience.InterfaceAudience; @InterfaceAudience.Private public class DisablePeerProcedure extends ModifyPeerProcedure { - private static final Log LOG = LogFactory.getLog(DisablePeerProcedure.class); + private static final Logger LOG = LoggerFactory.getLogger(DisablePeerProcedure.class); public DisablePeerProcedure() { } http://git-wip-us.apache.org/repos/asf/hbase/blob/62a2a8c9/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/EnablePeerProcedure.java
[32/50] [abbrv] hbase git commit: HBASE-19617 Remove ReplicationQueues, use ReplicationQueueStorage directly
http://git-wip-us.apache.org/repos/asf/hbase/blob/bef6d77b/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 89d9cd9..222ff6c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java @@ -34,18 +34,21 @@ import java.util.TreeSet; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.Server; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; @@ -60,7 +63,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeers; import org.apache.hadoop.hbase.replication.ReplicationQueueInfo; -import org.apache.hadoop.hbase.replication.ReplicationQueues; +import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; import org.apache.hadoop.hbase.replication.ReplicationTracker; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; @@ -96,7 +99,7 @@ public class ReplicationSourceManager implements ReplicationListener { private final List sources; // List of all the sources we got from died RSs private final List oldsources; - private final ReplicationQueues replicationQueues; + private final ReplicationQueueStorage queueStorage; private final ReplicationTracker replicationTracker; private final ReplicationPeers replicationPeers; // UUID for this cluster @@ -131,7 +134,7 @@ public class ReplicationSourceManager implements ReplicationListener { /** * Creates a replication manager and sets the watch on all the other registered region servers - * @param replicationQueues the interface for manipulating replication queues + * @param queueStorage the interface for manipulating replication queues * @param replicationPeers * @param replicationTracker * @param conf the configuration to use @@ -141,14 +144,14 @@ public class ReplicationSourceManager implements ReplicationListener { * @param oldLogDir the directory where old logs are archived * @param clusterId */ - public ReplicationSourceManager(ReplicationQueues replicationQueues, + public ReplicationSourceManager(ReplicationQueueStorage queueStorage, ReplicationPeers replicationPeers, ReplicationTracker replicationTracker, Configuration conf, Server server, FileSystem fs, Path logDir, Path oldLogDir, UUID clusterId, WALFileLengthProvider walFileLengthProvider) throws IOException { //CopyOnWriteArrayList is thread-safe. //Generally, reading is more than modifying. this.sources = new CopyOnWriteArrayList<>(); -this.replicationQueues = replicationQueues; +this.queueStorage = queueStorage; this.replicationPeers = replicationPeers; this.replicationTracker = replicationTracker; this.server = server; @@ -185,6 +188,19 @@ public class ReplicationSourceManager implements ReplicationListener { connection = ConnectionFactory.createConnection(conf); } + @FunctionalInterface + private interface ReplicationQueueOperation { +void exec() throws ReplicationException; + } + + private void abortWhenFail(ReplicationQueueOperation op) { +try { + op.exec(); +} catch (ReplicationException e) { + server.abort("Failed to operate on replication queue", e); +} + } + /** * Provide the id of the peer and a log key and this method will figure which * wal it belongs to and will log, for this region server, the current @@ -196,12 +212,13 @@ public class ReplicationSourceManager implements ReplicationListener { * @param queueRecovered indicates if this queue comes from another region server * @param holdLogInZK if
[41/50] [abbrv] hbase git commit: HBASE-19873 addendum add missing rule for new tests
HBASE-19873 addendum add missing rule for new tests Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/1d883eb5 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/1d883eb5 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/1d883eb5 Branch: refs/heads/HBASE-19397-branch-2 Commit: 1d883eb563384a28c49468c6d04d94716d4a0f25 Parents: c445b31 Author: zhangduoAuthored: Tue Jan 30 09:40:23 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 15:23:52 2018 +0800 -- .../hbase/replication/TestZKReplicationPeerStorage.java | 7 ++- .../hbase/replication/TestZKReplicationQueueStorage.java | 8 +++- .../hbase/replication/TestReplicationProcedureRetry.java | 7 ++- .../apache/hadoop/hbase/util/TestHBaseFsckReplication.java | 6 ++ 4 files changed, 25 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/1d883eb5/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java -- diff --git a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java index 3eb11da..3290fb0 100644 --- a/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java +++ b/hbase-replication/src/test/java/org/apache/hadoop/hbase/replication/TestZKReplicationPeerStorage.java @@ -33,19 +33,24 @@ import java.util.Map; import java.util.Random; import java.util.Set; import java.util.stream.Stream; - +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseZKTestingUtility; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ ReplicationTests.class, MediumTests.class }) public class TestZKReplicationPeerStorage { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestZKReplicationPeerStorage.class); + private static final HBaseZKTestingUtility UTIL = new HBaseZKTestingUtility(); private static ZKReplicationPeerStorage STORAGE; http://git-wip-us.apache.org/repos/asf/hbase/blob/1d883eb5/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 786730f..2c01a26 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 @@ -27,8 +27,8 @@ import java.util.Arrays; import java.util.List; import java.util.Set; import java.util.SortedSet; - import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseZKTestingUtility; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -38,11 +38,17 @@ import org.apache.zookeeper.KeeperException; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; @Category({ ReplicationTests.class, MediumTests.class }) public class TestZKReplicationQueueStorage { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestZKReplicationQueueStorage.class); + private static final HBaseZKTestingUtility UTIL = new HBaseZKTestingUtility(); private static ZKReplicationQueueStorage STORAGE; http://git-wip-us.apache.org/repos/asf/hbase/blob/1d883eb5/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationProcedureRetry.java index ab35b46..a2ae0b4 100644 ---
[19/50] [abbrv] hbase git commit: HBASE-19543 Abstract a replication storage interface to extract the zk specific code
http://git-wip-us.apache.org/repos/asf/hbase/blob/9b97636c/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java deleted file mode 100644 index b6f8784..000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/ReplicationManager.java +++ /dev/null @@ -1,199 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase.master.replication; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.regex.Pattern; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.Abortable; -import org.apache.hadoop.hbase.ReplicationPeerNotFoundException; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.replication.BaseReplicationEndpoint; -import org.apache.hadoop.hbase.replication.ReplicationException; -import org.apache.hadoop.hbase.replication.ReplicationFactory; -import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; -import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; -import org.apache.hadoop.hbase.replication.ReplicationPeers; -import org.apache.hadoop.hbase.replication.ReplicationQueuesClient; -import org.apache.hadoop.hbase.replication.ReplicationQueuesClientArguments; -import org.apache.hadoop.hbase.zookeeper.ZKWatcher; -import org.apache.yetus.audience.InterfaceAudience; - -/** - * Manages and performs all replication admin operations. - * - * Used to add/remove a replication peer. - */ -@InterfaceAudience.Private -public class ReplicationManager { - private final ReplicationQueuesClient replicationQueuesClient; - private final ReplicationPeers replicationPeers; - - public ReplicationManager(Configuration conf, ZKWatcher zkw, Abortable abortable) - throws IOException { -try { - this.replicationQueuesClient = ReplicationFactory - .getReplicationQueuesClient(new ReplicationQueuesClientArguments(conf, abortable, zkw)); - this.replicationQueuesClient.init(); - this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, -this.replicationQueuesClient, abortable); - this.replicationPeers.init(); -} catch (Exception e) { - throw new IOException("Failed to construct ReplicationManager", e); -} - } - - public void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) - throws ReplicationException { -checkPeerConfig(peerConfig); -replicationPeers.registerPeer(peerId, peerConfig, enabled); -replicationPeers.peerConnected(peerId); - } - - public void removeReplicationPeer(String peerId) throws ReplicationException { -replicationPeers.peerDisconnected(peerId); -replicationPeers.unregisterPeer(peerId); - } - - public void enableReplicationPeer(String peerId) throws ReplicationException { -this.replicationPeers.enablePeer(peerId); - } - - public void disableReplicationPeer(String peerId) throws ReplicationException { -this.replicationPeers.disablePeer(peerId); - } - - public ReplicationPeerConfig getPeerConfig(String peerId) - throws ReplicationException, ReplicationPeerNotFoundException { -ReplicationPeerConfig peerConfig = replicationPeers.getReplicationPeerConfig(peerId); -if (peerConfig == null) { - throw new ReplicationPeerNotFoundException(peerId); -} -return peerConfig; - } - - public void updatePeerConfig(String peerId, ReplicationPeerConfig peerConfig) - throws ReplicationException, IOException { -checkPeerConfig(peerConfig); -this.replicationPeers.updatePeerConfig(peerId, peerConfig); - } - - public List listReplicationPeers(Pattern pattern) - throws ReplicationException { -List peers = new ArrayList<>(); -List peerIds = replicationPeers.getAllPeerIds(); -for (String peerId :
[45/50] [abbrv] hbase git commit: HBASE-19748 TestRegionReplicaFailover and TestRegionReplicaReplicationEndpoint UT hangs
HBASE-19748 TestRegionReplicaFailover and TestRegionReplicaReplicationEndpoint UT hangs Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/affbdb04 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/affbdb04 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/affbdb04 Branch: refs/heads/HBASE-19397-branch-2 Commit: affbdb047411fa1e5711922edc629b78be316f42 Parents: 23e4303 Author: huzhengAuthored: Wed Jan 10 15:00:30 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 15:23:52 2018 +0800 -- .../main/java/org/apache/hadoop/hbase/master/HMaster.java | 9 - 1 file changed, 4 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/affbdb04/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 1f3730d..718f5db 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 @@ -40,7 +40,6 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Objects; -import java.util.Optional; import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; @@ -71,6 +70,7 @@ import org.apache.hadoop.hbase.MasterNotRunningException; 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.ServerLoad; import org.apache.hadoop.hbase.ServerMetricsBuilder; import org.apache.hadoop.hbase.ServerName; @@ -3414,13 +3414,12 @@ public class HMaster extends HRegionServer implements MasterServices { cpHost.preGetReplicationPeerConfig(peerId); } LOG.info(getClientIdAuditPrefix() + " get replication peer config, id=" + peerId); -Optional peerConfig = - this.replicationPeerManager.getPeerConfig(peerId); - +ReplicationPeerConfig peerConfig = this.replicationPeerManager.getPeerConfig(peerId) +.orElseThrow(() -> new ReplicationPeerNotFoundException(peerId)); if (cpHost != null) { cpHost.postGetReplicationPeerConfig(peerId); } -return peerConfig.orElse(null); +return peerConfig; } @Override
[06/50] [abbrv] hbase git commit: HBASE-20027 Add test TestClusterPortAssignment
HBASE-20027 Add test TestClusterPortAssignment LocalHBaseCluster forces random port assignment for sake of concurrent unit test execution friendliness, but we still need a positive test for RPC and info port assignment. Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/67b7ce38 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/67b7ce38 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/67b7ce38 Branch: refs/heads/HBASE-19397-branch-2 Commit: 67b7ce38fbc3c3f67dbb4e6e948cf72518f835d8 Parents: 5c988c9 Author: Andrew PurtellAuthored: Tue Feb 20 17:43:58 2018 -0800 Committer: Andrew Purtell Committed: Wed Feb 21 13:12:30 2018 -0800 -- .../apache/hadoop/hbase/LocalHBaseCluster.java | 15 +++- .../apache/hadoop/hbase/MiniHBaseCluster.java | 8 +- .../hadoop/hbase/TestClusterPortAssignment.java | 80 3 files changed, 96 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/67b7ce38/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java index 06199f7..e19e53b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java @@ -66,6 +66,8 @@ public class LocalHBaseCluster { public static final String LOCAL = "local"; /** 'local:' */ public static final String LOCAL_COLON = LOCAL + ":"; + public static final String ASSIGN_RANDOM_PORTS = "hbase.localcluster.assign.random.ports"; + private final Configuration conf; private final Class masterClass; private final Class regionServerClass; @@ -139,10 +141,15 @@ public class LocalHBaseCluster { // Always have masters and regionservers come up on port '0' so we don't // clash over default ports. -conf.set(HConstants.MASTER_PORT, "0"); -conf.set(HConstants.REGIONSERVER_PORT, "0"); -if (conf.getInt(HConstants.REGIONSERVER_INFO_PORT, 0) != -1) { - conf.set(HConstants.REGIONSERVER_INFO_PORT, "0"); +if (conf.getBoolean(ASSIGN_RANDOM_PORTS, true)) { + conf.set(HConstants.MASTER_PORT, "0"); + conf.set(HConstants.REGIONSERVER_PORT, "0"); + if (conf.getInt(HConstants.REGIONSERVER_INFO_PORT, 0) != -1) { +conf.set(HConstants.REGIONSERVER_INFO_PORT, "0"); + } + if (conf.getInt(HConstants.MASTER_INFO_PORT, 0) != -1) { +conf.set(HConstants.MASTER_INFO_PORT, "0"); + } } this.masterClass = (Class) http://git-wip-us.apache.org/repos/asf/hbase/blob/67b7ce38/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java index 9959e31..a652284 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java @@ -108,9 +108,11 @@ public class MiniHBaseCluster extends HBaseCluster { Class regionserverClass) throws IOException, InterruptedException { super(conf); -conf.set(HConstants.MASTER_PORT, "0"); -if (conf.getInt(HConstants.MASTER_INFO_PORT, 0) != -1) { - conf.set(HConstants.MASTER_INFO_PORT, "0"); +if (conf.getBoolean(LocalHBaseCluster.ASSIGN_RANDOM_PORTS, true)) { + conf.set(HConstants.MASTER_PORT, "0"); + if (conf.getInt(HConstants.MASTER_INFO_PORT, 0) != -1) { +conf.set(HConstants.MASTER_INFO_PORT, "0"); + } } // Hadoop 2 http://git-wip-us.apache.org/repos/asf/hbase/blob/67b7ce38/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClusterPortAssignment.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClusterPortAssignment.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClusterPortAssignment.java new file mode 100644 index 000..0c8247f --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClusterPortAssignment.java @@ -0,0 +1,80 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not
[05/50] [abbrv] hbase git commit: HBASE-19166 AsyncProtobufLogWriter persists ProtobufLogWriter as class name for backward compatibility
HBASE-19166 AsyncProtobufLogWriter persists ProtobufLogWriter as class name for backward compatibility Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/5c988c99 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/5c988c99 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/5c988c99 Branch: refs/heads/HBASE-19397-branch-2 Commit: 5c988c99606080599534a751f061750f241c0945 Parents: d8ec10e Author: tedyuAuthored: Wed Feb 21 13:05:54 2018 -0800 Committer: tedyu Committed: Wed Feb 21 13:05:54 2018 -0800 -- .../hbase/regionserver/wal/AsyncProtobufLogWriter.java | 9 + .../regionserver/wal/SecureAsyncProtobufLogWriter.java | 8 2 files changed, 17 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/5c988c99/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java index 67258ec..abdc24e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java @@ -108,6 +108,15 @@ public class AsyncProtobufLogWriter extends AbstractProtobufLogWriter this.channelClass = channelClass; } + /* + * @return class name which is recognized by hbase-1.x to avoid ProtobufLogReader throwing error: + * IOException: Got unknown writer class: AsyncProtobufLogWriter + */ + @Override + protected String getWriterClassName() { +return "ProtobufLogWriter"; + } + @Override public void append(Entry entry) { int buffered = output.buffered(); http://git-wip-us.apache.org/repos/asf/hbase/blob/5c988c99/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureAsyncProtobufLogWriter.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureAsyncProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureAsyncProtobufLogWriter.java index 7741b5e..e2d294a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureAsyncProtobufLogWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureAsyncProtobufLogWriter.java @@ -38,6 +38,14 @@ public class SecureAsyncProtobufLogWriter extends AsyncProtobufLogWriter { super(eventLoopGroup, channelClass); } + /* + * @return class name which is recognized by hbase-1.x to avoid ProtobufLogReader throwing error: + * IOException: Got unknown writer class: SecureAsyncProtobufLogWriter + */ + @Override + protected String getWriterClassName() { +return "SecureProtobufLogWriter"; + } @Override protected WALHeader buildWALHeader(Configuration conf, WALHeader.Builder builder) throws IOException {
[38/50] [abbrv] hbase git commit: Revert "HBASE-19767 Fix for Master web UI shows negative values for Remaining KVs" Applied prematurely.
Revert "HBASE-19767 Fix for Master web UI shows negative values for Remaining KVs" Applied prematurely. This reverts commit 61b55166bf7fe9edc4e8105f217463ed6e693d17. Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/2440f807 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/2440f807 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/2440f807 Branch: refs/heads/HBASE-19064 Commit: 2440f807bf7d077def819c616d4afa97a4e2539e Parents: b328807 Author: Michael StackAuthored: Wed Feb 21 18:02:25 2018 -0800 Committer: Michael Stack Committed: Wed Feb 21 18:02:25 2018 -0800 -- .../hadoop/hbase/regionserver/HRegionServer.java| 2 +- .../apache/hadoop/hbase/regionserver/HStore.java| 4 ++-- .../compactions/CompactionProgress.java | 16 .../hbase/regionserver/TestMajorCompaction.java | 2 +- 4 files changed, 8 insertions(+), 16 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/2440f807/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index a76dec2..49b7b80 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -1623,7 +1623,7 @@ public class HRegionServer extends HasThread implements storefileIndexSizeKB += store.getStorefilesRootLevelIndexSize() / 1024; CompactionProgress progress = store.getCompactionProgress(); if (progress != null) { -totalCompactingKVs += progress.getTotalCompactingKVs(); +totalCompactingKVs += progress.totalCompactingKVs; currentCompactedKVs += progress.currentCompactedKVs; } rootLevelIndexSizeKB += (int) (store.getStorefilesRootLevelIndexSize() / 1024); http://git-wip-us.apache.org/repos/asf/hbase/blob/2440f807/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index c0ef3ef..bef50b4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -1373,10 +1373,10 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat writeCompactionWalRecord(filesToCompact, sfs); replaceStoreFiles(filesToCompact, sfs); if (cr.isMajor()) { - majorCompactedCellsCount += getCompactionProgress().getTotalCompactingKVs(); + majorCompactedCellsCount += getCompactionProgress().totalCompactingKVs; majorCompactedCellsSize += getCompactionProgress().totalCompactedSize; } else { - compactedCellsCount += getCompactionProgress().getTotalCompactingKVs(); + compactedCellsCount += getCompactionProgress().totalCompactingKVs; compactedCellsSize += getCompactionProgress().totalCompactedSize; } long outputBytes = getTotalSize(sfs); http://git-wip-us.apache.org/repos/asf/hbase/blob/2440f807/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java index 577276e..d40651a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java @@ -20,8 +20,6 @@ package org.apache.hadoop.hbase.regionserver.compactions; import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * This class holds information relevant for tracking the progress of a @@ -34,10 +32,9 @@ import org.slf4j.LoggerFactory; */ @InterfaceAudience.Private public class CompactionProgress { - private static final Logger LOG = LoggerFactory.getLogger(CompactionProgress.class); /** the total compacting key values in currently running compaction */ - private long totalCompactingKVs; + public long totalCompactingKVs; /** the completed count of key values in
[48/50] [abbrv] hbase git commit: HBASE-19082 Reject read/write from client but accept write from replication in state S
HBASE-19082 Reject read/write from client but accept write from replication in state S Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/595ac441 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/595ac441 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/595ac441 Branch: refs/heads/HBASE-19064 Commit: 595ac441f9bb87b1f7cb83fc5253cfbd709f7927 Parents: b54b94b Author: zhangduoAuthored: Mon Feb 12 18:20:18 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 14:48:42 2018 +0800 -- .../org/apache/hadoop/hbase/HConstants.java | 3 - .../src/main/protobuf/MasterProcedure.proto | 3 +- .../hbase/replication/ReplicationUtils.java | 4 + ...ransitPeerSyncReplicationStateProcedure.java | 10 + .../hadoop/hbase/regionserver/HRegion.java | 5 +- .../hbase/regionserver/HRegionServer.java | 2 +- .../hbase/regionserver/RSRpcServices.java | 89 +++-- .../RejectRequestsFromClientStateChecker.java | 44 .../regionserver/ReplicationSink.java | 72 --- .../SyncReplicationPeerInfoProvider.java| 10 +- .../SyncReplicationPeerInfoProviderImpl.java| 19 +- .../hbase/wal/SyncReplicationWALProvider.java | 3 + .../org/apache/hadoop/hbase/wal/WALFactory.java | 4 +- .../hbase/replication/TestSyncReplication.java | 200 +++ .../wal/TestSyncReplicationWALProvider.java | 8 +- 15 files changed, 401 insertions(+), 75 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/595ac441/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 4664778..1cd6f89 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 @@ -1351,9 +1351,6 @@ public final class HConstants { public static final String NOT_IMPLEMENTED = "Not implemented"; - // TODO: need to find a better place to hold it. - public static final String SYNC_REPLICATION_ENABLED = "hbase.replication.sync.enabled"; - private HConstants() { // Can't be instantiated with this ctor. } http://git-wip-us.apache.org/repos/asf/hbase/blob/595ac441/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 8cc5c81..2bf634b 100644 --- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto +++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto @@ -382,7 +382,8 @@ enum PeerSyncReplicationStateTransitionState { REOPEN_ALL_REGIONS_IN_PEER = 5; TRANSIT_PEER_NEW_SYNC_REPLICATION_STATE = 6; REFRESH_PEER_SYNC_REPLICATION_STATE_ON_RS_END = 7; - POST_PEER_SYNC_REPLICATION_STATE_TRANSITION = 8; + CREATE_DIR_FOR_REMOTE_WAL = 8; + POST_PEER_SYNC_REPLICATION_STATE_TRANSITION = 9; } message PeerModificationStateData { http://git-wip-us.apache.org/repos/asf/hbase/blob/595ac441/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 d633be9..9d0c05d 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 @@ -37,6 +37,10 @@ import org.apache.yetus.audience.InterfaceAudience; @InterfaceAudience.Private public final class ReplicationUtils { + public static final String SYNC_REPLICATION_ENABLED = "hbase.replication.sync.enabled"; + + public static final String REPLICATION_ATTR_NAME = "__rep__"; + private ReplicationUtils() { } http://git-wip-us.apache.org/repos/asf/hbase/blob/595ac441/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java index c253bff..e53abc0 100644 ---
[49/50] [abbrv] hbase git commit: HBASE-19864 Use protobuf instead of enum.ordinal to store SyncReplicationState
HBASE-19864 Use protobuf instead of enum.ordinal to store SyncReplicationState Signed-off-by: zhangduoProject: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/fb268839 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/fb268839 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/fb268839 Branch: refs/heads/HBASE-19064 Commit: fb268839f5392b4a20f192b4c784fa6b52da45f8 Parents: fbab0eb Author: Guanghao Zhang Authored: Fri Jan 26 16:50:48 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 14:48:42 2018 +0800 -- .../replication/ReplicationPeerConfigUtil.java | 22 ++--- .../hbase/replication/SyncReplicationState.java | 17 + .../hbase/shaded/protobuf/RequestConverter.java | 7 +++--- .../src/main/protobuf/Replication.proto | 13 ++ .../replication/ZKReplicationPeerStorage.java | 25 +--- .../hadoop/hbase/master/MasterRpcServices.java | 9 --- ...ransitPeerSyncReplicationStateProcedure.java | 9 --- .../TestReplicationSourceManager.java | 2 +- 8 files changed, 67 insertions(+), 37 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/fb268839/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java index 86b49ea..5096824 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java @@ -398,7 +398,7 @@ public final class ReplicationPeerConfigUtil { ReplicationProtos.ReplicationState.State.ENABLED == desc.getState().getState(); ReplicationPeerConfig config = convert(desc.getConfig()); return new ReplicationPeerDescription(desc.getId(), enabled, config, - SyncReplicationState.valueOf(desc.getSyncReplicationState().getNumber())); + toSyncReplicationState(desc.getSyncReplicationState())); } public static ReplicationProtos.ReplicationPeerDescription @@ -406,17 +406,33 @@ public final class ReplicationPeerConfigUtil { ReplicationProtos.ReplicationPeerDescription.Builder builder = ReplicationProtos.ReplicationPeerDescription.newBuilder(); builder.setId(desc.getPeerId()); + ReplicationProtos.ReplicationState.Builder stateBuilder = ReplicationProtos.ReplicationState.newBuilder(); stateBuilder.setState(desc.isEnabled() ? ReplicationProtos.ReplicationState.State.ENABLED : ReplicationProtos.ReplicationState.State.DISABLED); builder.setState(stateBuilder.build()); + builder.setConfig(convert(desc.getPeerConfig())); -builder.setSyncReplicationState( - ReplicationProtos.SyncReplicationState.forNumber(desc.getSyncReplicationState().ordinal())); + builder.setSyncReplicationState(toSyncReplicationState(desc.getSyncReplicationState())); + return builder.build(); } + public static ReplicationProtos.SyncReplicationState + toSyncReplicationState(SyncReplicationState state) { +ReplicationProtos.SyncReplicationState.Builder syncReplicationStateBuilder = +ReplicationProtos.SyncReplicationState.newBuilder(); +syncReplicationStateBuilder + .setState(ReplicationProtos.SyncReplicationState.State.forNumber(state.ordinal())); +return syncReplicationStateBuilder.build(); + } + + public static SyncReplicationState + toSyncReplicationState(ReplicationProtos.SyncReplicationState state) { +return SyncReplicationState.valueOf(state.getState().getNumber()); + } + public static ReplicationPeerConfig appendTableCFsToReplicationPeerConfig( Map tableCfs, ReplicationPeerConfig peerConfig) { ReplicationPeerConfigBuilder builder = ReplicationPeerConfig.newBuilder(peerConfig); http://git-wip-us.apache.org/repos/asf/hbase/blob/fb268839/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java index bd144e9..a65b144 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/SyncReplicationState.java
[43/50] [abbrv] hbase git commit: HBASE-19747 Introduce a special WALProvider for synchronous replication
HBASE-19747 Introduce a special WALProvider for synchronous replication Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/c5a0c7e8 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c5a0c7e8 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c5a0c7e8 Branch: refs/heads/HBASE-19064 Commit: c5a0c7e8d1b97f9be0e58e056de2f38ddac157d1 Parents: 8187688 Author: zhangduoAuthored: Fri Jan 19 18:38:39 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 14:48:42 2018 +0800 -- .../hbase/regionserver/wal/AbstractFSWAL.java | 7 + .../hbase/regionserver/wal/AsyncFSWAL.java | 1 - .../hbase/regionserver/wal/DualAsyncFSWAL.java | 4 +- .../hadoop/hbase/regionserver/wal/FSHLog.java | 3 - .../regionserver/PeerActionListener.java| 33 +++ .../SynchronousReplicationPeerProvider.java | 35 +++ .../hadoop/hbase/wal/AbstractFSWALProvider.java | 1 + .../hadoop/hbase/wal/AsyncFSWALProvider.java| 18 +- .../hbase/wal/NettyAsyncFSWALConfigHelper.java | 8 +- .../hbase/wal/RegionGroupingProvider.java | 13 +- .../wal/SynchronousReplicationWALProvider.java | 225 +++ .../org/apache/hadoop/hbase/wal/WALFactory.java | 37 ++- .../org/apache/hadoop/hbase/wal/WALKeyImpl.java | 16 +- .../regionserver/TestCompactionPolicy.java | 1 + .../regionserver/TestFailedAppendAndSync.java | 122 +- .../hadoop/hbase/regionserver/TestHRegion.java | 24 +- .../TestHRegionWithInMemoryFlush.java | 7 - .../hbase/regionserver/TestRegionIncrement.java | 20 +- .../hbase/regionserver/TestWALLockup.java | 1 + .../regionserver/wal/AbstractTestWALReplay.java | 1 + .../regionserver/wal/ProtobufLogTestHelper.java | 44 +++- .../hbase/regionserver/wal/TestAsyncFSWAL.java | 13 +- .../regionserver/wal/TestAsyncWALReplay.java| 4 +- .../wal/TestCombinedAsyncWriter.java| 3 +- .../hbase/regionserver/wal/TestFSHLog.java | 15 +- .../hbase/regionserver/wal/TestWALReplay.java | 1 + .../apache/hadoop/hbase/wal/IOTestProvider.java | 2 - .../TestSynchronousReplicationWALProvider.java | 153 + 28 files changed, 659 insertions(+), 153 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/c5a0c7e8/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java index ce8dafa..4816d77 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java @@ -430,6 +430,13 @@ public abstract class AbstractFSWAL implements WAL { this.implClassName = getClass().getSimpleName(); } + /** + * Used to initialize the WAL. Usually just call rollWriter to create the first log writer. + */ + public void init() throws IOException { +rollWriter(); + } + @Override public void registerWALActionsListener(WALActionsListener listener) { this.listeners.add(listener); http://git-wip-us.apache.org/repos/asf/hbase/blob/c5a0c7e8/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java index 91fb5b3..3d8c036 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java @@ -247,7 +247,6 @@ public class AsyncFSWAL extends AbstractFSWAL { batchSize = conf.getLong(WAL_BATCH_SIZE, DEFAULT_WAL_BATCH_SIZE); waitOnShutdownInSeconds = conf.getInt(ASYNC_WAL_WAIT_ON_SHUTDOWN_IN_SECONDS, DEFAULT_ASYNC_WAL_WAIT_ON_SHUTDOWN_IN_SECONDS); -rollWriter(); } private static boolean waitingRoll(int epochAndState) { http://git-wip-us.apache.org/repos/asf/hbase/blob/c5a0c7e8/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/DualAsyncFSWAL.java index 42b0dae..0495337 100644 ---
[46/50] [abbrv] hbase git commit: HBASE-19857 Complete the procedure for adding a sync replication peer
HBASE-19857 Complete the procedure for adding a sync replication peer Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/fbab0eb9 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/fbab0eb9 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/fbab0eb9 Branch: refs/heads/HBASE-19064 Commit: fbab0eb9a8b58d3c4c6f0b8890413b47ab4708e1 Parents: c457396 Author: zhangduoAuthored: Thu Jan 25 20:09:00 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 14:48:42 2018 +0800 -- .../hbase/replication/ReplicationPeer.java | 9 + .../hbase/replication/ReplicationPeerImpl.java | 28 +-- .../hbase/replication/ReplicationPeers.java | 3 +- .../regionserver/PeerActionListener.java| 10 +- .../SyncReplicationPeerProvider.java| 35 +++ .../SynchronousReplicationPeerProvider.java | 35 --- .../hbase/wal/SyncReplicationWALProvider.java | 234 +++ .../wal/SynchronousReplicationWALProvider.java | 225 -- .../org/apache/hadoop/hbase/wal/WALFactory.java | 8 +- .../TestReplicationSourceManager.java | 3 + .../wal/TestSyncReplicationWALProvider.java | 153 .../TestSynchronousReplicationWALProvider.java | 153 12 files changed, 456 insertions(+), 440 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/fbab0eb9/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java -- diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java index 2da3cce..0196a9a 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java @@ -54,6 +54,15 @@ public interface ReplicationPeer { PeerState getPeerState(); /** + * Returns the sync replication state of the peer by reading local cache. + * + * If the peer is not a synchronous replication peer, a {@link SyncReplicationState#NONE} will be + * returned. + * @return the sync replication state + */ + SyncReplicationState getSyncReplicationState(); + + /** * Test whether the peer is enabled. * @return {@code true} if enabled, otherwise {@code false}. */ http://git-wip-us.apache.org/repos/asf/hbase/blob/fbab0eb9/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java -- diff --git a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java index d656466..ff3f662 100644 --- a/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java +++ b/hbase-replication/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerImpl.java @@ -36,6 +36,8 @@ public class ReplicationPeerImpl implements ReplicationPeer { private volatile PeerState peerState; + private volatile SyncReplicationState syncReplicationState; + private final List peerConfigListeners; /** @@ -45,12 +47,13 @@ public class ReplicationPeerImpl implements ReplicationPeer { * @param id string representation of this peer's identifier * @param peerConfig configuration for the replication peer */ - public ReplicationPeerImpl(Configuration conf, String id, boolean peerState, - ReplicationPeerConfig peerConfig) { + public ReplicationPeerImpl(Configuration conf, String id, ReplicationPeerConfig peerConfig, + boolean peerState, SyncReplicationState syncReplicationState) { this.conf = conf; this.id = id; this.peerState = peerState ? PeerState.ENABLED : PeerState.DISABLED; this.peerConfig = peerConfig; +this.syncReplicationState = syncReplicationState; this.peerConfigListeners = new ArrayList<>(); } @@ -77,37 +80,26 @@ public class ReplicationPeerImpl implements ReplicationPeer { return peerState; } - /** - * Get the peer config object - * @return the ReplicationPeerConfig for this peer - */ + @Override + public SyncReplicationState getSyncReplicationState() { +return syncReplicationState; + } + @Override public ReplicationPeerConfig getPeerConfig() { return peerConfig; } - /** - * Get the configuration object required to communicate with this peer - * @return configuration object - */ @Override public Configuration getConfiguration() { return conf; } - /** -
[36/50] [abbrv] hbase git commit: HBASE-19391 Calling HRegion#initializeRegionInternals from a region replica can still re-create a region directory
HBASE-19391 Calling HRegion#initializeRegionInternals from a region replica can still re-create a region directory Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/a0900857 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/a0900857 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/a0900857 Branch: refs/heads/HBASE-19064 Commit: a0900857c7d58e03a39794e96224bf6213307ce7 Parents: 13223c2 Author: Esteban GutierrezAuthored: Wed Feb 21 15:28:50 2018 -0800 Committer: huaxiangsun Committed: Wed Feb 21 15:56:36 2018 -0800 -- .../hbase/regionserver/HRegionFileSystem.java | 29 ++-- 1 file changed, 15 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/a0900857/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 6ad3f1a..3c2d392 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 @@ -969,28 +969,29 @@ public class HRegionFileSystem { public static HRegionFileSystem createRegionOnFileSystem(final Configuration conf, final FileSystem fs, final Path tableDir, final RegionInfo regionInfo) throws IOException { HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tableDir, regionInfo); -Path regionDir = regionFs.getRegionDir(); -if (fs.exists(regionDir)) { - LOG.warn("Trying to create a region that already exists on disk: " + regionDir); - throw new IOException("The specified region already exists on disk: " + regionDir); -} +// We only create a .regioninfo and the region directory if this is the default region replica +if (regionInfo.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) { + Path regionDir = regionFs.getRegionDir(); + if (fs.exists(regionDir)) { +LOG.warn("Trying to create a region that already exists on disk: " + regionDir); +throw new IOException("The specified region already exists on disk: " + regionDir); + } -// Create the region directory -if (!createDirOnFileSystem(fs, conf, regionDir)) { - LOG.warn("Unable to create the region directory: " + regionDir); - throw new IOException("Unable to create region directory: " + regionDir); -} + // Create the region directory + if (!createDirOnFileSystem(fs, conf, regionDir)) { +LOG.warn("Unable to create the region directory: " + regionDir); +throw new IOException("Unable to create region directory: " + regionDir); + } -// Write HRI to a file in case we need to recover hbase:meta -// Only primary replicas should write region info -if (regionInfo.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) { + // Write HRI to a file in case we need to recover hbase:meta regionFs.writeRegionInfoOnFilesystem(false); + return regionFs; } else { if (LOG.isDebugEnabled()) LOG.debug("Skipping creation of .regioninfo file for " + regionInfo); } -return regionFs; +return null; } /**
[34/50] [abbrv] hbase git commit: HBASE-19767 Fix for Master web UI shows negative values for Remaining KVs
HBASE-19767 Fix for Master web UI shows negative values for Remaining KVs Negative Remaining KVs and progress percent greater than 100 is because CompactionProgress#totalCompactingKVs is sometimes less than CompactionProgress#currentCompactedKVs. Changes add a getter to CompactionProgress#totalCompactingKVs and from inside getter warning is logged. currentCompactedKVs are return when totalCompactingKVs are less than current. Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/61b55166 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/61b55166 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/61b55166 Branch: refs/heads/HBASE-19064 Commit: 61b55166bf7fe9edc4e8105f217463ed6e693d17 Parents: 3a39942 Author: Umesh AgasheAuthored: Wed Feb 21 13:59:59 2018 -0800 Committer: Michael Stack Committed: Wed Feb 21 15:14:24 2018 -0800 -- .../hadoop/hbase/regionserver/HRegionServer.java| 2 +- .../apache/hadoop/hbase/regionserver/HStore.java| 4 ++-- .../compactions/CompactionProgress.java | 16 .../hbase/regionserver/TestMajorCompaction.java | 2 +- 4 files changed, 16 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/61b55166/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 49b7b80..a76dec2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -1623,7 +1623,7 @@ public class HRegionServer extends HasThread implements storefileIndexSizeKB += store.getStorefilesRootLevelIndexSize() / 1024; CompactionProgress progress = store.getCompactionProgress(); if (progress != null) { -totalCompactingKVs += progress.totalCompactingKVs; +totalCompactingKVs += progress.getTotalCompactingKVs(); currentCompactedKVs += progress.currentCompactedKVs; } rootLevelIndexSizeKB += (int) (store.getStorefilesRootLevelIndexSize() / 1024); http://git-wip-us.apache.org/repos/asf/hbase/blob/61b55166/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index bef50b4..c0ef3ef 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -1373,10 +1373,10 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat writeCompactionWalRecord(filesToCompact, sfs); replaceStoreFiles(filesToCompact, sfs); if (cr.isMajor()) { - majorCompactedCellsCount += getCompactionProgress().totalCompactingKVs; + majorCompactedCellsCount += getCompactionProgress().getTotalCompactingKVs(); majorCompactedCellsSize += getCompactionProgress().totalCompactedSize; } else { - compactedCellsCount += getCompactionProgress().totalCompactingKVs; + compactedCellsCount += getCompactionProgress().getTotalCompactingKVs(); compactedCellsSize += getCompactionProgress().totalCompactedSize; } long outputBytes = getTotalSize(sfs); http://git-wip-us.apache.org/repos/asf/hbase/blob/61b55166/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java index d40651a..577276e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java @@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.regionserver.compactions; import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * This class holds information relevant for tracking the progress of a @@ -32,9 +34,10 @@ import org.apache.yetus.audience.InterfaceAudience; */ @InterfaceAudience.Private public class CompactionProgress
[47/50] [abbrv] hbase git commit: HBASE-19935 Only allow table replication for sync replication for now
HBASE-19935 Only allow table replication for sync replication for now Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/3d720648 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/3d720648 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/3d720648 Branch: refs/heads/HBASE-19064 Commit: 3d7206481679519986433adf1d41691d8a0ae08d Parents: fb26883 Author: Guanghao ZhangAuthored: Tue Feb 6 16:00:59 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 14:48:42 2018 +0800 -- .../replication/ReplicationPeerConfig.java | 9 +++ .../replication/ReplicationPeerManager.java | 34 - .../replication/TestReplicationAdmin.java | 73 ++-- .../wal/TestCombinedAsyncWriter.java| 6 ++ .../wal/TestSyncReplicationWALProvider.java | 6 ++ 5 files changed, 102 insertions(+), 26 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/3d720648/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 4c10c46..69565a7 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 @@ -25,6 +25,8 @@ import java.util.List; import java.util.Map; import java.util.Set; import java.util.TreeMap; + +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; @@ -217,6 +219,13 @@ public class ReplicationPeerConfig { return this.remoteWALDir; } + /** + * Use remote wal dir to decide whether a peer is sync replication peer + */ + public boolean isSyncReplication() { +return !StringUtils.isBlank(this.remoteWALDir); + } + public static ReplicationPeerConfigBuilder newBuilder() { return new ReplicationPeerConfigBuilderImpl(); } http://git-wip-us.apache.org/repos/asf/hbase/blob/3d720648/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 9336fbd..6bfd9c9 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 @@ -167,7 +167,7 @@ public class ReplicationPeerManager { " does not match new remote wal dir '" + peerConfig.getRemoteWALDir() + "'"); } -if (oldPeerConfig.getRemoteWALDir() != null) { +if (oldPeerConfig.isSyncReplication()) { if (!ReplicationUtils.isKeyConfigEqual(oldPeerConfig, peerConfig)) { throw new DoNotRetryIOException( "Changing the replicated namespace/table config on a synchronous replication " @@ -195,8 +195,8 @@ public class ReplicationPeerManager { } ReplicationPeerConfig copiedPeerConfig = ReplicationPeerConfig.newBuilder(peerConfig).build(); SyncReplicationState syncReplicationState = -StringUtils.isBlank(peerConfig.getRemoteWALDir()) ? SyncReplicationState.NONE -: SyncReplicationState.DOWNGRADE_ACTIVE; +copiedPeerConfig.isSyncReplication() ? SyncReplicationState.DOWNGRADE_ACTIVE +: SyncReplicationState.NONE; peerStorage.addPeer(peerId, copiedPeerConfig, enabled, syncReplicationState); peers.put(peerId, new ReplicationPeerDescription(peerId, enabled, copiedPeerConfig, syncReplicationState)); @@ -316,9 +316,37 @@ public class ReplicationPeerManager { peerConfig.getTableCFsMap()); } +if (peerConfig.isSyncReplication()) { + checkPeerConfigForSyncReplication(peerConfig); +} + checkConfiguredWALEntryFilters(peerConfig); } + private void checkPeerConfigForSyncReplication(ReplicationPeerConfig peerConfig) + throws DoNotRetryIOException { +// This is used to reduce the difficulty for implementing the sync replication state transition +// as we need to reopen all the related regions. +// TODO: Add namespace, replicat_all flag back +if (peerConfig.replicateAllUserTables()) { + throw new DoNotRetryIOException( + "Only
[42/50] [abbrv] hbase git commit: HBASE-19957 General framework to transit sync replication state
HBASE-19957 General framework to transit sync replication state Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/b54b94ba Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/b54b94ba Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/b54b94ba Branch: refs/heads/HBASE-19064 Commit: b54b94bab415926a07685341bb115a0dee0341a9 Parents: 3d72064 Author: zhangduoAuthored: Fri Feb 9 18:33:28 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 14:48:42 2018 +0800 -- .../replication/ReplicationPeerConfig.java | 2 - .../replication/ReplicationPeerDescription.java | 5 +- .../hbase/replication/SyncReplicationState.java | 19 +- .../org/apache/hadoop/hbase/HConstants.java | 3 + .../src/main/protobuf/MasterProcedure.proto | 20 ++- .../hbase/replication/ReplicationPeerImpl.java | 45 - .../replication/ReplicationPeerStorage.java | 25 ++- .../hbase/replication/ReplicationPeers.java | 27 ++- .../replication/ZKReplicationPeerStorage.java | 65 +-- .../hbase/coprocessor/MasterObserver.java | 7 +- .../org/apache/hadoop/hbase/master/HMaster.java | 4 +- .../hbase/master/MasterCoprocessorHost.java | 12 +- .../replication/AbstractPeerProcedure.java | 14 +- .../master/replication/ModifyPeerProcedure.java | 15 +- .../replication/RefreshPeerProcedure.java | 18 +- .../replication/ReplicationPeerManager.java | 107 +++- ...ransitPeerSyncReplicationStateProcedure.java | 175 --- .../hbase/regionserver/HRegionServer.java | 35 ++-- .../regionserver/ReplicationSourceService.java | 11 +- .../regionserver/PeerActionListener.java| 4 +- .../regionserver/PeerProcedureHandler.java | 16 +- .../regionserver/PeerProcedureHandlerImpl.java | 55 +- .../regionserver/RefreshPeerCallable.java | 7 + .../replication/regionserver/Replication.java | 22 ++- .../regionserver/ReplicationSourceManager.java | 41 +++-- .../SyncReplicationPeerInfoProvider.java| 43 + .../SyncReplicationPeerInfoProviderImpl.java| 71 .../SyncReplicationPeerMappingManager.java | 48 + .../SyncReplicationPeerProvider.java| 35 .../hbase/wal/SyncReplicationWALProvider.java | 35 ++-- .../org/apache/hadoop/hbase/wal/WALFactory.java | 47 ++--- .../replication/TestReplicationAdmin.java | 3 +- .../TestReplicationSourceManager.java | 5 +- .../wal/TestSyncReplicationWALProvider.java | 36 ++-- 34 files changed, 752 insertions(+), 325 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/b54b94ba/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 69565a7..79b3a1d 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 @@ -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 java.util.Collection; @@ -25,7 +24,6 @@ import java.util.List; import java.util.Map; import java.util.Set; import java.util.TreeMap; - import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.util.Bytes; http://git-wip-us.apache.org/repos/asf/hbase/blob/b54b94ba/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java index 2d077c5..b0c27bb 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerDescription.java @@ -20,7 +20,10 @@ package org.apache.hadoop.hbase.replication; import org.apache.yetus.audience.InterfaceAudience; /** - * The POJO equivalent of ReplicationProtos.ReplicationPeerDescription + * The POJO equivalent of ReplicationProtos.ReplicationPeerDescription. + * + * To developer, here we do not store the new sync replication state since it is just an + * intermediate state and this class is
[32/50] [abbrv] hbase git commit: HBASE-20042 TestRegionServerAbort flakey
HBASE-20042 TestRegionServerAbort flakey Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/3f82098d Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/3f82098d Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/3f82098d Branch: refs/heads/HBASE-19064 Commit: 3f82098d4b7ae595aa6702d3fb7cc2fac682691b Parents: 173a5bf Author: Michael StackAuthored: Wed Feb 21 13:24:32 2018 -0800 Committer: Michael Stack Committed: Wed Feb 21 13:26:10 2018 -0800 -- .../hbase/regionserver/TestRegionServerAbort.java | 12 1 file changed, 12 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/3f82098d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java index fdb6887..0c778fb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import java.io.IOException; +import java.util.List; import java.util.Optional; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -52,6 +53,7 @@ import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil; +import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hdfs.DFSConfigKeys; @@ -146,6 +148,16 @@ public class TestRegionServerAbort { // should have triggered an abort due to FileNotFoundException // verify that the regionserver is stopped +List regions = null; +do { + regions = cluster.findRegionsForTable(tableName); + if (regions != null && regions.size() > 0) { +break; + } + LOG.warn("Waiting on regions for {} to online"); + Threads.sleep(100); +} while(true); + HRegion firstRegion = cluster.findRegionsForTable(tableName).get(0); assertNotNull(firstRegion); assertNotNull(firstRegion.getRegionServerServices());
[23/50] [abbrv] hbase git commit: HBASE-20032 Receving multiple warnings for missing reporting.plugins.plugin.version
HBASE-20032 Receving multiple warnings for missing reporting.plugins.plugin.version Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/0068b95c Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/0068b95c Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/0068b95c Branch: refs/heads/HBASE-19064 Commit: 0068b95c854d13e359b619aaa9879b5cbb993729 Parents: b75a8f4 Author: Artem ErvitsAuthored: Tue Feb 20 15:15:34 2018 -0800 Committer: Michael Stack Committed: Tue Feb 20 15:16:22 2018 -0800 -- pom.xml | 4 1 file changed, 4 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/0068b95c/pom.xml -- diff --git a/pom.xml b/pom.xml index d33478a..951747e 100755 --- a/pom.xml +++ b/pom.xml @@ -3491,6 +3491,7 @@ maven-project-info-reports-plugin +2.9 @@ -3520,6 +3521,7 @@ org.apache.maven.plugins maven-javadoc-plugin +3.0.0 @@ -3721,6 +3723,7 @@ org.apache.maven.plugins maven-checkstyle-plugin +${maven.checkstyle.version} target/** @@ -3729,6 +3732,7 @@ org.scala-tools maven-scala-plugin +2.15.2
[24/50] [abbrv] hbase git commit: HBASE-20017 BufferedMutatorImpl submit the same mutation repeatedly
HBASE-20017 BufferedMutatorImpl submit the same mutation repeatedly Signed-off-by: Andrew PurtellProject: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/79d9403a Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/79d9403a Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/79d9403a Branch: refs/heads/HBASE-19064 Commit: 79d9403a79cca60e614834659e3d9005d5482cac Parents: 0068b95 Author: Chia-Ping Tsai Authored: Sun Feb 18 21:45:04 2018 +0800 Committer: Andrew Purtell Committed: Tue Feb 20 16:59:48 2018 -0800 -- .../hbase/client/BufferedMutatorImpl.java | 47 +++- .../hadoop/hbase/client/TestAsyncProcess.java | 44 ++ .../hbase/regionserver/wal/ReaderBase.java | 2 +- 3 files changed, 81 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/79d9403a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java index 9d24b4d..d4bc811 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java @@ -295,7 +295,7 @@ public class BufferedMutatorImpl implements BufferedMutator { break; } AsyncRequestFuture asf; - try (QueueRowAccess access = new QueueRowAccess()) { + try (QueueRowAccess access = createQueueRowAccess()) { if (access.isEmpty()) { // It means someone has gotten the ticker to run the flush. break; @@ -406,16 +406,46 @@ public class BufferedMutatorImpl implements BufferedMutator { return currentWriteBufferSize.get(); } + /** + * Count the mutations which haven't been processed. + * @return count of undealt mutation + */ @VisibleForTesting int size() { return undealtMutationCount.get(); } - private class QueueRowAccess implements RowAccess, Closeable { + /** + * Count the mutations which haven't been flushed + * @return count of unflushed mutation + */ + @VisibleForTesting + int getUnflushedSize() { +return writeAsyncBuffer.size(); + } + + @VisibleForTesting + QueueRowAccess createQueueRowAccess() { +return new QueueRowAccess(); + } + + @VisibleForTesting + class QueueRowAccess implements RowAccess, Closeable { private int remainder = undealtMutationCount.getAndSet(0); +private Mutation last = null; + +private void restoreLastMutation() { + // restore the last mutation since it isn't submitted + if (last != null) { +writeAsyncBuffer.add(last); +currentWriteBufferSize.addAndGet(last.heapSize()); +last = null; + } +} @Override public void close() { + restoreLastMutation(); if (remainder > 0) { undealtMutationCount.addAndGet(remainder); remainder = 0; @@ -425,25 +455,22 @@ public class BufferedMutatorImpl implements BufferedMutator { @Override public Iterator iterator() { return new Iterator() { -private final Iterator iter = writeAsyncBuffer.iterator(); private int countDown = remainder; -private Mutation last = null; @Override public boolean hasNext() { - if (countDown <= 0) { -return false; - } - return iter.hasNext(); + return countDown > 0; } @Override public Row next() { + restoreLastMutation(); if (!hasNext()) { throw new NoSuchElementException(); } - last = iter.next(); + last = writeAsyncBuffer.poll(); if (last == null) { throw new NoSuchElementException(); } + currentWriteBufferSize.addAndGet(-last.heapSize()); --countDown; return last; } @@ -452,8 +479,6 @@ public class BufferedMutatorImpl implements BufferedMutator { if (last == null) { throw new IllegalStateException(); } - iter.remove(); - currentWriteBufferSize.addAndGet(-last.heapSize()); --remainder; last = null; } http://git-wip-us.apache.org/repos/asf/hbase/blob/79d9403a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java -- diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncProcess.java
[40/50] [abbrv] hbase git commit: HBASE-19781 Add a new cluster state flag for synchronous replication
HBASE-19781 Add a new cluster state flag for synchronous replication Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/c4573964 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c4573964 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c4573964 Branch: refs/heads/HBASE-19064 Commit: c457396424c304cc91e55385baf4430e2dc68b0a Parents: c5a0c7e Author: Guanghao ZhangAuthored: Mon Jan 22 11:44:49 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 14:48:42 2018 +0800 -- .../org/apache/hadoop/hbase/client/Admin.java | 39 + .../apache/hadoop/hbase/client/AsyncAdmin.java | 31 .../hadoop/hbase/client/AsyncHBaseAdmin.java| 7 + .../hbase/client/ConnectionImplementation.java | 9 ++ .../apache/hadoop/hbase/client/HBaseAdmin.java | 26 +++ .../hadoop/hbase/client/RawAsyncHBaseAdmin.java | 15 ++ .../client/ShortCircuitMasterConnection.java| 9 ++ .../replication/ReplicationPeerConfigUtil.java | 26 +-- .../replication/ReplicationPeerDescription.java | 10 +- .../hbase/replication/SyncReplicationState.java | 48 ++ .../hbase/shaded/protobuf/RequestConverter.java | 10 ++ .../src/main/protobuf/Master.proto | 4 + .../src/main/protobuf/MasterProcedure.proto | 6 +- .../src/main/protobuf/Replication.proto | 20 +++ .../replication/ReplicationPeerStorage.java | 18 ++- .../hbase/replication/ReplicationUtils.java | 1 + .../replication/ZKReplicationPeerStorage.java | 60 +-- .../replication/TestReplicationStateBasic.java | 23 ++- .../TestZKReplicationPeerStorage.java | 12 +- .../hbase/coprocessor/MasterObserver.java | 23 +++ .../org/apache/hadoop/hbase/master/HMaster.java | 12 ++ .../hbase/master/MasterCoprocessorHost.java | 21 +++ .../hadoop/hbase/master/MasterRpcServices.java | 17 ++ .../hadoop/hbase/master/MasterServices.java | 9 ++ .../procedure/PeerProcedureInterface.java | 2 +- .../replication/ReplicationPeerManager.java | 51 +- ...ransitPeerSyncReplicationStateProcedure.java | 159 +++ .../hbase/security/access/AccessController.java | 8 + .../replication/TestReplicationAdmin.java | 62 .../hbase/master/MockNoopMasterServices.java| 11 +- .../cleaner/TestReplicationHFileCleaner.java| 4 +- .../TestReplicationTrackerZKImpl.java | 6 +- .../TestReplicationSourceManager.java | 3 +- .../security/access/TestAccessController.java | 16 ++ .../hbase/util/TestHBaseFsckReplication.java| 5 +- .../src/main/ruby/hbase/replication_admin.rb| 15 ++ hbase-shell/src/main/ruby/shell.rb | 1 + .../src/main/ruby/shell/commands/list_peers.rb | 6 +- .../transit_peer_sync_replication_state.rb | 44 + .../test/ruby/hbase/replication_admin_test.rb | 24 +++ 40 files changed, 818 insertions(+), 55 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/c4573964/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java index b8546fa..167d6f3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java @@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException; import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException; import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException; import org.apache.hadoop.hbase.snapshot.SnapshotCreationException; @@ -2648,6 +2649,44 @@ public interface Admin extends Abortable, Closeable { List listReplicationPeers(Pattern pattern) throws IOException; /** + * Transit current cluster to a new state in a synchronous replication peer. + * @param peerId a short name that identifies the peer + * @param state a new state of current cluster + * @throws IOException if a remote or network exception occurs + */ + void transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState state) + throws IOException; + + /** + * Transit current cluster to a new state in a synchronous replication peer. But does not block + * and wait for it. + * + * You can use Future.get(long, TimeUnit) to wait on the
[27/50] [abbrv] hbase git commit: HBASE-19953 Ensure post DDL hooks are only called after successful operations
HBASE-19953 Ensure post DDL hooks are only called after successful operations The 1.x functionality of Master DDL operations is that "post" observer hooks are only invoked when the DDL action was successful. With the async-ness of ProcV2, we find ourselves in a case where the post-hook may be invoked before the Procedure runs and fails. We need to introduce some blocking to wait and see if the Procedure is going to fail on a precondition before invoking the hook. Signed-off-by: Michael StackProject: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/d9b8dcc1 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/d9b8dcc1 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/d9b8dcc1 Branch: refs/heads/HBASE-19064 Commit: d9b8dcc1d300ae114febc22dbc71866088387111 Parents: a27ef55 Author: Josh Elser Authored: Thu Feb 15 18:00:09 2018 -0500 Committer: Josh Elser Committed: Wed Feb 21 14:02:49 2018 -0500 -- .../hadoop/hbase/master/ClusterSchema.java | 10 +- .../hbase/master/ClusterSchemaServiceImpl.java | 16 +- .../org/apache/hadoop/hbase/master/HMaster.java | 43 ++- .../hbase/master/TableNamespaceManager.java | 3 +- .../AbstractStateMachineNamespaceProcedure.java | 13 + .../procedure/CreateNamespaceProcedure.java | 22 +- .../procedure/DeleteNamespaceProcedure.java | 34 +- .../procedure/ModifyNamespaceProcedure.java | 30 +- .../master/procedure/ProcedurePrepareLatch.java | 14 + .../procedure/TestMasterObserverPostCalls.java | 368 +++ 10 files changed, 518 insertions(+), 35 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/d9b8dcc1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java index 746f9ad..56a1f33 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java @@ -22,6 +22,7 @@ import java.io.InterruptedIOException; import java.util.List; import org.apache.hadoop.hbase.NamespaceDescriptor; +import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.util.NonceKey; @@ -79,32 +80,35 @@ public interface ClusterSchema { * Create a new Namespace. * @param namespaceDescriptor descriptor for new Namespace * @param nonceKey A unique identifier for this operation from the client or process. + * @param latch A latch to block on for precondition validation * @return procedure id * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException} *as well as {@link IOException} */ - long createNamespace(NamespaceDescriptor namespaceDescriptor, NonceKey nonceKey) + long createNamespace(NamespaceDescriptor namespaceDescriptor, NonceKey nonceKey, ProcedurePrepareLatch latch) throws IOException; /** * Modify an existing Namespace. * @param nonceKey A unique identifier for this operation from the client or process. + * @param latch A latch to block on for precondition validation * @return procedure id * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException} *as well as {@link IOException} */ - long modifyNamespace(NamespaceDescriptor descriptor, NonceKey nonceKey) + long modifyNamespace(NamespaceDescriptor descriptor, NonceKey nonceKey, ProcedurePrepareLatch latch) throws IOException; /** * Delete an existing Namespace. * Only empty Namespaces (no tables) can be removed. * @param nonceKey A unique identifier for this operation from the client or process. + * @param latch A latch to block on for precondition validation * @return procedure id * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException} *as well as {@link IOException} */ - long deleteNamespace(String name, NonceKey nonceKey) + long deleteNamespace(String name, NonceKey nonceKey, ProcedurePrepareLatch latch) throws IOException; /** http://git-wip-us.apache.org/repos/asf/hbase/blob/d9b8dcc1/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
[33/50] [abbrv] hbase git commit: HBASE-2004 TestClientClusterStatus is flakey
HBASE-2004 TestClientClusterStatus is flakey Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/3a399422 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/3a399422 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/3a399422 Branch: refs/heads/HBASE-19064 Commit: 3a3994223c5d634bdd7ef01ef7f31ff860849575 Parents: 3f82098 Author: Michael StackAuthored: Wed Feb 21 14:52:10 2018 -0800 Committer: Michael Stack Committed: Wed Feb 21 14:53:31 2018 -0800 -- .../hadoop/hbase/TestClientClusterStatus.java | 16 +--- 1 file changed, 9 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/3a399422/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java index bfdae8a..392ff6e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterStatus.java @@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; +import org.apache.hadoop.hbase.util.Threads; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -100,13 +101,14 @@ public class TestClientClusterStatus { @Test public void testNone() throws Exception { -ClusterStatus status0 - = new ClusterStatus(ADMIN.getClusterMetrics(EnumSet.allOf(Option.class))); -ClusterStatus status1 - = new ClusterStatus(ADMIN.getClusterMetrics(EnumSet.noneOf(Option.class))); -Assert.assertEquals(status0, status1); -checkPbObjectNotNull(status0); -checkPbObjectNotNull(status1); +ClusterMetrics status0 = ADMIN.getClusterMetrics(EnumSet.allOf(Option.class)); +ClusterMetrics status1 = ADMIN.getClusterMetrics(EnumSet.noneOf(Option.class)); +// Do a rough compare. More specific compares can fail because all regions not deployed yet +// or more requests than expected. +Assert.assertEquals(status0.getLiveServerMetrics().size(), +status1.getLiveServerMetrics().size()); +checkPbObjectNotNull(new ClusterStatus(status0)); +checkPbObjectNotNull(new ClusterStatus(status1)); } @Test
[30/50] [abbrv] hbase git commit: HBASE-19166 AsyncProtobufLogWriter persists ProtobufLogWriter as class name for backward compatibility
HBASE-19166 AsyncProtobufLogWriter persists ProtobufLogWriter as class name for backward compatibility Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/bf5f0344 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/bf5f0344 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/bf5f0344 Branch: refs/heads/HBASE-19064 Commit: bf5f034463d357f31e2c7d02c6477c2fcd93d7f4 Parents: 5d994a2 Author: tedyuAuthored: Wed Feb 21 13:04:59 2018 -0800 Committer: tedyu Committed: Wed Feb 21 13:04:59 2018 -0800 -- .../hbase/regionserver/wal/AsyncProtobufLogWriter.java | 9 + .../regionserver/wal/SecureAsyncProtobufLogWriter.java | 8 2 files changed, 17 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/bf5f0344/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java index 67258ec..abdc24e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncProtobufLogWriter.java @@ -108,6 +108,15 @@ public class AsyncProtobufLogWriter extends AbstractProtobufLogWriter this.channelClass = channelClass; } + /* + * @return class name which is recognized by hbase-1.x to avoid ProtobufLogReader throwing error: + * IOException: Got unknown writer class: AsyncProtobufLogWriter + */ + @Override + protected String getWriterClassName() { +return "ProtobufLogWriter"; + } + @Override public void append(Entry entry) { int buffered = output.buffered(); http://git-wip-us.apache.org/repos/asf/hbase/blob/bf5f0344/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureAsyncProtobufLogWriter.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureAsyncProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureAsyncProtobufLogWriter.java index 7741b5e..e2d294a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureAsyncProtobufLogWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureAsyncProtobufLogWriter.java @@ -38,6 +38,14 @@ public class SecureAsyncProtobufLogWriter extends AsyncProtobufLogWriter { super(eventLoopGroup, channelClass); } + /* + * @return class name which is recognized by hbase-1.x to avoid ProtobufLogReader throwing error: + * IOException: Got unknown writer class: SecureAsyncProtobufLogWriter + */ + @Override + protected String getWriterClassName() { +return "SecureProtobufLogWriter"; + } @Override protected WALHeader buildWALHeader(Configuration conf, WALHeader.Builder builder) throws IOException {
[44/50] [abbrv] hbase git commit: HBASE-19083 Introduce a new log writer which can write to two HDFSes
HBASE-19083 Introduce a new log writer which can write to two HDFSes Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/f044dc49 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/f044dc49 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/f044dc49 Branch: refs/heads/HBASE-19064 Commit: f044dc4961ed328a8ad09f2de7c12a3334503f06 Parents: 2440f80 Author: zhangduoAuthored: Thu Jan 11 21:08:02 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 14:48:42 2018 +0800 -- .../hbase/regionserver/wal/AsyncFSWAL.java | 21 +-- .../regionserver/wal/CombinedAsyncWriter.java | 134 ++ .../hbase/regionserver/wal/DualAsyncFSWAL.java | 67 + .../wal/AbstractTestProtobufLog.java| 110 +++ .../regionserver/wal/ProtobufLogTestHelper.java | 99 ++ .../regionserver/wal/TestAsyncProtobufLog.java | 32 + .../wal/TestCombinedAsyncWriter.java| 136 +++ .../hbase/regionserver/wal/TestProtobufLog.java | 14 +- .../regionserver/wal/WriterOverAsyncWriter.java | 63 + 9 files changed, 533 insertions(+), 143 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/f044dc49/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java index d22d1ec..91fb5b3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java @@ -606,12 +606,16 @@ public class AsyncFSWAL extends AbstractFSWAL { } } - @Override - protected AsyncWriter createWriterInstance(Path path) throws IOException { + protected final AsyncWriter createAsyncWriter(FileSystem fs, Path path) throws IOException { return AsyncFSWALProvider.createAsyncWriter(conf, fs, path, false, eventLoopGroup, channelClass); } + @Override + protected AsyncWriter createWriterInstance(Path path) throws IOException { +return createAsyncWriter(fs, path); + } + private void waitForSafePoint() { consumeLock.lock(); try { @@ -631,13 +635,12 @@ public class AsyncFSWAL extends AbstractFSWAL { } } - private long closeWriter() { -AsyncWriter oldWriter = this.writer; -if (oldWriter != null) { - long fileLength = oldWriter.getLength(); + protected final long closeWriter(AsyncWriter writer) { +if (writer != null) { + long fileLength = writer.getLength(); closeExecutor.execute(() -> { try { - oldWriter.close(); + writer.close(); } catch (IOException e) { LOG.warn("close old writer failed", e); } @@ -652,7 +655,7 @@ public class AsyncFSWAL extends AbstractFSWAL { protected void doReplaceWriter(Path oldPath, Path newPath, AsyncWriter nextWriter) throws IOException { waitForSafePoint(); -long oldFileLen = closeWriter(); +long oldFileLen = closeWriter(this.writer); logRollAndSetupWalProps(oldPath, newPath, oldFileLen); this.writer = nextWriter; if (nextWriter != null && nextWriter instanceof AsyncProtobufLogWriter) { @@ -677,7 +680,7 @@ public class AsyncFSWAL extends AbstractFSWAL { @Override protected void doShutdown() throws IOException { waitForSafePoint(); -closeWriter(); +closeWriter(this.writer); closeExecutor.shutdown(); try { if (!closeExecutor.awaitTermination(waitOnShutdownInSeconds, TimeUnit.SECONDS)) { http://git-wip-us.apache.org/repos/asf/hbase/blob/f044dc49/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CombinedAsyncWriter.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CombinedAsyncWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CombinedAsyncWriter.java new file mode 100644 index 000..8ecfede --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/CombinedAsyncWriter.java @@ -0,0 +1,134 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a
[22/50] [abbrv] hbase git commit: HBASE-19954 Separate TestBlockReorder into individual tests to avoid ShutdownHook suppression error against hadoop3
HBASE-19954 Separate TestBlockReorder into individual tests to avoid ShutdownHook suppression error against hadoop3 Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/b75a8f40 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/b75a8f40 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/b75a8f40 Branch: refs/heads/HBASE-19064 Commit: b75a8f4047e2a33031b121dfe091a992809ebae1 Parents: 6b84393 Author: Ted YuAuthored: Tue Feb 20 14:56:20 2018 -0800 Committer: Michael Stack Committed: Tue Feb 20 14:56:20 2018 -0800 -- .../hadoop/hbase/fs/TestBlockReorder.java | 258 + .../hbase/fs/TestBlockReorderBlockLocation.java | 159 +++ .../hbase/fs/TestBlockReorderMultiBlocks.java | 279 +++ 3 files changed, 443 insertions(+), 253 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/b75a8f40/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java index 59d2229..a75c0d0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/fs/TestBlockReorder.java @@ -17,15 +17,10 @@ */ package org.apache.hadoop.hbase.fs; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.lang.reflect.Field; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.net.BindException; import java.net.ServerSocket; -import java.util.List; -import java.util.concurrent.CountDownLatch; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.FSDataInputStream; @@ -35,32 +30,14 @@ 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.MiniHBaseCluster; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.master.LoadBalancer; -import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.Region; -import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MiscTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; -import org.apache.hadoop.hdfs.DFSClient; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; -import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; -import org.apache.hadoop.hdfs.protocol.DirectoryListing; -import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.server.datanode.DataNode; -import org.apache.hadoop.ipc.RemoteException; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -74,6 +51,11 @@ import org.slf4j.LoggerFactory; /** * Tests for the hdfs fix from HBASE-6435. + * + * Please don't add new subtest which involves starting / stopping MiniDFSCluster in this class. + * When stopping MiniDFSCluster, shutdown hooks would be cleared in hadoop's ShutdownHookManager + * in hadoop 3. + * This leads to 'Failed suppression of fs shutdown hook' error in region server. */ @Category({MiscTests.class, LargeTests.class}) public class TestBlockReorder { @@ -249,234 +231,4 @@ public class TestBlockReorder { } } - /** - * Test that the hook works within HBase, including when there are multiple blocks. - */ - @Test() - public void testHBaseCluster() throws Exception { -byte[] sb = Bytes.toBytes("sb"); -htu.startMiniZKCluster(); - -MiniHBaseCluster hbm = htu.startMiniHBaseCluster(1, 1); -hbm.waitForActiveAndReadyMaster(); -HRegionServer targetRs = LoadBalancer.isTablesOnMaster(hbm.getConf())? hbm.getMaster(): - hbm.getRegionServer(0); - -// We want to have a datanode with the same name as the region server, so -// we're going to get the regionservername, and start a new datanode with this name. -
[01/50] [abbrv] hbase git commit: HBASE-19998 Flakey TestVisibilityLabelsWithDefaultVisLabelService [Forced Update!]
Repository: hbase Updated Branches: refs/heads/HBASE-19064 33d0606a2 -> 0ac769246 (forced update) HBASE-19998 Flakey TestVisibilityLabelsWithDefaultVisLabelService Only call server.checkIfShouldMoveSystemRegionAsync if a node has been added. Do not call it if only one regionserver in cluster. Make it so ServerCrashProcedure runs before it. Add logging if server.checkIfShouldMoveSystemRegionAsync was responsible for MOVE (Previous was a mystery when it cut in). Previous we'd call it when there was a nodeChildrenChanged. These happen before nodeDeleted. If a server crashed, checkIfShouldMoveSystemRegionAsync could run first, find the server that had not yet registered as crashed, find system tables on it and then try to move them. It would fail because server would not respond to RPC. The region move would then be waiting on the servercrashprocedure to wake it up when done processing but this move had locked the region so SCP couldn't run Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/50c705da Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/50c705da Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/50c705da Branch: refs/heads/HBASE-19064 Commit: 50c705dad9825d3095352b997be35a2568bd6190 Parents: 816d860 Author: Michael StackAuthored: Wed Feb 14 22:32:29 2018 -0800 Committer: Michael Stack Committed: Thu Feb 15 06:08:55 2018 -0800 -- .../hadoop/hbase/master/MasterServices.java | 5 + .../hbase/master/RegionServerTracker.java | 16 +++--- .../master/assignment/AssignmentManager.java| 23 ++-- .../master/assignment/MoveRegionProcedure.java | 1 - 4 files changed, 39 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/50c705da/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java index 9d371bd..0e552d6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java @@ -486,6 +486,11 @@ public interface MasterServices extends Server { public String getRegionServerVersion(final ServerName sn); + /** + * Called when a new RegionServer is added to the cluster. + * Checks if new server has a newer version than any existing server and will move system tables + * there if so. + */ public void checkIfShouldMoveSystemRegionAsync(); /** http://git-wip-us.apache.org/repos/asf/hbase/blob/50c705da/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java index 29218e2..81fc589 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionServerTracker.java @@ -104,9 +104,6 @@ public class RegionServerTracker extends ZKListener { } } } -if (server.isInitialized()) { - server.checkIfShouldMoveSystemRegionAsync(); -} } private void remove(final ServerName sn) { @@ -116,6 +113,19 @@ public class RegionServerTracker extends ZKListener { } @Override + public void nodeCreated(String path) { +if (path.startsWith(watcher.znodePaths.rsZNode)) { + String serverName = ZKUtil.getNodeName(path); + LOG.info("RegionServer ephemeral node created, adding [" + serverName + "]"); + if (server.isInitialized()) { +// Only call the check to move servers if a RegionServer was added to the cluster; in this +// case it could be a server with a new version so it makes sense to run the check. +server.checkIfShouldMoveSystemRegionAsync(); + } +} + } + + @Override public void nodeDeleted(String path) { if (path.startsWith(watcher.znodePaths.rsZNode)) { String serverName = ZKUtil.getNodeName(path); http://git-wip-us.apache.org/repos/asf/hbase/blob/50c705da/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
[39/50] [abbrv] hbase git commit: HBASE-19781 Add a new cluster state flag for synchronous replication
http://git-wip-us.apache.org/repos/asf/hbase/blob/c4573964/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java index 8911982..f5eca39 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckReplication.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeerStorage; import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; +import org.apache.hadoop.hbase.replication.SyncReplicationState; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter.ERROR_CODE; @@ -67,9 +68,9 @@ public class TestHBaseFsckReplication { String peerId1 = "1"; String peerId2 = "2"; peerStorage.addPeer(peerId1, ReplicationPeerConfig.newBuilder().setClusterKey("key").build(), - true); + true, SyncReplicationState.NONE); peerStorage.addPeer(peerId2, ReplicationPeerConfig.newBuilder().setClusterKey("key").build(), - true); + true, SyncReplicationState.NONE); for (int i = 0; i < 10; i++) { queueStorage.addWAL(ServerName.valueOf("localhost", 1 + i, 10 + i), peerId1, "file-" + i); http://git-wip-us.apache.org/repos/asf/hbase/blob/c4573964/hbase-shell/src/main/ruby/hbase/replication_admin.rb -- diff --git a/hbase-shell/src/main/ruby/hbase/replication_admin.rb b/hbase-shell/src/main/ruby/hbase/replication_admin.rb index ba7d191..d5d4844 100644 --- a/hbase-shell/src/main/ruby/hbase/replication_admin.rb +++ b/hbase-shell/src/main/ruby/hbase/replication_admin.rb @@ -20,6 +20,7 @@ include Java java_import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil +java_import org.apache.hadoop.hbase.replication.SyncReplicationState java_import org.apache.hadoop.hbase.replication.ReplicationPeerConfig java_import org.apache.hadoop.hbase.util.Bytes java_import org.apache.hadoop.hbase.zookeeper.ZKConfig @@ -329,6 +330,20 @@ module Hbase '!' + ReplicationPeerConfigUtil.convertToString(tableCFs) end +# Transit current cluster to a new state in the specified synchronous +# replication peer +def transit_peer_sync_replication_state(id, state) + if 'ACTIVE'.eql?(state) +@admin.transitReplicationPeerSyncReplicationState(id, SyncReplicationState::ACTIVE) + elsif 'DOWNGRADE_ACTIVE'.eql?(state) +@admin.transitReplicationPeerSyncReplicationState(id, SyncReplicationState::DOWNGRADE_ACTIVE) + elsif 'STANDBY'.eql?(state) +@admin.transitReplicationPeerSyncReplicationState(id, SyncReplicationState::STANDBY) + else +raise(ArgumentError, 'synchronous replication state must be ACTIVE, DOWNGRADE_ACTIVE or STANDBY') + end +end + #-- # Enables a table's replication switch def enable_tablerep(table_name) http://git-wip-us.apache.org/repos/asf/hbase/blob/c4573964/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 507c0a9..0ed71ae 100644 --- a/hbase-shell/src/main/ruby/shell.rb +++ b/hbase-shell/src/main/ruby/shell.rb @@ -395,6 +395,7 @@ Shell.load_command_group( get_peer_config list_peer_configs update_peer_config +transit_peer_sync_replication_state ] ) http://git-wip-us.apache.org/repos/asf/hbase/blob/c4573964/hbase-shell/src/main/ruby/shell/commands/list_peers.rb -- diff --git a/hbase-shell/src/main/ruby/shell/commands/list_peers.rb b/hbase-shell/src/main/ruby/shell/commands/list_peers.rb index caeab86..aa10fda 100644 --- a/hbase-shell/src/main/ruby/shell/commands/list_peers.rb +++ b/hbase-shell/src/main/ruby/shell/commands/list_peers.rb @@ -39,8 +39,8 @@ EOF peers = replication_admin.list_peers formatter.header(%w[PEER_ID CLUSTER_KEY ENDPOINT_CLASSNAME -REMOTE_ROOT_DIR STATE REPLICATE_ALL -NAMESPACES TABLE_CFS BANDWIDTH]) +REMOTE_ROOT_DIR SYNC_REPLICATION_STATE STATE +REPLICATE_ALL NAMESPACES TABLE_CFS BANDWIDTH])
[41/50] [abbrv] hbase git commit: HBASE-19957 General framework to transit sync replication state
http://git-wip-us.apache.org/repos/asf/hbase/blob/b54b94ba/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 85b2e85..7b8c43b 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 @@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.replication.ReplicationException; import org.apache.hadoop.hbase.replication.ReplicationListener; import org.apache.hadoop.hbase.replication.ReplicationPeer; import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState; +import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.ReplicationPeers; import org.apache.hadoop.hbase.replication.ReplicationQueueInfo; import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; @@ -136,6 +137,8 @@ public class ReplicationSourceManager implements ReplicationListener { // For recovered source, the queue id's format is peer_id-servername-* private final ConcurrentMap> walsByIdRecoveredQueues; + private final SyncReplicationPeerMappingManager syncReplicationPeerMappingManager; + private final Configuration conf; private final FileSystem fs; // The paths to the latest log of each wal group, for new coming peers @@ -172,9 +175,8 @@ public class ReplicationSourceManager implements ReplicationListener { public ReplicationSourceManager(ReplicationQueueStorage queueStorage, ReplicationPeers replicationPeers, ReplicationTracker replicationTracker, Configuration conf, Server server, FileSystem fs, Path logDir, Path oldLogDir, UUID clusterId, - WALFileLengthProvider walFileLengthProvider) throws IOException { -// CopyOnWriteArrayList is thread-safe. -// Generally, reading is more than modifying. + WALFileLengthProvider walFileLengthProvider, + SyncReplicationPeerMappingManager syncReplicationPeerMappingManager) throws IOException { this.sources = new ConcurrentHashMap<>(); this.queueStorage = queueStorage; this.replicationPeers = replicationPeers; @@ -187,10 +189,11 @@ public class ReplicationSourceManager implements ReplicationListener { this.fs = fs; this.logDir = logDir; this.oldLogDir = oldLogDir; -this.sleepBeforeFailover = conf.getLong("replication.sleep.before.failover", 3); // 30 - // seconds +// 30 seconds +this.sleepBeforeFailover = conf.getLong("replication.sleep.before.failover", 3); this.clusterId = clusterId; this.walFileLengthProvider = walFileLengthProvider; +this.syncReplicationPeerMappingManager = syncReplicationPeerMappingManager; this.replicationTracker.registerListener(this); // It's preferable to failover 1 RS at a time, but with good zk servers // more could be processed at the same time. @@ -254,8 +257,11 @@ public class ReplicationSourceManager implements ReplicationListener { } /** - * 1. Add peer to replicationPeers 2. Add the normal source and related replication queue 3. Add - * HFile Refs + * + * Add peer to replicationPeers + * Add the normal source and related replication queue + * Add HFile Refs + * * @param peerId the id of replication peer */ public void addPeer(String peerId) throws IOException { @@ -274,13 +280,16 @@ public class ReplicationSourceManager implements ReplicationListener { } /** - * 1. Remove peer for replicationPeers 2. Remove all the recovered sources for the specified id - * and related replication queues 3. Remove the normal source and related replication queue 4. - * Remove HFile Refs + * + * Remove peer for replicationPeers + * Remove all the recovered sources for the specified id and related replication queues + * Remove the normal source and related replication queue + * Remove HFile Refs + * * @param peerId the id of the replication peer */ public void removePeer(String peerId) { -replicationPeers.removePeer(peerId); +ReplicationPeer peer = replicationPeers.removePeer(peerId); String terminateMessage = "Replication stream was removed by a user"; List oldSourcesToDelete = new ArrayList<>(); // synchronized on oldsources to avoid adding recovered source for the to-be-removed peer @@ -311,7 +320,10 @@ public class ReplicationSourceManager implements ReplicationListener { deleteQueue(peerId);
[16/50] [abbrv] hbase git commit: HBASE-20020 Make sure we throw DoNotRetryIOException when ConnectionImplementation is closed
HBASE-20020 Make sure we throw DoNotRetryIOException when ConnectionImplementation is closed Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/b7685307 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/b7685307 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/b7685307 Branch: refs/heads/HBASE-19064 Commit: b7685307e4cc72035fc1b8737faec2e06a7c97df Parents: 391790d Author: zhangduoAuthored: Tue Feb 20 16:50:03 2018 +0800 Committer: Michael Stack Committed: Tue Feb 20 06:12:13 2018 -0800 -- .../hadoop/hbase/MasterNotRunningException.java | 4 +- .../hadoop/hbase/client/ClusterConnection.java | 30 ++- .../hbase/client/ConnectionImplementation.java | 86 +--- .../hadoop/hbase/client/ConnectionUtils.java| 8 +- .../org/apache/hadoop/hbase/client/HTable.java | 5 +- .../hadoop/hbase/client/MasterCallable.java | 11 ++- .../hbase/client/TestSnapshotFromAdmin.java | 4 +- .../hbase/client/TestHBaseAdminNoCluster.java | 4 +- 8 files changed, 59 insertions(+), 93 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/b7685307/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterNotRunningException.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterNotRunningException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterNotRunningException.java index 1ff17ac..35cdecb 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterNotRunningException.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterNotRunningException.java @@ -18,15 +18,13 @@ */ package org.apache.hadoop.hbase; -import java.io.IOException; - import org.apache.yetus.audience.InterfaceAudience; /** * Thrown if the master is not running */ @InterfaceAudience.Public -public class MasterNotRunningException extends IOException { +public class MasterNotRunningException extends HBaseIOException { private static final long serialVersionUID = (1L << 23) - 1L; /** default constructor */ public MasterNotRunningException() { http://git-wip-us.apache.org/repos/asf/hbase/blob/b7685307/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java index 7294559..3e055b0 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java @@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; import java.util.List; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MasterNotRunningException; @@ -29,12 +28,12 @@ import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ZooKeeperConnectionException; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; +import org.apache.yetus.audience.InterfaceAudience; + import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService; /** Internal methods on Connection that should not be used by user code. */ @InterfaceAudience.Private @@ -224,7 +223,7 @@ public interface ClusterConnection extends Connection { /** * Returns a {@link MasterKeepAliveConnection} to the active master */ - MasterService.BlockingInterface getMaster() throws IOException; + MasterKeepAliveConnection getMaster() throws IOException; /** * Get the admin service for master. @@ -258,9 +257,8 @@ public interface ClusterConnection extends Connection { * @return Location of row. * @throws IOException if a remote or network exception occurs */ - HRegionLocation getRegionLocation(TableName tableName, byte [] row, -boolean reload) - throws IOException; + HRegionLocation getRegionLocation(TableName tableName, byte[] row, boolean reload) + throws IOException; /** * Clear any caches that pertain to server name sn. @@ -269,24 +267,6 @@ public interface ClusterConnection extends Connection { void clearCaches(final ServerName sn);
[04/50] [abbrv] hbase git commit: HBASE-19988 Fixed chatty log from HRegion#lockRowsAndBuildMiniBatch() when interrupted while waiting on a row lock
HBASE-19988 Fixed chatty log from HRegion#lockRowsAndBuildMiniBatch() when interrupted while waiting on a row lock Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/9497eac4 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/9497eac4 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/9497eac4 Branch: refs/heads/HBASE-19064 Commit: 9497eac4e01a12acfa6660b5b16e73a45bcf5707 Parents: 40f8d20 Author: Umesh AgasheAuthored: Mon Feb 12 17:49:47 2018 -0800 Committer: Michael Stack Committed: Fri Feb 16 07:01:56 2018 -0800 -- .../java/org/apache/hadoop/hbase/regionserver/HRegion.java | 6 -- 1 file changed, 4 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/9497eac4/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 0fffe88..6170888 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 @@ -3166,8 +3166,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi try { // if atomic then get exclusive lock, else shared lock rowLock = region.getRowLockInternal(mutation.getRow(), !isAtomic(), prevRowLock); -} catch (TimeoutIOException e) { - // We will retry when other exceptions, but we should stop if we timeout . +} catch (TimeoutIOException|InterruptedIOException e) { + // NOTE: We will retry when other exceptions, but we should stop if we receive + // TimeoutIOException or InterruptedIOException as operation has timed out or + // interrupted respectively. throw e; } catch (IOException ioe) { LOG.warn("Failed getting lock, row=" + Bytes.toStringBinary(mutation.getRow()), ioe);
[37/50] [abbrv] hbase git commit: HBASE-20038 TestLockProcedure.testTimeout is flakey
HBASE-20038 TestLockProcedure.testTimeout is flakey Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/b328807d Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/b328807d Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/b328807d Branch: refs/heads/HBASE-19064 Commit: b328807d25ec0d2537371dd51d8ad79c841c3cec Parents: a090085 Author: zhangduoAuthored: Wed Feb 21 21:20:24 2018 +0800 Committer: Michael Stack Committed: Wed Feb 21 17:24:19 2018 -0800 -- .../org/apache/hadoop/hbase/master/locking/TestLockProcedure.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/b328807d/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java index 85b00d0..c985fa7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/locking/TestLockProcedure.java @@ -284,7 +284,7 @@ public class TestLockProcedure { sendHeartbeatAndCheckLocked(procId, true); Thread.sleep(HEARTBEAT_TIMEOUT / 2); sendHeartbeatAndCheckLocked(procId, true); -Thread.sleep(2 * HEARTBEAT_TIMEOUT); +Thread.sleep(4 * HEARTBEAT_TIMEOUT); sendHeartbeatAndCheckLocked(procId, false); ProcedureTestingUtility.waitProcedure(procExec, procId); ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
[19/50] [abbrv] hbase git commit: HBASE-20024 TestMergeTableRegionsProcedure is STILL flakey
HBASE-20024 TestMergeTableRegionsProcedure is STILL flakey Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/51cea3e2 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/51cea3e2 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/51cea3e2 Branch: refs/heads/HBASE-19064 Commit: 51cea3e2c3c75b82148f14ca7903a88daa3738d9 Parents: 2b19698 Author: Michael StackAuthored: Tue Feb 20 07:21:52 2018 -0800 Committer: Michael Stack Committed: Tue Feb 20 11:08:27 2018 -0800 -- .../hadoop/hbase/procedure2/ProcedureExecutor.java | 8 .../hadoop/hbase/procedure2/StateMachineProcedure.java | 9 ++--- .../master/assignment/MergeTableRegionsProcedure.java | 8 .../master/assignment/SplitTableRegionProcedure.java | 13 ++--- .../hbase/master/procedure/DeleteTableProcedure.java | 4 ++-- .../master/procedure/MasterProcedureScheduler.java | 2 +- .../assignment/TestMergeTableRegionsProcedure.java | 2 +- .../procedure/MasterProcedureTestingUtility.java | 2 +- 8 files changed, 29 insertions(+), 19 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/51cea3e2/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java -- diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java index af0a61b..665d223 100644 --- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java +++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java @@ -931,18 +931,18 @@ public class ProcedureExecutor { * Send an abort notification the specified procedure. * Depending on the procedure implementation the abort can be considered or ignored. * @param procId the procedure to abort - * @return true if the procedure exist and has received the abort, otherwise false. + * @return true if the procedure exists and has received the abort, otherwise false. */ public boolean abort(final long procId) { return abort(procId, true); } /** - * Send an abort notification the specified procedure. - * Depending on the procedure implementation the abort can be considered or ignored. + * Send an abort notification to the specified procedure. + * Depending on the procedure implementation, the abort can be considered or ignored. * @param procId the procedure to abort * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted? - * @return true if the procedure exist and has received the abort, otherwise false. + * @return true if the procedure exists and has received the abort, otherwise false. */ public boolean abort(final long procId, final boolean mayInterruptIfRunning) { final Procedure proc = procedures.get(procId); http://git-wip-us.apache.org/repos/asf/hbase/blob/51cea3e2/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java -- diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java index 20bba58..c530386 100644 --- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java +++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java @@ -209,18 +209,13 @@ public abstract class StateMachineProcedure @Override protected boolean abort(final TEnvironment env) { -final boolean isDebugEnabled = LOG.isDebugEnabled(); final TState state = getCurrentState(); -if (isDebugEnabled) { - LOG.debug("abort requested for " + this + " state=" + state); -} - +LOG.debug("Abort requested for {}", this); if (hasMoreState()) { aborted.set(true); return true; -} else if (isDebugEnabled) { - LOG.debug("ignoring abort request on state=" + state + " for " + this); } +LOG.debug("Ignoring abort request on {}", this); return false; } http://git-wip-us.apache.org/repos/asf/hbase/blob/51cea3e2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
[07/50] [abbrv] hbase git commit: HBASE-20011 Disable TestRestoreSnapshotFromClientWithRegionReplicas; it is flakey. Needs attention.
HBASE-20011 Disable TestRestoreSnapshotFromClientWithRegionReplicas; it is flakey. Needs attention. Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/abf7de70 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/abf7de70 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/abf7de70 Branch: refs/heads/HBASE-19064 Commit: abf7de702cb1851e70eef4afb849469a322102fa Parents: 0d6acfa Author: Michael StackAuthored: Fri Feb 16 14:45:55 2018 -0800 Committer: Michael Stack Committed: Fri Feb 16 14:46:24 2018 -0800 -- .../main/java/org/apache/hadoop/hbase/regionserver/HRegion.java | 4 .../client/TestRestoreSnapshotFromClientWithRegionReplicas.java | 2 ++ 2 files changed, 2 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/abf7de70/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 6170888..9464fdb 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 @@ -892,10 +892,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi if (this.getRegionInfo().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) { status.setStatus("Writing region info on filesystem"); fs.checkRegionInfoOnFilesystem(); -} else { - if (LOG.isDebugEnabled()) { -LOG.debug("Skipping creation of .regioninfo file for " + this.getRegionInfo()); - } } // Initialize all the HStores http://git-wip-us.apache.org/repos/asf/hbase/blob/abf7de70/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientWithRegionReplicas.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientWithRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientWithRegionReplicas.java index d301098..839542a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientWithRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRestoreSnapshotFromClientWithRegionReplicas.java @@ -25,10 +25,12 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.ClassRule; +import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; +@Ignore // Disabled because flakey. See HBASE-20006. @Category({LargeTests.class, ClientTests.class}) public class TestRestoreSnapshotFromClientWithRegionReplicas extends TestRestoreSnapshotFromClient {
[05/50] [abbrv] hbase git commit: HBASE-19991 hbase-rest test fail against h3
HBASE-19991 hbase-rest test fail against h3 Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/0d6acfa0 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/0d6acfa0 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/0d6acfa0 Branch: refs/heads/HBASE-19064 Commit: 0d6acfa0cf878172ef63d4c5900293f95c0a0fea Parents: 8cf7a9d Author: Mike DrobAuthored: Tue Feb 13 14:35:26 2018 -0600 Committer: Mike Drob Committed: Fri Feb 16 09:12:15 2018 -0600 -- hbase-rest/pom.xml | 6 pom.xml| 86 - 2 files changed, 85 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/0d6acfa0/hbase-rest/pom.xml -- diff --git a/hbase-rest/pom.xml b/hbase-rest/pom.xml index d4ab0b4..e70e596 100644 --- a/hbase-rest/pom.xml +++ b/hbase-rest/pom.xml @@ -411,12 +411,6 @@ org.apache.hadoop hadoop-mapreduce-client-core test - - - com.google.guava - guava - - org.apache.hadoop http://git-wip-us.apache.org/repos/asf/hbase/blob/0d6acfa0/pom.xml -- diff --git a/pom.xml b/pom.xml index 1cb5082..d33478a 100755 --- a/pom.xml +++ b/pom.xml @@ -1417,6 +1417,7 @@ 9.2.19.v20160908 3.1.0 2.0.1 + 2.25.1 2.3.2 3.0.1-b08 @@ -2502,6 +2503,7 @@ org.apache.hadoop hadoop-hdfs +${hadoop-two.version} org.apache.htrace @@ -2540,7 +2542,6 @@ guava -${hadoop-two.version} org.apache.hadoop @@ -2735,6 +2736,10 @@ org.codehaus.jackson jackson-mapper-asl + + com.google.guava + guava + @@ -2775,6 +2780,10 @@ jersey-core + com.sun.jersey + jersey-server + + javax.servlet.jsp jsp-api @@ -2843,6 +2852,24 @@ com.google.guava guava + + com.sun.jersey + jersey-core + + + + + org.apache.hadoop + hadoop-hdfs + ${hadoop-three.version} + test-jar + tests + test + + + com.sun.jersey + jersey-core + @@ -2917,6 +2944,19 @@ org.apache.hadoop + hadoop-common + test-jar + tests + ${hadoop-three.version} + + + com.sun.jersey + jersey-core + + + + + org.apache.hadoop hadoop-client ${hadoop-three.version} @@ -2963,6 +3003,50 @@ + org.apache.hadoop + hadoop-yarn-server-nodemanager + ${hadoop-three.version} + + + com.sun.jersey + jersey-core + + + + + org.apache.hadoop + hadoop-yarn-server-resourcemanager + ${hadoop-three.version} + + + com.sun.jersey + jersey-core + + + + + org.apache.hadoop + hadoop-yarn-server-timelineservice + ${hadoop-three.version} + + + javax.ws.rs + jsr311-api + + + + + org.apache.hadoop + hadoop-yarn-common + ${hadoop-three.version} + + + com.sun.jersey + jersey-core + + + + org.apache.hadoop hadoop-minikdc ${hadoop-three.version}
[09/50] [abbrv] hbase git commit: HBASE-20014 TestAdmin1 Times out
HBASE-20014 TestAdmin1 Times out Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/96989510 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/96989510 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/96989510 Branch: refs/heads/HBASE-19064 Commit: 969895105c77aaa34f4678d1924e83ebda7edb0f Parents: dad90f6 Author: Michael StackAuthored: Fri Feb 16 20:55:29 2018 -0800 Committer: Michael Stack Committed: Fri Feb 16 20:57:10 2018 -0800 -- .../src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java | 2 +- src/main/asciidoc/_chapters/developer.adoc | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/96989510/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java -- diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java index d47b213..a41e383 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java @@ -61,7 +61,7 @@ public final class HBaseClassTestRule implements TestRule { for (Class c : categories[0].value()) { if (c == SmallTests.class || c == MediumTests.class || c == LargeTests.class) { // All tests have a 10minute timeout. -return TimeUnit.MINUTES.toSeconds(10); +return TimeUnit.MINUTES.toSeconds(13); } } throw new IllegalArgumentException( http://git-wip-us.apache.org/repos/asf/hbase/blob/96989510/src/main/asciidoc/_chapters/developer.adoc -- diff --git a/src/main/asciidoc/_chapters/developer.adoc b/src/main/asciidoc/_chapters/developer.adoc index 3d8275f..3dbfe74 100644 --- a/src/main/asciidoc/_chapters/developer.adoc +++ b/src/main/asciidoc/_chapters/developer.adoc @@ -999,7 +999,7 @@ is that of the local test; i.e. in the TestTimeout Test Class, you'd pass `TestTimeout.class` to the `CLASS_RULE` instead of the `TestHRegionInfo.class` we have above. The `CLASS_RULE` is where we'll enforce timeouts (currently set at a hard-limit of -ten minutes for all tests) and other cross-unit test facility. +thirteen! minutes for all tests -- 780 seconds) and other cross-unit test facility. The test is in the `SmallTest` Category. Categories can be arbitrary and provided as a list but each test MUST
[15/50] [abbrv] hbase git commit: HBASE-19978 The keepalive logic is incomplete in ProcedureExecutor
HBASE-19978 The keepalive logic is incomplete in ProcedureExecutor Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/391790dd Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/391790dd Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/391790dd Branch: refs/heads/HBASE-19064 Commit: 391790ddb05f0db0ffd67e0c2f2f961157af7be3 Parents: a9a6eed Author: zhangduoAuthored: Mon Feb 19 08:08:06 2018 +0800 Committer: Michael Stack Committed: Mon Feb 19 17:13:47 2018 -0800 -- .../hbase/procedure2/DelayedProcedure.java | 28 ++ .../hadoop/hbase/procedure2/InlineChore.java| 42 +++ .../hbase/procedure2/ProcedureExecutor.java | 284 +-- .../hbase/procedure2/StoppableThread.java | 54 .../hbase/procedure2/TimeoutExecutorThread.java | 140 + .../org/apache/hadoop/hbase/master/HMaster.java | 11 +- .../master/procedure/TestProcedurePriority.java | 180 7 files changed, 522 insertions(+), 217 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/391790dd/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/DelayedProcedure.java -- diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/DelayedProcedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/DelayedProcedure.java new file mode 100644 index 000..fcec0b7 --- /dev/null +++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/DelayedProcedure.java @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.procedure2; + +import org.apache.hadoop.hbase.procedure2.util.DelayedUtil; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +class DelayedProcedure extends DelayedUtil.DelayedContainerWithTimestamp { + public DelayedProcedure(Procedure procedure) { +super(procedure, procedure.getTimeoutTimestamp()); + } +} http://git-wip-us.apache.org/repos/asf/hbase/blob/391790dd/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/InlineChore.java -- diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/InlineChore.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/InlineChore.java new file mode 100644 index 000..32b4922 --- /dev/null +++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/InlineChore.java @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.procedure2; + +import org.apache.hadoop.hbase.procedure2.util.DelayedUtil; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.yetus.audience.InterfaceAudience; + +/** + * Inline Chores (executors internal chores). + */ +@InterfaceAudience.Private +abstract class InlineChore extends DelayedUtil.DelayedObject implements Runnable { + + private long timeout; + + public abstract int getTimeoutInterval(); + + protected void refreshTimeout() { +this.timeout = EnvironmentEdgeManager.currentTime() + getTimeoutInterval();
[28/50] [abbrv] hbase git commit: HBASE-20031 Unable to run integration test using mvn due to missing HBaseClassTestRule
HBASE-20031 Unable to run integration test using mvn due to missing HBaseClassTestRule Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/401227ba Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/401227ba Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/401227ba Branch: refs/heads/HBASE-19064 Commit: 401227ba6aeb3c7767e88d41a7fa2990b3717648 Parents: d9b8dcc Author: tedyuAuthored: Wed Feb 21 12:03:32 2018 -0800 Committer: tedyu Committed: Wed Feb 21 12:03:32 2018 -0800 -- .../java/org/apache/hadoop/hbase/HBaseClassTestRule.java | 4 .../apache/hadoop/hbase/HBaseClassTestRuleChecker.java| 10 ++ src/main/asciidoc/_chapters/developer.adoc| 3 ++- 3 files changed, 16 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/401227ba/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java -- diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java index a41e383..c3bef0f 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRule.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase; import java.util.concurrent.TimeUnit; +import org.apache.hadoop.hbase.testclassification.IntegrationTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -63,6 +64,9 @@ public final class HBaseClassTestRule implements TestRule { // All tests have a 10minute timeout. return TimeUnit.MINUTES.toSeconds(13); } + if (c == IntegrationTests.class) { +return TimeUnit.MINUTES.toSeconds(Long.MAX_VALUE); + } } throw new IllegalArgumentException( clazz.getName() + " does not have SmallTests/MediumTests/LargeTests in @Category"); http://git-wip-us.apache.org/repos/asf/hbase/blob/401227ba/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRuleChecker.java -- diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRuleChecker.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRuleChecker.java index 97c657f..c374903 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRuleChecker.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseClassTestRuleChecker.java @@ -22,8 +22,12 @@ import static org.junit.Assert.fail; import java.lang.reflect.Field; import java.lang.reflect.Modifier; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.hbase.testclassification.IntegrationTests; import org.apache.yetus.audience.InterfaceAudience; import org.junit.ClassRule; +import org.junit.experimental.categories.Category; import org.junit.runner.Description; import org.junit.runner.notification.RunListener; import org.junit.runner.notification.RunListener.ThreadSafe; @@ -37,6 +41,12 @@ public class HBaseClassTestRuleChecker extends RunListener { @Override public void testStarted(Description description) throws Exception { +Category[] categories = description.getTestClass().getAnnotationsByType(Category.class); +for (Class c : categories[0].value()) { + if (c == IntegrationTests.class) { +return; + } +} for (Field field : description.getTestClass().getFields()) { if (Modifier.isStatic(field.getModifiers()) && field.getType() == HBaseClassTestRule.class && field.isAnnotationPresent(ClassRule.class)) { http://git-wip-us.apache.org/repos/asf/hbase/blob/401227ba/src/main/asciidoc/_chapters/developer.adoc -- diff --git a/src/main/asciidoc/_chapters/developer.adoc b/src/main/asciidoc/_chapters/developer.adoc index 3dbfe74..c752cd7 100644 --- a/src/main/asciidoc/_chapters/developer.adoc +++ b/src/main/asciidoc/_chapters/developer.adoc @@ -1014,7 +1014,7 @@ The first three categories, `small`, `medium`, and `large`, are for test cases w type `$ mvn test`. In other words, these three categorizations are for HBase unit tests. The `integration` category is not for unit tests, but for integration tests. -These are run when you invoke `$ mvn verify`. +These are normally run when you invoke `$ mvn verify`. Integration tests are described in < >. Keep reading to figure which annotation of the
[14/50] [abbrv] hbase git commit: HBASE-19950 Introduce a ColumnValueFilter
HBASE-19950 Introduce a ColumnValueFilter Signed-off-by: Chia-Ping TsaiProject: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/a9a6eed3 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/a9a6eed3 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/a9a6eed3 Branch: refs/heads/HBASE-19064 Commit: a9a6eed372fa11dd2851635ac2c4bb6e1ca9dba7 Parents: f3bb9b9 Author: Reid Chan Authored: Tue Feb 13 16:01:28 2018 +0800 Committer: Chia-Ping Tsai Committed: Tue Feb 20 04:56:13 2018 +0800 -- .../hadoop/hbase/filter/ColumnValueFilter.java | 241 +++ .../hadoop/hbase/filter/CompareFilter.java | 10 +- .../apache/hadoop/hbase/filter/ParseFilter.java | 4 +- .../src/main/protobuf/Filter.proto | 7 + hbase-protocol/src/main/protobuf/Filter.proto | 9 +- .../apache/hadoop/hbase/filter/TestFilter.java | 155 .../hbase/filter/TestFilterSerialization.java | 9 + .../hadoop/hbase/filter/TestParseFilter.java| 11 + 8 files changed, 440 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/a9a6eed3/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java new file mode 100644 index 000..0795165 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java @@ -0,0 +1,241 @@ +/** + * + * 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.filter; + +import java.io.IOException; +import java.util.ArrayList; + +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.CompareOperator; +import org.apache.hadoop.hbase.PrivateCellUtil; +import org.apache.hadoop.hbase.exceptions.DeserializationException; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.yetus.audience.InterfaceAudience; + +import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; +import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException; +import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; + +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; + +/** + * Different from {@link SingleColumnValueFilter} which returns an entire row + * when specified condition is matched, {@link ColumnValueFilter} return the matched cell only. + * + * This filter is used to filter cells based on column and value. + * It takes a {@link org.apache.hadoop.hbase.CompareOperator} operator (<, <=, =, !=, >, >=), and + * and a {@link ByteArrayComparable} comparator. + */ +@InterfaceAudience.Public +public class ColumnValueFilter extends FilterBase { + private final byte[] family; + private final byte[] qualifier; + private final CompareOperator op; + private final ByteArrayComparable comparator; + + // This flag is used to speed up seeking cells when matched column is found, such that following + // columns in the same row can be skipped faster by NEXT_ROW instead of NEXT_COL. + private boolean columnFound = false; + + public ColumnValueFilter(final byte[] family, final byte[] qualifier, + final CompareOperator op, final byte[] value) { +this(family, qualifier, op, new BinaryComparator(value)); + } + + public ColumnValueFilter(final byte[] family, final byte[] qualifier, + final CompareOperator op, + final ByteArrayComparable comparator) { +this.family = Preconditions.checkNotNull(family, "family should not be null."); +this.qualifier =
[50/50] [abbrv] hbase git commit: HBASE-19078 Add a remote peer cluster wal directory config for synchronous replication
HBASE-19078 Add a remote peer cluster wal directory config for synchronous replication Signed-off-by: zhangduoProject: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/81876883 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/81876883 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/81876883 Branch: refs/heads/HBASE-19064 Commit: 8187688316da60a506189b33ee754256184b46ff Parents: f044dc4 Author: Guanghao Zhang Authored: Sat Jan 13 18:55:28 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 14:48:42 2018 +0800 -- .../replication/ReplicationPeerConfigUtil.java | 6 ++ .../replication/ReplicationPeerConfig.java | 21 +- .../ReplicationPeerConfigBuilder.java | 7 ++ .../src/main/protobuf/Replication.proto | 1 + .../replication/ReplicationPeerManager.java | 15 .../replication/TestReplicationAdmin.java | 77 .../src/main/ruby/hbase/replication_admin.rb| 17 +++-- hbase-shell/src/main/ruby/hbase_constants.rb| 1 + .../src/main/ruby/shell/commands/add_peer.rb| 21 +- .../src/main/ruby/shell/commands/list_peers.rb | 19 - .../test/ruby/hbase/replication_admin_test.rb | 16 11 files changed, 188 insertions(+), 13 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/81876883/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java index a234a9b..642149b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationPeerConfigUtil.java @@ -315,6 +315,9 @@ public final class ReplicationPeerConfigUtil { excludeNamespacesList.stream().map(ByteString::toStringUtf8).collect(Collectors.toSet())); } +if (peer.hasRemoteWALDir()) { + builder.setRemoteWALDir(peer.getRemoteWALDir()); +} return builder.build(); } @@ -371,6 +374,9 @@ public final class ReplicationPeerConfigUtil { } } +if (peerConfig.getRemoteWALDir() != null) { + builder.setRemoteWALDir(peerConfig.getRemoteWALDir()); +} return builder.build(); } http://git-wip-us.apache.org/repos/asf/hbase/blob/81876883/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java index bf8d030..4c10c46 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerConfig.java @@ -46,6 +46,8 @@ public class ReplicationPeerConfig { private Map excludeTableCFsMap = null; private Set excludeNamespaces = null; private long bandwidth = 0; + // Used by synchronous replication + private String remoteWALDir; private ReplicationPeerConfig(ReplicationPeerConfigBuilderImpl builder) { this.clusterKey = builder.clusterKey; @@ -64,6 +66,7 @@ public class ReplicationPeerConfig { builder.excludeNamespaces != null ? Collections.unmodifiableSet(builder.excludeNamespaces) : null; this.bandwidth = builder.bandwidth; +this.remoteWALDir = builder.remoteWALDir; } private Map @@ -210,6 +213,10 @@ public class ReplicationPeerConfig { return this; } + public String getRemoteWALDir() { +return this.remoteWALDir; + } + public static ReplicationPeerConfigBuilder newBuilder() { return new ReplicationPeerConfigBuilderImpl(); } @@ -223,7 +230,8 @@ public class ReplicationPeerConfig { .setReplicateAllUserTables(peerConfig.replicateAllUserTables()) .setExcludeTableCFsMap(peerConfig.getExcludeTableCFsMap()) .setExcludeNamespaces(peerConfig.getExcludeNamespaces()) -.setBandwidth(peerConfig.getBandwidth()); +.setBandwidth(peerConfig.getBandwidth()) +.setRemoteWALDir(peerConfig.getRemoteWALDir()); return builder; } @@ -250,6 +258,8 @@ public class ReplicationPeerConfig { private long bandwidth = 0; +private String remoteWALDir = null; + @Override public
[31/50] [abbrv] hbase git commit: HBASE-20027 Add test TestClusterPortAssignment
HBASE-20027 Add test TestClusterPortAssignment LocalHBaseCluster forces random port assignment for sake of concurrent unit test execution friendliness, but we still need a positive test for RPC and info port assignment. Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/173a5bf1 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/173a5bf1 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/173a5bf1 Branch: refs/heads/HBASE-19064 Commit: 173a5bf1f1ff2f60ea9ef92bdef7a3026597ec0d Parents: bf5f034 Author: Andrew PurtellAuthored: Tue Feb 20 17:43:58 2018 -0800 Committer: Andrew Purtell Committed: Wed Feb 21 13:12:30 2018 -0800 -- .../apache/hadoop/hbase/LocalHBaseCluster.java | 15 +++- .../apache/hadoop/hbase/MiniHBaseCluster.java | 8 +- .../hadoop/hbase/TestClusterPortAssignment.java | 80 3 files changed, 96 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/173a5bf1/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java index 06199f7..e19e53b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java @@ -66,6 +66,8 @@ public class LocalHBaseCluster { public static final String LOCAL = "local"; /** 'local:' */ public static final String LOCAL_COLON = LOCAL + ":"; + public static final String ASSIGN_RANDOM_PORTS = "hbase.localcluster.assign.random.ports"; + private final Configuration conf; private final Class masterClass; private final Class regionServerClass; @@ -139,10 +141,15 @@ public class LocalHBaseCluster { // Always have masters and regionservers come up on port '0' so we don't // clash over default ports. -conf.set(HConstants.MASTER_PORT, "0"); -conf.set(HConstants.REGIONSERVER_PORT, "0"); -if (conf.getInt(HConstants.REGIONSERVER_INFO_PORT, 0) != -1) { - conf.set(HConstants.REGIONSERVER_INFO_PORT, "0"); +if (conf.getBoolean(ASSIGN_RANDOM_PORTS, true)) { + conf.set(HConstants.MASTER_PORT, "0"); + conf.set(HConstants.REGIONSERVER_PORT, "0"); + if (conf.getInt(HConstants.REGIONSERVER_INFO_PORT, 0) != -1) { +conf.set(HConstants.REGIONSERVER_INFO_PORT, "0"); + } + if (conf.getInt(HConstants.MASTER_INFO_PORT, 0) != -1) { +conf.set(HConstants.MASTER_INFO_PORT, "0"); + } } this.masterClass = (Class) http://git-wip-us.apache.org/repos/asf/hbase/blob/173a5bf1/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java index 9959e31..a652284 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java @@ -108,9 +108,11 @@ public class MiniHBaseCluster extends HBaseCluster { Class regionserverClass) throws IOException, InterruptedException { super(conf); -conf.set(HConstants.MASTER_PORT, "0"); -if (conf.getInt(HConstants.MASTER_INFO_PORT, 0) != -1) { - conf.set(HConstants.MASTER_INFO_PORT, "0"); +if (conf.getBoolean(LocalHBaseCluster.ASSIGN_RANDOM_PORTS, true)) { + conf.set(HConstants.MASTER_PORT, "0"); + if (conf.getInt(HConstants.MASTER_INFO_PORT, 0) != -1) { +conf.set(HConstants.MASTER_INFO_PORT, "0"); + } } // Hadoop 2 http://git-wip-us.apache.org/repos/asf/hbase/blob/173a5bf1/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClusterPortAssignment.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClusterPortAssignment.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClusterPortAssignment.java new file mode 100644 index 000..0c8247f --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClusterPortAssignment.java @@ -0,0 +1,80 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this
[45/50] [abbrv] hbase git commit: HBASE-19990 Create remote wal directory when transitting to state S
HBASE-19990 Create remote wal directory when transitting to state S Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/0ac76924 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/0ac76924 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/0ac76924 Branch: refs/heads/HBASE-19064 Commit: 0ac76924616157a13a905817896549441c576683 Parents: 595ac44 Author: zhangduoAuthored: Wed Feb 14 16:01:16 2018 +0800 Committer: zhangduo Committed: Thu Feb 22 14:48:42 2018 +0800 -- .../procedure2/ProcedureYieldException.java | 9 -- .../hbase/replication/ReplicationUtils.java | 2 ++ .../hadoop/hbase/master/MasterFileSystem.java | 23 .../master/procedure/MasterProcedureEnv.java| 5 ...ransitPeerSyncReplicationStateProcedure.java | 29 .../hbase/replication/TestSyncReplication.java | 8 ++ 6 files changed, 58 insertions(+), 18 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/0ac76924/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureYieldException.java -- diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureYieldException.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureYieldException.java index 0487ac5b..dbb9981 100644 --- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureYieldException.java +++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureYieldException.java @@ -15,16 +15,21 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.procedure2; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; -// TODO: Not used yet +/** + * Indicate that a procedure wants to be rescheduled. Usually because there are something wrong but + * we do not want to fail the procedure. + * + * TODO: need to support scheduling after a delay. + */ @InterfaceAudience.Private @InterfaceStability.Stable public class ProcedureYieldException extends ProcedureException { + /** default constructor */ public ProcedureYieldException() { super(); http://git-wip-us.apache.org/repos/asf/hbase/blob/0ac76924/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 9d0c05d..9699e7b 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 @@ -41,6 +41,8 @@ public final class ReplicationUtils { public static final String REPLICATION_ATTR_NAME = "__rep__"; + public static final String REMOTE_WAL_DIR_NAME = "remoteWALs"; + private ReplicationUtils() { } http://git-wip-us.apache.org/repos/asf/hbase/blob/0ac76924/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..9bb8858 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 @@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.log.HBaseMarkers; import org.apache.hadoop.hbase.mob.MobConstants; import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore; import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.replication.ReplicationUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.util.FSUtils; @@ -133,7 +134,6 @@ public class MasterFileSystem { * Idempotent. */ private void createInitialFileSystemLayout() throws IOException { - final String[] protectedSubDirs = new String[] { HConstants.BASE_NAMESPACE_DIR, HConstants.HFILE_ARCHIVE_DIRECTORY, @@ -145,7 +145,8 @@ public class MasterFileSystem { HConstants.HREGION_LOGDIR_NAME, HConstants.HREGION_OLDLOGDIR_NAME, HConstants.CORRUPT_DIR_NAME, - WALProcedureStore.MASTER_PROCEDURE_LOGDIR +
[11/50] [abbrv] hbase git commit: HBASE-20015 TestMergeTableRegionsProcedure and TestRegionMergeTransactionOnCluster flakey
HBASE-20015 TestMergeTableRegionsProcedure and TestRegionMergeTransactionOnCluster flakey Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/f3ff55a2 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/f3ff55a2 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/f3ff55a2 Branch: refs/heads/HBASE-19064 Commit: f3ff55a2b4bb7a8b4980fdbb5b1f7a8d033631f3 Parents: 68d509b Author: Michael StackAuthored: Fri Feb 16 22:59:32 2018 -0800 Committer: Michael Stack Committed: Sat Feb 17 11:38:26 2018 -0800 -- .../master/assignment/MergeTableRegionsProcedure.java | 14 ++ .../master/assignment/SplitTableRegionProcedure.java | 1 + 2 files changed, 11 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/f3ff55a2/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java index 4bccab7..bd41208 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java @@ -255,10 +255,16 @@ public class MergeTableRegionsProcedure throw new UnsupportedOperationException(this + " unhandled state=" + state); } } catch (IOException e) { - LOG.warn("Error trying to merge regions " + RegionInfo.getShortNameToLog(regionsToMerge) + -" in the table " + getTableName() + " (in state=" + state + ")", e); - - setFailure("master-merge-regions", e); + String msg = "Error trying to merge regions " + +RegionInfo.getShortNameToLog(regionsToMerge) + " in the table " + getTableName() + + " (in state=" + state + ")"; + if (!isRollbackSupported(state)) { +// We reach a state that cannot be rolled back. We just need to keep retry. +LOG.warn(msg, e); + } else { +LOG.error(msg, e); +setFailure("master-merge-regions", e); + } } return Flow.HAS_MORE_STATE; } http://git-wip-us.apache.org/repos/asf/hbase/blob/f3ff55a2/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 c139566..e898d6a 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 @@ -270,6 +270,7 @@ public class SplitTableRegionProcedure } else { LOG.error(msg, e); setFailure(e); +setFailure("master-split-regions", e); } } // if split fails, need to call ((HRegion)parent).clearSplit() when it is a force split
[21/50] [abbrv] hbase git commit: HBASE-19400 Add missing security checks in MasterRpcServices
HBASE-19400 Add missing security checks in MasterRpcServices - Added ADMIN permission check for following rpc calls: normalize, setNormalizerRunning, runCatalogScan, enableCatalogJanitor, runCleanerChore, setCleanerChoreRunning, execMasterService, execProcedure, execProcedureWithRet - Moved authorizationEnabled check to start of AccessChecker's functions. Currently, and IDK why, we call authManager.authorize() first and then discard its result if authorizationEnabled is false. Weird. Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/6b843936 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/6b843936 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/6b843936 Branch: refs/heads/HBASE-19064 Commit: 6b8439366134b6749895b6019bc068d169865b48 Parents: 148356a Author: Apekshit SharmaAuthored: Tue Feb 13 12:33:43 2018 -0800 Committer: Apekshit Sharma Committed: Tue Feb 20 13:37:20 2018 -0700 -- .../hbase/rsgroup/RSGroupAdminEndpoint.java | 3 +- .../hadoop/hbase/master/MasterRpcServices.java | 120 - .../hbase/regionserver/HRegionServer.java | 2 +- .../hbase/regionserver/RSRpcServices.java | 25 +- .../hbase/security/access/AccessChecker.java| 102 .../hbase/security/access/AccessController.java | 5 +- .../security/access/TestAccessController.java | 13 +- .../access/TestAdminOnlyOperations.java | 244 +++ 8 files changed, 389 insertions(+), 125 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/6b843936/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java -- diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java index 435d138..7fec32d 100644 --- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java +++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java @@ -79,7 +79,6 @@ import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.security.access.AccessChecker; import org.apache.hadoop.hbase.security.access.Permission.Action; -import org.apache.hadoop.hbase.security.access.TableAuthManager; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; @@ -127,7 +126,7 @@ public class RSGroupAdminEndpoint implements MasterCoprocessor, MasterObserver { @Override public void stop(CoprocessorEnvironment env) { -TableAuthManager.release(accessChecker.getAuthManager()); +accessChecker.stop(); } @Override http://git-wip-us.apache.org/repos/asf/hbase/blob/6b843936/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java index 377a9c6..b4f0faf 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java @@ -86,6 +86,7 @@ import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.access.AccessChecker; import org.apache.hadoop.hbase.security.access.AccessController; +import org.apache.hadoop.hbase.security.access.Permission; import org.apache.hadoop.hbase.security.visibility.VisibilityController; import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; @@ -349,6 +350,24 @@ public class MasterRpcServices extends RSRpcServices return new MasterAnnotationReadingPriorityFunction(this); } + /** + * Checks for the following pre-checks in order: + * + * Master is initialized + * Rpc caller has admin permissions + * + * @param requestName name of rpc request. Used in reporting failures to provide context. + * @throws ServiceException If any of the above listed pre-check fails. + */ + private void rpcPreCheck(String requestName) throws ServiceException { +try { + master.checkInitialized(); + requirePermission(requestName, Permission.Action.ADMIN); +} catch (IOException ioe) { + throw new ServiceException(ioe); +} + } + enum BalanceSwitchMode { SYNC, ASYNC @@ -393,25
[08/50] [abbrv] hbase git commit: HBASE-19680 BufferedMutatorImpl#mutate should wait the result from AP in order to throw the failed mutations
HBASE-19680 BufferedMutatorImpl#mutate should wait the result from AP in order to throw the failed mutations Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/dad90f6c Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/dad90f6c Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/dad90f6c Branch: refs/heads/HBASE-19064 Commit: dad90f6cce5bc51e43e3778068d5e45bcb1c9de0 Parents: abf7de7 Author: Chia-Ping TsaiAuthored: Sat Feb 17 07:16:14 2018 +0800 Committer: Chia-Ping Tsai Committed: Sat Feb 17 07:33:58 2018 +0800 -- .../hadoop/hbase/client/AsyncProcess.java | 54 + .../hadoop/hbase/client/AsyncRequestFuture.java | 6 +- .../hbase/client/AsyncRequestFutureImpl.java| 3 +- .../hbase/client/BufferedMutatorImpl.java | 241 +-- .../hbase/client/ConnectionImplementation.java | 2 +- .../org/apache/hadoop/hbase/client/HTable.java | 7 +- .../hadoop/hbase/client/HTableMultiplexer.java | 2 +- .../apache/hadoop/hbase/client/RowAccess.java | 4 +- .../hadoop/hbase/client/TestAsyncProcess.java | 90 +++ .../TestAsyncProcessWithRegionException.java| 2 +- .../hbase/client/HConnectionTestingUtility.java | 2 +- 11 files changed, 171 insertions(+), 242 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/dad90f6c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java index 6c4118c..de7449b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java @@ -65,17 +65,12 @@ import org.apache.hadoop.hbase.util.Bytes; * The class manages internally the retries. * * - * The class can be constructed in regular mode, or "global error" mode. In global error mode, - * AP tracks errors across all calls (each "future" also has global view of all errors). That - * mode is necessary for backward compat with HTable behavior, where multiple submissions are - * made and the errors can propagate using any put/flush call, from previous calls. - * In "regular" mode, the errors are tracked inside the Future object that is returned. + * The errors are tracked inside the Future object that is returned. * The results are always tracked inside the Future object and can be retrieved when the call * has finished. Partial results can also be retrieved if some part of multi-request failed. * * - * This class is thread safe in regular mode; in global error code, submitting operations and - * retrieving errors from different threads may be not thread safe. + * This class is thread safe. * Internally, the class is thread safe enough to manage simultaneously new submission and results * arising from older operations. * @@ -144,7 +139,6 @@ class AsyncProcess { final ClusterConnection connection; private final RpcRetryingCallerFactory rpcCallerFactory; final RpcControllerFactory rpcFactory; - final BatchErrors globalErrors; // Start configuration settings. final int startLogErrorsCnt; @@ -168,14 +162,12 @@ class AsyncProcess { private static final int DEFAULT_LOG_DETAILS_PERIOD = 1; private final int periodToLog; AsyncProcess(ClusterConnection hc, Configuration conf, - RpcRetryingCallerFactory rpcCaller, boolean useGlobalErrors, - RpcControllerFactory rpcFactory) { + RpcRetryingCallerFactory rpcCaller, RpcControllerFactory rpcFactory) { if (hc == null) { throw new IllegalArgumentException("ClusterConnection cannot be null."); } this.connection = hc; -this.globalErrors = useGlobalErrors ? new BatchErrors() : null; this.id = COUNTER.incrementAndGet(); @@ -445,10 +437,10 @@ class AsyncProcess { private Consumer getLogger(TableName tableName, long max) { return (currentInProgress) -> { - LOG.info("#" + id + (max < 0 ? ", waiting for any free slot" - : ", waiting for some tasks to finish. Expected max=" - + max) + ", tasksInProgress=" + currentInProgress + - " hasError=" + hasError() + (tableName == null ? "" : ", tableName=" + tableName)); + LOG.info("#" + id + (max < 0 ? + ", waiting for any free slot" : + ", waiting for some tasks to finish. Expected max=" + max) + ", tasksInProgress=" + + currentInProgress + (tableName == null ? "" : ", tableName=" + tableName)); }; } @@ -460,38 +452,6 @@ class AsyncProcess { void decTaskCounters(Collection regions,
[20/50] [abbrv] hbase git commit: HBASE-20029 @Ignore TestQuotaThrottle and TestReplicasClient#testCancelOfMultiGet
HBASE-20029 @Ignore TestQuotaThrottle and TestReplicasClient#testCancelOfMultiGet Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/148356a7 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/148356a7 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/148356a7 Branch: refs/heads/HBASE-19064 Commit: 148356a7bfeacce17aa85029348e5548579e78d6 Parents: 51cea3e Author: Michael StackAuthored: Tue Feb 20 12:08:35 2018 -0800 Committer: Michael Stack Committed: Tue Feb 20 12:09:47 2018 -0800 -- .../java/org/apache/hadoop/hbase/client/TestReplicasClient.java| 2 ++ .../java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java | 2 ++ 2 files changed, 4 insertions(+) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/148356a7/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java index c55c6ca..311f651 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicasClient.java @@ -61,6 +61,7 @@ import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; import org.junit.ClassRule; +import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; @@ -616,6 +617,7 @@ public class TestReplicasClient { } } + @Ignore // Disabled because it is flakey. Fails 17% on constrained GCE. %3 on Apache. @Test public void testCancelOfMultiGet() throws Exception { openRegion(hriSecondary); http://git-wip-us.apache.org/repos/asf/hbase/blob/148356a7/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java index a7b8d9d..3b06d92 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/TestQuotaThrottle.java @@ -41,11 +41,13 @@ import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; +import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +@Ignore // Disabled because flakey. Fails ~30% on a resource constrained GCE though not on Apache. @Category({RegionServerTests.class, MediumTests.class}) public class TestQuotaThrottle {
[17/50] [abbrv] hbase git commit: HBASE-20021 TestFromClientSideWithCoprocessor is flakey
HBASE-20021 TestFromClientSideWithCoprocessor is flakey Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/69d2becc Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/69d2becc Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/69d2becc Branch: refs/heads/HBASE-19064 Commit: 69d2becc73deeb0217a816fe923d4bcd70f3d65f Parents: b768530 Author: zhangduoAuthored: Tue Feb 20 20:07:27 2018 +0800 Committer: Michael Stack Committed: Tue Feb 20 06:24:42 2018 -0800 -- .../hadoop/hbase/client/TestFromClientSide.java| 17 ++--- .../client/TestFromClientSideWithCoprocessor.java | 17 + 2 files changed, 15 insertions(+), 19 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/69d2becc/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java index b028880..29d3439 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java @@ -139,24 +139,27 @@ public class TestFromClientSide { @Rule public TestName name = new TestName(); - @BeforeClass - public static void setUpBeforeClass() throws Exception { + protected static final void initialize(Class... cps) throws Exception { // Uncomment the following lines if more verbosity is needed for // debugging (see HBASE-12285 for details). -//((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL); -//((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL); -//((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL); +// ((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL); +// ((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL); +// ((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL); // make sure that we do not get the same ts twice, see HBASE-19731 for more details. EnvironmentEdgeManager.injectEdge(new NonRepeatedEnvironmentEdge()); Configuration conf = TEST_UTIL.getConfiguration(); conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, -MultiRowMutationEndpoint.class.getName()); + Arrays.stream(cps).map(Class::getName).toArray(String[]::new)); conf.setBoolean("hbase.table.sanity.checks", true); // enable for below tests -conf.setLong(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, 600); // We need more than one region server in this test TEST_UTIL.startMiniCluster(SLAVES); } + @BeforeClass + public static void setUpBeforeClass() throws Exception { +initialize(MultiRowMutationEndpoint.class); + } + @AfterClass public static void tearDownAfterClass() throws Exception { TEST_UTIL.shutdownMiniCluster(); http://git-wip-us.apache.org/repos/asf/hbase/blob/69d2becc/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java index 5bf70c4..37d0135 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSideWithCoprocessor.java @@ -17,9 +17,7 @@ */ package org.apache.hadoop.hbase.client; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint; import org.apache.hadoop.hbase.regionserver.NoOpScanPolicyObserver; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -29,23 +27,18 @@ import org.junit.ClassRule; import org.junit.experimental.categories.Category; /** - * Test all client operations with a coprocessor that - * just implements the default flush/compact/scan policy. + * Test all client operations with a coprocessor that just implements the default flush/compact/scan + * policy. */ -@Category({LargeTests.class, ClientTests.class}) +@Category({ LargeTests.class, ClientTests.class }) public class TestFromClientSideWithCoprocessor extends TestFromClientSide { @ClassRule public static final HBaseClassTestRule CLASS_RULE = -
[18/50] [abbrv] hbase git commit: HBASE-20023 CompactionTool command line examples are incorrect
HBASE-20023 CompactionTool command line examples are incorrect Signed-off-by: tedyuProject: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/2b196989 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/2b196989 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/2b196989 Branch: refs/heads/HBASE-19064 Commit: 2b1969898f76e227ba3c89c7e5e8127f11236496 Parents: 69d2bec Author: Artem Ervits Authored: Tue Feb 20 10:15:27 2018 -0500 Committer: tedyu Committed: Tue Feb 20 08:13:12 2018 -0800 -- .../org/apache/hadoop/hbase/regionserver/CompactionTool.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/2b196989/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java -- diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java index 00ad54b..78db6fc 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java @@ -460,10 +460,10 @@ public class CompactionTool extends Configured implements Tool { System.err.println(); System.err.println("Examples:"); System.err.println(" To compact the full 'TestTable' using MapReduce:"); -System.err.println(" $ hbase " + this.getClass().getName() + " -mapred hdfs:///hbase/data/default/TestTable"); +System.err.println(" $ hbase " + this.getClass().getName() + " -mapred hdfs://hbase/data/default/TestTable"); System.err.println(); System.err.println(" To compact column family 'x' of the table 'TestTable' region 'abc':"); -System.err.println(" $ hbase " + this.getClass().getName() + " hdfs:///hbase/data/default/TestTable/abc/x"); +System.err.println(" $ hbase " + this.getClass().getName() + " hdfs://hbase/data/default/TestTable/abc/x"); } public static void main(String[] args) throws Exception {
[02/50] [abbrv] hbase git commit: HBASE-19903 Split TestShell so it will not time out
HBASE-19903 Split TestShell so it will not time out Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/01a28693 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/01a28693 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/01a28693 Branch: refs/heads/HBASE-19064 Commit: 01a2869363d45c7692f3984f31ab8c7a50d117fc Parents: 50c705d Author: zhangduoAuthored: Thu Feb 15 21:11:14 2018 +0800 Committer: zhangduo Committed: Fri Feb 16 00:17:19 2018 +0800 -- .../hadoop/hbase/client/TestAdminShell.java | 42 .../hadoop/hbase/client/TestQuotasShell.java| 42 .../apache/hadoop/hbase/client/TestShell.java | 6 +-- .../hadoop/hbase/client/TestShellNoCluster.java | 4 +- .../hadoop/hbase/client/TestTableShell.java | 42 5 files changed, 131 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/01a28693/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestAdminShell.java -- diff --git a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestAdminShell.java b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestAdminShell.java new file mode 100644 index 000..1835d88 --- /dev/null +++ b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestAdminShell.java @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.jruby.embed.PathType; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ ClientTests.class, LargeTests.class }) +public class TestAdminShell extends AbstractTestShell { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = +HBaseClassTestRule.forClass(TestAdminShell.class); + + @Test + public void testRunShellTests() throws IOException { +System.setProperty("shell.test.include", "admin_test.rb"); +// Start all ruby tests +jruby.runScriptlet(PathType.ABSOLUTE, "src/test/ruby/tests_runner.rb"); + } +} http://git-wip-us.apache.org/repos/asf/hbase/blob/01a28693/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestQuotasShell.java -- diff --git a/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestQuotasShell.java b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestQuotasShell.java new file mode 100644 index 000..482bf0f --- /dev/null +++ b/hbase-shell/src/test/java/org/apache/hadoop/hbase/client/TestQuotasShell.java @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.client; + +import java.io.IOException; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.jruby.embed.PathType;
[25/50] [abbrv] hbase git commit: HBASE-19506: The CellChunkMap index chunks are usually small, so in order to prevent memory underutilization, HBASE-19506 presents small chunks preallocated in a smal
HBASE-19506: The CellChunkMap index chunks are usually small, so in order to prevent memory underutilization, HBASE-19506 presents small chunks preallocated in a small pool Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/92d04d57 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/92d04d57 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/92d04d57 Branch: refs/heads/HBASE-19064 Commit: 92d04d57516391376d34aaca6307e5906ffa4ae8 Parents: 79d9403 Author: anastasAuthored: Wed Feb 21 10:12:25 2018 +0200 Committer: anastas Committed: Wed Feb 21 10:12:25 2018 +0200 -- .../regionserver/CellChunkImmutableSegment.java | 76 +++-- .../hadoop/hbase/regionserver/CellChunkMap.java | 17 +- .../apache/hadoop/hbase/regionserver/Chunk.java | 4 + .../hadoop/hbase/regionserver/ChunkCreator.java | 324 ++- .../regionserver/ImmutableMemStoreLAB.java | 28 +- .../hadoop/hbase/regionserver/MemStoreLAB.java | 22 +- .../hbase/regionserver/MemStoreLABImpl.java | 74 +++-- .../regionserver/TestCompactingMemStore.java| 3 +- .../regionserver/TestMemStoreChunkPool.java | 6 +- .../hbase/regionserver/TestMemStoreLAB.java | 8 +- .../TestMemstoreLABWithoutPool.java | 2 +- 11 files changed, 377 insertions(+), 187 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/92d04d57/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java index 53458f1..0eebfb5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java @@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.util.ClassSize; import org.apache.yetus.audience.InterfaceAudience; - /** * CellChunkImmutableSegment extends the API supported by a {@link Segment}, * and {@link ImmutableSegment}. This immutable segment is working with CellSet with @@ -43,6 +42,7 @@ public class CellChunkImmutableSegment extends ImmutableSegment { public static final long DEEP_OVERHEAD_CCM = ImmutableSegment.DEEP_OVERHEAD + ClassSize.CELL_CHUNK_MAP; + public static final float INDEX_CHUNK_UNUSED_SPACE_PRECENTAGE = 0.1f; / CONSTRUCTORS / /** @@ -135,20 +135,12 @@ public class CellChunkImmutableSegment extends ImmutableSegment { private void initializeCellSet(int numOfCells, MemStoreSegmentsIterator iterator, MemStoreCompactionStrategy.Action action) { -// calculate how many chunks we will need for index -int chunkSize = ChunkCreator.getInstance().getChunkSize(); -int numOfCellsInChunk = CellChunkMap.NUM_OF_CELL_REPS_IN_CHUNK; -int numberOfChunks = calculateNumberOfChunks(numOfCells, numOfCellsInChunk); int numOfCellsAfterCompaction = 0; int currentChunkIdx = 0; int offsetInCurentChunk = ChunkCreator.SIZEOF_CHUNK_HEADER; int numUniqueKeys=0; Cell prev = null; -// all index Chunks are allocated from ChunkCreator -Chunk[] chunks = new Chunk[numberOfChunks]; -for (int i=0; i < numberOfChunks; i++) { - chunks[i] = this.getMemStoreLAB().getNewExternalChunk(); -} +Chunk[] chunks = allocIndexChunks(numOfCells); while (iterator.hasNext()) {// the iterator hides the elimination logic for compaction boolean alreadyCopied = false; Cell c = iterator.next(); @@ -161,7 +153,7 @@ public class CellChunkImmutableSegment extends ImmutableSegment { c = copyCellIntoMSLAB(c); alreadyCopied = true; } - if (offsetInCurentChunk + ClassSize.CELL_CHUNK_MAP_ENTRY > chunkSize) { + if (offsetInCurentChunk + ClassSize.CELL_CHUNK_MAP_ENTRY > chunks[currentChunkIdx].size) { currentChunkIdx++; // continue to the next index chunk offsetInCurentChunk = ChunkCreator.SIZEOF_CHUNK_HEADER; } @@ -207,15 +199,7 @@ public class CellChunkImmutableSegment extends ImmutableSegment { int numOfCells, KeyValueScanner segmentScanner, CellSet oldCellSet, MemStoreCompactionStrategy.Action action) { Cell curCell; -// calculate how many chunks we will need for metadata -int chunkSize = ChunkCreator.getInstance().getChunkSize(); -int numOfCellsInChunk = CellChunkMap.NUM_OF_CELL_REPS_IN_CHUNK; -int
[03/50] [abbrv] hbase git commit: HBASE-20006 TestRestoreSnapshotFromClientWithRegionReplicas is flakey Remove assert in splittableregionprocedure. It was in the prepare. Was causing fail in legit cas
HBASE-20006 TestRestoreSnapshotFromClientWithRegionReplicas is flakey Remove assert in splittableregionprocedure. It was in the prepare. Was causing fail in legit case where a region split follows a table split BEFORE the parent has been GC'd. The region split finds the parent in SPLIT state which is right. The assert was having us fail. No need. Also disabled TestHTrace since not supported in 2.0.0 and flakey. Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/40f8d20c Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/40f8d20c Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/40f8d20c Branch: refs/heads/HBASE-19064 Commit: 40f8d20cf7b297a9324319190d03d93563230d6e Parents: 01a2869 Author: Michael StackAuthored: Thu Feb 15 20:43:47 2018 -0800 Committer: Michael Stack Committed: Thu Feb 15 20:46:32 2018 -0800 -- .../hadoop/hbase/master/assignment/SplitTableRegionProcedure.java | 1 - .../main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java | 2 +- .../test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java | 2 ++ 3 files changed, 3 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/40f8d20c/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 be0741d..c139566 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 @@ -225,7 +225,6 @@ public class SplitTableRegionProcedure setNextState(SplitTableRegionState.SPLIT_TABLE_REGION_PRE_OPERATION); break; } else { - assert isFailed() : "split region should have an exception here"; return Flow.NO_MORE_STATE; } case SPLIT_TABLE_REGION_PRE_OPERATION: http://git-wip-us.apache.org/repos/asf/hbase/blob/40f8d20c/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java index a32a493..58f8f81 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java @@ -120,7 +120,7 @@ public class StoreUtils { if (comparator.compareRows(midKey, firstKey) == 0 || comparator.compareRows(midKey, lastKey) == 0) { if (LOG.isDebugEnabled()) { -LOG.debug("cannot split because midkey is the same as first or last row"); +LOG.debug("cannot split {} because midkey is the same as first or last row", file); } return Optional.empty(); } http://git-wip-us.apache.org/repos/asf/hbase/blob/40f8d20c/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java index c39afdd..4513dd2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java @@ -38,6 +38,7 @@ import org.apache.htrace.core.TraceScope; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -45,6 +46,7 @@ import org.junit.rules.TestName; import org.apache.hbase.thirdparty.com.google.common.collect.Sets; +@Ignore // We don't support htrace in hbase-2.0.0 and this flakey is a little flakey. @Category({MiscTests.class, MediumTests.class}) public class TestHTraceHooks {
[13/50] [abbrv] hbase git commit: HBASE-18294 Reduce global heap pressure: flush based on heap occupancy
HBASE-18294 Reduce global heap pressure: flush based on heap occupancy Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/f3bb9b96 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/f3bb9b96 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/f3bb9b96 Branch: refs/heads/HBASE-19064 Commit: f3bb9b9613beaefbee0a53be1fb66b94fde1ce19 Parents: f3ff55a Author: eshcarAuthored: Sun Feb 18 09:55:44 2018 +0200 Committer: eshcar Committed: Sun Feb 18 09:55:44 2018 +0200 -- .../hadoop/hbase/client/ConnectionUtils.java| 2 +- .../apache/hadoop/hbase/client/Mutation.java| 2 +- .../org/apache/hadoop/hbase/client/Result.java | 2 +- .../apache/hadoop/hbase/ByteBufferKeyValue.java | 2 +- .../java/org/apache/hadoop/hbase/CellUtil.java | 2 +- .../apache/hadoop/hbase/PrivateCellUtil.java| 7 +- .../hbase/util/MapReduceExtendedCell.java | 2 +- .../hadoop/hbase/io/hfile/HFileBlockIndex.java | 2 +- .../hbase/regionserver/AbstractMemStore.java| 4 +- .../regionserver/CSLMImmutableSegment.java | 3 +- .../regionserver/CellArrayImmutableSegment.java | 9 +- .../regionserver/CellChunkImmutableSegment.java | 52 ++- .../hbase/regionserver/CompactingMemStore.java | 8 +- .../hbase/regionserver/CompactionPipeline.java | 47 +++--- .../regionserver/CompositeImmutableSegment.java | 12 +- .../hbase/regionserver/DefaultMemStore.java | 4 +- .../regionserver/FlushAllLargeStoresPolicy.java | 2 +- .../regionserver/FlushLargeStoresPolicy.java| 52 --- .../FlushNonSloppyStoresFirstPolicy.java| 2 +- .../hadoop/hbase/regionserver/HRegion.java | 144 ++- .../hbase/regionserver/HRegionServer.java | 33 - .../hadoop/hbase/regionserver/HStore.java | 2 +- .../regionserver/ImmutableMemStoreLAB.java | 12 ++ .../hbase/regionserver/ImmutableSegment.java| 4 + .../hbase/regionserver/MemStoreFlusher.java | 92 +--- .../hadoop/hbase/regionserver/MemStoreLAB.java | 8 +- .../hbase/regionserver/MemStoreLABImpl.java | 10 ++ .../hadoop/hbase/regionserver/MemStoreSize.java | 52 ++- .../hbase/regionserver/MemStoreSizing.java | 58 +++- .../hbase/regionserver/MemStoreSnapshot.java| 16 +-- .../MetricsTableWrapperAggregateImpl.java | 2 +- .../hbase/regionserver/MutableSegment.java | 7 +- .../hadoop/hbase/regionserver/Region.java | 16 ++- .../regionserver/RegionServerAccounting.java| 45 +++--- .../regionserver/RegionServicesForStores.java | 4 +- .../hadoop/hbase/regionserver/Segment.java | 115 ++- .../hadoop/hbase/regionserver/StoreScanner.java | 2 +- .../org/apache/hadoop/hbase/wal/WALEdit.java| 2 +- .../hadoop/hbase/TestGlobalMemStoreSize.java| 4 +- .../hbase/TestPartialResultsFromClientSide.java | 2 +- ...TestServerSideScanMetricsFromClientSide.java | 3 +- .../hbase/client/TestAsyncRegionAdminApi.java | 12 +- .../hadoop/hbase/client/TestClientPushback.java | 6 +- .../hbase/client/TestFlushFromClient.java | 14 +- .../hadoop/hbase/client/TestSizeFailures.java | 6 +- ...NegativeMemStoreSizeWithSlowCoprocessor.java | 3 +- .../regionserver/TestCompactingMemStore.java| 8 +- .../TestCompactingToCellFlatMapMemStore.java| 2 +- .../TestEndToEndSplitTransaction.java | 2 +- .../hadoop/hbase/regionserver/TestHRegion.java | 20 +-- .../regionserver/TestHRegionReplayEvents.java | 36 ++--- .../hadoop/hbase/regionserver/TestHStore.java | 6 +- .../regionserver/TestPerColumnFamilyFlush.java | 20 +-- .../TestRegionServerAccounting.java | 20 +-- .../hbase/regionserver/TestWALLockup.java | 4 +- .../TestWalAndCompactingMemStoreFlush.java | 18 +-- 56 files changed, 649 insertions(+), 377 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java -- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java index 1a093f8..c9e994f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java @@ -320,7 +320,7 @@ public final class ConnectionUtils { long estimatedHeapSizeOfResult = 0; // We don't make Iterator here for (Cell cell : rs.rawCells()) { - estimatedHeapSizeOfResult += PrivateCellUtil.estimatedHeapSizeOf(cell); + estimatedHeapSizeOfResult +=
[26/50] [abbrv] hbase git commit: HBASE-20037 Race when calling SequenceIdAccounting.resetHighest
HBASE-20037 Race when calling SequenceIdAccounting.resetHighest Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/a27ef55a Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/a27ef55a Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/a27ef55a Branch: refs/heads/HBASE-19064 Commit: a27ef55a4074a5b69590c134ce7c0cb09124cdb4 Parents: 92d04d5 Author: zhangduoAuthored: Wed Feb 21 18:19:01 2018 +0800 Committer: Michael Stack Committed: Wed Feb 21 10:05:02 2018 -0800 -- .../hbase/regionserver/wal/AbstractFSWAL.java | 37 + .../hbase/regionserver/wal/AsyncFSWAL.java | 42 ++-- .../wal/AsyncProtobufLogWriter.java | 4 +- .../hadoop/hbase/regionserver/wal/FSHLog.java | 6 +-- .../regionserver/wal/ProtobufLogWriter.java | 11 +++-- .../regionserver/wal/SequenceIdAccounting.java | 40 ++- 6 files changed, 74 insertions(+), 66 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/a27ef55a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java index 14fbe10..ce8dafa 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java @@ -661,9 +661,26 @@ public abstract class AbstractFSWAL implements WAL { } } + protected final void logRollAndSetupWalProps(Path oldPath, Path newPath, long oldFileLen) { +int oldNumEntries = this.numEntries.getAndSet(0); +String newPathString = newPath != null ? CommonFSUtils.getPath(newPath) : null; +if (oldPath != null) { + this.walFile2Props.put(oldPath, +new WalProps(this.sequenceIdAccounting.resetHighest(), oldFileLen)); + this.totalLogSize.addAndGet(oldFileLen); + LOG.info("Rolled WAL {} with entries={}, filesize={}; new WAL {}", +CommonFSUtils.getPath(oldPath), oldNumEntries, StringUtils.byteDesc(oldFileLen), +newPathString); +} else { + LOG.info("New WAL {}", newPathString); +} + } + /** + * * Cleans up current writer closing it and then puts in place the passed in * nextWriter. + * * * * In the case of creating a new WAL, oldPath will be null. @@ -672,26 +689,17 @@ public abstract class AbstractFSWAL implements WAL { * In the case of closing out this FSHLog with no further use newPath and nextWriter will be * null. * + * * @param oldPath may be null * @param newPath may be null * @param nextWriter may be null * @return the passed in newPath * @throws IOException if there is a problem flushing or closing the underlying FS */ + @VisibleForTesting Path replaceWriter(Path oldPath, Path newPath, W nextWriter) throws IOException { try (TraceScope scope = TraceUtil.createTrace("FSHFile.replaceWriter")) { - long oldFileLen = doReplaceWriter(oldPath, newPath, nextWriter); - int oldNumEntries = this.numEntries.getAndSet(0); - final String newPathString = (null == newPath ? null : CommonFSUtils.getPath(newPath)); - if (oldPath != null) { -this.walFile2Props.put(oldPath, - new WalProps(this.sequenceIdAccounting.resetHighest(), oldFileLen)); -this.totalLogSize.addAndGet(oldFileLen); -LOG.info("Rolled WAL " + CommonFSUtils.getPath(oldPath) + " with entries=" + oldNumEntries + - ", filesize=" + StringUtils.byteDesc(oldFileLen) + "; new WAL " + newPathString); - } else { -LOG.info("New WAL " + newPathString); - } + doReplaceWriter(oldPath, newPath, nextWriter); return newPath; } } @@ -1021,10 +1029,7 @@ public abstract class AbstractFSWAL implements WAL { protected abstract W createWriterInstance(Path path) throws IOException, CommonFSUtils.StreamLacksCapabilityException; - /** - * @return old wal file size - */ - protected abstract long doReplaceWriter(Path oldPath, Path newPath, W nextWriter) + protected abstract void doReplaceWriter(Path oldPath, Path newPath, W nextWriter) throws IOException; protected abstract void doShutdown() throws IOException; http://git-wip-us.apache.org/repos/asf/hbase/blob/a27ef55a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java -- diff --git
[35/50] [abbrv] hbase git commit: HBASE-20042 TestRegionServerAbort flakey; ADDENDUM, RETRY
HBASE-20042 TestRegionServerAbort flakey; ADDENDUM, RETRY Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/13223c21 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/13223c21 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/13223c21 Branch: refs/heads/HBASE-19064 Commit: 13223c217ca6cb84a96f3c70b8c38ec19eca729f Parents: 61b5516 Author: Michael StackAuthored: Wed Feb 21 15:26:21 2018 -0800 Committer: Michael Stack Committed: Wed Feb 21 15:26:21 2018 -0800 -- .../hbase/regionserver/TestRegionServerAbort.java | 17 +++-- 1 file changed, 3 insertions(+), 14 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/13223c21/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java -- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java index 0c778fb..d5cbad0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAbort.java @@ -144,21 +144,10 @@ public class TestRegionServerAbort { put.addColumn(FAMILY_BYTES, Bytes.toBytes("c"), new byte[]{}); put.setAttribute(StopBlockingRegionObserver.DO_ABORT, new byte[]{1}); -table.put(put); -// should have triggered an abort due to FileNotFoundException - -// verify that the regionserver is stopped -List regions = null; -do { - regions = cluster.findRegionsForTable(tableName); - if (regions != null && regions.size() > 0) { -break; - } - LOG.warn("Waiting on regions for {} to online"); - Threads.sleep(100); -} while(true); - +List regions = cluster.findRegionsForTable(tableName); HRegion firstRegion = cluster.findRegionsForTable(tableName).get(0); +table.put(put); +// Verify that the regionserver is stopped assertNotNull(firstRegion); assertNotNull(firstRegion.getRegionServerServices()); LOG.info("isAborted = " + firstRegion.getRegionServerServices().isAborted());
[29/50] [abbrv] hbase git commit: HBASE-20039 MR tests out to hbase-mapreduce mobile
HBASE-20039 MR tests out to hbase-mapreduce mobile Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/5d994a24 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/5d994a24 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/5d994a24 Branch: refs/heads/HBASE-19064 Commit: 5d994a24fc570ee5156df0d1356cb7b12305fb78 Parents: 401227b Author: Mike DrobAuthored: Wed Feb 21 10:53:42 2018 -0600 Committer: Mike Drob Committed: Wed Feb 21 14:54:30 2018 -0600 -- .../mapreduce/TestHBaseMRTestingUtility.java| 73 .../hadoop/hbase/TestHBaseTestingUtility.java | 32 - 2 files changed, 73 insertions(+), 32 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/5d994a24/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHBaseMRTestingUtility.java -- diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHBaseMRTestingUtility.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHBaseMRTestingUtility.java new file mode 100644 index 000..2467dca --- /dev/null +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHBaseMRTestingUtility.java @@ -0,0 +1,73 @@ +/** + * 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.mapreduce; + +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.testclassification.MapReduceTests; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +@Category({MapReduceTests.class, LargeTests.class}) +public class TestHBaseMRTestingUtility { + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestHBaseMRTestingUtility.class); + + @Test + public void testMRYarnConfigsPopulation() throws IOException { +Map dummyProps = new HashMap<>(); +dummyProps.put("mapreduce.jobtracker.address", "dummyhost:11234"); +dummyProps.put("yarn.resourcemanager.address", "dummyhost:11235"); +dummyProps.put("mapreduce.jobhistory.address", "dummyhost:11236"); +dummyProps.put("yarn.resourcemanager.scheduler.address", "dummyhost:11237"); +dummyProps.put("mapreduce.jobhistory.webapp.address", "dummyhost:11238"); +dummyProps.put("yarn.resourcemanager.webapp.address", "dummyhost:11239"); + +HBaseTestingUtility hbt = new HBaseTestingUtility(); + +// populate the mr props to the Configuration instance +for (Map.Entry entry : dummyProps.entrySet()) { + hbt.getConfiguration().set(entry.getKey(), entry.getValue()); +} + +for (Map.Entry entry : dummyProps.entrySet()) { + assertTrue("The Configuration for key " + entry.getKey() +" and value: " + entry.getValue() + + " is not populated correctly", hbt.getConfiguration().get(entry.getKey()).equals(entry.getValue())); +} + +hbt.startMiniMapReduceCluster(); + +// Confirm that MiniMapReduceCluster overwrites the mr properties and updates the Configuration +for (Map.Entry entry : dummyProps.entrySet()) { + assertFalse("The MR prop: " + entry.getValue() + " is not overwritten when map reduce mini"+ + "cluster is started", hbt.getConfiguration().get(entry.getKey()).equals(entry.getValue())); +} + +hbt.shutdownMiniMapReduceCluster(); + } +} http://git-wip-us.apache.org/repos/asf/hbase/blob/5d994a24/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseTestingUtility.java -- diff --git
[12/50] [abbrv] hbase git commit: HBASE-18294 Reduce global heap pressure: flush based on heap occupancy
http://git-wip-us.apache.org/repos/asf/hbase/blob/f3bb9b96/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAccounting.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAccounting.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAccounting.java index 7689fcd..1c627f7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAccounting.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAccounting.java @@ -31,17 +31,17 @@ import org.apache.hadoop.hbase.util.Pair; /** * RegionServerAccounting keeps record of some basic real time information about - * the Region Server. Currently, it keeps record the global memstore size and global memstore heap - * overhead. It also tracks the replay edits per region. + * the Region Server. Currently, it keeps record the global memstore size and global memstore + * on-heap and off-heap overhead. It also tracks the replay edits per region. */ @InterfaceAudience.Private public class RegionServerAccounting { // memstore data size - private final LongAdder globalMemstoreDataSize = new LongAdder(); - // memstore heap size. When off heap MSLAB in place, this will be only heap overhead of the Cell - // POJOs and entry overhead of them onto memstore. When on heap MSLAB, this will be include heap - // overhead as well as the cell data size. Ya cell data is in on heap area only then. - private final LongAdder globalMemstoreHeapSize = new LongAdder(); + private final LongAdder globalMemStoreDataSize = new LongAdder(); + // memstore heap size. + private final LongAdder globalMemStoreHeapSize = new LongAdder(); + // memstore off-heap size. + private final LongAdder globalMemStoreOffHeapSize = new LongAdder(); // Store the edits size during replaying WAL. Use this to roll back the // global memstore size once a region opening failed. @@ -114,14 +114,21 @@ public class RegionServerAccounting { * @return the global Memstore data size in the RegionServer */ public long getGlobalMemStoreDataSize() { -return globalMemstoreDataSize.sum(); +return globalMemStoreDataSize.sum(); } /** * @return the global memstore heap size in the RegionServer */ public long getGlobalMemStoreHeapSize() { -return this.globalMemstoreHeapSize.sum(); +return this.globalMemStoreHeapSize.sum(); + } + + /** + * @return the global memstore heap size in the RegionServer + */ + public long getGlobalMemStoreOffHeapSize() { +return this.globalMemStoreOffHeapSize.sum(); } /** @@ -129,13 +136,15 @@ public class RegionServerAccounting { *the global Memstore size */ public void incGlobalMemStoreSize(MemStoreSize memStoreSize) { -globalMemstoreDataSize.add(memStoreSize.getDataSize()); -globalMemstoreHeapSize.add(memStoreSize.getHeapSize()); +globalMemStoreDataSize.add(memStoreSize.getDataSize()); +globalMemStoreHeapSize.add(memStoreSize.getHeapSize()); +globalMemStoreOffHeapSize.add(memStoreSize.getOffHeapSize()); } public void decGlobalMemStoreSize(MemStoreSize memStoreSize) { -globalMemstoreDataSize.add(-memStoreSize.getDataSize()); -globalMemstoreHeapSize.add(-memStoreSize.getHeapSize()); +globalMemStoreDataSize.add(-memStoreSize.getDataSize()); +globalMemStoreHeapSize.add(-memStoreSize.getHeapSize()); +globalMemStoreOffHeapSize.add(-memStoreSize.getOffHeapSize()); } /** @@ -151,13 +160,13 @@ public class RegionServerAccounting { } } else { // If the configured memstore is offheap, check for two things - // 1) If the global memstore data size is greater than the configured + // 1) If the global memstore off-heap size is greater than the configured // 'hbase.regionserver.offheap.global.memstore.size' // 2) If the global memstore heap size is greater than the configured onheap // global memstore limit 'hbase.regionserver.global.memstore.size'. // We do this to avoid OOME incase of scenarios where the heap is occupied with // lot of onheap references to the cells in memstore - if (getGlobalMemStoreDataSize() >= globalMemStoreLimit) { + if (getGlobalMemStoreOffHeapSize() >= globalMemStoreLimit) { // Indicates that global memstore size is above the configured // 'hbase.regionserver.offheap.global.memstore.size' return FlushType.ABOVE_OFFHEAP_HIGHER_MARK; @@ -181,8 +190,8 @@ public class RegionServerAccounting { return FlushType.ABOVE_ONHEAP_LOWER_MARK; } } else { - if (getGlobalMemStoreDataSize() >= globalMemStoreLimitLowMark) { -// Indicates that the offheap memstore's data size is greater than the global memstore + if (getGlobalMemStoreOffHeapSize() >=
[10/50] [abbrv] hbase git commit: HBASE-20013 TestZKPermissionWatcher is flakey
HBASE-20013 TestZKPermissionWatcher is flakey Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/68d509bc Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/68d509bc Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/68d509bc Branch: refs/heads/HBASE-19064 Commit: 68d509bc1ff7a3bf69a596aed49f238b42ee0679 Parents: 9698951 Author: Michael StackAuthored: Fri Feb 16 20:11:03 2018 -0800 Committer: Michael Stack Committed: Fri Feb 16 23:22:23 2018 -0800 -- .../org/apache/hadoop/hbase/master/HMaster.java | 18 ++ 1 file changed, 14 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/68d509bc/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 02fbc02..b949aa9 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 @@ -2718,11 +2718,21 @@ public class HMaster extends HRegionServer implements MasterServices { } } - void checkInitialized() - throws PleaseHoldException, ServerNotRunningYetException, MasterNotRunningException { + public static class MasterStoppedException extends DoNotRetryIOException { +MasterStoppedException() { + super(); +} + } + + void checkInitialized() throws PleaseHoldException, ServerNotRunningYetException, + MasterNotRunningException, MasterStoppedException { checkServiceStarted(); -if (!isInitialized()) throw new PleaseHoldException("Master is initializing"); -if (isStopped()) throw new MasterNotRunningException(); +if (!isInitialized()) { + throw new PleaseHoldException("Master is initializing"); +} +if (isStopped()) { + throw new MasterStoppedException(); +} } /**
hbase git commit: Revert "HBASE-19767 Fix for Master web UI shows negative values for Remaining KVs" Applied prematurely.
Repository: hbase Updated Branches: refs/heads/master b328807d2 -> 2440f807b Revert "HBASE-19767 Fix for Master web UI shows negative values for Remaining KVs" Applied prematurely. This reverts commit 61b55166bf7fe9edc4e8105f217463ed6e693d17. Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/2440f807 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/2440f807 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/2440f807 Branch: refs/heads/master Commit: 2440f807bf7d077def819c616d4afa97a4e2539e Parents: b328807 Author: Michael StackAuthored: Wed Feb 21 18:02:25 2018 -0800 Committer: Michael Stack Committed: Wed Feb 21 18:02:25 2018 -0800 -- .../hadoop/hbase/regionserver/HRegionServer.java| 2 +- .../apache/hadoop/hbase/regionserver/HStore.java| 4 ++-- .../compactions/CompactionProgress.java | 16 .../hbase/regionserver/TestMajorCompaction.java | 2 +- 4 files changed, 8 insertions(+), 16 deletions(-) -- http://git-wip-us.apache.org/repos/asf/hbase/blob/2440f807/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index a76dec2..49b7b80 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -1623,7 +1623,7 @@ public class HRegionServer extends HasThread implements storefileIndexSizeKB += store.getStorefilesRootLevelIndexSize() / 1024; CompactionProgress progress = store.getCompactionProgress(); if (progress != null) { -totalCompactingKVs += progress.getTotalCompactingKVs(); +totalCompactingKVs += progress.totalCompactingKVs; currentCompactedKVs += progress.currentCompactedKVs; } rootLevelIndexSizeKB += (int) (store.getStorefilesRootLevelIndexSize() / 1024); http://git-wip-us.apache.org/repos/asf/hbase/blob/2440f807/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index c0ef3ef..bef50b4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -1373,10 +1373,10 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat writeCompactionWalRecord(filesToCompact, sfs); replaceStoreFiles(filesToCompact, sfs); if (cr.isMajor()) { - majorCompactedCellsCount += getCompactionProgress().getTotalCompactingKVs(); + majorCompactedCellsCount += getCompactionProgress().totalCompactingKVs; majorCompactedCellsSize += getCompactionProgress().totalCompactedSize; } else { - compactedCellsCount += getCompactionProgress().getTotalCompactingKVs(); + compactedCellsCount += getCompactionProgress().totalCompactingKVs; compactedCellsSize += getCompactionProgress().totalCompactedSize; } long outputBytes = getTotalSize(sfs); http://git-wip-us.apache.org/repos/asf/hbase/blob/2440f807/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java -- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java index 577276e..d40651a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionProgress.java @@ -20,8 +20,6 @@ package org.apache.hadoop.hbase.regionserver.compactions; import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * This class holds information relevant for tracking the progress of a @@ -34,10 +32,9 @@ import org.slf4j.LoggerFactory; */ @InterfaceAudience.Private public class CompactionProgress { - private static final Logger LOG = LoggerFactory.getLogger(CompactionProgress.class); /** the total compacting key values in currently running compaction */ - private long totalCompactingKVs; + public