HBASE-13209 Procedure V2 - master Add/Modify/Delete Column Family (Stephen Yuan Jiang)

This commit is contained in:
Matteo Bertozzi 2015-04-09 21:21:18 +01:00
parent c967595acb
commit 4cc464dae1
14 changed files with 6280 additions and 80 deletions

View File

@ -89,3 +89,49 @@ message DeleteTableStateData {
required TableName table_name = 2;
repeated RegionInfo region_info = 3;
}
enum AddColumnFamilyState {
ADD_COLUMN_FAMILY_PREPARE = 1;
ADD_COLUMN_FAMILY_PRE_OPERATION = 2;
ADD_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3;
ADD_COLUMN_FAMILY_POST_OPERATION = 4;
ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 5;
}
message AddColumnFamilyStateData {
required UserInformation user_info = 1;
required TableName table_name = 2;
required ColumnFamilySchema columnfamily_schema = 3;
optional TableSchema unmodified_table_schema = 4;
}
enum ModifyColumnFamilyState {
MODIFY_COLUMN_FAMILY_PREPARE = 1;
MODIFY_COLUMN_FAMILY_PRE_OPERATION = 2;
MODIFY_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3;
MODIFY_COLUMN_FAMILY_POST_OPERATION = 4;
MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 5;
}
message ModifyColumnFamilyStateData {
required UserInformation user_info = 1;
required TableName table_name = 2;
required ColumnFamilySchema columnfamily_schema = 3;
optional TableSchema unmodified_table_schema = 4;
}
enum DeleteColumnFamilyState {
DELETE_COLUMN_FAMILY_PREPARE = 1;
DELETE_COLUMN_FAMILY_PRE_OPERATION = 2;
DELETE_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR = 3;
DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT = 4;
DELETE_COLUMN_FAMILY_POST_OPERATION = 5;
DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS = 6;
}
message DeleteColumnFamilyStateData {
required UserInformation user_info = 1;
required TableName table_name = 2;
required bytes columnfamily_name = 3;
optional TableSchema unmodified_table_schema = 4;
}

View File

@ -93,14 +93,14 @@ 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.TableAddFamilyHandler;
import org.apache.hadoop.hbase.master.handler.TableDeleteFamilyHandler;
import org.apache.hadoop.hbase.master.handler.TableModifyFamilyHandler;
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.MasterProcedureConstants;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.ModifyColumnFamilyProcedure;
import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
@ -1649,8 +1649,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
return;
}
}
//TODO: we should process this (and some others) in an executor
new TableAddFamilyHandler(tableName, columnDescriptor, this, this).prepare().process();
// Execute the operation synchronously - wait for the operation to complete before continuing.
long procId =
this.procedureExecutor.submitProcedure(new AddColumnFamilyProcedure(procedureExecutor
.getEnvironment(), tableName, columnDescriptor));
ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
if (cpHost != null) {
cpHost.postAddColumn(tableName, columnDescriptor);
}
@ -1668,8 +1671,13 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
}
LOG.info(getClientIdAuditPrefix() + " modify " + descriptor);
new TableModifyFamilyHandler(tableName, descriptor, this, this)
.prepare().process();
// Execute the operation synchronously - wait for the operation to complete before continuing.
long procId =
this.procedureExecutor.submitProcedure(new ModifyColumnFamilyProcedure(procedureExecutor
.getEnvironment(), tableName, descriptor));
ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
if (cpHost != null) {
cpHost.postModifyColumn(tableName, descriptor);
}
@ -1685,7 +1693,13 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
}
LOG.info(getClientIdAuditPrefix() + " delete " + Bytes.toString(columnName));
new TableDeleteFamilyHandler(tableName, columnName, this, this).prepare().process();
// Execute the operation synchronously - wait for the operation to complete before continuing.
long procId =
this.procedureExecutor.submitProcedure(new DeleteColumnFamilyProcedure(procedureExecutor
.getEnvironment(), tableName, columnName));
ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
if (cpHost != null) {
cpHost.postDeleteColumn(tableName, columnName);
}

View File

