HBASE-21078 [amv2] CODE-BUG NPE in RTP doing Unassign
This commit is contained in:
parent
e26ca63f88
commit
d954031d50
|
@ -421,7 +421,10 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure<TE
|
|||
sb.append(", state="); // pState for Procedure State as opposed to any other kind.
|
||||
toStringState(sb);
|
||||
|
||||
sb.append(", hasLock=").append(locked);
|
||||
// Only print out locked if actually locked. Most of the time it is not.
|
||||
if (this.locked) {
|
||||
sb.append(", locked=").append(locked);
|
||||
}
|
||||
|
||||
if (hasException()) {
|
||||
sb.append(", exception=" + getException());
|
||||
|
|
|
@ -1725,7 +1725,8 @@ public class ProcedureExecutor<TEnvironment> {
|
|||
// children have completed, move parent to front of the queue.
|
||||
store.update(parent);
|
||||
scheduler.addFront(parent);
|
||||
LOG.info("Finished subprocedure(s) of " + parent + "; resume parent processing.");
|
||||
LOG.info("Finished subprocedure pid={}, resume processing parent {}",
|
||||
procedure.getProcId(), parent);
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,17 +24,19 @@ import java.io.IOException;
|
|||
import org.apache.hadoop.hbase.HBaseIOException;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.DoNotRetryRegionException;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.master.RegionPlan;
|
||||
import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineRegionProcedure;
|
||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MoveRegionState;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.MoveRegionStateData;
|
||||
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* Procedure that implements a RegionPlan.
|
||||
|
@ -55,6 +57,11 @@ public class MoveRegionProcedure extends AbstractStateMachineRegionProcedure<Mov
|
|||
super();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
protected RegionPlan getPlan() {
|
||||
return this.plan;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param check whether we should do some checks in the constructor. We will skip the checks if we
|
||||
* are reopening a region as this may fail the whole procedure and cause stuck. We will
|
||||
|
@ -91,6 +98,13 @@ public class MoveRegionProcedure extends AbstractStateMachineRegionProcedure<Mov
|
|||
}
|
||||
break;
|
||||
case MOVE_REGION_UNASSIGN:
|
||||
try {
|
||||
checkOnline(env, this.plan.getRegionInfo());
|
||||
} catch (DoNotRetryRegionException dnrre) {
|
||||
LOG.info("Skipping move, {} is not online; {}", getRegion().getEncodedName(), this,
|
||||
dnrre);
|
||||
return Flow.NO_MORE_STATE;
|
||||
}
|
||||
addChildProcedure(new UnassignProcedure(plan.getRegionInfo(), plan.getSource(),
|
||||
plan.getDestination(), true));
|
||||
setNextState(MoveRegionState.MOVE_REGION_ASSIGN);
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.client.RegionInfo;
|
|||
import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
|
||||
import org.apache.hadoop.hbase.favored.FavoredNodesManager;
|
||||
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
|
||||
import org.apache.hadoop.hbase.master.RegionState;
|
||||
import org.apache.hadoop.hbase.master.RegionState.State;
|
||||
import org.apache.hadoop.hbase.master.assignment.RegionStates.RegionStateNode;
|
||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
||||
|
@ -116,9 +117,7 @@ public class UnassignProcedure extends RegionTransitionProcedure {
|
|||
this.destinationServer = destinationServer;
|
||||
this.force = force;
|
||||
this.removeAfterUnassigning = removeAfterUnassigning;
|
||||
|
||||
// we don't need REGION_TRANSITION_QUEUE, we jump directly to sending the request
|
||||
setTransitionState(RegionTransitionState.REGION_TRANSITION_DISPATCH);
|
||||
setTransitionState(RegionTransitionState.REGION_TRANSITION_QUEUE);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -179,9 +178,21 @@ public class UnassignProcedure extends RegionTransitionProcedure {
|
|||
|
||||
@Override
|
||||
protected boolean startTransition(final MasterProcedureEnv env, final RegionStateNode regionNode) {
|
||||
// nothing to do here. we skip the step in the constructor
|
||||
// by jumping to REGION_TRANSITION_DISPATCH
|
||||
throw new UnsupportedOperationException();
|
||||
// Check region is actually unassignable now we have lock on it. If not skirt to end.
|
||||
// It could have had its status changed on us post construction... perhaps a split removed
|
||||
// the region we are to unassign (a split and a move happening near-concurrently).
|
||||
// Don't overcheck. A region is set to have a SPLITTING state if it is the parent and it is
|
||||
// being split. Regions that are in this RSN state are unassignable. Regions that are SPLIT
|
||||
// are not.
|
||||
RegionStates regionStates = env.getAssignmentManager().getRegionStates();
|
||||
RegionState rs = regionStates.getRegionState(regionNode.getRegionInfo());
|
||||
// Don't try unassigning regions that are closed or split. RSN state could have been set
|
||||
// after our creation but before we got the region lock.
|
||||
if (rs.isClosing() || rs.isClosed() || rs.isSplit() || rs.isMerged()) {
|
||||
LOG.info("NOT unassignable {}, skipping {}", rs, this);
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -3215,7 +3215,6 @@ public class HRegionServer extends HasThread implements
|
|||
}
|
||||
|
||||
/**
|
||||
* @param regionName
|
||||
* @return HRegion for the passed binary <code>regionName</code> or null if
|
||||
* named region is not member of the online regions.
|
||||
*/
|
||||
|
|
|
@ -0,0 +1,174 @@
|
|||
/**
|
||||
* 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.assignment;
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseIOException;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.master.RegionPlan;
|
||||
import org.apache.hadoop.hbase.master.RegionState;
|
||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* Like TestRegionMove in regionserver package but in here in this package so I can get access to
|
||||
* Procedure internals to mess with the assignment to manufacture states seen out on clusters.
|
||||
*/
|
||||
@Category({LargeTests.class})
|
||||
public class TestRegionMove2 {
|
||||
private final static Logger LOG = LoggerFactory.getLogger(TestRegionMove2.class);
|
||||
|
||||
@ClassRule
|
||||
public static final HBaseClassTestRule CLASS_RULE =
|
||||
HBaseClassTestRule.forClass(TestRegionMove2.class);
|
||||
|
||||
@Rule
|
||||
public TestName name = new TestName();
|
||||
|
||||
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
|
||||
@BeforeClass
|
||||
public static void startCluster() throws Exception {
|
||||
TEST_UTIL.startMiniCluster(2);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void stopCluster() throws Exception {
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that we make it through to the end if parent Region is offlined between start of this
|
||||
* Move and when we go to run the move UnassignProcedure.
|
||||
*/
|
||||
@Test
|
||||
public void testMoveOfRegionOfflinedPostStart() throws Exception {
|
||||
TableName tableName = TableName.valueOf(this.name.getMethodName());
|
||||
// Create a table with more than one region
|
||||
byte [] cf = Bytes.toBytes("cf");
|
||||
Table t = TEST_UTIL.createMultiRegionTable(tableName, cf, 10);
|
||||
TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
|
||||
HRegionServer rs1 = null;
|
||||
HRegionServer rs2 = null;
|
||||
List<RegionInfo> regionsOnRS1ForTable = null;
|
||||
try (Admin admin = TEST_UTIL.getAdmin()) {
|
||||
// Write an update to each region
|
||||
for (RegionInfo regionInfo : admin.getRegions(tableName)) {
|
||||
byte[] startKey = regionInfo.getStartKey();
|
||||
// StartKey of first region is "empty", which would throw an error if we try to Put that.
|
||||
byte[] rowKey =
|
||||
org.apache.hbase.thirdparty.com.google.common.primitives.Bytes.concat(startKey,
|
||||
Bytes.toBytes("1"));
|
||||
Put p = new Put(rowKey);
|
||||
p.addColumn(cf, Bytes.toBytes("q1"), Bytes.toBytes("value"));
|
||||
t.put(p);
|
||||
}
|
||||
|
||||
// Get a Region which is on the first RS
|
||||
rs1 = TEST_UTIL.getRSForFirstRegionInTable(tableName);
|
||||
rs2 = TEST_UTIL.getOtherRegionServer(rs1);
|
||||
regionsOnRS1ForTable = admin.getRegions(rs1.getServerName()).stream().
|
||||
filter((regionInfo) -> regionInfo.getTable().equals(tableName)).
|
||||
collect(Collectors.toList());
|
||||
}
|
||||
assertTrue("Expected to find at least one region for " + tableName + " on " +
|
||||
rs1.getServerName() + ", but found none", !regionsOnRS1ForTable.isEmpty());
|
||||
final RegionInfo regionToMove = regionsOnRS1ForTable.get(0);
|
||||
HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
|
||||
|
||||
// Try to move the region. HackedMoveRegionProcedure should intercede and mess up the region
|
||||
// state setting it to SPLIT when we run the UnassignProcedure part of move region.
|
||||
// Then when we go to do the unassignprocedure, we should notice the region-to-move is not
|
||||
// online.... spew some log, and then fast-track to the end of the unassign. The assign under
|
||||
// move will also notice that the parent is not-online but SPLIT and will skip it... so the
|
||||
// move will "succeed" but we won't have moved the region!
|
||||
RegionPlan rp = new RegionPlan(regionToMove, rs1.getServerName(), rs2.getServerName());
|
||||
MasterProcedureEnv env = master.getMasterProcedureExecutor().getEnvironment();
|
||||
HackedMoveRegionProcedure p = new HackedMoveRegionProcedure(env, rp);
|
||||
master.getMasterProcedureExecutor().submitProcedure(p);
|
||||
ProcedureTestingUtility.waitProcedure(master.getMasterProcedureExecutor(), p);
|
||||
// Split should have been called.
|
||||
assertTrue(p.split.get());
|
||||
// The region should not have been moved!
|
||||
assertTrue(rs1.getOnlineRegion(regionToMove.getRegionName()) != null);
|
||||
}
|
||||
|
||||
/**
|
||||
* Class just so we can mess around with RegionStateNode state at a particular point in the
|
||||
* Procedure to try and mess it up.
|
||||
*/
|
||||
public static class HackedMoveRegionProcedure extends MoveRegionProcedure {
|
||||
/**
|
||||
* Set to true after we hack this regions RSN to SPLIT
|
||||
*/
|
||||
public static AtomicBoolean split = new AtomicBoolean(false);
|
||||
|
||||
// Required by the Procedure framework to create the procedure on replay
|
||||
public HackedMoveRegionProcedure() {
|
||||
super();
|
||||
}
|
||||
|
||||
public HackedMoveRegionProcedure(MasterProcedureEnv env, RegionPlan plan)
|
||||
throws HBaseIOException {
|
||||
super(env, plan, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Flow executeFromState(MasterProcedureEnv env,
|
||||
MasterProcedureProtos.MoveRegionState state) throws InterruptedException {
|
||||
Flow flow = null;
|
||||
switch (state) {
|
||||
case MOVE_REGION_UNASSIGN:
|
||||
// Just before the unassign, flip the state to SPLIT. The unassign should exit!
|
||||
RegionStates.RegionStateNode rsn =
|
||||
env.getAssignmentManager().getRegionStates().getOrCreateRegionStateNode(getRegion());
|
||||
rsn.setState(RegionState.State.SPLIT);
|
||||
LOG.info("HACKED RSN, setting it to SPLIT: {}", rsn);
|
||||
split.set(true);
|
||||
default:
|
||||
flow = super.executeFromState(env, state);
|
||||
}
|
||||
return flow;
|
||||
}
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue