HBASE-13211 Procedure V2 - master Enable/Disable table (Stephen Yuan Jiang)
This commit is contained in:
parent
4ae8b8cc52
commit
57c70f0af8
File diff suppressed because it is too large
Load Diff
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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
|
||||
// ==========================================================================
|
||||
|
|
Loading…
Reference in New Issue