HBASE-13211 Procedure V2 - master Enable/Disable table (Stephen Yuan Jiang)

This commit is contained in:
Matteo Bertozzi 2015-04-09 21:52:02 +01:00
parent 4ae8b8cc52
commit 57c70f0af8
10 changed files with 3723 additions and 48 deletions

View File

@ -135,3 +135,33 @@ message DeleteColumnFamilyStateData {
required bytes columnfamily_name = 3;
optional TableSchema unmodified_table_schema = 4;
}
enum EnableTableState {
ENABLE_TABLE_PREPARE = 1;
ENABLE_TABLE_PRE_OPERATION = 2;
ENABLE_TABLE_SET_ENABLING_TABLE_STATE = 3;
ENABLE_TABLE_MARK_REGIONS_ONLINE = 4;
ENABLE_TABLE_SET_ENABLED_TABLE_STATE = 5;
ENABLE_TABLE_POST_OPERATION = 6;
}
message EnableTableStateData {
required UserInformation user_info = 1;
required TableName table_name = 2;
required bool skip_table_state_check = 3;
}
enum DisableTableState {
DISABLE_TABLE_PREPARE = 1;
DISABLE_TABLE_PRE_OPERATION = 2;
DISABLE_TABLE_SET_DISABLING_TABLE_STATE = 3;
DISABLE_TABLE_MARK_REGIONS_OFFLINE = 4;
DISABLE_TABLE_SET_DISABLED_TABLE_STATE = 5;
DISABLE_TABLE_POST_OPERATION = 6;
}
message DisableTableStateData {
required UserInformation user_info = 1;
required TableName table_name = 2;
required bool skip_table_state_check = 3;
}

View File

