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:
parent
8cfc46d8d0
commit
237229cade
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
}
|
Loading…
Reference in New Issue