HBASE-4083 If Enable table is not completed and is partial, then scanning of the table is not working
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1151341 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
2485279bfa
commit
5f2aa2fea6
|
@ -175,6 +175,8 @@ Release 0.91.0 - Unreleased
|
|||
(Allan Yan)
|
||||
HBASE-3845 data loss because lastSeqWritten can miss memstore edits
|
||||
(Prakash Khemani and ramkrishna.s.vasudevan)
|
||||
HBASE-4083 If Enable table is not completed and is partial, then scanning of
|
||||
the table is not working (ramkrishna.s.vasudevan)
|
||||
|
||||
IMPROVEMENTS
|
||||
HBASE-3290 Max Compaction Size (Nicolas Spiegelberg via Stack)
|
||||
|
|
|
@ -82,6 +82,7 @@ import org.apache.hadoop.hbase.filter.SkipFilter;
|
|||
import org.apache.hadoop.hbase.filter.ValueFilter;
|
||||
import org.apache.hadoop.hbase.filter.WhileMatchFilter;
|
||||
import org.apache.hadoop.hbase.filter.WritableByteArrayComparable;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||
|
@ -234,6 +235,9 @@ public class HbaseObjectWritable implements Writable, WritableWithSize, Configur
|
|||
addToMap(ColumnRangeFilter.class, code++);
|
||||
|
||||
addToMap(HServerLoad.class, code++);
|
||||
|
||||
addToMap(RegionOpeningState.class, code++);
|
||||
|
||||
}
|
||||
|
||||
private Class<?> declaredClass;
|
||||
|
|
|
@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.client.coprocessor.Exec;
|
|||
import org.apache.hadoop.hbase.client.coprocessor.ExecResult;
|
||||
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
|
||||
import org.apache.hadoop.hbase.filter.WritableByteArrayComparable;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.hadoop.ipc.VersionedProtocol;
|
||||
|
@ -326,10 +327,17 @@ public interface HRegionInterface extends VersionedProtocol, Stoppable, Abortabl
|
|||
|
||||
/**
|
||||
* Opens the specified region.
|
||||
* @param region region to open
|
||||
*
|
||||
* @param region
|
||||
* region to open
|
||||
* @return RegionOpeningState
|
||||
* OPENED - if region opened succesfully.
|
||||
* ALREADY_OPENED - if the region was already opened.
|
||||
* FAILED_OPENING - if region opening failed.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
public void openRegion(final HRegionInfo region) throws IOException;
|
||||
public RegionOpeningState openRegion(final HRegionInfo region) throws IOException;
|
||||
|
||||
/**
|
||||
* Opens the specified regions.
|
||||
|
|
|
@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.NotServingRegionException;
|
|||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.Stoppable;
|
||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
||||
import org.apache.hadoop.hbase.catalog.MetaReader;
|
||||
import org.apache.hadoop.hbase.catalog.RootLocationEditor;
|
||||
|
@ -56,10 +57,12 @@ import org.apache.hadoop.hbase.client.Result;
|
|||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
import org.apache.hadoop.hbase.executor.RegionTransitionData;
|
||||
import org.apache.hadoop.hbase.executor.EventHandler.EventType;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
|
||||
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
|
||||
import org.apache.hadoop.hbase.master.LoadBalancer.RegionPlan;
|
||||
import org.apache.hadoop.hbase.master.handler.ClosedRegionHandler;
|
||||
import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
|
||||
import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
|
||||
import org.apache.hadoop.hbase.master.handler.OpenedRegionHandler;
|
||||
import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
|
||||
import org.apache.hadoop.hbase.master.handler.SplitRegionHandler;
|
||||
|
@ -1233,7 +1236,33 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
state.update(RegionState.State.PENDING_OPEN, System.currentTimeMillis(),
|
||||
plan.getDestination());
|
||||
// Send OPEN RPC. This can fail if the server on other end is is not up.
|
||||
serverManager.sendRegionOpen(plan.getDestination(), state.getRegion());
|
||||
RegionOpeningState regionOpenState = serverManager.sendRegionOpen(plan
|
||||
.getDestination(), state.getRegion());
|
||||
if (regionOpenState == RegionOpeningState.ALREADY_OPENED) {
|
||||
// Remove region from in-memory transition and unassigned node from ZK
|
||||
// While trying to enable the table the regions of the table were
|
||||
// already enabled.
|
||||
String encodedRegionName = state.getRegion()
|
||||
.getEncodedName();
|
||||
try {
|
||||
ZKAssign.deleteOfflineNode(master.getZooKeeper(), encodedRegionName);
|
||||
} catch (KeeperException.NoNodeException e) {
|
||||
if(LOG.isDebugEnabled()){
|
||||
LOG.debug("The unassigned node "+encodedRegionName+" doesnot exist.");
|
||||
}
|
||||
} catch (KeeperException e) {
|
||||
master.abort(
|
||||
"Error deleting OFFLINED node in ZK for transition ZK node ("
|
||||
+ encodedRegionName + ")", e);
|
||||
}
|
||||
synchronized (this.regionsInTransition) {
|
||||
this.regionsInTransition.remove(plan.getRegionInfo()
|
||||
.getEncodedName());
|
||||
}
|
||||
synchronized (this.regions) {
|
||||
this.regions.put(plan.getRegionInfo(), plan.getDestination());
|
||||
}
|
||||
}
|
||||
break;
|
||||
} catch (Throwable t) {
|
||||
LOG.warn("Failed assignment of " +
|
||||
|
@ -1770,23 +1799,25 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
new TreeMap<ServerName, List<Pair<HRegionInfo, Result>>>();
|
||||
// store all the table names in disabling state
|
||||
Set<String> disablingTables = new HashSet<String>(1);
|
||||
// store all the enabling state tablenames.
|
||||
Set<String> enablingTables = new HashSet<String>(1);
|
||||
// Iterate regions in META
|
||||
for (Result result : results) {
|
||||
Pair<HRegionInfo, ServerName> region = MetaReader.metaRowToRegionPair(result);
|
||||
if (region == null) continue;
|
||||
HRegionInfo regionInfo = region.getFirst();
|
||||
String disablingTableName = regionInfo.getTableNameAsString();
|
||||
ServerName regionLocation = region.getSecond();
|
||||
String tableName = regionInfo.getTableNameAsString();
|
||||
if (regionLocation == null) {
|
||||
// Region not being served, add to region map with no assignment
|
||||
// If this needs to be assigned out, it will also be in ZK as RIT
|
||||
// add if the table is not in disabled state
|
||||
if (false == checkIfRegionBelongsToDisabled(regionInfo)) {
|
||||
this.regions.put(regionInfo, null);
|
||||
}
|
||||
if (checkIfRegionBelongsToDisabling(regionInfo)) {
|
||||
disablingTables.add(disablingTableName);
|
||||
// add if the table is not in disabled and enabling state
|
||||
if (false == checkIfRegionBelongsToDisabled(regionInfo)
|
||||
&& false == checkIfRegionsBelongsToEnabling(regionInfo)) {
|
||||
regions.put(regionInfo, regionLocation);
|
||||
}
|
||||
addTheTablesInPartialState(disablingTables, enablingTables, regionInfo,
|
||||
tableName);
|
||||
} else if (!this.serverManager.isServerOnline(regionLocation)) {
|
||||
// Region is located on a server that isn't online
|
||||
List<Pair<HRegionInfo, Result>> offlineRegions =
|
||||
|
@ -1798,22 +1829,51 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
offlineRegions.add(new Pair<HRegionInfo,Result>(regionInfo, result));
|
||||
} else {
|
||||
// Region is being served and on an active server
|
||||
// add only if region not in disabled table
|
||||
if (false == checkIfRegionBelongsToDisabled(regionInfo)) {
|
||||
// add only if region not in disabled and enabling table
|
||||
if (false == checkIfRegionBelongsToDisabled(regionInfo)
|
||||
&& false == checkIfRegionsBelongsToEnabling(regionInfo)) {
|
||||
regions.put(regionInfo, regionLocation);
|
||||
addToServers(regionLocation, regionInfo);
|
||||
}
|
||||
if (checkIfRegionBelongsToDisabling(regionInfo)) {
|
||||
disablingTables.add(disablingTableName);
|
||||
}
|
||||
addTheTablesInPartialState(disablingTables, enablingTables, regionInfo,
|
||||
tableName);
|
||||
}
|
||||
}
|
||||
// Recover the tables that were not fully moved to DISABLED state.
|
||||
// These tables are in DISABLING state when the master restarted/switched.
|
||||
boolean isWatcherCreated = recoverTableInDisablingState(disablingTables);
|
||||
recoverTableInEnablingState(enablingTables, isWatcherCreated);
|
||||
return offlineServers;
|
||||
}
|
||||
|
||||
private void addTheTablesInPartialState(Set<String> disablingTables,
|
||||
Set<String> enablingTables, HRegionInfo regionInfo,
|
||||
String disablingTableName) {
|
||||
if (checkIfRegionBelongsToDisabling(regionInfo)) {
|
||||
disablingTables.add(disablingTableName);
|
||||
} else if (checkIfRegionsBelongsToEnabling(regionInfo)) {
|
||||
enablingTables.add(disablingTableName);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Recover the tables that were not fully moved to DISABLED state. These
|
||||
* tables are in DISABLING state when the master restarted/switched.
|
||||
*
|
||||
* @param disablingTables
|
||||
* @return
|
||||
* @throws KeeperException
|
||||
* @throws TableNotFoundException
|
||||
* @throws IOException
|
||||
*/
|
||||
private boolean recoverTableInDisablingState(Set<String> disablingTables)
|
||||
throws KeeperException, TableNotFoundException, IOException {
|
||||
boolean isWatcherCreated = false;
|
||||
if (disablingTables.size() != 0) {
|
||||
// Create a watcher on the zookeeper node
|
||||
ZKUtil.listChildrenAndWatchForNewChildren(watcher,
|
||||
watcher.assignmentZNode);
|
||||
isWatcherCreated = true;
|
||||
for (String tableName : disablingTables) {
|
||||
// Recover by calling DisableTableHandler
|
||||
LOG.info("The table " + tableName
|
||||
|
@ -1823,7 +1883,41 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
catalogTracker, this).process();
|
||||
}
|
||||
}
|
||||
return offlineServers;
|
||||
return isWatcherCreated;
|
||||
}
|
||||
|
||||
/**
|
||||
* Recover the tables that are not fully moved to ENABLED state. These tables
|
||||
* are in ENABLING state when the master restarted/switched
|
||||
*
|
||||
* @param enablingTables
|
||||
* @param isWatcherCreated
|
||||
* @throws KeeperException
|
||||
* @throws TableNotFoundException
|
||||
* @throws IOException
|
||||
*/
|
||||
private void recoverTableInEnablingState(Set<String> enablingTables,
|
||||
boolean isWatcherCreated) throws KeeperException, TableNotFoundException,
|
||||
IOException {
|
||||
if (enablingTables.size() != 0) {
|
||||
if (false == isWatcherCreated) {
|
||||
ZKUtil.listChildrenAndWatchForNewChildren(watcher,
|
||||
watcher.assignmentZNode);
|
||||
}
|
||||
for (String tableName : enablingTables) {
|
||||
// Recover by calling DisableTableHandler
|
||||
LOG.info("The table " + tableName
|
||||
+ " is in ENABLING state. Hence recovering by moving the table"
|
||||
+ " to ENABLED state.");
|
||||
new EnableTableHandler(this.master, tableName.getBytes(),
|
||||
catalogTracker, this).process();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private boolean checkIfRegionsBelongsToEnabling(HRegionInfo regionInfo) {
|
||||
String tableName = regionInfo.getTableNameAsString();
|
||||
return getZKTable().isEnablingTable(tableName);
|
||||
}
|
||||
|
||||
private boolean checkIfRegionBelongsToDisabled(HRegionInfo regionInfo) {
|
||||
|
|
|
@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.client.HConnection;
|
|||
import org.apache.hadoop.hbase.client.HConnectionManager;
|
||||
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
|
||||
import org.apache.hadoop.hbase.ipc.HRegionInterface;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
|
||||
import org.apache.hadoop.hbase.master.handler.MetaServerShutdownHandler;
|
||||
import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||
|
@ -392,15 +393,15 @@ public class ServerManager {
|
|||
* @param server server to open a region
|
||||
* @param region region to open
|
||||
*/
|
||||
public void sendRegionOpen(final ServerName server, HRegionInfo region)
|
||||
public RegionOpeningState sendRegionOpen(final ServerName server, HRegionInfo region)
|
||||
throws IOException {
|
||||
HRegionInterface hri = getServerConnection(server);
|
||||
if (hri == null) {
|
||||
LOG.warn("Attempting to send OPEN RPC to server " + server.toString() +
|
||||
" failed because no RPC connection found to this server");
|
||||
return;
|
||||
return RegionOpeningState.FAILED_OPENING;
|
||||
}
|
||||
hri.openRegion(region);
|
||||
return hri.openRegion(region);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -118,6 +118,7 @@ import org.apache.hadoop.hbase.regionserver.handler.OpenRootHandler;
|
|||
import org.apache.hadoop.hbase.regionserver.metrics.RegionServerMetrics;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALObserver;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
|
||||
import org.apache.hadoop.hbase.replication.regionserver.Replication;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -2287,12 +2288,18 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
|
||||
@Override
|
||||
@QosPriority(priority=HIGH_QOS)
|
||||
public void openRegion(HRegionInfo region)
|
||||
public RegionOpeningState openRegion(HRegionInfo region)
|
||||
throws IOException {
|
||||
checkOpen();
|
||||
if (this.regionsInTransitionInRS.contains(region.getEncodedNameAsBytes())) {
|
||||
throw new RegionAlreadyInTransitionException("open", region.getEncodedName());
|
||||
}
|
||||
HRegion onlineRegion = this.getFromOnlineRegions(region.getEncodedName());
|
||||
if (null != onlineRegion) {
|
||||
LOG.warn("Attempted open of " + region.getEncodedName()
|
||||
+ " but already online on this server");
|
||||
return RegionOpeningState.ALREADY_OPENED;
|
||||
}
|
||||
LOG.info("Received request to open region: " +
|
||||
region.getRegionNameAsString());
|
||||
HTableDescriptor htd = this.tableDescriptors.get(region.getTableName());
|
||||
|
@ -2303,6 +2310,7 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
} else {
|
||||
this.service.submit(new OpenRegionHandler(this, this, region, htd));
|
||||
}
|
||||
return RegionOpeningState.OPENED;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -0,0 +1,29 @@
|
|||
/**
|
||||
* Copyright 2011 The Apache Software Foundation
|
||||
*
|
||||
* 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.regionserver;
|
||||
|
||||
public enum RegionOpeningState {
|
||||
|
||||
OPENED,
|
||||
|
||||
ALREADY_OPENED,
|
||||
|
||||
FAILED_OPENING;
|
||||
}
|
|
@ -83,11 +83,6 @@ public class OpenRegionHandler extends EventHandler {
|
|||
|
||||
// Check that this region is not already online
|
||||
HRegion region = this.rsServices.getFromOnlineRegions(encodedName);
|
||||
if (region != null) {
|
||||
LOG.warn("Attempted open of " + name +
|
||||
" but already online on this server");
|
||||
return;
|
||||
}
|
||||
|
||||
// If fails, just return. Someone stole the region from under us.
|
||||
// Calling transitionZookeeperOfflineToOpening initalizes this.version.
|
||||
|
|
Loading…
Reference in New Issue