@ -87,14 +87,14 @@ import org.apache.hadoop.hbase.master.balancer.ClusterStatusChore;
import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
import org.apache.hadoop.hbase.master.handler.DispatchMergingRegionHandler;
import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
import org.apache.hadoop.hbase.master.handler.TruncateTableHandler;
import org.apache.hadoop.hbase.master.procedure.AddColumnFamilyProcedure;
import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
import org.apache.hadoop.hbase.master.procedure.DeleteColumnFamilyProcedure;
import org.apache.hadoop.hbase.master.procedure.DeleteTableProcedure;
import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.ModifyColumnFamilyProcedure;
@ -1681,11 +1681,24 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
cpHost.preEnableTable(tableName);
}
LOG.info(getClientIdAuditPrefix() + " enable " + tableName);
this.service.submit(new EnableTableHandler(this, tableName,
assignmentManager, tableLockManager, false).prepare());
// Execute the operation asynchronously - client will check the progress of the operation
final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createLatch();
long procId =
this.procedureExecutor.submitProcedure(new EnableTableProcedure(procedureExecutor
.getEnvironment(), tableName, false, prepareLatch));
// Before returning to client, we want to make sure that the table is prepared to be
// enabled (the table is locked and the table state is set).
//
// Note: if the procedure throws exception, we will catch it and rethrow.
prepareLatch.await();
if (cpHost != null) {
cpHost.postEnableTable(tableName);
}
}
// TODO: return procId as part of client-side change
// return procId;
}
@Override
@ -1695,11 +1708,25 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
cpHost.preDisableTable(tableName);
}
LOG.info(getClientIdAuditPrefix() + " disable " + tableName);
this.service.submit(new DisableTableHandler(this, tableName,
assignmentManager, tableLockManager, false).prepare());
// Execute the operation asynchronously - client will check the progress of the operation
final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createLatch();
// Execute the operation asynchronously - client will check the progress of the operation
long procId =
this.procedureExecutor.submitProcedure(new DisableTableProcedure(procedureExecutor
.getEnvironment(), tableName, false, prepareLatch));
// Before returning to client, we want to make sure that the table is prepared to be
// enabled (the table is locked and the table state is set).
//
// Note: if the procedure throws exception, we will catch it and rethrow.
prepareLatch.await();
if (cpHost != null) {
cpHost.postDisableTable(tableName);
}
// TODO: return procId as part of client-side change
// return procId;
}
/**

View File

@ -0,0 +1,540 @@
/**
* 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.procedure;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.security.PrivilegedExceptionAction;
import java.util.List;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.BulkAssigner;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.master.RegionStates;
import org.apache.hadoop.hbase.master.TableStateManager;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableState;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.htrace.Trace;
@InterfaceAudience.Private
public class DisableTableProcedure
extends StateMachineProcedure<MasterProcedureEnv, DisableTableState>
implements TableProcedureInterface {
private static final Log LOG = LogFactory.getLog(DisableTableProcedure.class);
private final AtomicBoolean aborted = new AtomicBoolean(false);
// This is for back compatible with 1.0 asynchronized operations.
private final ProcedurePrepareLatch syncLatch;
private TableName tableName;
private boolean skipTableStateCheck;
private UserGroupInformation user;
private Boolean traceEnabled = null;
enum MarkRegionOfflineOpResult {
MARK_ALL_REGIONS_OFFLINE_SUCCESSFUL,
BULK_ASSIGN_REGIONS_FAILED,
MARK_ALL_REGIONS_OFFLINE_INTERRUPTED,
}
public DisableTableProcedure() {
syncLatch = null;
}
/**
* Constructor
* @param env MasterProcedureEnv
* @param tableName the table to operate on
* @param skipTableStateCheck whether to check table state
* @throws IOException
*/
public DisableTableProcedure(
final MasterProcedureEnv env,
final TableName tableName,
final boolean skipTableStateCheck) throws IOException {
this(env, tableName, skipTableStateCheck, null);
}
/**
* Constructor
* @param env MasterProcedureEnv
* @param tableName the table to operate on
* @param skipTableStateCheck whether to check table state
* @throws IOException
*/
public DisableTableProcedure(
final MasterProcedureEnv env,
final TableName tableName,
final boolean skipTableStateCheck,
final ProcedurePrepareLatch syncLatch) throws IOException {
this.tableName = tableName;
this.skipTableStateCheck = skipTableStateCheck;
this.user = env.getRequestUser().getUGI();
// Compatible with 1.0: We use latch to make sure that this procedure implementation is
// compatible with 1.0 asynchronized operations. We need to lock the table and check
// whether the Disable operation could be performed (table exists and online; table state
// is ENABLED). Once it is done, we are good to release the latch and the client can
// start asynchronously wait for the operation.
//
// Note: the member syncLatch could be null if we are in failover or recovery scenario.
// This is ok for backward compatible, as 1.0 client would not able to peek at procedure.
this.syncLatch = syncLatch;
}
@Override
protected Flow executeFromState(final MasterProcedureEnv env, final DisableTableState state) {
if (isTraceEnabled()) {
LOG.trace(this + " execute state=" + state);
}
try {
switch (state) {
case DISABLE_TABLE_PREPARE:
if (prepareDisable(env)) {
setNextState(DisableTableState.DISABLE_TABLE_PRE_OPERATION);
} else {
assert isFailed() : "disable should have an exception here";
return Flow.NO_MORE_STATE;
}
break;
case DISABLE_TABLE_PRE_OPERATION:
preDisable(env, state);
setNextState(DisableTableState.DISABLE_TABLE_SET_DISABLING_TABLE_STATE);
break;
case DISABLE_TABLE_SET_DISABLING_TABLE_STATE:
setTableStateToDisabling(env, tableName);
setNextState(DisableTableState.DISABLE_TABLE_MARK_REGIONS_OFFLINE);
break;
case DISABLE_TABLE_MARK_REGIONS_OFFLINE:
if (markRegionsOffline(env, tableName, true) ==
MarkRegionOfflineOpResult.MARK_ALL_REGIONS_OFFLINE_SUCCESSFUL) {
setNextState(DisableTableState.DISABLE_TABLE_SET_DISABLED_TABLE_STATE);
} else {
LOG.trace("Retrying later to disable the missing regions");
}
break;
case DISABLE_TABLE_SET_DISABLED_TABLE_STATE:
setTableStateToDisabled(env, tableName);
setNextState(DisableTableState.DISABLE_TABLE_POST_OPERATION);
break;
case DISABLE_TABLE_POST_OPERATION:
postDisable(env, state);
return Flow.NO_MORE_STATE;
default:
throw new UnsupportedOperationException("unhandled state=" + state);
}
} catch (InterruptedException|IOException e) {
LOG.warn("Retriable error trying to disable table=" + tableName + " state=" + state, e);
}
return Flow.HAS_MORE_STATE;
}
@Override
protected void rollbackState(final MasterProcedureEnv env, final DisableTableState state)
throws IOException {
if (state == DisableTableState.DISABLE_TABLE_PREPARE) {
// nothing to rollback, prepare-disable is just table-state checks.
// We can fail if the table does not exist or is not disabled.
ProcedurePrepareLatch.releaseLatch(syncLatch, this);
return;
}
// The delete doesn't have a rollback. The execution will succeed, at some point.
throw new UnsupportedOperationException("unhandled state=" + state);
}
@Override
protected DisableTableState getState(final int stateId) {
return DisableTableState.valueOf(stateId);
}
@Override
protected int getStateId(final DisableTableState state) {
return state.getNumber();
}
@Override
protected DisableTableState getInitialState() {
return DisableTableState.DISABLE_TABLE_PREPARE;
}
@Override
protected void setNextState(final DisableTableState state) {
if (aborted.get()) {
setAbortFailure("disable-table", "abort requested");
} else {
super.setNextState(state);
}
}
@Override
public boolean abort(final MasterProcedureEnv env) {
aborted.set(true);
return true;
}
@Override
protected boolean acquireLock(final MasterProcedureEnv env) {
if (!env.isInitialized()) return false;
return env.getProcedureQueue().tryAcquireTableWrite(
tableName,
EventType.C_M_DISABLE_TABLE.toString());
}
@Override
protected void releaseLock(final MasterProcedureEnv env) {
env.getProcedureQueue().releaseTableWrite(tableName);
}
@Override
public void serializeStateData(final OutputStream stream) throws IOException {
super.serializeStateData(stream);
MasterProcedureProtos.DisableTableStateData.Builder disableTableMsg =
MasterProcedureProtos.DisableTableStateData.newBuilder()
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
.setTableName(ProtobufUtil.toProtoTableName(tableName))
.setSkipTableStateCheck(skipTableStateCheck);
disableTableMsg.build().writeDelimitedTo(stream);
}
@Override
public void deserializeStateData(final InputStream stream) throws IOException {
super.deserializeStateData(stream);
MasterProcedureProtos.DisableTableStateData disableTableMsg =
MasterProcedureProtos.DisableTableStateData.parseDelimitedFrom(stream);
user = MasterProcedureUtil.toUserInfo(disableTableMsg.getUserInfo());
tableName = ProtobufUtil.toTableName(disableTableMsg.getTableName());
skipTableStateCheck = disableTableMsg.getSkipTableStateCheck();
}
@Override
public void toStringClassDetails(StringBuilder sb) {
sb.append(getClass().getSimpleName());
sb.append(" (table=");
sb.append(tableName);
sb.append(") user=");
sb.append(user);
}
@Override
public TableName getTableName() {
return tableName;
}
@Override
public TableOperationType getTableOperationType() {
return TableOperationType.DISABLE;
}
/**
* Action before any real action of disabling table. Set the exception in the procedure instead
* of throwing it. This approach is to deal with backward compatible with 1.0.
* @param env MasterProcedureEnv
* @throws IOException
*/
private boolean prepareDisable(final MasterProcedureEnv env) throws IOException {
boolean canTableBeDisabled = true;
if (tableName.equals(TableName.META_TABLE_NAME)) {
setFailure("master-disable-table", new ConstraintException("Cannot disable catalog table"));
canTableBeDisabled = false;
} else if (!MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), tableName)) {
setFailure("master-disable-table", new TableNotFoundException(tableName));
canTableBeDisabled = false;
} else if (!skipTableStateCheck) {
// There could be multiple client requests trying to disable or enable
// the table at the same time. Ensure only the first request is honored
// After that, no other requests can be accepted until the table reaches
// DISABLED or ENABLED.
//
// Note: in 1.0 release, we called TableStateManager.setTableStateIfInStates() to set
// the state to DISABLING from ENABLED. The implementation was done before table lock
// was implemented. With table lock, there is no need to set the state here (it will
// set the state later on). A quick state check should be enough for us to move forward.
TableStateManager tsm =
env.getMasterServices().getAssignmentManager().getTableStateManager();
if (!tsm.getTableState(tableName).equals(TableState.State.ENABLED)) {
LOG.info("Table " + tableName + " isn't enabled; skipping disable");
setFailure("master-disable-table", new TableNotEnabledException(tableName));
canTableBeDisabled = false;
}
}
// We are done the check. Future actions in this procedure could be done asynchronously.
ProcedurePrepareLatch.releaseLatch(syncLatch, this);
return canTableBeDisabled;
}
/**
* Action before disabling table.
* @param env MasterProcedureEnv
* @param state the procedure state
* @throws IOException
* @throws InterruptedException
*/
protected void preDisable(final MasterProcedureEnv env, final DisableTableState state)
throws IOException, InterruptedException {
runCoprocessorAction(env, state);
}
/**
* Mark table state to Disabling
* @param env MasterProcedureEnv
* @throws IOException
*/
protected static void setTableStateToDisabling(
final MasterProcedureEnv env,
final TableName tableName) throws IOException {
// Set table disabling flag up in zk.
env.getMasterServices().getAssignmentManager().getTableStateManager().setTableState(
tableName,
TableState.State.DISABLING);
}
/**
* Mark regions of the table offline with retries
* @param env MasterProcedureEnv
* @param tableName the target table
* @param retryRequired whether to retry if the first run failed
* @return whether the operation is fully completed or being interrupted.
* @throws IOException
*/
protected static MarkRegionOfflineOpResult markRegionsOffline(
final MasterProcedureEnv env,
final TableName tableName,
final Boolean retryRequired) throws IOException {
// Dev consideration: add a config to control max number of retry. For now, it is hard coded.
int maxTry = (retryRequired ? 10 : 1);
MarkRegionOfflineOpResult operationResult =
MarkRegionOfflineOpResult.BULK_ASSIGN_REGIONS_FAILED;
do {
try {
operationResult = markRegionsOffline(env, tableName);
if (operationResult == MarkRegionOfflineOpResult.MARK_ALL_REGIONS_OFFLINE_SUCCESSFUL) {
break;
}
maxTry--;
} catch (Exception e) {
LOG.warn("Received exception while marking regions online. tries left: " + maxTry, e);
maxTry--;
if (maxTry > 0) {
continue; // we still have some retry left, try again.
}
throw e;
}
} while (maxTry > 0);
if (operationResult != MarkRegionOfflineOpResult.MARK_ALL_REGIONS_OFFLINE_SUCCESSFUL) {
LOG.warn("Some or all regions of the Table '" + tableName + "' were still online");
}
return operationResult;
}
/**
* Mark regions of the table offline
* @param env MasterProcedureEnv
* @param tableName the target table
* @return whether the operation is fully completed or being interrupted.
* @throws IOException
*/
private static MarkRegionOfflineOpResult markRegionsOffline(
final MasterProcedureEnv env,
final TableName tableName) throws IOException {
// Get list of online regions that are of this table. Regions that are
// already closed will not be included in this list; i.e. the returned
// list is not ALL regions in a table, its all online regions according
// to the in-memory state on this master.
MarkRegionOfflineOpResult operationResult =
MarkRegionOfflineOpResult.MARK_ALL_REGIONS_OFFLINE_SUCCESSFUL;
final List<HRegionInfo> regions =
env.getMasterServices().getAssignmentManager().getRegionStates()
.getRegionsOfTable(tableName);
if (regions.size() > 0) {
LOG.info("Offlining " + regions.size() + " regions.");
BulkDisabler bd = new BulkDisabler(env, tableName, regions);
try {
if (!bd.bulkAssign()) {
operationResult = MarkRegionOfflineOpResult.BULK_ASSIGN_REGIONS_FAILED;
}
} catch (InterruptedException e) {
LOG.warn("Disable was interrupted");
// Preserve the interrupt.
Thread.currentThread().interrupt();
operationResult = MarkRegionOfflineOpResult.MARK_ALL_REGIONS_OFFLINE_INTERRUPTED;
}
}
return operationResult;
}
/**
* Mark table state to Disabled
* @param env MasterProcedureEnv
* @throws IOException
*/
protected static void setTableStateToDisabled(
final MasterProcedureEnv env,
final TableName tableName) throws IOException {
// Flip the table to disabled
env.getMasterServices().getAssignmentManager().getTableStateManager().setTableState(
tableName,
TableState.State.DISABLED);
LOG.info("Disabled table, " + tableName + ", is completed.");
}
/**
* Action after disabling table.
* @param env MasterProcedureEnv
* @param state the procedure state
* @throws IOException
* @throws InterruptedException
*/
protected void postDisable(final MasterProcedureEnv env, final DisableTableState state)
throws IOException, InterruptedException {
runCoprocessorAction(env, state);
}
/**
* The procedure could be restarted from a different machine. If the variable is null, we need to
* retrieve it.
* @return traceEnabled
*/
private Boolean isTraceEnabled() {
if (traceEnabled == null) {
traceEnabled = LOG.isTraceEnabled();
}
return traceEnabled;
}
/**
* Coprocessor Action.
* @param env MasterProcedureEnv
* @param state the procedure state
* @throws IOException
* @throws InterruptedException
*/
private void runCoprocessorAction(final MasterProcedureEnv env, final DisableTableState state)
throws IOException, InterruptedException {
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
user.doAs(new PrivilegedExceptionAction<Void>() {
@Override
public Void run() throws Exception {
switch (state) {
case DISABLE_TABLE_PRE_OPERATION:
cpHost.preDisableTableHandler(tableName);
break;
case DISABLE_TABLE_POST_OPERATION:
cpHost.postDisableTableHandler(tableName);
break;
default:
throw new UnsupportedOperationException(this + " unhandled state=" + state);
}
return null;
}
});
}
}
/**
* Run bulk disable.
*/
private static class BulkDisabler extends BulkAssigner {
private final AssignmentManager assignmentManager;
private final List<HRegionInfo> regions;
private final TableName tableName;
private final int waitingTimeForEvents;
public BulkDisabler(final MasterProcedureEnv env, final TableName tableName,
final List<HRegionInfo> regions) {
super(env.getMasterServices());
this.assignmentManager = env.getMasterServices().getAssignmentManager();
this.tableName = tableName;
this.regions = regions;
this.waitingTimeForEvents =
env.getMasterServices().getConfiguration()
.getInt("hbase.master.event.waiting.time", 1000);
}
@Override
protected void populatePool(ExecutorService pool) {
RegionStates regionStates = assignmentManager.getRegionStates();
for (final HRegionInfo region : regions) {
if (regionStates.isRegionInTransition(region)
&& !regionStates.isRegionInState(region, RegionState.State.FAILED_CLOSE)) {
continue;
}
pool.execute(Trace.wrap("DisableTableHandler.BulkDisabler", new Runnable() {
@Override
public void run() {
assignmentManager.unassign(region);
}
}));
}
}
@Override
protected boolean waitUntilDone(long timeout) throws InterruptedException {
long startTime = EnvironmentEdgeManager.currentTime();
long remaining = timeout;
List<HRegionInfo> regions = null;
long lastLogTime = startTime;
while (!server.isStopped() && remaining > 0) {
Thread.sleep(waitingTimeForEvents);
regions = assignmentManager.getRegionStates().getRegionsOfTable(tableName);
long now = EnvironmentEdgeManager.currentTime();
// Don't log more than once every ten seconds. Its obnoxious. And only log table regions
// if we are waiting a while for them to go down...
if (LOG.isDebugEnabled() && ((now - lastLogTime) > 10000)) {
lastLogTime = now;
LOG.debug("Disable waiting until done; " + remaining + " ms remaining; " + regions);
}
if (regions.isEmpty()) break;
remaining = timeout - (now - startTime);
}
return regions != null && regions.isEmpty();
}
}
}

