HBASE-18893 remove add/delete/modify column

This commit is contained in:
Mike Drob 2017-10-17 16:47:41 -05:00
parent 880b26d7d8
commit a1bc20ab58
15 changed files with 44 additions and 2386 deletions

View File

@ -148,52 +148,6 @@ message DeleteNamespaceStateData {
optional NamespaceDescriptor namespace_descriptor = 2;
}
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;
}
enum EnableTableState {
ENABLE_TABLE_PREPARE = 1;
ENABLE_TABLE_PRE_OPERATION = 2;

View File

@ -270,148 +270,6 @@ public interface MasterObserver {
final TableName tableName,
final TableDescriptor htd) throws IOException {}
/**
* Called prior to adding a new column family to the table. Called as part of
* add column RPC call.
*
* @param ctx the environment to interact with the framework and master
* @param tableName the name of the table
* @param columnFamily the ColumnFamilyDescriptor
*/
default void preAddColumnFamily(final ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException {}
/**
* Called after the new column family has been created. Called as part of
* add column RPC call.
*
* @param ctx the environment to interact with the framework and master
* @param tableName the name of the table
* @param columnFamily the ColumnFamilyDescriptor
*/
default void postAddColumnFamily(final ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException {}
/**
* Called prior to adding a new column family to the table. Called as part of
* add column procedure.
*
* @param ctx the environment to interact with the framework and master
* @param tableName the name of the table
* @param columnFamily the ColumnFamilyDescriptor
*/
default void preAddColumnFamilyAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName,
final ColumnFamilyDescriptor columnFamily) throws IOException {}
/**
* Called after the new column family has been created. Called as part of
* add column procedure.
*
* @param ctx the environment to interact with the framework and master
* @param tableName the name of the table
* @param columnFamily the ColumnFamilyDescriptor
*/
default void postCompletedAddColumnFamilyAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName,
final ColumnFamilyDescriptor columnFamily) throws IOException {}
/**
* Called prior to modifying a column family's attributes. Called as part of
* modify column RPC call.
*
* @param ctx the environment to interact with the framework and master
* @param tableName the name of the table
* @param columnFamily the ColumnFamilyDescriptor
*/
default void preModifyColumnFamily(final ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException {}
/**
* Called after the column family has been updated. Called as part of modify
* column RPC call.
*
* @param ctx the environment to interact with the framework and master
* @param tableName the name of the table
* @param columnFamily the ColumnFamilyDescriptor
*/
default void postModifyColumnFamily(final ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException {}
/**
* Called prior to modifying a column family's attributes. Called as part of
* modify column procedure.
*
* @param ctx the environment to interact with the framework and master
* @param tableName the name of the table
* @param columnFamily the ColumnFamilyDescriptor
*/
default void preModifyColumnFamilyAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName,
final ColumnFamilyDescriptor columnFamily) throws IOException {}
/**
* Called after the column family has been updated. Called as part of modify
* column procedure.
*
* @param ctx the environment to interact with the framework and master
* @param tableName the name of the table
* @param columnFamily the ColumnFamilyDescriptor
*/
default void postCompletedModifyColumnFamilyAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName,
final ColumnFamilyDescriptor columnFamily) throws IOException {}
/**
* Called prior to deleting the entire column family. Called as part of
* delete column RPC call.
*
* @param ctx the environment to interact with the framework and master
* @param tableName the name of the table
* @param columnFamily the column
*/
default void preDeleteColumnFamily(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName, final byte[] columnFamily) throws IOException {}
/**
* Called after the column family has been deleted. Called as part of delete
* column RPC call.
*
* @param ctx the environment to interact with the framework and master
* @param tableName the name of the table
* @param columnFamily the column family
*/
default void postDeleteColumnFamily(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName, final byte[] columnFamily) throws IOException {}
/**
* Called prior to deleting the entire column family. Called as part of
* delete column procedure.
*
* @param ctx the environment to interact with the framework and master
* @param tableName the name of the table
* @param columnFamily the column family
*/
default void preDeleteColumnFamilyAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName, final byte[] columnFamily) throws IOException {}
/**
* Called after the column family has been deleted. Called as part of
* delete column procedure.
*
* @param ctx the environment to interact with the framework and master
* @param tableName the name of the table
* @param columnFamily the column family
*/
default void postCompletedDeleteColumnFamilyAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName, final byte[] columnFamily) throws IOException {}
/**
* Called prior to enabling a table. Called as part of enable table RPC call.
* It can't bypass the default action, e.g., ctx.bypass() won't have effect.

View File

@ -63,6 +63,7 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.MasterNotRunningException;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
@ -114,9 +115,7 @@ import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerChore;
import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerFactory;
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;
@ -124,7 +123,6 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureScheduler;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
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.RecoverMetaProcedure;
@ -2158,37 +2156,22 @@ public class HMaster extends HRegionServer implements MasterServices {
@Override
public long addColumn(
final TableName tableName,
final ColumnFamilyDescriptor columnDescriptor,
final ColumnFamilyDescriptor column,
final long nonceGroup,
final long nonce)
throws IOException {
checkInitialized();
checkCompression(columnDescriptor);
checkEncryption(conf, columnDescriptor);
checkReplicationScope(columnDescriptor);
checkTableExists(tableName);
return MasterProcedureUtil.submitProcedure(
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
@Override
protected void run() throws IOException {
if (getMaster().getMasterCoprocessorHost().preAddColumn(tableName, columnDescriptor)) {
return;
}
TableDescriptor old = getTableDescriptors().get(tableName);
if (old.hasColumnFamily(column.getName())) {
throw new InvalidFamilyOperationException("Column family '" + column.getNameAsString()
+ "' in table '" + tableName + "' already exists so cannot be added");
}
// Execute the operation synchronously, wait for the operation to complete before continuing
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
submitProcedure(new AddColumnFamilyProcedure(procedureExecutor.getEnvironment(),
tableName, columnDescriptor, latch));
latch.await();
getMaster().getMasterCoprocessorHost().postAddColumn(tableName, columnDescriptor);
}
@Override
protected String getDescription() {
return "AddColumnFamilyProcedure";
}
});
TableDescriptor newDesc = TableDescriptorBuilder
.newBuilder(old).addColumnFamily(column).build();
return modifyTable(tableName, newDesc, nonceGroup, nonce);
}
@Override
@ -2199,35 +2182,20 @@ public class HMaster extends HRegionServer implements MasterServices {
final long nonce)
throws IOException {
checkInitialized();
checkCompression(descriptor);
checkEncryption(conf, descriptor);
checkReplicationScope(descriptor);
checkTableExists(tableName);
return MasterProcedureUtil.submitProcedure(
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
@Override
protected void run() throws IOException {
if (getMaster().getMasterCoprocessorHost().preModifyColumn(tableName, descriptor)) {
return;
}
TableDescriptor old = getTableDescriptors().get(tableName);
if (! old.hasColumnFamily(descriptor.getName())) {
throw new InvalidFamilyOperationException("Family '" + descriptor.getNameAsString()
+ "' does not exist, so it cannot be modified");
}
LOG.info(getClientIdAuditPrefix() + " modify " + descriptor);
TableDescriptor td = TableDescriptorBuilder
.newBuilder(old)
.modifyColumnFamily(descriptor)
.build();
// Execute the operation synchronously - wait for the operation to complete before
// continuing.
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
submitProcedure(new ModifyColumnFamilyProcedure(procedureExecutor.getEnvironment(),
tableName, descriptor, latch));
latch.await();
getMaster().getMasterCoprocessorHost().postModifyColumn(tableName, descriptor);
}
@Override
protected String getDescription() {
return "ModifyColumnFamilyProcedure";
}
});
return modifyTable(tableName, td, nonceGroup, nonce);
}
@Override
@ -2238,31 +2206,22 @@ public class HMaster extends HRegionServer implements MasterServices {
final long nonce)
throws IOException {
checkInitialized();
checkTableExists(tableName);
return MasterProcedureUtil.submitProcedure(
new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
@Override
protected void run() throws IOException {
if (getMaster().getMasterCoprocessorHost().preDeleteColumn(tableName, columnName)) {
return;
}
TableDescriptor old = getTableDescriptors().get(tableName);
LOG.info(getClientIdAuditPrefix() + " delete " + Bytes.toString(columnName));
if (! old.hasColumnFamily(columnName)) {
throw new InvalidFamilyOperationException("Family '" + Bytes.toString(columnName)
+ "' does not exist, so it cannot be deleted");
}
if (old.getColumnFamilyCount() == 1) {
throw new InvalidFamilyOperationException("Family '" + Bytes.toString(columnName)
+ "' is the only column family in the table, so it cannot be deleted");
}
// Execute the operation synchronously - wait for the operation to complete before continuing.
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
submitProcedure(new DeleteColumnFamilyProcedure(procedureExecutor.getEnvironment(),
tableName, columnName, latch));
latch.await();
getMaster().getMasterCoprocessorHost().postDeleteColumn(tableName, columnName);
}
@Override
protected String getDescription() {
return "DeleteColumnFamilyProcedure";
}
});
TableDescriptor td = TableDescriptorBuilder
.newBuilder(old).removeColumnFamily(columnName).build();
return modifyTable(tableName, td, nonceGroup, nonce);
}
@Override
@ -2441,15 +2400,20 @@ public class HMaster extends HRegionServer implements MasterServices {
});
}
private void checkTableExists(final TableName tableName)
throws IOException, TableNotFoundException {
if (!MetaTableAccessor.tableExists(getConnection(), tableName)) {
throw new TableNotFoundException(tableName);
}
}
@Override
public void checkTableModifiable(final TableName tableName)
throws IOException, TableNotFoundException, TableNotDisabledException {
if (isCatalogTable(tableName)) {
throw new IOException("Can't modify catalog tables");
}
if (!MetaTableAccessor.tableExists(getConnection(), tableName)) {
throw new TableNotFoundException(tableName);
}
checkTableExists(tableName);
if (!getTableStateManager().isTableState(tableName, TableState.State.DISABLED)) {
throw new TableNotDisabledException(tableName);
}

View File

@ -456,139 +456,6 @@ public class MasterCoprocessorHost
});
}
public boolean preAddColumn(final TableName tableName, final ColumnFamilyDescriptor columnFamily)
throws IOException {
return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
@Override
public void call(MasterObserver observer) throws IOException {
observer.preAddColumnFamily(this, tableName, columnFamily);
}
});
}
public void postAddColumn(final TableName tableName, final ColumnFamilyDescriptor columnFamily)
throws IOException {
execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
@Override
public void call(MasterObserver observer) throws IOException {
observer.postAddColumnFamily(this, tableName, columnFamily);
}
});
}
public boolean preAddColumnFamilyAction(
final TableName tableName,
final ColumnFamilyDescriptor columnFamily,
final User user)
throws IOException {
return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
@Override
public void call(MasterObserver observer) throws IOException {
observer.preAddColumnFamilyAction(this, tableName, columnFamily);
}
});
}
public void postCompletedAddColumnFamilyAction(
final TableName tableName,
final ColumnFamilyDescriptor columnFamily,
final User user)
throws IOException {
execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
@Override
public void call(MasterObserver observer) throws IOException {
observer.postCompletedAddColumnFamilyAction(this, tableName, columnFamily);
}
});
}
public boolean preModifyColumn(final TableName tableName,
final ColumnFamilyDescriptor columnFamily) throws IOException {
return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
@Override
public void call(MasterObserver observer) throws IOException {
observer.preModifyColumnFamily(this, tableName, columnFamily);
}
});
}
public void postModifyColumn(final TableName tableName, final ColumnFamilyDescriptor columnFamily)
throws IOException {
execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
@Override
public void call(MasterObserver observer) throws IOException {
observer.postModifyColumnFamily(this, tableName, columnFamily);
}
});
}
public boolean preModifyColumnFamilyAction(
final TableName tableName,
final ColumnFamilyDescriptor columnFamily,
final User user) throws IOException {
return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
@Override
public void call(MasterObserver observer) throws IOException {
observer.preModifyColumnFamilyAction(this, tableName, columnFamily);
}
});
}
public void postCompletedModifyColumnFamilyAction(
final TableName tableName,
final ColumnFamilyDescriptor columnFamily,
final User user) throws IOException {
execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
@Override
public void call(MasterObserver observer) throws IOException {
observer.postCompletedModifyColumnFamilyAction(this, tableName, columnFamily);
}
});
}
public boolean preDeleteColumn(final TableName tableName, final byte[] columnFamily)
throws IOException {
return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
@Override
public void call(MasterObserver observer) throws IOException {
observer.preDeleteColumnFamily(this, tableName, columnFamily);
}
});
}
public void postDeleteColumn(final TableName tableName, final byte[] columnFamily)
throws IOException {
execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
@Override
public void call(MasterObserver observer) throws IOException {
observer.postDeleteColumnFamily(this, tableName, columnFamily);
}
});
}
public boolean preDeleteColumnFamilyAction(
final TableName tableName,
final byte[] columnFamily,
final User user)
throws IOException {
return execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
@Override
public void call(MasterObserver observer) throws IOException {
observer.preDeleteColumnFamilyAction(this, tableName, columnFamily);
}
});
}
public void postCompletedDeleteColumnFamilyAction(
final TableName tableName, final byte[] columnFamily, final User user) throws IOException {
execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation(user) {
@Override
public void call(MasterObserver observer) throws IOException {
observer.postCompletedDeleteColumnFamilyAction(this, tableName, columnFamily);
}
});
}
public void preEnableTable(final TableName tableName) throws IOException {
execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
@Override

View File

@ -1,358 +0,0 @@
/**
* 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.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.AddColumnFamilyState;
/**
* The procedure to add a column family to an existing table.
*/
@InterfaceAudience.Private
public class AddColumnFamilyProcedure
extends AbstractStateMachineTableProcedure<AddColumnFamilyState> {
private static final Log LOG = LogFactory.getLog(AddColumnFamilyProcedure.class);
private TableName tableName;
private TableDescriptor unmodifiedTableDescriptor;
private ColumnFamilyDescriptor cfDescriptor;
private List<RegionInfo> regionInfoList;
private Boolean traceEnabled;
public AddColumnFamilyProcedure() {
super();
this.unmodifiedTableDescriptor = null;
this.regionInfoList = null;
this.traceEnabled = null;
}
public AddColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
final ColumnFamilyDescriptor cfDescriptor) throws IOException {
this(env, tableName, cfDescriptor, null);
}
public AddColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
final ColumnFamilyDescriptor cfDescriptor, final ProcedurePrepareLatch latch) {
super(env, latch);
this.tableName = tableName;
this.cfDescriptor = cfDescriptor;
this.unmodifiedTableDescriptor = null;
this.regionInfoList = null;
this.traceEnabled = null;
}
@Override
protected Flow executeFromState(final MasterProcedureEnv env, final AddColumnFamilyState state)
throws InterruptedException {
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:
if (env.getAssignmentManager().isTableEnabled(getTableName())) {
addChildProcedure(env.getAssignmentManager()
.createReopenProcedures(getRegionInfoList(env)));
}
return Flow.NO_MORE_STATE;
default:
throw new UnsupportedOperationException(this + " unhandled state=" + state);
}
} catch (IOException e) {
if (isRollbackSupported(state)) {
setFailure("master-add-columnfamily", e);
} else {
LOG.warn("Retriable error trying to add the column family " + getColumnFamilyName() +
" to the table " + tableName + " (in state=" + state + ")", e);
}
}
return Flow.HAS_MORE_STATE;
}
@Override
protected void rollbackState(final MasterProcedureEnv env, final AddColumnFamilyState state)
throws IOException {
if (state == AddColumnFamilyState.ADD_COLUMN_FAMILY_PREPARE ||
state == AddColumnFamilyState.ADD_COLUMN_FAMILY_PRE_OPERATION) {
// nothing to rollback, pre is just table-state checks.
// We can fail if the table does not exist or is not disabled.
// TODO: coprocessor rollback semantic is still undefined.
return;
}
// The procedure doesn't have a rollback. The execution will succeed, at some point.
throw new UnsupportedOperationException("unhandled state=" + state);
}
@Override
protected boolean isRollbackSupported(final AddColumnFamilyState state) {
switch (state) {
case ADD_COLUMN_FAMILY_PREPARE:
case ADD_COLUMN_FAMILY_PRE_OPERATION:
return true;
default:
return false;
}
}
@Override
protected void completionCleanup(final MasterProcedureEnv env) {
releaseSyncLatch();
}
@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 serializeStateData(ProcedureStateSerializer serializer)
throws IOException {
super.serializeStateData(serializer);
MasterProcedureProtos.AddColumnFamilyStateData.Builder addCFMsg =
MasterProcedureProtos.AddColumnFamilyStateData.newBuilder()
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
.setTableName(ProtobufUtil.toProtoTableName(tableName))
.setColumnfamilySchema(ProtobufUtil.toColumnFamilySchema(cfDescriptor));
if (unmodifiedTableDescriptor != null) {
addCFMsg
.setUnmodifiedTableSchema(ProtobufUtil.toTableSchema(unmodifiedTableDescriptor));
}
serializer.serialize(addCFMsg.build());
}
@Override
protected void deserializeStateData(ProcedureStateSerializer serializer)
throws IOException {
super.deserializeStateData(serializer);
MasterProcedureProtos.AddColumnFamilyStateData addCFMsg =
serializer.deserialize(MasterProcedureProtos.AddColumnFamilyStateData.class);
setUser(MasterProcedureUtil.toUserInfo(addCFMsg.getUserInfo()));
tableName = ProtobufUtil.toTableName(addCFMsg.getTableName());
cfDescriptor = ProtobufUtil.toColumnFamilyDescriptor(addCFMsg.getColumnfamilySchema());
if (addCFMsg.hasUnmodifiedTableSchema()) {
unmodifiedTableDescriptor = ProtobufUtil.toTableDescriptor(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(")");
}
@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.
checkTableModifiable(env);
// In order to update the descriptor, we need to retrieve the old descriptor for comparison.
unmodifiedTableDescriptor = env.getMasterServices().getTableDescriptors().get(tableName);
if (unmodifiedTableDescriptor == null) {
throw new IOException("TableDescriptor missing for " + tableName);
}
if (unmodifiedTableDescriptor.hasColumnFamily(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());
TableDescriptor htd = env.getMasterServices().getTableDescriptors().get(tableName);
if (htd.hasColumnFamily(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;
}
env.getMasterServices().getTableDescriptors().add(
TableDescriptorBuilder.newBuilder(htd)
.addColumnFamily(cfDescriptor).build());
}
/**
* Restore the table descriptor back to pre-add
* @param env MasterProcedureEnv
* @throws IOException
**/
private void restoreTableDescriptor(final MasterProcedureEnv env) throws IOException {
TableDescriptor htd = env.getMasterServices().getTableDescriptors().get(tableName);
if (htd.hasColumnFamily(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(), cfDescriptor.isMobEnabled());
env.getMasterServices().getTableDescriptors().add(unmodifiedTableDescriptor);
// Make sure regions are opened after table descriptor is updated.
//reOpenAllRegionsIfTableIsOnline(env);
// TODO: NUKE ROLLBACK!!!!
}
}
/**
* 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);
}
/**
* 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) {
switch (state) {
case ADD_COLUMN_FAMILY_PRE_OPERATION:
cpHost.preAddColumnFamilyAction(tableName, cfDescriptor, getUser());
break;
case ADD_COLUMN_FAMILY_POST_OPERATION:
cpHost.postCompletedAddColumnFamilyAction(tableName, cfDescriptor, getUser());
break;
default:
throw new UnsupportedOperationException(this + " unhandled state=" + state);
}
}
}
private List<RegionInfo> getRegionInfoList(final MasterProcedureEnv env) throws IOException {
if (regionInfoList == null) {
regionInfoList = env.getAssignmentManager().getRegionStates()
.getRegionsOfTable(getTableName());
}
return regionInfoList;
}
}

View File

@ -1,371 +0,0 @@
/**
* 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.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteColumnFamilyState;
/**
* The procedure to delete a column family from an existing table.
*/
@InterfaceAudience.Private
public class DeleteColumnFamilyProcedure
extends AbstractStateMachineTableProcedure<DeleteColumnFamilyState> {
private static final Log LOG = LogFactory.getLog(DeleteColumnFamilyProcedure.class);
private TableDescriptor unmodifiedTableDescriptor;
private TableName tableName;
private byte [] familyName;
private boolean hasMob;
private List<RegionInfo> regionInfoList;
private Boolean traceEnabled;
public DeleteColumnFamilyProcedure() {
super();
this.unmodifiedTableDescriptor = null;
this.regionInfoList = null;
this.traceEnabled = null;
}
public DeleteColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
final byte[] familyName) {
this(env, tableName, familyName, null);
}
public DeleteColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
final byte[] familyName, final ProcedurePrepareLatch latch) {
super(env, latch);
this.tableName = tableName;
this.familyName = familyName;
this.unmodifiedTableDescriptor = null;
this.regionInfoList = null;
this.traceEnabled = null;
}
@Override
protected Flow executeFromState(final MasterProcedureEnv env, DeleteColumnFamilyState state)
throws InterruptedException {
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:
if (env.getAssignmentManager().isTableEnabled(getTableName())) {
addChildProcedure(env.getAssignmentManager()
.createReopenProcedures(getRegionInfoList(env)));
}
return Flow.NO_MORE_STATE;
default:
throw new UnsupportedOperationException(this + " unhandled state=" + state);
}
} catch (IOException e) {
if (isRollbackSupported(state)) {
setFailure("master-delete-columnfamily", e);
} else {
LOG.warn("Retriable error trying to delete the column family " + getColumnFamilyName() +
" from table " + tableName + " (in state=" + state + ")", e);
}
}
return Flow.HAS_MORE_STATE;
}
@Override
protected void rollbackState(final MasterProcedureEnv env, final DeleteColumnFamilyState state)
throws IOException {
if (state == DeleteColumnFamilyState.DELETE_COLUMN_FAMILY_PREPARE ||
state == DeleteColumnFamilyState.DELETE_COLUMN_FAMILY_PRE_OPERATION) {
// nothing to rollback, pre is just table-state checks.
// We can fail if the table does not exist or is not disabled.
// TODO: coprocessor rollback semantic is still undefined.
return;
}
// The procedure doesn't have a rollback. The execution will succeed, at some point.
throw new UnsupportedOperationException("unhandled state=" + state);
}
@Override
protected boolean isRollbackSupported(final DeleteColumnFamilyState state) {
switch (state) {
case DELETE_COLUMN_FAMILY_PRE_OPERATION:
case DELETE_COLUMN_FAMILY_PREPARE:
return true;
default:
return false;
}
}
@Override
protected void completionCleanup(final MasterProcedureEnv env) {
releaseSyncLatch();
}
@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 serializeStateData(ProcedureStateSerializer serializer)
throws IOException {
super.serializeStateData(serializer);
MasterProcedureProtos.DeleteColumnFamilyStateData.Builder deleteCFMsg =
MasterProcedureProtos.DeleteColumnFamilyStateData.newBuilder()
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
.setTableName(ProtobufUtil.toProtoTableName(tableName))
.setColumnfamilyName(UnsafeByteOperations.unsafeWrap(familyName));
if (unmodifiedTableDescriptor != null) {
deleteCFMsg
.setUnmodifiedTableSchema(ProtobufUtil.toTableSchema(unmodifiedTableDescriptor));
}
serializer.serialize(deleteCFMsg.build());
}
@Override
protected void deserializeStateData(ProcedureStateSerializer serializer)
throws IOException {
super.deserializeStateData(serializer);
MasterProcedureProtos.DeleteColumnFamilyStateData deleteCFMsg =
serializer.deserialize(MasterProcedureProtos.DeleteColumnFamilyStateData.class);
setUser(MasterProcedureUtil.toUserInfo(deleteCFMsg.getUserInfo()));
tableName = ProtobufUtil.toTableName(deleteCFMsg.getTableName());
familyName = deleteCFMsg.getColumnfamilyName().toByteArray();
if (deleteCFMsg.hasUnmodifiedTableSchema()) {
unmodifiedTableDescriptor = ProtobufUtil.toTableDescriptor(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(")");
}
@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.
checkTableModifiable(env);
// In order to update the descriptor, we need to retrieve the old descriptor for comparison.
unmodifiedTableDescriptor = env.getMasterServices().getTableDescriptors().get(tableName);
if (unmodifiedTableDescriptor == null) {
throw new IOException("TableDescriptor missing for " + tableName);
}
if (!unmodifiedTableDescriptor.hasColumnFamily(familyName)) {
throw new InvalidFamilyOperationException("Family '" + getColumnFamilyName()
+ "' does not exist, so it cannot be deleted");
}
if (unmodifiedTableDescriptor.getColumnFamilyCount() == 1) {
throw new InvalidFamilyOperationException("Family '" + getColumnFamilyName()
+ "' is the only column family in the table, so it cannot be deleted");
}
// whether mob family
hasMob = unmodifiedTableDescriptor.getColumnFamily(familyName).isMobEnabled();
}
/**
* 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());
TableDescriptor htd = env.getMasterServices().getTableDescriptors().get(tableName);
if (!htd.hasColumnFamily(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;
}
env.getMasterServices().getTableDescriptors().add(
TableDescriptorBuilder.newBuilder(htd).removeColumnFamily(familyName).build());
}
/**
* Restore back to the old descriptor
* @param env MasterProcedureEnv
* @throws IOException
**/
private void restoreTableDescriptor(final MasterProcedureEnv env) throws IOException {
env.getMasterServices().getTableDescriptors().add(unmodifiedTableDescriptor);
// Make sure regions are opened after table descriptor is updated.
//reOpenAllRegionsIfTableIsOnline(env);
// TODO: NUKE ROLLBACK!!!!
}
/**
* Remove the column family from the file system
**/
private void deleteFromFs(final MasterProcedureEnv env) throws IOException {
MasterDDLOperationHelper.deleteColumnFamilyFromFileSystem(env, tableName,
getRegionInfoList(env), familyName, hasMob);
}
/**
* 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);
}
/**
* 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) {
switch (state) {
case DELETE_COLUMN_FAMILY_PRE_OPERATION:
cpHost.preDeleteColumnFamilyAction(tableName, familyName, getUser());
break;
case DELETE_COLUMN_FAMILY_POST_OPERATION:
cpHost.postCompletedDeleteColumnFamilyAction(tableName, familyName, getUser());
break;
default:
throw new UnsupportedOperationException(this + " unhandled state=" + state);
}
}
}
private List<RegionInfo> getRegionInfoList(final MasterProcedureEnv env) throws IOException {
if (regionInfoList == null) {
regionInfoList = env.getAssignmentManager().getRegionStates()
.getRegionsOfTable(getTableName());
}
return regionInfoList;
}
}

View File

@ -1,323 +0,0 @@
/**
* 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 org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyColumnFamilyState;
/**
* The procedure to modify a column family from an existing table.
*/
@InterfaceAudience.Private
public class ModifyColumnFamilyProcedure
extends AbstractStateMachineTableProcedure<ModifyColumnFamilyState> {
private static final Log LOG = LogFactory.getLog(ModifyColumnFamilyProcedure.class);
private TableName tableName;
private TableDescriptor unmodifiedtableDescriptor;
private ColumnFamilyDescriptor cfDescriptor;
private Boolean traceEnabled;
public ModifyColumnFamilyProcedure() {
super();
this.unmodifiedtableDescriptor = null;
this.traceEnabled = null;
}
public ModifyColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
final ColumnFamilyDescriptor cfDescriptor) {
this(env, tableName, cfDescriptor, null);
}
public ModifyColumnFamilyProcedure(final MasterProcedureEnv env, final TableName tableName,
final ColumnFamilyDescriptor cfDescriptor, final ProcedurePrepareLatch latch) {
super(env, latch);
this.tableName = tableName;
this.cfDescriptor = cfDescriptor;
this.unmodifiedtableDescriptor = null;
this.traceEnabled = null;
}
@Override
protected Flow executeFromState(final MasterProcedureEnv env,
final ModifyColumnFamilyState state) throws InterruptedException {
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:
if (env.getAssignmentManager().isTableEnabled(getTableName())) {
addChildProcedure(env.getAssignmentManager().createReopenProcedures(getTableName()));
}
return Flow.NO_MORE_STATE;
default:
throw new UnsupportedOperationException(this + " unhandled state=" + state);
}
} catch (IOException e) {
if (isRollbackSupported(state)) {
setFailure("master-modify-columnfamily", e);
} else {
LOG.warn("Retriable error trying to disable table=" + tableName +
" (in state=" + state + ")", e);
}
}
return Flow.HAS_MORE_STATE;
}
@Override
protected void rollbackState(final MasterProcedureEnv env, final ModifyColumnFamilyState state)
throws IOException {
if (state == ModifyColumnFamilyState.MODIFY_COLUMN_FAMILY_PREPARE ||
state == ModifyColumnFamilyState.MODIFY_COLUMN_FAMILY_PRE_OPERATION) {
// nothing to rollback, pre-modify is just checks.
// TODO: coprocessor rollback semantic is still undefined.
return;
}
// The delete doesn't have a rollback. The execution will succeed, at some point.
throw new UnsupportedOperationException("unhandled state=" + state);
}
@Override
protected boolean isRollbackSupported(final ModifyColumnFamilyState state) {
switch (state) {
case MODIFY_COLUMN_FAMILY_PRE_OPERATION:
case MODIFY_COLUMN_FAMILY_PREPARE:
return true;
default:
return false;
}
}
@Override
protected void completionCleanup(final MasterProcedureEnv env) {
releaseSyncLatch();
}
@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 serializeStateData(ProcedureStateSerializer serializer)
throws IOException {
super.serializeStateData(serializer);
MasterProcedureProtos.ModifyColumnFamilyStateData.Builder modifyCFMsg =
MasterProcedureProtos.ModifyColumnFamilyStateData.newBuilder()
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(getUser()))
.setTableName(ProtobufUtil.toProtoTableName(tableName))
.setColumnfamilySchema(ProtobufUtil.toColumnFamilySchema(cfDescriptor));
if (unmodifiedtableDescriptor != null) {
modifyCFMsg
.setUnmodifiedTableSchema(ProtobufUtil.toTableSchema(unmodifiedtableDescriptor));
}
serializer.serialize(modifyCFMsg.build());
}
@Override
protected void deserializeStateData(ProcedureStateSerializer serializer)
throws IOException {
super.deserializeStateData(serializer);
MasterProcedureProtos.ModifyColumnFamilyStateData modifyCFMsg =
serializer.deserialize(MasterProcedureProtos.ModifyColumnFamilyStateData.class);
setUser(MasterProcedureUtil.toUserInfo(modifyCFMsg.getUserInfo()));
tableName = ProtobufUtil.toTableName(modifyCFMsg.getTableName());
cfDescriptor = ProtobufUtil.toColumnFamilyDescriptor(modifyCFMsg.getColumnfamilySchema());
if (modifyCFMsg.hasUnmodifiedTableSchema()) {
unmodifiedtableDescriptor = ProtobufUtil.toTableDescriptor(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(")");
}
@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.
checkTableModifiable(env);
unmodifiedtableDescriptor = env.getMasterServices().getTableDescriptors().get(tableName);
if (unmodifiedtableDescriptor == null) {
throw new IOException("TableDescriptor missing for " + tableName);
}
if (!unmodifiedtableDescriptor.hasColumnFamily(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());
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(env.getMasterServices().getTableDescriptors().get(tableName));
builder.modifyColumnFamily(cfDescriptor);
env.getMasterServices().getTableDescriptors().add(builder.build());
}
/**
* Restore back to the old descriptor
* @param env MasterProcedureEnv
* @throws IOException
**/
private void restoreTableDescriptor(final MasterProcedureEnv env) throws IOException {
env.getMasterServices().getTableDescriptors().add(unmodifiedtableDescriptor);
// Make sure regions are opened after table descriptor is updated.
//reOpenAllRegionsIfTableIsOnline(env);
// TODO: NUKE ROLLBACK!!!!
}
/**
* 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);
}
/**
* 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) {
switch (state) {
case MODIFY_COLUMN_FAMILY_PRE_OPERATION:
cpHost.preModifyColumnFamilyAction(tableName, cfDescriptor, getUser());
break;
case MODIFY_COLUMN_FAMILY_POST_OPERATION:
cpHost.postCompletedModifyColumnFamilyAction(tableName, cfDescriptor, getUser());
break;
default:
throw new UnsupportedOperationException(this + " unhandled state=" + state);
}
}
}
}

View File

@ -1168,6 +1168,7 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
@Override
public void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName,
TableDescriptor htd) throws IOException {
// TODO: potentially check if this is a add/modify/delete column operation
requirePermission(getActiveUser(c), "modifyTable", tableName, null, null,
Action.ADMIN, Action.CREATE);
}
@ -1193,45 +1194,6 @@ public class AccessController implements MasterCoprocessor, RegionCoprocessor,
});
}
@Override
public void preAddColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName, ColumnFamilyDescriptor columnFamily)
throws IOException {
requireTablePermission(getActiveUser(ctx), "addColumn", tableName, columnFamily.getName(), null,
Action.ADMIN, Action.CREATE);
}
@Override
public void preModifyColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName, ColumnFamilyDescriptor columnFamily)
throws IOException {
requirePermission(getActiveUser(ctx), "modifyColumn", tableName, columnFamily.getName(), null,
Action.ADMIN, Action.CREATE);
}
@Override
public void preDeleteColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName, byte[] columnFamily) throws IOException {
requirePermission(getActiveUser(ctx), "deleteColumn", tableName, columnFamily, null,
Action.ADMIN, Action.CREATE);
}
@Override
public void postDeleteColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName, final byte[] columnFamily) throws IOException {
final Configuration conf = ctx.getEnvironment().getConfiguration();
User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
@Override
public Void run() throws Exception {
try (Table table = ctx.getEnvironment().getConnection().
getTable(AccessControlLists.ACL_TABLE_NAME)) {
AccessControlLists.removeTablePermissions(conf, tableName, columnFamily, table);
}
return null;
}
});
}
@Override
public void preEnableTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName)
throws IOException {

View File

@ -244,40 +244,6 @@ public class VisibilityController implements MasterCoprocessor, RegionCoprocesso
}
}
@Override
public void preAddColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName, ColumnFamilyDescriptor columnFamily)
throws IOException {
if (!authorizationEnabled) {
return;
}
if (LABELS_TABLE_NAME.equals(tableName)) {
throw new ConstraintException("Cannot alter " + LABELS_TABLE_NAME);
}
}
@Override
public void preModifyColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException {
if (!authorizationEnabled) {
return;
}
if (LABELS_TABLE_NAME.equals(tableName)) {
throw new ConstraintException("Cannot alter " + LABELS_TABLE_NAME);
}
}
@Override
public void preDeleteColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName, byte[] columnFamily) throws IOException {
if (!authorizationEnabled) {
return;
}
if (LABELS_TABLE_NAME.equals(tableName)) {
throw new ConstraintException("Cannot alter " + LABELS_TABLE_NAME);
}
}
@Override
public void preDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
throws IOException {

View File

@ -21,7 +21,6 @@ package org.apache.hadoop.hbase.coprocessor;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
@ -45,7 +44,6 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.MasterSwitchType;
@ -518,77 +516,6 @@ public class TestMasterObserver {
return preListNamespaceDescriptorsCalled && !postListNamespaceDescriptorsCalled;
}
@Override
public void preAddColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName, ColumnFamilyDescriptor columnFamily
) throws IOException {
if (bypass) {
ctx.bypass();
}
preAddColumnCalled = true;
}
@Override
public void postAddColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException {
postAddColumnCalled = true;
}
public boolean wasAddColumnCalled() {
return preAddColumnCalled && postAddColumnCalled;
}
public boolean preAddColumnCalledOnly() {
return preAddColumnCalled && !postAddColumnCalled;
}
@Override
public void preModifyColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException {
if (bypass) {
ctx.bypass();
}
preModifyColumnCalled = true;
}
@Override
public void postModifyColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName, ColumnFamilyDescriptor columnFamily) throws IOException {
postModifyColumnCalled = true;
}
public boolean wasModifyColumnCalled() {
return preModifyColumnCalled && postModifyColumnCalled;
}
public boolean preModifyColumnCalledOnly() {
return preModifyColumnCalled && !postModifyColumnCalled;
}
@Override
public void preDeleteColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName, byte[] columnFamily) throws IOException {
if (bypass) {
ctx.bypass();
}
preDeleteColumnCalled = true;
}
@Override
public void postDeleteColumnFamily(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName, byte[] columnFamily) throws IOException {
postDeleteColumnCalled = true;
}
public boolean wasDeleteColumnCalled() {
return preDeleteColumnCalled && postDeleteColumnCalled;
}
public boolean preDeleteColumnCalledOnly() {
return preDeleteColumnCalled && !postDeleteColumnCalled;
}
@Override
public void preEnableTable(ObserverContext<MasterCoprocessorEnvironment> env,
TableName tableName) throws IOException {
@ -1084,87 +1011,6 @@ public class TestMasterObserver {
return preModifyTableActionCalled && !postCompletedModifyTableActionCalled;
}
@Override
public void preAddColumnFamilyAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName,
final ColumnFamilyDescriptor columnFamily) throws IOException {
if (bypass) {
ctx.bypass();
}
preAddColumnFamilyActionCalled = true;
}
@Override
public void postCompletedAddColumnFamilyAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName,
final ColumnFamilyDescriptor columnFamily) throws IOException {
postCompletedAddColumnFamilyActionCalled = true;
}
public boolean wasAddColumnFamilyActionCalled() {
return preAddColumnFamilyActionCalled && postCompletedAddColumnFamilyActionCalled;
}
public boolean preAddColumnFamilyActionCalledOnly() {
return preAddColumnFamilyActionCalled && !postCompletedAddColumnFamilyActionCalled;
}
@Override
public void preModifyColumnFamilyAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName,
final ColumnFamilyDescriptor columnFamily) throws IOException {
if (bypass) {
ctx.bypass();
}
preModifyColumnFamilyActionCalled = true;
}
@Override
public void postCompletedModifyColumnFamilyAction(
ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName,
ColumnFamilyDescriptor columnFamily) throws IOException {
postCompletedModifyColumnFamilyActionCalled = true;
}
public boolean wasModifyColumnFamilyActionCalled() {
return preModifyColumnFamilyActionCalled && postCompletedModifyColumnFamilyActionCalled;
}
public boolean preModifyColumnFamilyActionCalledOnly() {
return preModifyColumnFamilyActionCalled && !postCompletedModifyColumnFamilyActionCalled;
}
@Override
public void preDeleteColumnFamilyAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName,
final byte[] columnFamily) throws IOException {
if (bypass) {
ctx.bypass();
}
preDeleteColumnFamilyActionCalled = true;
}
@Override
public void postCompletedDeleteColumnFamilyAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx,
final TableName tableName,
final byte[] columnFamily) throws IOException {
postCompletedDeleteColumnFamilyActionCalled = true;
}
public boolean wasDeleteColumnFamilyActionCalled() {
return preDeleteColumnFamilyActionCalled && postCompletedDeleteColumnFamilyActionCalled;
}
public boolean preDeleteColumnFamilyActionCalledOnly() {
return preDeleteColumnFamilyActionCalled && !postCompletedDeleteColumnFamilyActionCalled;
}
@Override
public void preEnableTableAction(
final ObserverContext<MasterCoprocessorEnvironment> ctx, final TableName tableName)
throws IOException {
@ -1580,18 +1426,6 @@ public class TestMasterObserver {
assertTrue("Test table should have been modified",
cp.wasModifyTableCalled());
// add a column family
admin.addColumnFamily(tableName, new HColumnDescriptor(TEST_FAMILY2));
assertTrue("New column family shouldn't have been added to test table",
cp.preAddColumnCalledOnly());
// modify a column family
HColumnDescriptor hcd1 = new HColumnDescriptor(TEST_FAMILY2);
hcd1.setMaxVersions(25);
admin.modifyColumnFamily(tableName, hcd1);
assertTrue("Second column family should be modified",
cp.preModifyColumnCalledOnly());
// truncate table
admin.truncateTable(tableName, false);
@ -1634,21 +1468,6 @@ public class TestMasterObserver {
modifyTableSync(admin, tableName, htd);
assertTrue("Test table should have been modified",
cp.wasModifyTableCalled());
// add a column family
admin.addColumnFamily(tableName, new HColumnDescriptor(TEST_FAMILY2));
assertTrue("New column family should have been added to test table",
cp.wasAddColumnCalled());
assertTrue("Add column handler should be called.",
cp.wasAddColumnFamilyActionCalled());
// modify a column family
HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY2);
hcd.setMaxVersions(25);
admin.modifyColumnFamily(tableName, hcd);
assertTrue("Second column family should be modified",
cp.wasModifyColumnCalled());
assertTrue("Modify table handler should be called.",
cp.wasModifyColumnFamilyActionCalled());
// enable
assertFalse(cp.wasEnableTableCalled());
@ -1664,19 +1483,6 @@ public class TestMasterObserver {
admin.disableTable(tableName);
assertTrue(admin.isTableDisabled(tableName));
// delete column
assertFalse("No column family deleted yet", cp.wasDeleteColumnCalled());
assertFalse("Delete table column handler should not be called.",
cp.wasDeleteColumnFamilyActionCalled());
admin.deleteColumnFamily(tableName, TEST_FAMILY2);
HTableDescriptor tableDesc = admin.getTableDescriptor(tableName);
assertNull("'"+Bytes.toString(TEST_FAMILY2)+"' should have been removed",
tableDesc.getFamily(TEST_FAMILY2));
assertTrue("Coprocessor should have been called on column delete",
cp.wasDeleteColumnCalled());
assertTrue("Delete table column handler should be called.",
cp.wasDeleteColumnFamilyActionCalled());
// delete table
assertFalse("No table deleted yet", cp.wasDeleteTableCalled());
assertFalse("Delete table handler should not be called.",

View File

@ -1,190 +0,0 @@
/**
* 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.hbase.CategoryBasedTimeout;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.junit.rules.TestRule;
@Category({MasterTests.class, MediumTests.class})
public class TestAddColumnFamilyProcedure extends TestTableDDLProcedureBase {
private static final Log LOG = LogFactory.getLog(TestAddColumnFamilyProcedure.class);
@Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
withLookingForStuckThread(true).build();
@Rule public TestName name = new TestName();
@Test(timeout = 60000)
public void testAddColumnFamily() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
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(getMaster(), tableName, cf1);
// Test 2: Add a column family offline
UTIL.getAdmin().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(getMaster(), tableName, cf2);
}
@Test(timeout=60000)
public void testAddSameColumnFamilyTwice() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
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(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
Procedure<?> result = procExec.getResult(procId2);
assertTrue(result.isFailed());
LOG.debug("Add failed with exception: " + result.getException());
assertTrue(
ProcedureTestingUtility.getExceptionCause(result) instanceof InvalidFamilyOperationException);
// Do the same add the existing column family - this time offline
UTIL.getAdmin().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(
ProcedureTestingUtility.getExceptionCause(result) instanceof InvalidFamilyOperationException);
}
@Test(timeout = 60000)
public void testRecoveryAndDoubleExecutionOffline() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
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.getAdmin().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
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
MasterProcedureTestingUtility.validateColumnFamilyAddition(getMaster(), tableName, cf4);
}
@Test(timeout = 60000)
public void testRecoveryAndDoubleExecutionOnline() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
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
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
MasterProcedureTestingUtility.validateColumnFamilyAddition(getMaster(), tableName, cf5);
}
@Test(timeout = 60000)
public void testRollbackAndDoubleExecution() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
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 = 0; // failing at "pre operations"
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
MasterProcedureTestingUtility.validateColumnFamilyDeletion(getMaster(), tableName, cf6);
}
}

View File

@ -1,211 +0,0 @@
/**
* 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.hbase.CategoryBasedTimeout;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.junit.rules.TestRule;
@Category({MasterTests.class, MediumTests.class})
public class TestDeleteColumnFamilyProcedure extends TestTableDDLProcedureBase {
private static final Log LOG = LogFactory.getLog(TestDeleteColumnFamilyProcedure.class);
@Rule public final TestRule timeout = CategoryBasedTimeout.builder().withTimeout(this.getClass()).
withLookingForStuckThread(true).build();
@Rule public TestName name = new TestName();
@Test(timeout = 60000)
public void testDeleteColumnFamily() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
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(getMaster(), tableName, cf1);
// Test 2: delete the column family that exists offline
UTIL.getAdmin().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(name.getMethodName());
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(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
Procedure<?> result = procExec.getResult(procId2);
assertTrue(result.isFailed());
LOG.debug("Delete online failed with exception: " + result.getException());
assertTrue(
ProcedureTestingUtility.getExceptionCause(result) instanceof InvalidFamilyOperationException);
// Try again, this time with table disabled.
UTIL.getAdmin().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(
ProcedureTestingUtility.getExceptionCause(result) instanceof InvalidFamilyOperationException);
}
@Test(timeout=60000)
public void testDeleteNonExistingColumnFamily() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
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);
Procedure<?> result = procExec.getResult(procId1);
assertTrue(result.isFailed());
LOG.debug("Delete failed with exception: " + result.getException());
assertTrue(
ProcedureTestingUtility.getExceptionCause(result) instanceof InvalidFamilyOperationException);
}
@Test(timeout=60000)
public void testRecoveryAndDoubleExecutionOffline() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final String cf4 = "cf4";
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
// create the table
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1", "f2", "f3", cf4);
UTIL.getAdmin().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
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
MasterProcedureTestingUtility.validateColumnFamilyDeletion(getMaster(), tableName, cf4);
}
@Test(timeout = 60000)
public void testRecoveryAndDoubleExecutionOnline() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
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
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
MasterProcedureTestingUtility.validateColumnFamilyDeletion(getMaster(), tableName, cf5);
}
@Test(timeout = 60000)
public void testRollbackAndDoubleExecution() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
final String cf5 = "cf5";
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
// create the table
RegionInfo[] 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()));
int numberOfSteps = 0; // failing at pre operation
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
MasterProcedureTestingUtility.validateTableCreation(
getMaster(), tableName, regions, "f1", "f2", "f3", cf5);
}
}

View File

@ -1,183 +0,0 @@
/**
* 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.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
@Category({MasterTests.class, MediumTests.class})
public class TestModifyColumnFamilyProcedure extends TestTableDDLProcedureBase {
private static final Log LOG = LogFactory.getLog(TestModifyColumnFamilyProcedure.class);
@Rule
public TestName name = new TestName();
@Test(timeout = 60000)
public void testModifyColumnFamily() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
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.getAdmin().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(name.getMethodName());
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);
Procedure<?> result = procExec.getResult(procId1);
assertTrue(result.isFailed());
LOG.debug("Modify failed with exception: " + result.getException());
assertTrue(
ProcedureTestingUtility.getExceptionCause(result) instanceof InvalidFamilyOperationException);
}
@Test(timeout=60000)
public void testRecoveryAndDoubleExecutionOffline() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
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.getAdmin().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
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
MasterProcedureTestingUtility.validateColumnFamilyModification(UTIL.getHBaseCluster()
.getMaster(), tableName, cf3, columnDescriptor);
}
@Test(timeout = 60000)
public void testRecoveryAndDoubleExecutionOnline() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
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
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(procExec, procId);
MasterProcedureTestingUtility.validateColumnFamilyModification(UTIL.getHBaseCluster()
.getMaster(), tableName, cf4, columnDescriptor);
}
@Test(timeout = 60000)
public void testRollbackAndDoubleExecution() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName());
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));
int numberOfSteps = 0; // failing at pre operation
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(procExec, procId, numberOfSteps);
}
}

View File

@ -446,57 +446,6 @@ public class TestAccessController extends SecureTestUtil {
verifyDenied(truncateTable, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
}
@Test (timeout=180000)
public void testAddColumn() throws Exception {
final HColumnDescriptor hcd = new HColumnDescriptor("fam_new");
AccessTestAction action = new AccessTestAction() {
@Override
public Object run() throws Exception {
ACCESS_CONTROLLER.preAddColumnFamily(ObserverContextImpl.createAndPrepare(CP_ENV), TEST_TABLE,
hcd);
return null;
}
};
verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_CREATE, USER_OWNER, USER_GROUP_CREATE,
USER_GROUP_ADMIN);
verifyDenied(action, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
}
@Test (timeout=180000)
public void testModifyColumn() throws Exception {
final HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY);
hcd.setMaxVersions(10);
AccessTestAction action = new AccessTestAction() {
@Override
public Object run() throws Exception {
ACCESS_CONTROLLER.preModifyColumnFamily(ObserverContextImpl.createAndPrepare(CP_ENV),
TEST_TABLE, hcd);
return null;
}
};
verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_CREATE, USER_OWNER, USER_ADMIN_CF,
USER_GROUP_CREATE, USER_GROUP_ADMIN);
verifyDenied(action, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
}
@Test (timeout=180000)
public void testDeleteColumn() throws Exception {
AccessTestAction action = new AccessTestAction() {
@Override
public Object run() throws Exception {
ACCESS_CONTROLLER.preDeleteColumnFamily(ObserverContextImpl.createAndPrepare(CP_ENV),
TEST_TABLE, TEST_FAMILY);
return null;
}
};
verifyAllowed(action, SUPERUSER, USER_ADMIN, USER_CREATE, USER_OWNER, USER_ADMIN_CF,
USER_GROUP_CREATE, USER_GROUP_ADMIN);
verifyDenied(action, USER_RW, USER_RO, USER_NONE, USER_GROUP_READ, USER_GROUP_WRITE);
}
@Test (timeout=180000)
public void testTableDisable() throws Exception {
AccessTestAction disableTable = new AccessTestAction() {

View File

@ -522,38 +522,6 @@ public class TestWithDisabledAuthorization extends SecureTestUtil {
}
}, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
// preAddColumnFamily
verifyAllowed(new AccessTestAction() {
@Override
public Object run() throws Exception {
HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY2);
ACCESS_CONTROLLER.preAddColumnFamily(ObserverContextImpl.createAndPrepare(CP_ENV),
TEST_TABLE.getTableName(), hcd);
return null;
}
}, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
// preModifyColumnFamily
verifyAllowed(new AccessTestAction() {
@Override
public Object run() throws Exception {
HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY2);
ACCESS_CONTROLLER.preModifyColumnFamily(ObserverContextImpl.createAndPrepare(CP_ENV),
TEST_TABLE.getTableName(), hcd);
return null;
}
}, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
// preDeleteColumnFamily
verifyAllowed(new AccessTestAction() {
@Override
public Object run() throws Exception {
ACCESS_CONTROLLER.preDeleteColumnFamily(ObserverContextImpl.createAndPrepare(CP_ENV),
TEST_TABLE.getTableName(), TEST_FAMILY2);
return null;
}
}, SUPERUSER, USER_ADMIN, USER_RW, USER_RO, USER_OWNER, USER_CREATE, USER_QUAL, USER_NONE);
// preEnableTable
verifyAllowed(new AccessTestAction() {
@Override