@ -21,11 +21,11 @@ package org.apache.hadoop.hbase.master.handler;
import java.io.IOException;
import java.util.List;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Handles adding a new family to an existing table.
* Handles Deleting a column family from an existing table.
*/
@InterfaceAudience.Private
public class TableDeleteFamilyHandler extends TableEventHandler {

View File

@ -0,0 +1,407 @@
/**
* 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.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
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.AddColumnFamilyState;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.security.UserGroupInformation;
/**
* The procedure to add a column family to an existing table.
*/
@InterfaceAudience.Private
public class AddColumnFamilyProcedure
extends StateMachineProcedure<MasterProcedureEnv, AddColumnFamilyState>
implements TableProcedureInterface {
private static final Log LOG = LogFactory.getLog(AddColumnFamilyProcedure.class);
private final AtomicBoolean aborted = new AtomicBoolean(false);
private TableName tableName;
private HTableDescriptor unmodifiedHTableDescriptor;
private HColumnDescriptor cfDescriptor;
private UserGroupInformation user;
private List<HRegionInfo> regionInfoList;
private Boolean traceEnabled;
public AddColumnFamilyProcedure() {
this.unmodifiedHTableDescriptor = null;
this.regionInfoList = null;
this.traceEnabled = null;
}
public AddColumnFamilyProcedure(
final MasterProcedureEnv env,
final TableName tableName,
final HColumnDescriptor cfDescriptor) throws IOException {
this.tableName = tableName;
this.cfDescriptor = cfDescriptor;
this.user = env.getRequestUser().getUGI();
this.unmodifiedHTableDescriptor = null;
this.regionInfoList = null;
this.traceEnabled = null;
}
@Override
protected Flow executeFromState(final MasterProcedureEnv env, final AddColumnFamilyState state) {
if (isTraceEnabled()) {
LOG.trace(this + " execute state=" + state);
}
try {
switch (state) {
case ADD_COLUMN_FAMILY_PREPARE:
prepareAdd(env);
setNextState(AddColumnFamilyState.ADD_COLUMN_FAMILY_PRE_OPERATION);
break;
case ADD_COLUMN_FAMILY_PRE_OPERATION:
preAdd(env, state);
setNextState(AddColumnFamilyState.ADD_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR);
break;
case ADD_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR:
updateTableDescriptor(env);
setNextState(AddColumnFamilyState.ADD_COLUMN_FAMILY_POST_OPERATION);
break;
case ADD_COLUMN_FAMILY_POST_OPERATION:
postAdd(env, state);
setNextState(AddColumnFamilyState.ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS);
break;
case ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
reOpenAllRegionsIfTableIsOnline(env);
return Flow.NO_MORE_STATE;
default:
throw new UnsupportedOperationException(this + " unhandled state=" + state);
}
} catch (InterruptedException|IOException e) {
LOG.warn("Error trying to add the column family" + getColumnFamilyName() + " to the table "
+ tableName + " (in state=" + state + ")", e);
setFailure("master-add-columnfamily", e);
}
return Flow.HAS_MORE_STATE;
}
@Override
protected void rollbackState(final MasterProcedureEnv env, final AddColumnFamilyState state)
throws IOException {
if (isTraceEnabled()) {
LOG.trace(this + " rollback state=" + state);
}
try {
switch (state) {
case ADD_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
break; // Nothing to undo.
case ADD_COLUMN_FAMILY_POST_OPERATION:
// TODO-MAYBE: call the coprocessor event to undo?
break;
case ADD_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR:
restoreTableDescriptor(env);
break;
case ADD_COLUMN_FAMILY_PRE_OPERATION:
// TODO-MAYBE: call the coprocessor event to undo?
break;
case ADD_COLUMN_FAMILY_PREPARE:
break; // nothing to do
default:
throw new UnsupportedOperationException(this + " 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 rollback attempt step " + state + " for adding the column family"
+ getColumnFamilyName() + " to the table " + tableName, e);
throw e;
}
}
@Override
protected AddColumnFamilyState getState(final int stateId) {
return AddColumnFamilyState.valueOf(stateId);
}
@Override
protected int getStateId(final AddColumnFamilyState state) {
return state.getNumber();
}
@Override
protected AddColumnFamilyState getInitialState() {
return AddColumnFamilyState.ADD_COLUMN_FAMILY_PREPARE;
}
@Override
protected void setNextState(AddColumnFamilyState state) {
if (aborted.get()) {
setAbortFailure("add-columnfamily", "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_ADD_FAMILY.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.AddColumnFamilyStateData.Builder addCFMsg =
MasterProcedureProtos.AddColumnFamilyStateData.newBuilder()
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
.setTableName(ProtobufUtil.toProtoTableName(tableName))
.setColumnfamilySchema(cfDescriptor.convert());
if (unmodifiedHTableDescriptor != null) {
addCFMsg.setUnmodifiedTableSchema(unmodifiedHTableDescriptor.convert());
}
addCFMsg.build().writeDelimitedTo(stream);
}
@Override
public void deserializeStateData(final InputStream stream) throws IOException {
super.deserializeStateData(stream);
MasterProcedureProtos.AddColumnFamilyStateData addCFMsg =
MasterProcedureProtos.AddColumnFamilyStateData.parseDelimitedFrom(stream);
user = MasterProcedureUtil.toUserInfo(addCFMsg.getUserInfo());
tableName = ProtobufUtil.toTableName(addCFMsg.getTableName());
cfDescriptor = HColumnDescriptor.convert(addCFMsg.getColumnfamilySchema());
if (addCFMsg.hasUnmodifiedTableSchema()) {
unmodifiedHTableDescriptor = HTableDescriptor.convert(addCFMsg.getUnmodifiedTableSchema());
}
}
@Override
public void toStringClassDetails(StringBuilder sb) {
sb.append(getClass().getSimpleName());
sb.append(" (table=");
sb.append(tableName);
sb.append(", columnfamily=");
if (cfDescriptor != null) {
sb.append(getColumnFamilyName());
} else {
sb.append("Unknown");
}
sb.append(") user=");
sb.append(user);
}
@Override
public TableName getTableName() {
return tableName;
}
@Override
public TableOperationType getTableOperationType() {
return TableOperationType.EDIT;
}
/**
* Action before any real action of adding column family.
* @param env MasterProcedureEnv
* @throws IOException
*/
private void prepareAdd(final MasterProcedureEnv env) throws IOException {
// Checks whether the table is allowed to be modified.
MasterDDLOperationHelper.checkTableModifiable(env, tableName);
// In order to update the descriptor, we need to retrieve the old descriptor for comparison.
unmodifiedHTableDescriptor = env.getMasterServices().getTableDescriptors().get(tableName);
if (unmodifiedHTableDescriptor == null) {
throw new IOException("HTableDescriptor missing for " + tableName);
}
if (unmodifiedHTableDescriptor.hasFamily(cfDescriptor.getName())) {
throw new InvalidFamilyOperationException("Column family '" + getColumnFamilyName()
+ "' in table '" + tableName + "' already exists so cannot be added");
}
}
/**
* Action before adding column family.
* @param env MasterProcedureEnv
* @param state the procedure state
* @throws IOException
* @throws InterruptedException
*/
private void preAdd(final MasterProcedureEnv env, final AddColumnFamilyState state)
throws IOException, InterruptedException {
runCoprocessorAction(env, state);
}
/**
* Add the column family to the file system
*/
private void updateTableDescriptor(final MasterProcedureEnv env) throws IOException {
// Update table descriptor
LOG.info("AddColumn. Table = " + tableName + " HCD = " + cfDescriptor.toString());
HTableDescriptor htd = env.getMasterServices().getTableDescriptors().get(tableName);
if (htd.hasFamily(cfDescriptor.getName())) {
// It is possible to reach this situation, as we could already add the column family
// to table descriptor, but the master failover happens before we complete this state.
// We should be able to handle running this function multiple times without causing problem.
return;
}
htd.addFamily(cfDescriptor);
env.getMasterServices().getTableDescriptors().add(htd);
}
/**
* Restore the table descriptor back to pre-add
* @param env MasterProcedureEnv
* @throws IOException
**/
private void restoreTableDescriptor(final MasterProcedureEnv env) throws IOException {
HTableDescriptor htd = env.getMasterServices().getTableDescriptors().get(tableName);
if (htd.hasFamily(cfDescriptor.getName())) {
// Remove the column family from file system and update the table descriptor to
// the before-add-column-family-state
MasterDDLOperationHelper.deleteColumnFamilyFromFileSystem(env, tableName,
getRegionInfoList(env), cfDescriptor.getName());
env.getMasterServices().getTableDescriptors().add(unmodifiedHTableDescriptor);
// Make sure regions are opened after table descriptor is updated.
reOpenAllRegionsIfTableIsOnline(env);
}
}
/**
* Action after adding column family.
* @param env MasterProcedureEnv
* @param state the procedure state
* @throws IOException
* @throws InterruptedException
*/
private void postAdd(final MasterProcedureEnv env, final AddColumnFamilyState state)
throws IOException, InterruptedException {
runCoprocessorAction(env, state);
}
/**
* Last action from the procedure - executed when online schema change is supported.
* @param env MasterProcedureEnv
* @throws IOException
*/
private void reOpenAllRegionsIfTableIsOnline(final MasterProcedureEnv env) throws IOException {
// This operation only run when the table is enabled.
if (!env.getMasterServices().getAssignmentManager().getTableStateManager()
.isTableState(getTableName(), ZooKeeperProtos.Table.State.ENABLED)) {
return;
}
if (MasterDDLOperationHelper.reOpenAllRegions(env, getTableName(), getRegionInfoList(env))) {
LOG.info("Completed add column family operation on table " + getTableName());
} else {
LOG.warn("Error on reopening the regions on table " + getTableName());
}
}
/**
* 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;
}
private String getColumnFamilyName() {
return cfDescriptor.getNameAsString();
}
/**
* Coprocessor Action.
* @param env MasterProcedureEnv
* @param state the procedure state
* @throws IOException
* @throws InterruptedException
*/
private void runCoprocessorAction(final MasterProcedureEnv env, final AddColumnFamilyState 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 ADD_COLUMN_FAMILY_PRE_OPERATION:
cpHost.preAddColumnHandler(tableName, cfDescriptor);
break;
case ADD_COLUMN_FAMILY_POST_OPERATION:
cpHost.postAddColumnHandler(tableName, cfDescriptor);
break;
default:
throw new UnsupportedOperationException(this + " unhandled state=" + state);
}
return null;
}
});
}
}
private List<HRegionInfo> getRegionInfoList(final MasterProcedureEnv env) throws IOException {
if (regionInfoList == null) {
regionInfoList = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
}
return regionInfoList;
}
}

View File

