HBASE-22707 [HBCK2] MasterRpcServices assigns method should try to reload regions from meta if the passed regions isn't found under AssignmentManager RegionsStateStore

Signed-off-by: stack <stack@apache.org>
This commit is contained in:
Wellington Chevreuil 2019-08-01 16:48:26 +01:00
parent 8cfc46d8d0
commit 237229cade
7 changed files with 285 additions and 45 deletions

View File

@ -59,6 +59,8 @@ import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
import org.apache.hadoop.hbase.filter.RowFilter;
import org.apache.hadoop.hbase.filter.SubstringComparator;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
import org.apache.hadoop.hbase.ipc.ServerRpcController;
import org.apache.hadoop.hbase.master.RegionState;
@ -372,6 +374,25 @@ public class MetaTableAccessor {
return get(getMetaHTable(connection), get);
}
/**
* Scans META table for a row whose key contains the specified <B>regionEncodedName</B>,
* returning a single related <code>Result</code> instance if any row is found, null otherwise.
*
* @param connection the connection to query META table.
* @param regionEncodedName the region encoded name to look for at META.
* @return <code>Result</code> instance with the row related info in META, null otherwise.
* @throws IOException if any errors occur while querying META.
*/
public static Result scanByRegionEncodedName(Connection connection,
String regionEncodedName) throws IOException {
RowFilter rowFilter = new RowFilter(CompareOperator.EQUAL,
new SubstringComparator(regionEncodedName));
Scan scan = getMetaScan(connection, 1);
scan.setFilter(rowFilter);
ResultScanner resultScanner = getMetaHTable(connection).getScanner(scan);
return resultScanner.next();
}
/**
* Get regions from the merge qualifier of the specified merged region
* @return null if it doesn't contain merge qualifier, else two merge regions

View File

@ -2417,7 +2417,9 @@ public class MasterRpcServices extends RSRpcServices
String encodedRegionName = Bytes.toString(rs.getValue().toByteArray());
RegionState regionState = this.master.getAssignmentManager().getRegionStates().
getRegionState(encodedRegionName);
ri = regionState == null? null: regionState.getRegion();
ri = regionState == null ?
this.master.getAssignmentManager().loadRegionFromMeta(encodedRegionName) :
regionState.getRegion();
break;
default:
break;

View File

@ -1383,51 +1383,82 @@ public class AssignmentManager {
}
}
/* AM internal RegionStateStore.RegionStateVisitor implementation. To be used when
* scanning META table for region rows, using RegionStateStore utility methods. RegionStateStore
* methods will convert Result into proper RegionInfo instances, but those would still need to be
* added into AssignmentManager.regionStates in-memory cache.
* RegionMetaLoadingVisitor.visitRegionState method provides the logic for adding RegionInfo
* instances as loaded from latest META scan into AssignmentManager.regionStates.
*/
private class RegionMetaLoadingVisitor implements RegionStateStore.RegionStateVisitor {
@Override
public void visitRegionState(Result result, final RegionInfo regionInfo, final State state,
final ServerName regionLocation, final ServerName lastHost, final long openSeqNum) {
if (state == null && regionLocation == null && lastHost == null &&
openSeqNum == SequenceId.NO_SEQUENCE_ID) {
// This is a row with nothing in it.
LOG.warn("Skipping empty row={}", result);
return;
}
State localState = state;
if (localState == null) {
// No region state column data in hbase:meta table! Are I doing a rolling upgrade from
// hbase1 to hbase2? Am I restoring a SNAPSHOT or otherwise adding a region to hbase:meta?
// In any of these cases, state is empty. For now, presume OFFLINE but there are probably
// cases where we need to probe more to be sure this correct; TODO informed by experience.
LOG.info(regionInfo.getEncodedName() + " regionState=null; presuming " + State.OFFLINE);
localState = State.OFFLINE;
}
RegionStateNode regionNode = regionStates.getOrCreateRegionStateNode(regionInfo);
// Do not need to lock on regionNode, as we can make sure that before we finish loading
// meta, all the related procedures can not be executed. The only exception is for meta
// region related operations, but here we do not load the informations for meta region.
regionNode.setState(localState);
regionNode.setLastHost(lastHost);
regionNode.setRegionLocation(regionLocation);
regionNode.setOpenSeqNum(openSeqNum);
// Note: keep consistent with other methods, see region(Opening|Opened|Closing)
// RIT/ServerCrash handling should take care of the transiting regions.
if (localState.matches(State.OPEN, State.OPENING, State.CLOSING, State.SPLITTING,
State.MERGING)) {
assert regionLocation != null : "found null region location for " + regionNode;
regionStates.addRegionToServer(regionNode);
} else if (localState == State.OFFLINE || regionInfo.isOffline()) {
regionStates.addToOfflineRegions(regionNode);
}
if (regionNode.getProcedure() != null) {
regionNode.getProcedure().stateLoaded(AssignmentManager.this, regionNode);
}
}
};
/**
* Query META if the given <code>RegionInfo</code> exists, adding to
* <code>AssignmentManager.regionStateStore</code> cache if the region is found in META.
* @param regionEncodedName encoded name for the region to be loaded from META into
* <code>AssignmentManager.regionStateStore</code> cache
* @return <code>RegionInfo</code> instance for the given region if it is present in META
* and got successfully loaded into <code>AssignmentManager.regionStateStore</code>
* cache, <b>null</b> otherwise.
* @throws UnknownRegionException if any errors occur while querying meta.
*/
public RegionInfo loadRegionFromMeta(String regionEncodedName) throws UnknownRegionException {
try {
RegionMetaLoadingVisitor visitor = new RegionMetaLoadingVisitor();
regionStateStore.visitMetaForRegion(regionEncodedName, visitor);
return regionStates.getRegionState(regionEncodedName) == null ? null :
regionStates.getRegionState(regionEncodedName).getRegion();
} catch(IOException e) {
LOG.error("Error trying to load region {} from META", regionEncodedName, e);
throw new UnknownRegionException("Error while trying load region from meta");
}
}
private void loadMeta() throws IOException {
// TODO: use a thread pool
regionStateStore.visitMeta(new RegionStateStore.RegionStateVisitor() {
@Override
public void visitRegionState(Result result, final RegionInfo regionInfo, final State state,
final ServerName regionLocation, final ServerName lastHost, final long openSeqNum) {
if (state == null && regionLocation == null && lastHost == null &&
openSeqNum == SequenceId.NO_SEQUENCE_ID) {
// This is a row with nothing in it.
LOG.warn("Skipping empty row={}", result);
return;
}
State localState = state;
if (localState == null) {
// No region state column data in hbase:meta table! Are I doing a rolling upgrade from
// hbase1 to hbase2? Am I restoring a SNAPSHOT or otherwise adding a region to hbase:meta?
// In any of these cases, state is empty. For now, presume OFFLINE but there are probably
// cases where we need to probe more to be sure this correct; TODO informed by experience.
LOG.info(regionInfo.getEncodedName() + " regionState=null; presuming " + State.OFFLINE);
localState = State.OFFLINE;
}
RegionStateNode regionNode = regionStates.getOrCreateRegionStateNode(regionInfo);
// Do not need to lock on regionNode, as we can make sure that before we finish loading
// meta, all the related procedures can not be executed. The only exception is for meta
// region related operations, but here we do not load the informations for meta region.
regionNode.setState(localState);
regionNode.setLastHost(lastHost);
regionNode.setRegionLocation(regionLocation);
regionNode.setOpenSeqNum(openSeqNum);
// Note: keep consistent with other methods, see region(Opening|Opened|Closing)
// RIT/ServerCrash handling should take care of the transiting regions.
if (localState.matches(State.OPEN, State.OPENING, State.CLOSING, State.SPLITTING,
State.MERGING)) {
assert regionLocation != null : "found null region location for " + regionNode;
regionStates.addRegionToServer(regionNode);
} else if (localState == State.OFFLINE || regionInfo.isOffline()) {
regionStates.addToOfflineRegions(regionNode);
}
if (regionNode.getProcedure() != null) {
regionNode.getProcedure().stateLoaded(AssignmentManager.this, regionNode);
}
}
});
regionStateStore.visitMeta(new RegionMetaLoadingVisitor());
// every assignment is blocked until meta is loaded.
wakeMetaLoadedEvent();
}