View File

@ -0,0 +1,582 @@
/**
* 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.procedure;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.security.PrivilegedExceptionAction;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.BulkAssigner;
import org.apache.hadoop.hbase.master.GeneralBulkAssigner;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.RegionStates;
import org.apache.hadoop.hbase.master.ServerManager;
import org.apache.hadoop.hbase.master.TableStateManager;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableState;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.security.UserGroupInformation;
@InterfaceAudience.Private
public class EnableTableProcedure
extends StateMachineProcedure<MasterProcedureEnv, EnableTableState>
implements TableProcedureInterface {
private static final Log LOG = LogFactory.getLog(EnableTableProcedure.class);
private final AtomicBoolean aborted = new AtomicBoolean(false);
// This is for back compatible with 1.0 asynchronized operations.
private final ProcedurePrepareLatch syncLatch;
private TableName tableName;
private boolean skipTableStateCheck;
private UserGroupInformation user;
private Boolean traceEnabled = null;
public EnableTableProcedure() {
syncLatch = null;
}
/**
* Constructor
* @param env MasterProcedureEnv
* @param tableName the table to operate on
* @param skipTableStateCheck whether to check table state
* @throws IOException
*/
public EnableTableProcedure(
final MasterProcedureEnv env,
final TableName tableName,
final boolean skipTableStateCheck) throws IOException {
this(env, tableName, skipTableStateCheck, null);
}
/**
* Constructor
* @param env MasterProcedureEnv
* @throws IOException
* @param tableName the table to operate on
* @param skipTableStateCheck whether to check table state
*/
public EnableTableProcedure(
final MasterProcedureEnv env,
final TableName tableName,
final boolean skipTableStateCheck,
final ProcedurePrepareLatch syncLatch) throws IOException {
this.tableName = tableName;
this.skipTableStateCheck = skipTableStateCheck;
this.user = env.getRequestUser().getUGI();
// Compatible with 1.0: We use latch to make sure that this procedure implementation is
// compatible with 1.0 asynchronized operations. We need to lock the table and check
// whether the Enable operation could be performed (table exists and offline; table state
// is DISABLED). Once it is done, we are good to release the latch and the client can
// start asynchronously wait for the operation.
//
// Note: the member syncLatch could be null if we are in failover or recovery scenario.
// This is ok for backward compatible, as 1.0 client would not able to peek at procedure.
this.syncLatch = syncLatch;
}
@Override
protected Flow executeFromState(final MasterProcedureEnv env, final EnableTableState state) {
if (isTraceEnabled()) {
LOG.trace(this + " execute state=" + state);
}
try {
switch (state) {
case ENABLE_TABLE_PREPARE:
if (prepareEnable(env)) {
setNextState(EnableTableState.ENABLE_TABLE_PRE_OPERATION);
} else {
assert isFailed() : "enable should have an exception here";
return Flow.NO_MORE_STATE;
}
break;
case ENABLE_TABLE_PRE_OPERATION:
preEnable(env, state);
setNextState(EnableTableState.ENABLE_TABLE_SET_ENABLING_TABLE_STATE);
break;
case ENABLE_TABLE_SET_ENABLING_TABLE_STATE:
setTableStateToEnabling(env, tableName);
setNextState(EnableTableState.ENABLE_TABLE_MARK_REGIONS_ONLINE);
break;
case ENABLE_TABLE_MARK_REGIONS_ONLINE:
markRegionsOnline(env, tableName, true);
setNextState(EnableTableState.ENABLE_TABLE_SET_ENABLED_TABLE_STATE);
break;
case ENABLE_TABLE_SET_ENABLED_TABLE_STATE:
setTableStateToEnabled(env, tableName);
setNextState(EnableTableState.ENABLE_TABLE_POST_OPERATION);
break;
case ENABLE_TABLE_POST_OPERATION:
postEnable(env, state);
return Flow.NO_MORE_STATE;
default:
throw new UnsupportedOperationException("unhandled state=" + state);
}
} catch (InterruptedException|IOException e) {
LOG.error("Error trying to enable table=" + tableName + " state=" + state, e);
setFailure("master-enable-table", e);
}
return Flow.HAS_MORE_STATE;
}
@Override
protected void rollbackState(final MasterProcedureEnv env, final EnableTableState state)
throws IOException {
if (isTraceEnabled()) {
LOG.trace(this + " rollback state=" + state);
}
try {
switch (state) {
case ENABLE_TABLE_POST_OPERATION:
// TODO-MAYBE: call the coprocessor event to undo (eg. DisableTableProcedure.preDisable())?
break;
case ENABLE_TABLE_SET_ENABLED_TABLE_STATE:
DisableTableProcedure.setTableStateToDisabling(env, tableName);
break;
case ENABLE_TABLE_MARK_REGIONS_ONLINE:
markRegionsOfflineDuringRecovery(env);
break;
case ENABLE_TABLE_SET_ENABLING_TABLE_STATE:
DisableTableProcedure.setTableStateToDisabled(env, tableName);
break;
case ENABLE_TABLE_PRE_OPERATION:
// TODO-MAYBE: call the coprocessor event to undo (eg. DisableTableProcedure.postDisable())?
break;
case ENABLE_TABLE_PREPARE:
// Nothing to undo for this state.
// We do need to count down the latch count so that we don't stuck.
ProcedurePrepareLatch.releaseLatch(syncLatch, this);
break;
default:
throw new UnsupportedOperationException("unhandled state=" + state);
}
} catch (IOException e) {
// This will be retried. Unless there is a bug in the code,
// this should be just a "temporary error" (e.g. network down)
LOG.warn("Failed enable table rollback attempt step=" + state + " table=" + tableName, e);
throw e;
}
}
@Override
protected EnableTableState getState(final int stateId) {
return EnableTableState.valueOf(stateId);
}
@Override
protected int getStateId(final EnableTableState state) {
return state.getNumber();
}
@Override
protected EnableTableState getInitialState() {
return EnableTableState.ENABLE_TABLE_PREPARE;
}
@Override
protected void setNextState(final EnableTableState state) {
if (aborted.get()) {
setAbortFailure("Enable-table", "abort requested");
} else {
super.setNextState(state);
}
}
@Override
public boolean abort(final MasterProcedureEnv env) {
aborted.set(true);
return true;
}
@Override
protected boolean acquireLock(final MasterProcedureEnv env) {
if (!env.isInitialized()) return false;
return env.getProcedureQueue().tryAcquireTableWrite(
tableName,
EventType.C_M_ENABLE_TABLE.toString());
}
@Override
protected void releaseLock(final MasterProcedureEnv env) {
env.getProcedureQueue().releaseTableWrite(tableName);
}
@Override
public void serializeStateData(final OutputStream stream) throws IOException {
super.serializeStateData(stream);
MasterProcedureProtos.EnableTableStateData.Builder enableTableMsg =
MasterProcedureProtos.EnableTableStateData.newBuilder()
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
.setTableName(ProtobufUtil.toProtoTableName(tableName))
.setSkipTableStateCheck(skipTableStateCheck);
enableTableMsg.build().writeDelimitedTo(stream);
}
@Override
public void deserializeStateData(final InputStream stream) throws IOException {
super.deserializeStateData(stream);
MasterProcedureProtos.EnableTableStateData enableTableMsg =
MasterProcedureProtos.EnableTableStateData.parseDelimitedFrom(stream);
user = MasterProcedureUtil.toUserInfo(enableTableMsg.getUserInfo());
tableName = ProtobufUtil.toTableName(enableTableMsg.getTableName());
skipTableStateCheck = enableTableMsg.getSkipTableStateCheck();
}
@Override
public void toStringClassDetails(StringBuilder sb) {
sb.append(getClass().getSimpleName());
sb.append(" (table=");
sb.append(tableName);
sb.append(") user=");
sb.append(user);
}
@Override
public TableName getTableName() {
return tableName;
}
@Override
public TableOperationType getTableOperationType() {
return TableOperationType.ENABLE;
}
/**
* Action before any real action of enabling table. Set the exception in the procedure instead
* of throwing it. This approach is to deal with backward compatible with 1.0.
* @param env MasterProcedureEnv
* @return whether the table passes the necessary checks
* @throws IOException
*/
private boolean prepareEnable(final MasterProcedureEnv env) throws IOException {
boolean canTableBeEnabled = true;
// Check whether table exists
if (!MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), tableName)) {
setFailure("master-enable-table", new TableNotFoundException(tableName));
canTableBeEnabled = false;
} else if (!skipTableStateCheck) {
// There could be multiple client requests trying to disable or enable
// the table at the same time. Ensure only the first request is honored
// After that, no other requests can be accepted until the table reaches
// DISABLED or ENABLED.
//
// Note: in 1.0 release, we called TableStateManager.setTableStateIfInStates() to set
// the state to ENABLING from DISABLED. The implementation was done before table lock
// was implemented. With table lock, there is no need to set the state here (it will
// set the state later on). A quick state check should be enough for us to move forward.
TableStateManager tsm = env.getMasterServices().getAssignmentManager().getTableStateManager();
if (!tsm.getTableState(tableName).equals(TableState.State.DISABLED)) {
LOG.info("Table " + tableName + " isn't disabled; skipping enable");
setFailure("master-enable-table", new TableNotDisabledException(this.tableName));
canTableBeEnabled = false;
}
}
// We are done the check. Future actions in this procedure could be done asynchronously.
ProcedurePrepareLatch.releaseLatch(syncLatch, this);
return canTableBeEnabled;
}
/**
* Action before enabling table.
* @param env MasterProcedureEnv
* @param state the procedure state
* @throws IOException
* @throws InterruptedException
*/
private void preEnable(final MasterProcedureEnv env, final EnableTableState state)
throws IOException, InterruptedException {
runCoprocessorAction(env, state);
}
/**
* Mark table state to Enabling
* @param env MasterProcedureEnv
* @param tableName the target table
* @throws IOException
*/
protected static void setTableStateToEnabling(
final MasterProcedureEnv env,
final TableName tableName) throws IOException {
// Set table disabling flag up in zk.
LOG.info("Attempting to enable the table " + tableName);
env.getMasterServices().getAssignmentManager().getTableStateManager().setTableState(
tableName,
TableState.State.ENABLING);
}
/**
* Mark offline regions of the table online with retry
* @param env MasterProcedureEnv
* @param tableName the target table
* @param retryRequired whether to retry if the first run failed
* @throws IOException
*/
protected static void markRegionsOnline(
final MasterProcedureEnv env,
final TableName tableName,
final Boolean retryRequired) throws IOException {
// This is best effort approach to make all regions of a table online. If we fail to do
// that, it is ok that the table has some offline regions; user can fix it manually.
// Dev consideration: add a config to control max number of retry. For now, it is hard coded.
int maxTry = (retryRequired ? 10 : 1);
boolean done = false;
do {
try {
done = markRegionsOnline(env, tableName);
if (done) {
break;
}
maxTry--;
} catch (Exception e) {
LOG.warn("Received exception while marking regions online. tries left: " + maxTry, e);
maxTry--;
if (maxTry > 0) {
continue; // we still have some retry left, try again.
}
throw e;
}
} while (maxTry > 0);
if (!done) {
LOG.warn("Some or all regions of the Table '" + tableName + "' were offline");
}
}
/**
* Mark offline regions of the table online
* @param env MasterProcedureEnv
* @param tableName the target table
* @return whether the operation is fully completed or being interrupted.
* @throws IOException
*/
private static boolean markRegionsOnline(final MasterProcedureEnv env, final TableName tableName)
throws IOException {
final AssignmentManager assignmentManager = env.getMasterServices().getAssignmentManager();
final MasterServices masterServices = env.getMasterServices();
final ServerManager serverManager = masterServices.getServerManager();
boolean done = false;
// Get the regions of this table. We're done when all listed
// tables are onlined.
List<Pair<HRegionInfo, ServerName>> tableRegionsAndLocations;
if (TableName.META_TABLE_NAME.equals(tableName)) {
tableRegionsAndLocations =
new MetaTableLocator().getMetaRegionsAndLocations(masterServices.getZooKeeper());
} else {
tableRegionsAndLocations =
MetaTableAccessor.getTableRegionsAndLocations(masterServices.getConnection(), tableName);
}
int countOfRegionsInTable = tableRegionsAndLocations.size();
Map<HRegionInfo, ServerName> regionsToAssign =
regionsToAssignWithServerName(env, tableRegionsAndLocations);
// need to potentially create some regions for the replicas
List<HRegionInfo> unrecordedReplicas =
AssignmentManager.replicaRegionsNotRecordedInMeta(new HashSet<HRegionInfo>(
regionsToAssign.keySet()), masterServices);
Map<ServerName, List<HRegionInfo>> srvToUnassignedRegs =
assignmentManager.getBalancer().roundRobinAssignment(unrecordedReplicas,
serverManager.getOnlineServersList());
if (srvToUnassignedRegs != null) {
for (Map.Entry<ServerName, List<HRegionInfo>> entry : srvToUnassignedRegs.entrySet()) {
for (HRegionInfo h : entry.getValue()) {
regionsToAssign.put(h, entry.getKey());
}
}
}
int offlineRegionsCount = regionsToAssign.size();
LOG.info("Table '" + tableName + "' has " + countOfRegionsInTable + " regions, of which "
+ offlineRegionsCount + " are offline.");
if (offlineRegionsCount == 0) {
return true;
}
List<ServerName> onlineServers = serverManager.createDestinationServersList();
Map<ServerName, List<HRegionInfo>> bulkPlan =
env.getMasterServices().getAssignmentManager().getBalancer()
.retainAssignment(regionsToAssign, onlineServers);
if (bulkPlan != null) {
LOG.info("Bulk assigning " + offlineRegionsCount + " region(s) across " + bulkPlan.size()
+ " server(s), retainAssignment=true");
BulkAssigner ba = new GeneralBulkAssigner(masterServices, bulkPlan, assignmentManager, true);
try {
if (ba.bulkAssign()) {
done = true;
}
} catch (InterruptedException e) {
LOG.warn("Enable operation was interrupted when enabling table '" + tableName + "'");
// Preserve the interrupt.
Thread.currentThread().interrupt();
}
} else {
LOG.info("Balancer was unable to find suitable servers for table " + tableName
+ ", leaving unassigned");
}
return done;
}
/**
* Mark regions of the table offline during recovery
* @param env MasterProcedureEnv
*/
private void markRegionsOfflineDuringRecovery(final MasterProcedureEnv env) {
try {
// This is a best effort attempt. We will move on even it does not succeed. We will retry
// several times until we giving up.
DisableTableProcedure.markRegionsOffline(env, tableName, true);
} catch (Exception e) {
LOG.debug("Failed to offline all regions of table " + tableName + ". Ignoring", e);
}
}
/**
* Mark table state to Enabled
* @param env MasterProcedureEnv
* @throws IOException
*/
protected static void setTableStateToEnabled(
final MasterProcedureEnv env,
final TableName tableName) throws IOException {
// Flip the table to Enabled
env.getMasterServices().getAssignmentManager().getTableStateManager().setTableState(
tableName,
TableState.State.ENABLED);
LOG.info("Table '" + tableName + "' was successfully enabled.");
}
/**
* Action after enabling table.
* @param env MasterProcedureEnv
* @param state the procedure state
* @throws IOException
* @throws InterruptedException
*/
private void postEnable(final MasterProcedureEnv env, final EnableTableState state)
throws IOException, InterruptedException {
runCoprocessorAction(env, state);
}
/**
* The procedure could be restarted from a different machine. If the variable is null, we need to
* retrieve it.
* @return traceEnabled
*/
private Boolean isTraceEnabled() {
if (traceEnabled == null) {
traceEnabled = LOG.isTraceEnabled();
}
return traceEnabled;
}
/**
* @param regionsInMeta
* @return List of regions neither in transition nor assigned.
* @throws IOException
*/
private static Map<HRegionInfo, ServerName> regionsToAssignWithServerName(
final MasterProcedureEnv env,
final List<Pair<HRegionInfo, ServerName>> regionsInMeta) throws IOException {
Map<HRegionInfo, ServerName> regionsToAssign =
new HashMap<HRegionInfo, ServerName>(regionsInMeta.size());
RegionStates regionStates = env.getMasterServices().getAssignmentManager().getRegionStates();
for (Pair<HRegionInfo, ServerName> regionLocation : regionsInMeta) {
HRegionInfo hri = regionLocation.getFirst();
ServerName sn = regionLocation.getSecond();
if (regionStates.isRegionOffline(hri)) {
regionsToAssign.put(hri, sn);
} else {
if (LOG.isDebugEnabled()) {
LOG.debug("Skipping assign for the region " + hri + " during enable table "
+ hri.getTable() + " because its already in tranition or assigned.");
}
}
}
return regionsToAssign;
}
/**
* Coprocessor Action.
* @param env MasterProcedureEnv
* @param state the procedure state
* @throws IOException
* @throws InterruptedException
*/
private void runCoprocessorAction(final MasterProcedureEnv env, final EnableTableState state)
throws IOException, InterruptedException {
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
user.doAs(new PrivilegedExceptionAction<Void>() {
@Override
public Void run() throws Exception {
switch (state) {
case ENABLE_TABLE_PRE_OPERATION:
cpHost.preEnableTableHandler(getTableName());
break;
case ENABLE_TABLE_POST_OPERATION:
cpHost.postEnableTableHandler(getTableName());
break;
default:
throw new UnsupportedOperationException(this + " unhandled state=" + state);
}
return null;
}
});
}
}
}