@ -0,0 +1,439 @@
/**
* 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.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.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
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.DeleteColumnFamilyState;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.security.UserGroupInformation;
/**
* The procedure to delete a column family from an existing table.
*/
@InterfaceAudience.Private
public class DeleteColumnFamilyProcedure
extends StateMachineProcedure<MasterProcedureEnv, DeleteColumnFamilyState>
implements TableProcedureInterface {
private static final Log LOG = LogFactory.getLog(DeleteColumnFamilyProcedure.class);
private final AtomicBoolean aborted = new AtomicBoolean(false);
private HTableDescriptor unmodifiedHTableDescriptor;
private TableName tableName;
private byte [] familyName;
private UserGroupInformation user;
private List<HRegionInfo> regionInfoList;
private Boolean traceEnabled;
public DeleteColumnFamilyProcedure() {
this.unmodifiedHTableDescriptor = null;
this.regionInfoList = null;
this.traceEnabled = null;
}
public DeleteColumnFamilyProcedure(
final MasterProcedureEnv env,
final TableName tableName,
final byte[] familyName) throws IOException {
this.tableName = tableName;
this.familyName = familyName;
this.user = env.getRequestUser().getUGI();
this.unmodifiedHTableDescriptor = null;
this.regionInfoList = null;
this.traceEnabled = null;
}
@Override
protected Flow executeFromState(final MasterProcedureEnv env, DeleteColumnFamilyState state) {
if (isTraceEnabled()) {
LOG.trace(this + " execute state=" + state);
}
try {
switch (state) {
case DELETE_COLUMN_FAMILY_PREPARE:
prepareDelete(env);
setNextState(DeleteColumnFamilyState.DELETE_COLUMN_FAMILY_PRE_OPERATION);
break;
case DELETE_COLUMN_FAMILY_PRE_OPERATION:
preDelete(env, state);
setNextState(DeleteColumnFamilyState.DELETE_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR);
break;
case DELETE_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR:
updateTableDescriptor(env);
setNextState(DeleteColumnFamilyState.DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT);
break;
case DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT:
deleteFromFs(env);
setNextState(DeleteColumnFamilyState.DELETE_COLUMN_FAMILY_POST_OPERATION);
break;
case DELETE_COLUMN_FAMILY_POST_OPERATION:
postDelete(env, state);
setNextState(DeleteColumnFamilyState.DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS);
break;
case DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
reOpenAllRegionsIfTableIsOnline(env);
return Flow.NO_MORE_STATE;
default:
throw new UnsupportedOperationException(this + " unhandled state=" + state);
}
} catch (InterruptedException|IOException e) {
if (!isRollbackSupported(state)) {
// We reach a state that cannot be rolled back. We just need to keep retry.
LOG.warn("Error trying to delete the column family " + getColumnFamilyName()
+ " from table " + tableName + "(in state=" + state + ")", e);
} else {
LOG.error("Error trying to delete the column family " + getColumnFamilyName()
+ " from table " + tableName + "(in state=" + state + ")", e);
setFailure("master-delete-column-family", e);
}
}
return Flow.HAS_MORE_STATE;
}
@Override
protected void rollbackState(final MasterProcedureEnv env, final DeleteColumnFamilyState state)
throws IOException {
if (isTraceEnabled()) {
LOG.trace(this + " rollback state=" + state);
}
try {
switch (state) {
case DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
break; // Nothing to undo.
case DELETE_COLUMN_FAMILY_POST_OPERATION:
// TODO-MAYBE: call the coprocessor event to undo?
break;
case DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT:
// Once we reach to this state - we could NOT rollback - as it is tricky to undelete
// the deleted files. We are not suppose to reach here, throw exception so that we know
// there is a code bug to investigate.
throw new UnsupportedOperationException(this + " rollback of state=" + state
+ " is unsupported.");
case DELETE_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR:
restoreTableDescriptor(env);
break;
case DELETE_COLUMN_FAMILY_PRE_OPERATION:
// TODO-MAYBE: call the coprocessor event to undo?
break;
case DELETE_COLUMN_FAMILY_PREPARE:
break; // nothing to do
default:
throw new UnsupportedOperationException(this + " 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 rollback attempt step " + state + " for deleting the column family"
+ getColumnFamilyName() + " to the table " + tableName, e);
throw e;
}
}
@Override
protected DeleteColumnFamilyState getState(final int stateId) {
return DeleteColumnFamilyState.valueOf(stateId);
}
@Override
protected int getStateId(final DeleteColumnFamilyState state) {
return state.getNumber();
}
@Override
protected DeleteColumnFamilyState getInitialState() {
return DeleteColumnFamilyState.DELETE_COLUMN_FAMILY_PREPARE;
}
@Override
protected void setNextState(DeleteColumnFamilyState state) {
if (aborted.get() && isRollbackSupported(state)) {
setAbortFailure("delete-columnfamily", "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_DELETE_FAMILY.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.DeleteColumnFamilyStateData.Builder deleteCFMsg =
MasterProcedureProtos.DeleteColumnFamilyStateData.newBuilder()
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
.setTableName(ProtobufUtil.toProtoTableName(tableName))
.setColumnfamilyName(ByteStringer.wrap(familyName));
if (unmodifiedHTableDescriptor != null) {
deleteCFMsg.setUnmodifiedTableSchema(unmodifiedHTableDescriptor.convert());
}
deleteCFMsg.build().writeDelimitedTo(stream);
}
@Override
public void deserializeStateData(final InputStream stream) throws IOException {
super.deserializeStateData(stream);
MasterProcedureProtos.DeleteColumnFamilyStateData deleteCFMsg =
MasterProcedureProtos.DeleteColumnFamilyStateData.parseDelimitedFrom(stream);
user = MasterProcedureUtil.toUserInfo(deleteCFMsg.getUserInfo());
tableName = ProtobufUtil.toTableName(deleteCFMsg.getTableName());
familyName = deleteCFMsg.getColumnfamilyName().toByteArray();
if (deleteCFMsg.hasUnmodifiedTableSchema()) {
unmodifiedHTableDescriptor = HTableDescriptor.convert(deleteCFMsg.getUnmodifiedTableSchema());
}
}
@Override
public void toStringClassDetails(StringBuilder sb) {
sb.append(getClass().getSimpleName());
sb.append(" (table=");
sb.append(tableName);
sb.append(", columnfamily=");
if (familyName != null) {
sb.append(getColumnFamilyName());
} else {
sb.append("Unknown");
}
sb.append(") user=");
sb.append(user);
}
@Override
public TableName getTableName() {
return tableName;
}
@Override
public TableOperationType getTableOperationType() {
return TableOperationType.EDIT;
}
/**
* Action before any real action of deleting column family.
* @param env MasterProcedureEnv
* @throws IOException
*/
private void prepareDelete(final MasterProcedureEnv env) throws IOException {
// Checks whether the table is allowed to be modified.
MasterDDLOperationHelper.checkTableModifiable(env, tableName);
// In order to update the descriptor, we need to retrieve the old descriptor for comparison.
unmodifiedHTableDescriptor = env.getMasterServices().getTableDescriptors().get(tableName);
if (unmodifiedHTableDescriptor == null) {
throw new IOException("HTableDescriptor missing for " + tableName);
}
if (!unmodifiedHTableDescriptor.hasFamily(familyName)) {
throw new InvalidFamilyOperationException("Family '" + getColumnFamilyName()
+ "' does not exist, so it cannot be deleted");
}
}
/**
* Action before deleting column family.
* @param env MasterProcedureEnv
* @param state the procedure state
* @throws IOException
* @throws InterruptedException
*/
private void preDelete(final MasterProcedureEnv env, final DeleteColumnFamilyState state)
throws IOException, InterruptedException {
runCoprocessorAction(env, state);
}
/**
* Remove the column family from the file system and update the table descriptor
*/
private void updateTableDescriptor(final MasterProcedureEnv env) throws IOException {
// Update table descriptor
LOG.info("DeleteColumn. Table = " + tableName + " family = " + getColumnFamilyName());
HTableDescriptor htd = env.getMasterServices().getTableDescriptors().get(tableName);
if (!htd.hasFamily(familyName)) {
// It is possible to reach this situation, as we could already delete the column family
// from table descriptor, but the master failover happens before we complete this state.
// We should be able to handle running this function multiple times without causing problem.
return;
}
htd.removeFamily(familyName);
env.getMasterServices().getTableDescriptors().add(htd);
}
/**
* Restore back to the old descriptor
* @param env MasterProcedureEnv
* @throws IOException
**/
private void restoreTableDescriptor(final MasterProcedureEnv env) throws IOException {
env.getMasterServices().getTableDescriptors().add(unmodifiedHTableDescriptor);
// Make sure regions are opened after table descriptor is updated.
reOpenAllRegionsIfTableIsOnline(env);
}
/**
* Remove the column family from the file system
**/
private void deleteFromFs(final MasterProcedureEnv env) throws IOException {
MasterDDLOperationHelper.deleteColumnFamilyFromFileSystem(env, tableName,
getRegionInfoList(env), familyName);
}
/**
* Action after deleting column family.
* @param env MasterProcedureEnv
* @param state the procedure state
* @throws IOException
* @throws InterruptedException
*/
private void postDelete(final MasterProcedureEnv env, final DeleteColumnFamilyState state)
throws IOException, InterruptedException {
runCoprocessorAction(env, state);
}
/**
* Last action from the procedure - executed when online schema change is supported.
* @param env MasterProcedureEnv
* @throws IOException
*/
private void reOpenAllRegionsIfTableIsOnline(final MasterProcedureEnv env) throws IOException {
// This operation only run when the table is enabled.
if (!env.getMasterServices().getAssignmentManager().getTableStateManager()
.isTableState(getTableName(), ZooKeeperProtos.Table.State.ENABLED)) {
return;
}
if (MasterDDLOperationHelper.reOpenAllRegions(env, getTableName(), getRegionInfoList(env))) {
LOG.info("Completed delete column family operation on table " + getTableName());
} else {
LOG.warn("Error on reopening the regions on table " + getTableName());
}
}
/**
* 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;
}
private String getColumnFamilyName() {
return Bytes.toString(familyName);
}
/**
* Coprocessor Action.
* @param env MasterProcedureEnv
* @param state the procedure state
* @throws IOException
* @throws InterruptedException
*/
private void runCoprocessorAction(final MasterProcedureEnv env,
final DeleteColumnFamilyState 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 DELETE_COLUMN_FAMILY_PRE_OPERATION:
cpHost.preDeleteColumnHandler(tableName, familyName);
break;
case DELETE_COLUMN_FAMILY_POST_OPERATION:
cpHost.postDeleteColumnHandler(tableName, familyName);
break;
default:
throw new UnsupportedOperationException(this + " unhandled state=" + state);
}
return null;
}
});
}
}
/*
* Check whether we are in the state that can be rollback
*/
private boolean isRollbackSupported(final DeleteColumnFamilyState state) {
switch (state) {
case DELETE_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
case DELETE_COLUMN_FAMILY_POST_OPERATION:
case DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT:
// It is not safe to rollback if we reach to these states.
return false;
default:
break;
}
return true;
}
private List<HRegionInfo> getRegionInfoList(final MasterProcedureEnv env) throws IOException {
if (regionInfoList == null) {
regionInfoList = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
}
return regionInfoList;
}
}