View File

@ -96,6 +96,23 @@ public class RegionStateStore {
});
}
/**
* Queries META table for the passed region encoded name,
* delegating action upon results to the <code>RegionStateVisitor</code>
* passed as second parameter.
* @param regionEncodedName encoded name for the Region we want to query META for.
* @param visitor The <code>RegionStateVisitor</code> instance to react over the query results.
* @throws IOException If some error occurs while querying META or parsing results.
*/
public void visitMetaForRegion(final String regionEncodedName, final RegionStateVisitor visitor)
throws IOException {
Result result = MetaTableAccessor.
scanByRegionEncodedName(master.getConnection(), regionEncodedName);
if (result != null) {
visitMetaEntry(visitor, result);
}
}
private void visitMetaEntry(final RegionStateVisitor visitor, final Result result)
throws IOException {
final RegionLocations rl = MetaTableAccessor.getRegionLocations(result);

View File

@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.ipc.DelegatingRpcScheduler;
import org.apache.hadoop.hbase.ipc.PriorityFunction;
import org.apache.hadoop.hbase.ipc.RpcScheduler;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.RSRpcServices;
import org.apache.hadoop.hbase.regionserver.SimpleRpcSchedulerFactory;
@ -866,7 +867,8 @@ public class TestMetaTableAccessor {
List<RegionInfo> regionInfos = Lists.newArrayList(parent);
MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3);
MetaTableAccessor.splitRegion(connection, parent, -1L, splitA, splitB, serverName0, 3);
MetaTableAccessor.splitRegion(connection, parent, -1L, splitA, splitB,
serverName0, 3);
Get get1 = new Get(splitA.getRegionName());
Result resultA = meta.get(get1);
Cell serverCellA = resultA.getColumnLatestCell(HConstants.CATALOG_FAMILY,
@ -890,5 +892,28 @@ public class TestMetaTableAccessor {
}
}
}
@Test
public void testScanByRegionEncodedNameExistingRegion() throws Exception {
final TableName tableName = TableName.valueOf("testScanByRegionEncodedNameExistingRegion");
UTIL.createTable(tableName, "cf");
final List<HRegion> regions = UTIL.getHBaseCluster().getRegions(tableName);
final String encodedName = regions.get(0).getRegionInfo().getEncodedName();
final Result result = MetaTableAccessor.scanByRegionEncodedName(UTIL.getConnection(),
encodedName);
assertNotNull(result);
assertTrue(result.advance());
final String resultingRowKey = CellUtil.getCellKeyAsString(result.current());
assertTrue(resultingRowKey.contains(encodedName));
UTIL.deleteTable(tableName);
}
@Test
public void testScanByRegionEncodedNameNonExistingRegion() throws Exception {
final String encodedName = "nonexistingregion";
final Result result = MetaTableAccessor.scanByRegionEncodedName(UTIL.getConnection(),
encodedName);
assertNull(result);
}
}

