HBASE-20660 Reopen regions using ReopenTableRegionsProcedure
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/c3e071af Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/c3e071af Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/c3e071af Branch: refs/heads/HBASE-19064 Commit: c3e071af4dafe74e7086d41038ed6dfce1c6c1bc Parents: 2a386ba Author: zhangduo <zhang...@apache.org> Authored: Thu May 31 09:53:44 2018 +0800 Committer: zhangduo <zhang...@apache.org> Committed: Tue Jun 26 14:26:32 2018 +0800 ---------------------------------------------------------------------- ...ransitPeerSyncReplicationStateProcedure.java | 24 ++++++-------------- 1 file changed, 7 insertions(+), 17 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/c3e071af/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 81ee6b6..66f67dd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/replication/TransitPeerSyncReplicationStateProcedure.java @@ -18,16 +18,14 @@ package org.apache.hadoop.hbase.master.replication; import java.io.IOException; -import java.util.List; -import java.util.stream.Collectors; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.DoNotRetryIOException; -import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; +import org.apache.hadoop.hbase.master.procedure.ReopenTableRegionsProcedure; import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; import org.apache.hadoop.hbase.procedure2.ProcedureYieldException; @@ -141,11 +139,10 @@ public class TransitPeerSyncReplicationStateProcedure } } - private List<RegionInfo> getRegionsToReopen(MasterProcedureEnv env) { - return env.getReplicationPeerManager().getPeerConfig(peerId).get().getTableCFsMap().keySet() - .stream() - .flatMap(tn -> env.getAssignmentManager().getRegionStates().getRegionsOfTable(tn).stream()) - .collect(Collectors.toList()); + private void reopenRegions(MasterProcedureEnv env) { + addChildProcedure( + env.getReplicationPeerManager().getPeerConfig(peerId).get().getTableCFsMap().keySet().stream() + .map(ReopenTableRegionsProcedure::new).toArray(ReopenTableRegionsProcedure[]::new)); } private void createDirForRemoteWAL(MasterProcedureEnv env) @@ -190,7 +187,7 @@ public class TransitPeerSyncReplicationStateProcedure } private void replayRemoteWAL() { - addChildProcedure(new RecoverStandbyProcedure[] { new RecoverStandbyProcedure(peerId) }); + addChildProcedure(new RecoverStandbyProcedure(peerId)); } @Override @@ -252,14 +249,7 @@ public class TransitPeerSyncReplicationStateProcedure : PeerSyncReplicationStateTransitionState.TRANSIT_PEER_NEW_SYNC_REPLICATION_STATE); return Flow.HAS_MORE_STATE; case REOPEN_ALL_REGIONS_IN_PEER: - try { - addChildProcedure( - env.getAssignmentManager().createReopenProcedures(getRegionsToReopen(env))); - } catch (IOException e) { - LOG.warn("Failed to schedule region reopen for peer {} when starting transiting sync " + - "replication peer state from {} to {}, retry", peerId, fromState, toState, e); - throw new ProcedureYieldException(); - } + reopenRegions(env); setNextState( PeerSyncReplicationStateTransitionState.TRANSIT_PEER_NEW_SYNC_REPLICATION_STATE); return Flow.HAS_MORE_STATE;