View File

@ -0,0 +1,382 @@
/**
* 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.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
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.ModifyColumnFamilyState;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.security.UserGroupInformation;
/**
* The procedure to modify a column family from an existing table.
*/
@InterfaceAudience.Private
public class ModifyColumnFamilyProcedure
extends StateMachineProcedure<MasterProcedureEnv, ModifyColumnFamilyState>
implements TableProcedureInterface {
private static final Log LOG = LogFactory.getLog(ModifyColumnFamilyProcedure.class);
private final AtomicBoolean aborted = new AtomicBoolean(false);
private TableName tableName;
private HTableDescriptor unmodifiedHTableDescriptor;
private HColumnDescriptor cfDescriptor;
private UserGroupInformation user;
private Boolean traceEnabled;
public ModifyColumnFamilyProcedure() {
this.unmodifiedHTableDescriptor = null;
this.traceEnabled = null;
}
public ModifyColumnFamilyProcedure(
final MasterProcedureEnv env,
final TableName tableName,
final HColumnDescriptor cfDescriptor) throws IOException {
this.tableName = tableName;
this.cfDescriptor = cfDescriptor;
this.user = env.getRequestUser().getUGI();
this.unmodifiedHTableDescriptor = null;
this.traceEnabled = null;
}
@Override
protected Flow executeFromState(final MasterProcedureEnv env,
final ModifyColumnFamilyState state) {
if (isTraceEnabled()) {
LOG.trace(this + " execute state=" + state);
}
try {
switch (state) {
case MODIFY_COLUMN_FAMILY_PREPARE:
prepareModify(env);
setNextState(ModifyColumnFamilyState.MODIFY_COLUMN_FAMILY_PRE_OPERATION);
break;
case MODIFY_COLUMN_FAMILY_PRE_OPERATION:
preModify(env, state);
setNextState(ModifyColumnFamilyState.MODIFY_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR);
break;
case MODIFY_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR:
updateTableDescriptor(env);
setNextState(ModifyColumnFamilyState.MODIFY_COLUMN_FAMILY_POST_OPERATION);
break;
case MODIFY_COLUMN_FAMILY_POST_OPERATION:
postModify(env, state);
setNextState(ModifyColumnFamilyState.MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS);
break;
case MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
reOpenAllRegionsIfTableIsOnline(env);
return Flow.NO_MORE_STATE;
default:
throw new UnsupportedOperationException(this + " unhandled state=" + state);
}
} catch (InterruptedException|IOException e) {
LOG.warn("Error trying to modify the column family " + getColumnFamilyName()
+ " of the table " + tableName + "(in state=" + state + ")", e);
setFailure("master-modify-columnfamily", e);
}
return Flow.HAS_MORE_STATE;
}
@Override
protected void rollbackState(final MasterProcedureEnv env, final ModifyColumnFamilyState state)
throws IOException {
if (isTraceEnabled()) {
LOG.trace(this + " rollback state=" + state);
}
try {
switch (state) {
case MODIFY_COLUMN_FAMILY_REOPEN_ALL_REGIONS:
break; // Nothing to undo.
case MODIFY_COLUMN_FAMILY_POST_OPERATION:
// TODO-MAYBE: call the coprocessor event to undo?
break;
case MODIFY_COLUMN_FAMILY_UPDATE_TABLE_DESCRIPTOR:
restoreTableDescriptor(env);
break;
case MODIFY_COLUMN_FAMILY_PRE_OPERATION:
// TODO-MAYBE: call the coprocessor event to undo?
break;
case MODIFY_COLUMN_FAMILY_PREPARE:
break; // nothing to do
default:
throw new UnsupportedOperationException(this + " 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 rollback attempt step " + state + " for adding the column family"
+ getColumnFamilyName() + " to the table " + tableName, e);
throw e;
}
}
@Override
protected ModifyColumnFamilyState getState(final int stateId) {
return ModifyColumnFamilyState.valueOf(stateId);
}
@Override
protected int getStateId(final ModifyColumnFamilyState state) {
return state.getNumber();
}
@Override
protected ModifyColumnFamilyState getInitialState() {
return ModifyColumnFamilyState.MODIFY_COLUMN_FAMILY_PREPARE;
}
@Override
protected void setNextState(ModifyColumnFamilyState state) {
if (aborted.get()) {
setAbortFailure("modify-columnfamily", "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_MODIFY_FAMILY.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.ModifyColumnFamilyStateData.Builder modifyCFMsg =
MasterProcedureProtos.ModifyColumnFamilyStateData.newBuilder()
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(user))
.setTableName(ProtobufUtil.toProtoTableName(tableName))
.setColumnfamilySchema(cfDescriptor.convert());
if (unmodifiedHTableDescriptor != null) {
modifyCFMsg.setUnmodifiedTableSchema(unmodifiedHTableDescriptor.convert());
}
modifyCFMsg.build().writeDelimitedTo(stream);
}
@Override
public void deserializeStateData(final InputStream stream) throws IOException {
super.deserializeStateData(stream);
MasterProcedureProtos.ModifyColumnFamilyStateData modifyCFMsg =
MasterProcedureProtos.ModifyColumnFamilyStateData.parseDelimitedFrom(stream);
user = MasterProcedureUtil.toUserInfo(modifyCFMsg.getUserInfo());
tableName = ProtobufUtil.toTableName(modifyCFMsg.getTableName());
cfDescriptor = HColumnDescriptor.convert(modifyCFMsg.getColumnfamilySchema());
if (modifyCFMsg.hasUnmodifiedTableSchema()) {
unmodifiedHTableDescriptor = HTableDescriptor.convert(modifyCFMsg.getUnmodifiedTableSchema());
}
}
@Override
public void toStringClassDetails(StringBuilder sb) {
sb.append(getClass().getSimpleName());
sb.append(" (table=");
sb.append(tableName);
sb.append(", columnfamily=");
if (cfDescriptor != null) {
sb.append(getColumnFamilyName());
} else {
sb.append("Unknown");
}
sb.append(") user=");
sb.append(user);
}
@Override
public TableName getTableName() {
return tableName;
}
@Override
public TableOperationType getTableOperationType() {
return TableOperationType.EDIT;
}
/**
* Action before any real action of modifying column family.
* @param env MasterProcedureEnv
* @throws IOException
*/
private void prepareModify(final MasterProcedureEnv env) throws IOException {
// Checks whether the table is allowed to be modified.
MasterDDLOperationHelper.checkTableModifiable(env, tableName);
unmodifiedHTableDescriptor = env.getMasterServices().getTableDescriptors().get(tableName);
if (unmodifiedHTableDescriptor == null) {
throw new IOException("HTableDescriptor missing for " + tableName);
}
if (!unmodifiedHTableDescriptor.hasFamily(cfDescriptor.getName())) {
throw new InvalidFamilyOperationException("Family '" + getColumnFamilyName()
+ "' does not exist, so it cannot be modified");
}
}
/**
* Action before modifying column family.
* @param env MasterProcedureEnv
* @param state the procedure state
* @throws IOException
* @throws InterruptedException
*/
private void preModify(final MasterProcedureEnv env, final ModifyColumnFamilyState state)
throws IOException, InterruptedException {
runCoprocessorAction(env, state);
}
/**
* Modify the column family from the file system
*/
private void updateTableDescriptor(final MasterProcedureEnv env) throws IOException {
// Update table descriptor
LOG.info("ModifyColumnFamily. Table = " + tableName + " HCD = " + cfDescriptor.toString());
HTableDescriptor htd = env.getMasterServices().getTableDescriptors().get(tableName);
htd.modifyFamily(cfDescriptor);
env.getMasterServices().getTableDescriptors().add(htd);
}
/**
* Restore back to the old descriptor
* @param env MasterProcedureEnv
* @throws IOException
**/
private void restoreTableDescriptor(final MasterProcedureEnv env) throws IOException {
env.getMasterServices().getTableDescriptors().add(unmodifiedHTableDescriptor);
// Make sure regions are opened after table descriptor is updated.
reOpenAllRegionsIfTableIsOnline(env);
}
/**
* Action after modifying column family.
* @param env MasterProcedureEnv
* @param state the procedure state
* @throws IOException
* @throws InterruptedException
*/
private void postModify(final MasterProcedureEnv env, final ModifyColumnFamilyState state)
throws IOException, InterruptedException {
runCoprocessorAction(env, state);
}
/**
* Last action from the procedure - executed when online schema change is supported.
* @param env MasterProcedureEnv
* @throws IOException
*/
private void reOpenAllRegionsIfTableIsOnline(final MasterProcedureEnv env) throws IOException {
// This operation only run when the table is enabled.
if (!env.getMasterServices().getAssignmentManager().getTableStateManager()
.isTableState(getTableName(), ZooKeeperProtos.Table.State.ENABLED)) {
return;
}
List<HRegionInfo> regionInfoList = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
if (MasterDDLOperationHelper.reOpenAllRegions(env, getTableName(), regionInfoList)) {
LOG.info("Completed add column family operation on table " + getTableName());
} else {
LOG.warn("Error on reopening the regions on table " + getTableName());
}
}
/**
* 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;
}
private String getColumnFamilyName() {
return cfDescriptor.getNameAsString();
}
/**
* Coprocessor Action.
* @param env MasterProcedureEnv
* @param state the procedure state
* @throws IOException
* @throws InterruptedException
*/
private void runCoprocessorAction(final MasterProcedureEnv env,
final ModifyColumnFamilyState 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 MODIFY_COLUMN_FAMILY_PRE_OPERATION:
cpHost.preModifyColumnHandler(tableName, cfDescriptor);
break;
case MODIFY_COLUMN_FAMILY_POST_OPERATION:
cpHost.postModifyColumnHandler(tableName, cfDescriptor);
break;
default:
throw new UnsupportedOperationException(this + " unhandled state=" + state);
}
return null;
}
});
}
}
}