View File

@ -18,12 +18,14 @@
package org.apache.hadoop.hbase.master.assignment;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
@ -285,4 +287,43 @@ public class TestAssignmentManager extends TestAssignmentManagerBase {
assertEquals(reopenFailedCount, reopenProcMetrics.getFailedCounter().getCount());
assertCloseThenOpen();
}
@Test
public void testLoadRegionFromMetaAfterRegionManuallyAdded() throws Exception {
try {
this.util.startMiniCluster();
final AssignmentManager am = this.util.getHBaseCluster().getMaster().getAssignmentManager();
final TableName tableName = TableName.
valueOf("testLoadRegionFromMetaAfterRegionManuallyAdded");
this.util.createTable(tableName, "f");
RegionInfo hri = createRegionInfo(tableName, 1);
assertNull("RegionInfo was just instantiated by the test, but "
+ "shouldn't be in AM regionStates yet.", am.getRegionStates().getRegionState(hri));
MetaTableAccessor.addRegionToMeta(this.util.getConnection(), hri);
assertNull("RegionInfo was manually added in META, but "
+ "shouldn't be in AM regionStates yet.", am.getRegionStates().getRegionState(hri));
hri = am.loadRegionFromMeta(hri.getEncodedName());
assertEquals(hri.getEncodedName(),
am.getRegionStates().getRegionState(hri).getRegion().getEncodedName());
}finally {
this.util.killMiniHBaseCluster();
}
}
@Test
public void testLoadRegionFromMetaRegionNotInMeta() throws Exception {
try {
this.util.startMiniCluster();
final AssignmentManager am = this.util.getHBaseCluster().getMaster().getAssignmentManager();
final TableName tableName = TableName.valueOf("testLoadRegionFromMetaRegionNotInMeta");
this.util.createTable(tableName, "f");
final RegionInfo hri = createRegionInfo(tableName, 1);
assertNull("RegionInfo was just instantiated by the test, but "
+ "shouldn't be in AM regionStates yet.", am.getRegionStates().getRegionState(hri));
assertNull("RegionInfo was never added in META, should had returned null.",
am.loadRegionFromMeta(hri.getEncodedName()));
}finally {
this.util.killMiniHBaseCluster();
}
}
}