View File

@ -18,9 +18,9 @@
package org.apache.hadoop.hbase.master.procedure;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.TableName;
/**
* Procedures that operates on a specific Table (e.g. create, delete, snapshot, ...)
@ -29,7 +29,9 @@ import org.apache.hadoop.hbase.TableName;
@InterfaceAudience.Private
@InterfaceStability.Evolving
public interface TableProcedureInterface {
public enum TableOperationType { CREATE, DELETE, EDIT, READ };
public enum TableOperationType {
CREATE, DELETE, DISABLE, EDIT, ENABLE, READ,
};
/**
* @return the name of the table the procedure is operating on

View File

@ -36,7 +36,9 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.TableStateManager;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
@ -172,6 +174,18 @@ public class MasterProcedureTestingUtility {
return actualRegCount.get();
}
public static void validateTableIsEnabled(final HMaster master, final TableName tableName)
throws IOException {
TableStateManager tsm = master.getAssignmentManager().getTableStateManager();
assertTrue(tsm.getTableState(tableName).equals(TableState.State.ENABLED));
}
public static void validateTableIsDisabled(final HMaster master, final TableName tableName)
throws IOException {
TableStateManager tsm = master.getAssignmentManager().getTableStateManager();
assertTrue(tsm.getTableState(tableName).equals(TableState.State.DISABLED));
}
public static <TState> void testRecoveryAndDoubleExecution(
final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId,
final int numSteps, final TState[] states) throws Exception {

View File

@ -0,0 +1,182 @@
/**
* 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.procedure;
import static org.junit.Assert.assertTrue;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureResult;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category({MasterTests.class, MediumTests.class})
public class TestDisableTableProcedure {
private static final Log LOG = LogFactory.getLog(TestDisableTableProcedure.class);
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static void setupConf(Configuration conf) {
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
}
@BeforeClass
public static void setupCluster() throws Exception {
setupConf(UTIL.getConfiguration());
UTIL.startMiniCluster(1);
}
@AfterClass
public static void cleanupTest() throws Exception {
try {
UTIL.shutdownMiniCluster();
} catch (Exception e) {
LOG.warn("failure shutting down cluster", e);
}
}
@Before
public void setup() throws Exception {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
}
@After
public void tearDown() throws Exception {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
for (HTableDescriptor htd: UTIL.getHBaseAdmin().listTables()) {
LOG.info("Tear down, remove table=" + htd.getTableName());
UTIL.deleteTable(htd.getTableName());
}
}
@Test(timeout = 60000)
public void testDisableTable() throws Exception {
final TableName tableName = TableName.valueOf("testDisableTable");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2");
// Disable the table
long procId = procExec.submitProcedure(
new DisableTableProcedure(procExec.getEnvironment(), tableName, false));
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
MasterProcedureTestingUtility.validateTableIsDisabled(UTIL.getHBaseCluster().getMaster(),
tableName);
}
@Test(timeout = 60000)
public void testDisableTableMultipleTimes() throws Exception {
final TableName tableName = TableName.valueOf("testDisableTableMultipleTimes");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2");
// Disable the table
long procId1 = procExec.submitProcedure(new DisableTableProcedure(
procExec.getEnvironment(), tableName, false));
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
MasterProcedureTestingUtility.validateTableIsDisabled(UTIL.getHBaseCluster().getMaster(),
tableName);
// Disable the table again - expect failure
long procId2 = procExec.submitProcedure(new DisableTableProcedure(
procExec.getEnvironment(), tableName, false));
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId2);
ProcedureResult result = procExec.getResult(procId2);
assertTrue(result.isFailed());
LOG.debug("Disable failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof TableNotEnabledException);
// Disable the table - expect failure from ProcedurePrepareLatch
try {
final ProcedurePrepareLatch prepareLatch = new ProcedurePrepareLatch.CompatibilityLatch();
long procId3 = procExec.submitProcedure(new DisableTableProcedure(
procExec.getEnvironment(), tableName, false, prepareLatch));
prepareLatch.await();
Assert.fail("Disable should throw exception through latch.");
} catch (TableNotEnabledException tnee) {
// Expected
LOG.debug("Disable failed with expected exception.");
}
// Disable the table again with skipping table state check flag (simulate recovery scenario)
long procId4 = procExec.submitProcedure(new DisableTableProcedure(
procExec.getEnvironment(), tableName, true));
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId4);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId4);
MasterProcedureTestingUtility.validateTableIsDisabled(UTIL.getHBaseCluster().getMaster(),
tableName);
}
@Test(timeout=60000)
public void testRecoveryAndDoubleExecution() throws Exception {
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
final byte[][] splitKeys = new byte[][] {
Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
};
MasterProcedureTestingUtility.createTable(procExec, tableName, splitKeys, "f1", "f2");
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the Disable procedure && kill the executor
long procId =
procExec.submitProcedure(new DisableTableProcedure(procExec.getEnvironment(), tableName,
false));
// Restart the executor and execute the step twice
int numberOfSteps = DisableTableState.values().length;
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(
procExec,
procId,
numberOfSteps,
DisableTableState.values());
MasterProcedureTestingUtility.validateTableIsDisabled(UTIL.getHBaseCluster().getMaster(),
tableName);
}
private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
}
}

View File

@ -0,0 +1,193 @@
/**
* 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.procedure;
import static org.junit.Assert.assertTrue;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureResult;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category({MasterTests.class, MediumTests.class})
public class TestEnableTableProcedure {
private static final Log LOG = LogFactory.getLog(TestEnableTableProcedure.class);
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static void setupConf(Configuration conf) {
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
}
@BeforeClass
public static void setupCluster() throws Exception {
setupConf(UTIL.getConfiguration());
UTIL.startMiniCluster(1);
}
@AfterClass
public static void cleanupTest() throws Exception {
try {
UTIL.shutdownMiniCluster();
} catch (Exception e) {
LOG.warn("failure shutting down cluster", e);
}
}
@Before
public void setup() throws Exception {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
}
@After
public void tearDown() throws Exception {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
for (HTableDescriptor htd: UTIL.getHBaseAdmin().listTables()) {
LOG.info("Tear down, remove table=" + htd.getTableName());
UTIL.deleteTable(htd.getTableName());
}
}
@Test(timeout = 60000)
public void testEnableTable() throws Exception {
final TableName tableName = TableName.valueOf("testEnableTable");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2");
UTIL.getHBaseAdmin().disableTable(tableName);
// Enable the table
long procId = procExec.submitProcedure(
new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
MasterProcedureTestingUtility.validateTableIsEnabled(UTIL.getHBaseCluster().getMaster(),
tableName);
}
@Test(timeout=60000, expected=TableNotDisabledException.class)
public void testEnableNonDisabledTable() throws Exception {
final TableName tableName = TableName.valueOf("testEnableNonExistingTable");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2");
// Enable the table - expect failure
long procId1 = procExec.submitProcedure(
new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureResult result = procExec.getResult(procId1);
assertTrue(result.isFailed());
LOG.debug("Enable failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof TableNotDisabledException);
// Enable the table with skipping table state check flag (simulate recovery scenario)
long procId2 = procExec.submitProcedure(
new EnableTableProcedure(procExec.getEnvironment(), tableName, true));
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId2);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
// Enable the table - expect failure from ProcedurePrepareLatch
final ProcedurePrepareLatch prepareLatch = new ProcedurePrepareLatch.CompatibilityLatch();
long procId3 = procExec.submitProcedure(
new EnableTableProcedure(procExec.getEnvironment(), tableName, false, prepareLatch));
prepareLatch.await();
Assert.fail("Enable should throw exception through latch.");
}
@Test(timeout = 60000)
public void testRecoveryAndDoubleExecution() throws Exception {
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
final byte[][] splitKeys = new byte[][] {
Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
};
MasterProcedureTestingUtility.createTable(procExec, tableName, splitKeys, "f1", "f2");
UTIL.getHBaseAdmin().disableTable(tableName);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the Enable procedure && kill the executor
long procId = procExec.submitProcedure(
new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
// Restart the executor and execute the step twice
int numberOfSteps = EnableTableState.values().length;
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(
procExec,
procId,
numberOfSteps,
EnableTableState.values());
MasterProcedureTestingUtility.validateTableIsEnabled(UTIL.getHBaseCluster().getMaster(),
tableName);
}
@Test(timeout = 60000)
public void testRollbackAndDoubleExecution() throws Exception {
final TableName tableName = TableName.valueOf("testRollbackAndDoubleExecution");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
final byte[][] splitKeys = new byte[][] {
Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
};
MasterProcedureTestingUtility.createTable(procExec, tableName, splitKeys, "f1", "f2");
UTIL.getHBaseAdmin().disableTable(tableName);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the Enable procedure && kill the executor
long procId = procExec.submitProcedure(
new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
int numberOfSteps = EnableTableState.values().length - 2; // failing in the middle of proc
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(
procExec,
procId,
numberOfSteps,
EnableTableState.values());
MasterProcedureTestingUtility.validateTableIsDisabled(UTIL.getHBaseCluster().getMaster(),
tableName);
}
private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
}
}

View File

@ -37,8 +37,11 @@ import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableState;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableState;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
@ -211,6 +214,79 @@ public class TestMasterFailoverWithProcedures {
UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
}
// ==========================================================================
// Test Disable Table
// ==========================================================================
@Test(timeout=60000)
public void testDisableTableWithFailover() throws Exception {
// TODO: Should we try every step? (master failover takes long time)
// It is already covered by TestDisableTableProcedure
// but without the master restart, only the executor/store is restarted.
// Without Master restart we may not find bug in the procedure code
// like missing "wait" for resources to be available (e.g. RS)
testDisableTableWithFailoverAtStep(
DisableTableState.DISABLE_TABLE_MARK_REGIONS_OFFLINE.ordinal());
}
private void testDisableTableWithFailoverAtStep(final int step) throws Exception {
final TableName tableName = TableName.valueOf("testDisableTableWithFailoverAtStep" + step);
// create the table
final byte[][] splitKeys = new byte[][] {
Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
};
MasterProcedureTestingUtility.createTable(
getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the Delete procedure && kill the executor
long procId = procExec.submitProcedure(
new DisableTableProcedure(procExec.getEnvironment(), tableName, false));
testRecoveryAndDoubleExecution(UTIL, procId, step, DisableTableState.values());
MasterProcedureTestingUtility.validateTableIsDisabled(
UTIL.getHBaseCluster().getMaster(), tableName);
}
// ==========================================================================
// Test Enable Table
// ==========================================================================
@Test(timeout=60000)
public void testEnableTableWithFailover() throws Exception {
// TODO: Should we try every step? (master failover takes long time)
// It is already covered by TestEnableTableProcedure
// but without the master restart, only the executor/store is restarted.
// Without Master restart we may not find bug in the procedure code
// like missing "wait" for resources to be available (e.g. RS)
testEnableTableWithFailoverAtStep(
EnableTableState.ENABLE_TABLE_MARK_REGIONS_ONLINE.ordinal());
}
private void testEnableTableWithFailoverAtStep(final int step) throws Exception {
final TableName tableName = TableName.valueOf("testEnableTableWithFailoverAtStep" + step);
// create the table
final byte[][] splitKeys = new byte[][] {
Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
};
MasterProcedureTestingUtility.createTable(
getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
UTIL.getHBaseAdmin().disableTable(tableName);
ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the Delete procedure && kill the executor
long procId = procExec.submitProcedure(
new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
testRecoveryAndDoubleExecution(UTIL, procId, step, EnableTableState.values());
MasterProcedureTestingUtility.validateTableIsEnabled(
UTIL.getHBaseCluster().getMaster(), tableName);
}
// ==========================================================================
// Test Helpers
// ==========================================================================