View File

@ -102,37 +102,6 @@ public class TestTableLockManager {
TEST_UTIL.shutdownMiniCluster();
}
@Test(timeout = 600000)
public void testLockTimeoutException() throws Exception {
Configuration conf = TEST_UTIL.getConfiguration();
conf.setInt(TableLockManager.TABLE_WRITE_LOCK_TIMEOUT_MS, 3000);
prepareMiniCluster();
HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
master.getMasterCoprocessorHost().load(TestLockTimeoutExceptionMasterObserver.class,
0, TEST_UTIL.getConfiguration());
ExecutorService executor = Executors.newSingleThreadExecutor();
Future<Object> shouldFinish = executor.submit(new Callable<Object>() {
@Override
public Object call() throws Exception {
Admin admin = TEST_UTIL.getHBaseAdmin();
admin.deleteColumn(TABLE_NAME, FAMILY);
return null;
}
});
deleteColumn.await();
try {
Admin admin = TEST_UTIL.getHBaseAdmin();
admin.addColumn(TABLE_NAME, new HColumnDescriptor(NEW_FAMILY));
fail("Was expecting TableLockTimeoutException");
} catch (LockTimeoutException ex) {
//expected
}
shouldFinish.get();
}
public static class TestLockTimeoutExceptionMasterObserver extends BaseMasterObserver {
@Override
public void preDeleteColumnHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,

View File

@ -29,18 +29,24 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Table;
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.wal.WALSplitter;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
@ -62,10 +68,17 @@ public class TestTableDeleteFamilyHandler {
*/
@BeforeClass
public static void beforeAllTests() throws Exception {
TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
TEST_UTIL.startMiniCluster(2);
}
@AfterClass
public static void afterAllTests() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}
@Before
public void setup() throws IOException, InterruptedException {
// Create a table of three families. This will assign a region.
TEST_UTIL.createTable(TABLENAME, FAMILIES);
HTable t = new HTable(TEST_UTIL.getConfiguration(), TABLENAME);
@ -84,22 +97,17 @@ public class TestTableDeleteFamilyHandler {
TEST_UTIL.flush();
t.close();
}
@AfterClass
public static void afterAllTests() throws Exception {
TEST_UTIL.deleteTable(TABLENAME);
TEST_UTIL.shutdownMiniCluster();
}
@Before
public void setup() throws IOException, InterruptedException {
TEST_UTIL.ensureSomeRegionServersAvailable(2);
}
@After
public void cleanup() throws Exception {
TEST_UTIL.deleteTable(TABLENAME);
}
@Test
public void deleteColumnFamilyWithMultipleRegions() throws Exception {
Admin admin = TEST_UTIL.getHBaseAdmin();
HTableDescriptor beforehtd = admin.getTableDescriptor(TABLENAME);
@ -112,7 +120,6 @@ public class TestTableDeleteFamilyHandler {
assertEquals(3, beforehtd.getColumnFamilies().length);
HColumnDescriptor[] families = beforehtd.getColumnFamilies();
for (int i = 0; i < families.length; i++) {
assertTrue(families[i].getNameAsString().equals("cf" + (i + 1)));
}
@ -177,4 +184,95 @@ public class TestTableDeleteFamilyHandler {
}
}
@Test
public void deleteColumnFamilyTwice() throws Exception {
Admin admin = TEST_UTIL.getHBaseAdmin();
HTableDescriptor beforehtd = admin.getTableDescriptor(TABLENAME);
String cfToDelete = "cf1";
FileSystem fs = TEST_UTIL.getDFSCluster().getFileSystem();
// 1 - Check if table exists in descriptor
assertTrue(admin.isTableAvailable(TABLENAME));
// 2 - Check if all the target column family exist in descriptor
HColumnDescriptor[] families = beforehtd.getColumnFamilies();
Boolean foundCF = false;
int i;
for (i = 0; i < families.length; i++) {
if (families[i].getNameAsString().equals(cfToDelete)) {
foundCF = true;
break;
}
}
assertTrue(foundCF);
// 3 - Check if table exists in FS
Path tableDir = FSUtils.getTableDir(TEST_UTIL.getDefaultRootDirPath(), TABLENAME);
assertTrue(fs.exists(tableDir));
// 4 - Check if all the target column family exist in FS
FileStatus[] fileStatus = fs.listStatus(tableDir);
foundCF = false;
for (i = 0; i < fileStatus.length; i++) {
if (fileStatus[i].isDirectory() == true) {
FileStatus[] cf = fs.listStatus(fileStatus[i].getPath(), new PathFilter() {
@Override
public boolean accept(Path p) {
if (p.getName().contains(HConstants.RECOVERED_EDITS_DIR)) {
return false;
}
return true;
}
});
for (int j = 0; j < cf.length; j++) {
if (cf[j].isDirectory() == true && cf[j].getPath().getName().equals(cfToDelete)) {
foundCF = true;
break;
}
}
}
if (foundCF) {
break;
}
}
assertTrue(foundCF);
// TEST - Disable and delete the column family
if (admin.isTableEnabled(TABLENAME)) {
admin.disableTable(TABLENAME);
}
admin.deleteColumn(TABLENAME, Bytes.toBytes(cfToDelete));
// 5 - Check if the target column family is gone from the FS
fileStatus = fs.listStatus(tableDir);
for (i = 0; i < fileStatus.length; i++) {
if (fileStatus[i].isDirectory() == true) {
FileStatus[] cf = fs.listStatus(fileStatus[i].getPath(), new PathFilter() {
@Override
public boolean accept(Path p) {
if (WALSplitter.isSequenceIdFile(p)) {
return false;
}
return true;
}
});
for (int j = 0; j < cf.length; j++) {
if (cf[j].isDirectory() == true) {
assertFalse(cf[j].getPath().getName().equals(cfToDelete));
}
}
}
}
try {
// Test: delete again
admin.deleteColumn(TABLENAME, Bytes.toBytes(cfToDelete));
Assert.fail("Delete a non-exist column family should fail");
} catch (InvalidFamilyOperationException e) {
// Expected.
}
}
}

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.master.MasterFileSystem;
@ -37,6 +38,7 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Rule;
@ -121,6 +123,95 @@ public class TestTableDescriptorModification {
}
}
@Test
public void testAddSameColumnFamilyTwice() throws IOException {
Admin admin = TEST_UTIL.getHBaseAdmin();
// Create a table with one families
HTableDescriptor baseHtd = new HTableDescriptor(TABLE_NAME);
baseHtd.addFamily(new HColumnDescriptor(FAMILY_0));
admin.createTable(baseHtd);
admin.disableTable(TABLE_NAME);
try {
// Verify the table descriptor
verifyTableDescriptor(TABLE_NAME, FAMILY_0);
// Modify the table removing one family and verify the descriptor
admin.addColumn(TABLE_NAME, new HColumnDescriptor(FAMILY_1));
verifyTableDescriptor(TABLE_NAME, FAMILY_0, FAMILY_1);
try {
// Add same column family again - expect failure
admin.addColumn(TABLE_NAME, new HColumnDescriptor(FAMILY_1));
Assert.fail("Delete a non-exist column family should fail");
} catch (InvalidFamilyOperationException e) {
// Expected.
}
} finally {
admin.deleteTable(TABLE_NAME);
}
}
@Test
public void testModifyColumnFamily() throws IOException {
Admin admin = TEST_UTIL.getHBaseAdmin();
HColumnDescriptor cfDescriptor = new HColumnDescriptor(FAMILY_0);
int blockSize = cfDescriptor.getBlocksize();
// Create a table with one families
HTableDescriptor baseHtd = new HTableDescriptor(TABLE_NAME);
baseHtd.addFamily(cfDescriptor);
admin.createTable(baseHtd);
admin.disableTable(TABLE_NAME);
try {
// Verify the table descriptor
verifyTableDescriptor(TABLE_NAME, FAMILY_0);
int newBlockSize = 2 * blockSize;
cfDescriptor.setBlocksize(newBlockSize);
// Modify colymn family
admin.modifyColumn(TABLE_NAME, cfDescriptor);
HTableDescriptor htd = admin.getTableDescriptor(TABLE_NAME);
HColumnDescriptor hcfd = htd.getFamily(FAMILY_0);
assertTrue(hcfd.getBlocksize() == newBlockSize);
} finally {
admin.deleteTable(TABLE_NAME);
}
}
@Test
public void testModifyNonExistingColumnFamily() throws IOException {
Admin admin = TEST_UTIL.getHBaseAdmin();
HColumnDescriptor cfDescriptor = new HColumnDescriptor(FAMILY_1);
int blockSize = cfDescriptor.getBlocksize();
// Create a table with one families
HTableDescriptor baseHtd = new HTableDescriptor(TABLE_NAME);
baseHtd.addFamily(new HColumnDescriptor(FAMILY_0));
admin.createTable(baseHtd);
admin.disableTable(TABLE_NAME);
try {
// Verify the table descriptor
verifyTableDescriptor(TABLE_NAME, FAMILY_0);
int newBlockSize = 2 * blockSize;
cfDescriptor.setBlocksize(newBlockSize);
// Modify a column family that is not in the table.
try {
admin.modifyColumn(TABLE_NAME, cfDescriptor);
Assert.fail("Modify a non-exist column family should fail");
} catch (InvalidFamilyOperationException e) {
// Expected.
}
} finally {
admin.deleteTable(TABLE_NAME);
}
}
@Test
public void testDeleteColumn() throws IOException {
Admin admin = TEST_UTIL.getHBaseAdmin();
@ -142,6 +233,35 @@ public class TestTableDescriptorModification {
}
}
@Test
public void testDeleteSameColumnFamilyTwice() throws IOException {
Admin admin = TEST_UTIL.getHBaseAdmin();
// Create a table with two families
HTableDescriptor baseHtd = new HTableDescriptor(TABLE_NAME);
baseHtd.addFamily(new HColumnDescriptor(FAMILY_0));
baseHtd.addFamily(new HColumnDescriptor(FAMILY_1));
admin.createTable(baseHtd);
admin.disableTable(TABLE_NAME);
try {
// Verify the table descriptor
verifyTableDescriptor(TABLE_NAME, FAMILY_0, FAMILY_1);
// Modify the table removing one family and verify the descriptor
admin.deleteColumn(TABLE_NAME, FAMILY_1);
verifyTableDescriptor(TABLE_NAME, FAMILY_0);
try {
// Delete again - expect failure
admin.deleteColumn(TABLE_NAME, FAMILY_1);
Assert.fail("Delete a non-exist column family should fail");
} catch (Exception e) {
// Expected.
}
} finally {
admin.deleteTable(TABLE_NAME);
}
}
private void verifyTableDescriptor(final TableName tableName,
final byte[]... families) throws IOException {
Admin admin = TEST_UTIL.getHBaseAdmin();

View File

@ -295,6 +295,39 @@ public class MasterProcedureTestingUtility {
ProcedureTestingUtility.assertIsAbortException(procExec.getResult(procId));
}
public static void validateColumnFamilyAddition(final HMaster master, final TableName tableName,
final String family) throws IOException {
HTableDescriptor htd = master.getTableDescriptors().get(tableName);
assertTrue(htd != null);
assertTrue(htd.hasFamily(family.getBytes()));
}
public static void validateColumnFamilyDeletion(final HMaster master, final TableName tableName,
final String family) throws IOException {
// verify htd
HTableDescriptor htd = master.getTableDescriptors().get(tableName);
assertTrue(htd != null);
assertFalse(htd.hasFamily(family.getBytes()));
// verify fs
final FileSystem fs = master.getMasterFileSystem().getFileSystem();
final Path tableDir = FSUtils.getTableDir(master.getMasterFileSystem().getRootDir(), tableName);
for (Path regionDir: FSUtils.getRegionDirs(fs, tableDir)) {
final Path familyDir = new Path(regionDir, family);
assertFalse(family + " family dir should not exist", fs.exists(familyDir));
}
}
public static void validateColumnFamilyModification(final HMaster master,
final TableName tableName, final String family, HColumnDescriptor columnDescriptor)
throws IOException {
HTableDescriptor htd = master.getTableDescriptors().get(tableName);
assertTrue(htd != null);
HColumnDescriptor hcfd = htd.getFamily(family.getBytes());
assertTrue(hcfd.equals(columnDescriptor));
}
public static class InjectAbortOnLoadListener
implements ProcedureExecutor.ProcedureExecutorListener {
private final ProcedureExecutor<MasterProcedureEnv> procExec;

View File

@ -0,0 +1,245 @@
/**
* 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.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName;
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.AddColumnFamilyState;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(MediumTests.class)
public class TestAddColumnFamilyProcedure {
private static final Log LOG = LogFactory.getLog(TestAddColumnFamilyProcedure.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 testAddColumnFamily() throws Exception {
final TableName tableName = TableName.valueOf("testAddColumnFamily");
final String cf1 = "cf1";
final String cf2 = "cf2";
final HColumnDescriptor columnDescriptor1 = new HColumnDescriptor(cf1);
final HColumnDescriptor columnDescriptor2 = new HColumnDescriptor(cf2);
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f3");
// Test 1: Add a column family online
long procId1 =
procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName,
columnDescriptor1));
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
MasterProcedureTestingUtility.validateColumnFamilyAddition(UTIL.getHBaseCluster().getMaster(),
tableName, cf1);
// Test 2: Add a column family offline
UTIL.getHBaseAdmin().disableTable(tableName);
long procId2 =
procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName,
columnDescriptor2));
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId2);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
MasterProcedureTestingUtility.validateColumnFamilyAddition(UTIL.getHBaseCluster().getMaster(),
tableName, cf2);
}
@Test(timeout=60000)
public void testAddSameColumnFamilyTwice() throws Exception {
final TableName tableName = TableName.valueOf("testAddColumnFamilyTwice");
final String cf2 = "cf2";
final HColumnDescriptor columnDescriptor = new HColumnDescriptor(cf2);
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1");
// add the column family
long procId1 =
procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName,
columnDescriptor));
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
MasterProcedureTestingUtility.validateColumnFamilyAddition(UTIL.getHBaseCluster().getMaster(),
tableName, cf2);
// add the column family that exists
long procId2 =
procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName,
columnDescriptor));
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId2);
// Second add should fail with InvalidFamilyOperationException
ProcedureResult result = procExec.getResult(procId2);
assertTrue(result.isFailed());
LOG.debug("Add failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof InvalidFamilyOperationException);
// Do the same add the existing column family - this time offline
UTIL.getHBaseAdmin().disableTable(tableName);
long procId3 =
procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName,
columnDescriptor));
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId3);
// Second add should fail with InvalidFamilyOperationException
result = procExec.getResult(procId3);
assertTrue(result.isFailed());
LOG.debug("Add failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof InvalidFamilyOperationException);
}
@Test(timeout = 60000)
public void testRecoveryAndDoubleExecutionOffline() throws Exception {
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecutionOffline");
final String cf4 = "cf4";
final HColumnDescriptor columnDescriptor = new HColumnDescriptor(cf4);
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
// create the table
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2", "f3");
UTIL.getHBaseAdmin().disableTable(tableName);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the AddColumnFamily procedure && kill the executor
long procId =
procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName,
columnDescriptor));
// Restart the executor and execute the step twice
int numberOfSteps = AddColumnFamilyState.values().length;
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps,
AddColumnFamilyState.values());
MasterProcedureTestingUtility.validateColumnFamilyAddition(UTIL.getHBaseCluster().getMaster(),
tableName, cf4);
}
@Test(timeout = 60000)
public void testRecoveryAndDoubleExecutionOnline() throws Exception {
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecutionOnline");
final String cf5 = "cf5";
final HColumnDescriptor columnDescriptor = new HColumnDescriptor(cf5);
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
// create the table
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2", "f3");
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the AddColumnFamily procedure && kill the executor
long procId =
procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName,
columnDescriptor));
// Restart the executor and execute the step twice
int numberOfSteps = AddColumnFamilyState.values().length;
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps,
AddColumnFamilyState.values());
MasterProcedureTestingUtility.validateColumnFamilyAddition(UTIL.getHBaseCluster().getMaster(),
tableName, cf5);
}
@Test(timeout = 60000)
public void testRollbackAndDoubleExecution() throws Exception {
final TableName tableName = TableName.valueOf("testRollbackAndDoubleExecution");
final String cf6 = "cf6";
final HColumnDescriptor columnDescriptor = new HColumnDescriptor(cf6);
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
// create the table
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2");
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the AddColumnFamily procedure && kill the executor
long procId =
procExec.submitProcedure(new AddColumnFamilyProcedure(procExec.getEnvironment(), tableName,
columnDescriptor));
int numberOfSteps = AddColumnFamilyState.values().length - 2; // failing in the middle of proc
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps,
AddColumnFamilyState.values());
MasterProcedureTestingUtility.validateColumnFamilyDeletion(UTIL.getHBaseCluster().getMaster(),
tableName, cf6);
}
private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
}
}

View File

@ -0,0 +1,301 @@
/**
* 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.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName;
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.DeleteColumnFamilyState;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(MediumTests.class)
public class TestDeleteColumnFamilyProcedure {
private static final Log LOG = LogFactory.getLog(TestDeleteColumnFamilyProcedure.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 testDeleteColumnFamily() throws Exception {
final TableName tableName = TableName.valueOf("testDeleteColumnFamily");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
final String cf1 = "cf1";
final String cf2 = "cf2";
MasterProcedureTestingUtility.createTable(procExec, tableName, null, cf1, cf2, "f3");
// Test 1: delete the column family that exists online
long procId1 =
procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(),
tableName, cf1.getBytes()));
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
MasterProcedureTestingUtility.validateColumnFamilyDeletion(UTIL.getHBaseCluster().getMaster(),
tableName, cf1);
// Test 2: delete the column family that exists offline
UTIL.getHBaseAdmin().disableTable(tableName);
long procId2 =
procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(),
tableName, cf2.getBytes()));
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId2);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
}
@Test(timeout=60000)
public void testDeleteColumnFamilyTwice() throws Exception {
final TableName tableName = TableName.valueOf("testDeleteColumnFamilyTwice");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
final String cf2 = "cf2";
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", cf2);
// delete the column family that exists
long procId1 =
procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(),
tableName, cf2.getBytes()));
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
// First delete should succeed
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
MasterProcedureTestingUtility.validateColumnFamilyDeletion(UTIL.getHBaseCluster().getMaster(),
tableName, cf2);
// delete the column family that does not exist
long procId2 =
procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(),
tableName, cf2.getBytes()));
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId2);
// Second delete should fail with InvalidFamilyOperationException
ProcedureResult result = procExec.getResult(procId2);
assertTrue(result.isFailed());
LOG.debug("Delete online failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof InvalidFamilyOperationException);
// Try again, this time with table disabled.
UTIL.getHBaseAdmin().disableTable(tableName);
long procId3 =
procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(),
tableName, cf2.getBytes()));
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId3);
// Expect fail with InvalidFamilyOperationException
result = procExec.getResult(procId2);
assertTrue(result.isFailed());
LOG.debug("Delete offline failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof InvalidFamilyOperationException);
}
@Test(timeout=60000)
public void testDeleteNonExistingColumnFamily() throws Exception {
final TableName tableName = TableName.valueOf("testDeleteNonExistingColumnFamily");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
final String cf3 = "cf3";
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2");
// delete the column family that does not exist
long procId1 =
procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(),
tableName, cf3.getBytes()));
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureResult result = procExec.getResult(procId1);
assertTrue(result.isFailed());
LOG.debug("Delete failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof InvalidFamilyOperationException);
}
@Test(timeout=60000)
public void testRecoveryAndDoubleExecutionOffline() throws Exception {
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecutionOffline");
final String cf4 = "cf4";
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
// create the table
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2", "f3", cf4);
UTIL.getHBaseAdmin().disableTable(tableName);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the Delete procedure && kill the executor
long procId =
procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(),
tableName, cf4.getBytes()));
// Restart the executor and execute the step twice
int numberOfSteps = DeleteColumnFamilyState.values().length;
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps,
DeleteColumnFamilyState.values());
MasterProcedureTestingUtility.validateColumnFamilyDeletion(UTIL.getHBaseCluster().getMaster(),
tableName, cf4);
}
@Test(timeout = 60000)
public void testRecoveryAndDoubleExecutionOnline() throws Exception {
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecutionOnline");
final String cf5 = "cf5";
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
// create the table
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2", "f3", cf5);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the Delete procedure && kill the executor
long procId =
procExec.submitProcedure(new DeleteColumnFamilyProcedure(procExec.getEnvironment(),
tableName, cf5.getBytes()));
// Restart the executor and execute the step twice
int numberOfSteps = DeleteColumnFamilyState.values().length;
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps,
DeleteColumnFamilyState.values());
MasterProcedureTestingUtility.validateColumnFamilyDeletion(UTIL.getHBaseCluster().getMaster(),
tableName, cf5);
}
@Test(timeout = 60000)
public void testRollbackAndDoubleExecution() throws Exception {
final TableName tableName = TableName.valueOf("testRollbackAndDoubleExecution");
final String cf5 = "cf5";
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
// create the table
HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, "f1", "f2", "f3", cf5);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the Delete procedure && kill the executor
long procId = procExec.submitProcedure(
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes()));
// Failing before DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT we should trigger the rollback
// NOTE: the 1 (number before DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT step) is hardcoded,
// so you have to look at this test at least once when you add a new step.
int numberOfSteps = 1;
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(
procExec,
procId,
numberOfSteps,
DeleteColumnFamilyState.values());
MasterProcedureTestingUtility.validateTableCreation(
UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2", "f3", cf5);
}
@Test(timeout = 60000)
public void testRollbackAndDoubleExecutionAfterPONR() throws Exception {
final TableName tableName = TableName.valueOf("testRollbackAndDoubleExecutionAfterPONR");
final String cf5 = "cf5";
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
// create the table
HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, "f1", "f2", "f3", cf5);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the Delete procedure && kill the executor
long procId = procExec.submitProcedure(
new DeleteColumnFamilyProcedure(procExec.getEnvironment(), tableName, cf5.getBytes()));
// Failing after DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT we should not trigger the rollback.
// NOTE: the 4 (number of DELETE_COLUMN_FAMILY_DELETE_FS_LAYOUT + 1 step) is hardcoded,
// so you have to look at this test at least once when you add a new step.
int numberOfSteps = 4;
MasterProcedureTestingUtility.testRollbackAndDoubleExecutionAfterPONR(
procExec,
procId,
numberOfSteps,
DeleteColumnFamilyState.values());
MasterProcedureTestingUtility.validateColumnFamilyDeletion(
UTIL.getHBaseCluster().getMaster(), tableName, cf5);
}
private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
}
}

View File

@ -0,0 +1,237 @@
/**
* 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.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName;
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.ModifyColumnFamilyState;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category(MediumTests.class)
public class TestModifyColumnFamilyProcedure {
private static final Log LOG = LogFactory.getLog(TestModifyColumnFamilyProcedure.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 testModifyColumnFamily() throws Exception {
final TableName tableName = TableName.valueOf("testModifyColumnFamily");
final String cf1 = "cf1";
final HColumnDescriptor columnDescriptor = new HColumnDescriptor(cf1);
int oldBlockSize = columnDescriptor.getBlocksize();
int newBlockSize = 3 * oldBlockSize;
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
MasterProcedureTestingUtility.createTable(procExec, tableName, null, cf1, "f2");
// Test 1: modify the column family online
columnDescriptor.setBlocksize(newBlockSize);
long procId1 = procExec.submitProcedure(new ModifyColumnFamilyProcedure(
procExec.getEnvironment(), tableName, columnDescriptor));
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
MasterProcedureTestingUtility.validateColumnFamilyModification(UTIL.getHBaseCluster()
.getMaster(), tableName, cf1, columnDescriptor);
// Test 2: modify the column family offline
UTIL.getHBaseAdmin().disableTable(tableName);
columnDescriptor.setBlocksize(newBlockSize * 2);
long procId2 =
procExec.submitProcedure(new ModifyColumnFamilyProcedure(procExec.getEnvironment(),
tableName, columnDescriptor));
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId2);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
MasterProcedureTestingUtility.validateColumnFamilyModification(UTIL.getHBaseCluster()
.getMaster(), tableName, cf1, columnDescriptor);
}
@Test(timeout=60000)
public void testModifyNonExistingColumnFamily() throws Exception {
final TableName tableName = TableName.valueOf("testModifyExistingColumnFamily");
final String cf2 = "cf2";
final HColumnDescriptor columnDescriptor = new HColumnDescriptor(cf2);
int oldBlockSize = columnDescriptor.getBlocksize();
int newBlockSize = 2 * oldBlockSize;
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1");
// Modify the column family that does not exist
columnDescriptor.setBlocksize(newBlockSize);
long procId1 = procExec.submitProcedure(new ModifyColumnFamilyProcedure(
procExec.getEnvironment(), tableName, columnDescriptor));
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureResult result = procExec.getResult(procId1);
assertTrue(result.isFailed());
LOG.debug("Modify failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof InvalidFamilyOperationException);
}
@Test(timeout=60000)
public void testRecoveryAndDoubleExecutionOffline() throws Exception {
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecutionOffline");
final String cf3 = "cf3";
final HColumnDescriptor columnDescriptor = new HColumnDescriptor(cf3);
int oldBlockSize = columnDescriptor.getBlocksize();
int newBlockSize = 4 * oldBlockSize;
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
// create the table
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2", cf3);
UTIL.getHBaseAdmin().disableTable(tableName);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the Modify procedure && kill the executor
columnDescriptor.setBlocksize(newBlockSize);
long procId = procExec.submitProcedure(new ModifyColumnFamilyProcedure(
procExec.getEnvironment(), tableName, columnDescriptor));
// Restart the executor and execute the step twice
int numberOfSteps = ModifyColumnFamilyState.values().length;
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(
procExec,
procId,
numberOfSteps,
ModifyColumnFamilyState.values());
MasterProcedureTestingUtility.validateColumnFamilyModification(UTIL.getHBaseCluster()
.getMaster(), tableName, cf3, columnDescriptor);
}
@Test(timeout = 60000)
public void testRecoveryAndDoubleExecutionOnline() throws Exception {
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecutionOnline");
final String cf4 = "cf4";
final HColumnDescriptor columnDescriptor = new HColumnDescriptor(cf4);
int oldBlockSize = columnDescriptor.getBlocksize();
int newBlockSize = 4 * oldBlockSize;
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
// create the table
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2", cf4);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the Modify procedure && kill the executor
columnDescriptor.setBlocksize(newBlockSize);
long procId =
procExec.submitProcedure(new ModifyColumnFamilyProcedure(procExec.getEnvironment(),
tableName, columnDescriptor));
// Restart the executor and execute the step twice
int numberOfSteps = ModifyColumnFamilyState.values().length;
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId, numberOfSteps,
ModifyColumnFamilyState.values());
MasterProcedureTestingUtility.validateColumnFamilyModification(UTIL.getHBaseCluster()
.getMaster(), tableName, cf4, columnDescriptor);
}
@Test(timeout = 60000)
public void testRollbackAndDoubleExecution() throws Exception {
final TableName tableName = TableName.valueOf("testRollbackAndDoubleExecution");
final String cf3 = "cf3";
final HColumnDescriptor columnDescriptor = new HColumnDescriptor(cf3);
int oldBlockSize = columnDescriptor.getBlocksize();
int newBlockSize = 4 * oldBlockSize;
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
// create the table
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2", cf3);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the Modify procedure && kill the executor
columnDescriptor.setBlocksize(newBlockSize);
long procId = procExec.submitProcedure(new ModifyColumnFamilyProcedure(
procExec.getEnvironment(), tableName, columnDescriptor));
// Failing in the middle of proc
int numberOfSteps = ModifyColumnFamilyState.values().length - 2;
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(
procExec,
procId,
numberOfSteps,
ModifyColumnFamilyState.values());
}
private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
}
}