View File

@ -0,0 +1,103 @@
/**
* 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.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.junit.After;
import org.junit.Before;
import org.junit.ClassRule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@Category({ MasterTests.class, MediumTests.class })
public class TestRegionStateStore {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestRegionStateStore.class);
private static final Logger LOG = LoggerFactory.getLogger(TestRegionStateStore.class);
protected HBaseTestingUtility util;
@Before
public void setup() throws Exception {
util = new HBaseTestingUtility();
util.startMiniCluster();
}
@After
public void tearDown() throws Exception {
util.shutdownMiniCluster();
}
@Test
public void testVisitMetaForRegionExistingRegion() throws Exception {
final TableName tableName = TableName.valueOf("testVisitMetaForRegion");
util.createTable(tableName, "cf");
final List<HRegion> regions = util.getHBaseCluster().getRegions(tableName);
final String encodedName = regions.get(0).getRegionInfo().getEncodedName();
final RegionStateStore regionStateStore = util.getHBaseCluster().getMaster().
getAssignmentManager().getRegionStateStore();
final AtomicBoolean visitorCalled = new AtomicBoolean(false);
regionStateStore.visitMetaForRegion(encodedName, new RegionStateStore.RegionStateVisitor() {
@Override
public void visitRegionState(Result result, RegionInfo regionInfo, RegionState.State state,
ServerName regionLocation, ServerName lastHost, long openSeqNum) {
assertEquals(encodedName, regionInfo.getEncodedName());
visitorCalled.set(true);
}
});
assertTrue("Visitor has not been called.", visitorCalled.get());
}
@Test
public void testVisitMetaForRegionNonExistingRegion() throws Exception {
final String encodedName = "fakeencodedregionname";
final RegionStateStore regionStateStore = util.getHBaseCluster().getMaster().
getAssignmentManager().getRegionStateStore();
final AtomicBoolean visitorCalled = new AtomicBoolean(false);
regionStateStore.visitMetaForRegion(encodedName, new RegionStateStore.RegionStateVisitor() {
@Override
public void visitRegionState(Result result, RegionInfo regionInfo, RegionState.State state,
ServerName regionLocation, ServerName lastHost, long openSeqNum) {
visitorCalled.set(true);
}
});
assertFalse("Visitor has been called, but it shouldn't.", visitorCalled.get());
}
}