HBASE-20660 Reopen regions using ReopenTableRegionsProcedure

This commit is contained in:
zhangduo 2018-05-31 09:53:44 +08:00
parent 05295abd5b
commit 7448b045cc
1 changed files with 7 additions and 17 deletions

View File

@ -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;