HBASE-13455 Procedure V2 - master truncate table

This commit is contained in:
Matteo Bertozzi 2015-04-15 09:40:18 +01:00
parent d75326a797
commit 4788c6d1a8
9 changed files with 2381 additions and 92 deletions

View File

@ -75,6 +75,24 @@ message ModifyTableStateData {
required bool delete_column_family_in_modify = 4;
}
enum TruncateTableState {
TRUNCATE_TABLE_PRE_OPERATION = 1;
TRUNCATE_TABLE_REMOVE_FROM_META = 2;
TRUNCATE_TABLE_CLEAR_FS_LAYOUT = 3;
TRUNCATE_TABLE_CREATE_FS_LAYOUT = 4;
TRUNCATE_TABLE_ADD_TO_META = 5;
TRUNCATE_TABLE_ASSIGN_REGIONS = 6;
TRUNCATE_TABLE_POST_OPERATION = 7;
}
message TruncateTableStateData {
required UserInformation user_info = 1;
required bool preserve_splits = 2;
optional TableName table_name = 3;
optional TableSchema table_schema = 4;
repeated RegionInfo region_info = 5;
}
enum DeleteTableState {
DELETE_TABLE_PRE_OPERATION = 1;
DELETE_TABLE_REMOVE_FROM_META = 2;

View File

@ -88,7 +88,6 @@ import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
import org.apache.hadoop.hbase.master.handler.DispatchMergingRegionHandler;
import org.apache.hadoop.hbase.master.handler.TruncateTableHandler;
import org.apache.hadoop.hbase.master.procedure.AddColumnFamilyProcedure;
import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
import org.apache.hadoop.hbase.master.procedure.DeleteColumnFamilyProcedure;
@ -101,6 +100,7 @@ import org.apache.hadoop.hbase.master.procedure.ModifyColumnFamilyProcedure;
import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
import org.apache.hadoop.hbase.master.procedure.TruncateTableProcedure;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
@ -1599,9 +1599,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
cpHost.preTruncateTable(tableName);
}
LOG.info(getClientIdAuditPrefix() + " truncate " + tableName);
TruncateTableHandler handler = new TruncateTableHandler(tableName, this, this, preserveSplits);
handler.prepare();
handler.process();
long procId = this.procedureExecutor.submitProcedure(
new TruncateTableProcedure(procedureExecutor.getEnvironment(), tableName, preserveSplits));
ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
if (cpHost != null) {
cpHost.postTruncateTable(tableName);
}

View File

@ -106,14 +106,15 @@ public class DeleteTableProcedure
return Flow.NO_MORE_STATE;
}
preDelete(env);
// TODO: Move out... in the acquireLock()
LOG.debug("waiting for '" + getTableName() + "' regions in transition");
regions = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
assert regions != null && !regions.isEmpty() : "unexpected 0 regions";
ProcedureSyncWait.waitRegionInTransition(env, regions);
// Call coprocessors
preDelete(env);
setNextState(DeleteTableState.DELETE_TABLE_REMOVE_FROM_META);
break;
case DELETE_TABLE_REMOVE_FROM_META:

View File

@ -0,0 +1,291 @@
/**
* 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.InputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.exceptions.HBaseException;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableState;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
import org.apache.hadoop.security.UserGroupInformation;
@InterfaceAudience.Private
public class TruncateTableProcedure
extends StateMachineProcedure<MasterProcedureEnv, TruncateTableState>
implements TableProcedureInterface {
private static final Log LOG = LogFactory.getLog(TruncateTableProcedure.class);
private boolean preserveSplits;
private List<HRegionInfo> regions;
private UserGroupInformation user;
private HTableDescriptor hTableDescriptor;
private TableName tableName;
public TruncateTableProcedure() {
// Required by the Procedure framework to create the procedure on replay
}
public TruncateTableProcedure(final MasterProcedureEnv env, final TableName tableName,
boolean preserveSplits) throws IOException {
this.tableName = tableName;
this.preserveSplits = preserveSplits;
this.user = env.getRequestUser().getUGI();
}
@Override
protected Flow executeFromState(final MasterProcedureEnv env, TruncateTableState state) {
if (LOG.isTraceEnabled()) {
LOG.trace(this + " execute state=" + state);
}
try {
switch (state) {
case TRUNCATE_TABLE_PRE_OPERATION:
// Verify if we can truncate the table
if (!prepareTruncate(env)) {
assert isFailed() : "the truncate should have an exception here";
return Flow.NO_MORE_STATE;
}
// TODO: Move out... in the acquireLock()
LOG.debug("waiting for '" + getTableName() + "' regions in transition");
regions = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
assert regions != null && !regions.isEmpty() : "unexpected 0 regions";
ProcedureSyncWait.waitRegionInTransition(env, regions);
// Call coprocessors
preTruncate(env);
setNextState(TruncateTableState.TRUNCATE_TABLE_REMOVE_FROM_META);
break;
case TRUNCATE_TABLE_REMOVE_FROM_META:
hTableDescriptor = env.getMasterServices().getTableDescriptors()
.getDescriptor(tableName).getHTableDescriptor();
DeleteTableProcedure.deleteFromMeta(env, getTableName(), regions);
DeleteTableProcedure.deleteAssignmentState(env, getTableName());
setNextState(TruncateTableState.TRUNCATE_TABLE_CLEAR_FS_LAYOUT);
break;
case TRUNCATE_TABLE_CLEAR_FS_LAYOUT:
DeleteTableProcedure.deleteFromFs(env, getTableName(), regions, true);
if (!preserveSplits) {
// if we are not preserving splits, generate a new single region
regions = Arrays.asList(ModifyRegionUtils.createHRegionInfos(hTableDescriptor, null));
}
setNextState(TruncateTableState.TRUNCATE_TABLE_CREATE_FS_LAYOUT);
break;
case TRUNCATE_TABLE_CREATE_FS_LAYOUT:
regions = CreateTableProcedure.createFsLayout(env, hTableDescriptor, regions);
CreateTableProcedure.updateTableDescCache(env, getTableName());
setNextState(TruncateTableState.TRUNCATE_TABLE_ADD_TO_META);
break;
case TRUNCATE_TABLE_ADD_TO_META:
regions = CreateTableProcedure.addTableToMeta(env, hTableDescriptor, regions);
setNextState(TruncateTableState.TRUNCATE_TABLE_ASSIGN_REGIONS);
break;
case TRUNCATE_TABLE_ASSIGN_REGIONS:
CreateTableProcedure.assignRegions(env, getTableName(), regions);
setNextState(TruncateTableState.TRUNCATE_TABLE_POST_OPERATION);
hTableDescriptor = null;
regions = null;
break;
case TRUNCATE_TABLE_POST_OPERATION:
postTruncate(env);
LOG.debug("truncate '" + getTableName() + "' completed");
return Flow.NO_MORE_STATE;
default:
throw new UnsupportedOperationException("unhandled state=" + state);
}
} catch (HBaseException|IOException e) {
LOG.warn("Retriable error trying to truncate table=" + getTableName() + " state=" + state, e);
} catch (InterruptedException e) {
// if the interrupt is real, the executor will be stopped.
LOG.warn("Interrupted trying to truncate table=" + getTableName() + " state=" + state, e);
}
return Flow.HAS_MORE_STATE;
}
@Override
protected void rollbackState(final MasterProcedureEnv env, final TruncateTableState state) {
if (state == TruncateTableState.TRUNCATE_TABLE_PRE_OPERATION) {
// nothing to rollback, pre-truncate is just table-state checks.
// We can fail if the table does not exist or is not disabled.
return;
}
// The truncate doesn't have a rollback. The execution will succeed, at some point.
throw new UnsupportedOperationException("unhandled state=" + state);
}
@Override
protected TruncateTableState getState(final int stateId) {
return TruncateTableState.valueOf(stateId);
}
@Override
protected int getStateId(final TruncateTableState state) {
return state.getNumber();
}
@Override
protected TruncateTableState getInitialState() {
return TruncateTableState.TRUNCATE_TABLE_PRE_OPERATION;
}
@Override
public TableName getTableName() {
return tableName;
}
@Override
public TableOperationType getTableOperationType() {
return TableOperationType.EDIT;
}
@Override
public boolean abort(final MasterProcedureEnv env) {
// TODO: We may be able to abort if the procedure is not started yet.
return false;
}
@Override
protected boolean acquireLock(final MasterProcedureEnv env) {
if (!env.isInitialized()) return false;
return env.getProcedureQueue().tryAcquireTableWrite(getTableName(), "truncate table");
}
@Override
protected void releaseLock(final MasterProcedureEnv env) {
env.getProcedureQueue().releaseTableWrite(getTableName());
}
@Override
public void toStringClassDetails(StringBuilder sb) {
sb.append(getClass().getSimpleName());
sb.append(" (table=");
sb.append(getTableName());
sb.append(" preserveSplits=");
sb.append(preserveSplits);
sb.append(") user=");
sb.append(user);
}
@Override
public void serializeStateData(final OutputStream stream) throws IOException {
super.serializeStateData(stream);
MasterProcedureProtos.TruncateTableStateData.Builder state =
MasterProcedureProtos.TruncateTableStateData.newBuilder()
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(this.user))
.setPreserveSplits(preserveSplits);
if (hTableDescriptor != null) {
state.setTableSchema(hTableDescriptor.convert());
} else {
state.setTableName(ProtobufUtil.toProtoTableName(tableName));
}
if (regions != null) {
for (HRegionInfo hri: regions) {
state.addRegionInfo(HRegionInfo.convert(hri));
}
}
state.build().writeDelimitedTo(stream);
}
@Override
public void deserializeStateData(final InputStream stream) throws IOException {
super.deserializeStateData(stream);
MasterProcedureProtos.TruncateTableStateData state =
MasterProcedureProtos.TruncateTableStateData.parseDelimitedFrom(stream);
user = MasterProcedureUtil.toUserInfo(state.getUserInfo());
if (state.hasTableSchema()) {
hTableDescriptor = HTableDescriptor.convert(state.getTableSchema());
tableName = hTableDescriptor.getTableName();
} else {
tableName = ProtobufUtil.toTableName(state.getTableName());
}
preserveSplits = state.getPreserveSplits();
if (state.getRegionInfoCount() == 0) {
regions = null;
} else {
regions = new ArrayList<HRegionInfo>(state.getRegionInfoCount());
for (HBaseProtos.RegionInfo hri: state.getRegionInfoList()) {
regions.add(HRegionInfo.convert(hri));
}
}
}
private boolean prepareTruncate(final MasterProcedureEnv env) throws IOException {
try {
env.getMasterServices().checkTableModifiable(getTableName());
} catch (TableNotFoundException|TableNotDisabledException e) {
setFailure("master-truncate-table", e);
return false;
}
return true;
}
private boolean preTruncate(final MasterProcedureEnv env)
throws IOException, InterruptedException {
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
final TableName tableName = getTableName();
user.doAs(new PrivilegedExceptionAction<Void>() {
@Override
public Void run() throws Exception {
cpHost.preTruncateTableHandler(tableName);
return null;
}
});
}
return true;
}
private void postTruncate(final MasterProcedureEnv env)
throws IOException, InterruptedException {
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
final TableName tableName = getTableName();
user.doAs(new PrivilegedExceptionAction<Void>() {
@Override
public Void run() throws Exception {
cpHost.postTruncateTableHandler(tableName);
return null;
}
});
}
}
}

View File

@ -2290,6 +2290,18 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
return count;
}
/**
* Return the number of rows in the given table.
*/
public int countRows(final TableName tableName) throws IOException {
Table table = getConnection().getTable(tableName);
try {
return countRows(table);
} finally {
table.close();
}
}
/**
* Return an md5 digest of the entire contents of a table.
*/

View File

@ -35,6 +35,10 @@ import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.client.BufferedMutator;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.master.HMaster;
@ -44,6 +48,7 @@ import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.MD5Hash;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
@ -85,6 +90,7 @@ public class MasterProcedureTestingUtility {
final FileSystem fs = master.getMasterFileSystem().getFileSystem();
final Path tableDir = FSUtils.getTableDir(master.getMasterFileSystem().getRootDir(), tableName);
assertTrue(fs.exists(tableDir));
FSUtils.logFileSystemState(fs, tableDir, LOG);
List<Path> allRegionDirs = FSUtils.getRegionDirs(fs, tableDir);
for (int i = 0; i < regions.length; ++i) {
Path regionDir = new Path(tableDir, regions[i].getEncodedName());
@ -343,6 +349,43 @@ public class MasterProcedureTestingUtility {
assertTrue(hcfd.equals(columnDescriptor));
}
public static void loadData(final Connection connection, final TableName tableName,
int rows, final byte[][] splitKeys, final String... sfamilies) throws IOException {
byte[][] families = new byte[sfamilies.length][];
for (int i = 0; i < families.length; ++i) {
families[i] = Bytes.toBytes(sfamilies[i]);
}
BufferedMutator mutator = connection.getBufferedMutator(tableName);
// Ensure one row per region
assertTrue(rows >= splitKeys.length);
for (byte[] k: splitKeys) {
byte[] value = Bytes.add(Bytes.toBytes(System.currentTimeMillis()), k);
byte[] key = Bytes.add(k, Bytes.toBytes(MD5Hash.getMD5AsHex(value)));
mutator.mutate(createPut(families, key, value));
rows--;
}
// Add other extra rows. more rows, more files
while (rows-- > 0) {
byte[] value = Bytes.add(Bytes.toBytes(System.currentTimeMillis()), Bytes.toBytes(rows));
byte[] key = Bytes.toBytes(MD5Hash.getMD5AsHex(value));
mutator.mutate(createPut(families, key, value));
}
mutator.flush();
}
private static Put createPut(final byte[][] families, final byte[] key, final byte[] value) {
byte[] q = Bytes.toBytes("q");
Put put = new Put(key);
put.setDurability(Durability.SKIP_WAL);
for (byte[] family: families) {
put.add(family, q, value);
}
return put;
}
public static class InjectAbortOnLoadListener
implements ProcedureExecutor.ProcedureExecutorListener {
private final ProcedureExecutor<MasterProcedureEnv> procExec;

View File

@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTa
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableState;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableState;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableState;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -214,6 +215,67 @@ public class TestMasterFailoverWithProcedures {
UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
}
// ==========================================================================
// Test Truncate Table
// ==========================================================================
@Test(timeout=90000)
public void testTruncateWithFailover() throws Exception {
// TODO: Should we try every step? (master failover takes long time)
// It is already covered by TestTruncateTableProcedure
// but without the master restart, only the executor/store is restarted.
// Without Master restart we may not find bug in the procedure code
// like missing "wait" for resources to be available (e.g. RS)
testTruncateWithFailoverAtStep(true, TruncateTableState.TRUNCATE_TABLE_ADD_TO_META.ordinal());
}
private void testTruncateWithFailoverAtStep(final boolean preserveSplits, final int step)
throws Exception {
final TableName tableName = TableName.valueOf("testTruncateWithFailoverAtStep" + step);
// create the table
final String[] families = new String[] { "f1", "f2" };
final byte[][] splitKeys = new byte[][] {
Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
};
HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
getMasterProcedureExecutor(), tableName, splitKeys, families);
// load and verify that there are rows in the table
MasterProcedureTestingUtility.loadData(
UTIL.getConnection(), tableName, 100, splitKeys, families);
assertEquals(100, UTIL.countRows(tableName));
// disable the table
UTIL.getHBaseAdmin().disableTable(tableName);
ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the Truncate procedure && kill the executor
long procId = procExec.submitProcedure(
new TruncateTableProcedure(procExec.getEnvironment(), tableName, preserveSplits));
testRecoveryAndDoubleExecution(UTIL, procId, step, TruncateTableState.values());
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
UTIL.waitUntilAllRegionsAssigned(tableName);
// validate the table regions and layout
if (preserveSplits) {
assertEquals(1 + splitKeys.length, UTIL.getHBaseAdmin().getTableRegions(tableName).size());
} else {
regions = UTIL.getHBaseAdmin().getTableRegions(tableName).toArray(new HRegionInfo[1]);
assertEquals(1, regions.length);
}
MasterProcedureTestingUtility.validateTableCreation(
UTIL.getHBaseCluster().getMaster(), tableName, regions, families);
// verify that there are no rows in the table
assertEquals(0, UTIL.countRows(tableName));
// verify that the table is read/writable
MasterProcedureTestingUtility.loadData(
UTIL.getConnection(), tableName, 50, splitKeys, families);
assertEquals(50, UTIL.countRows(tableName));
}
// ==========================================================================
// Test Disable Table
// ==========================================================================

View File

@ -0,0 +1,246 @@
/**
* 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 org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureResult;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@Category({MasterTests.class, MediumTests.class})
public class TestTruncateTableProcedure {
private static final Log LOG = LogFactory.getLog(TestTruncateTableProcedure.class);
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static void setupConf(Configuration conf) {
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
}
@BeforeClass
public static void setupCluster() throws Exception {
setupConf(UTIL.getConfiguration());
UTIL.startMiniCluster(1);
}
@AfterClass
public static void cleanupTest() throws Exception {
try {
UTIL.shutdownMiniCluster();
} catch (Exception e) {
LOG.warn("failure shutting down cluster", e);
}
}
@Before
public void setup() throws Exception {
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
assertTrue("expected executor to be running", procExec.isRunning());
}
@After
public void tearDown() throws Exception {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
for (HTableDescriptor htd: UTIL.getHBaseAdmin().listTables()) {
LOG.info("Tear down, remove table=" + htd.getTableName());
UTIL.deleteTable(htd.getTableName());
}
}
@Test(timeout=60000)
public void testTruncateNotExistentTable() throws Exception {
final TableName tableName = TableName.valueOf("testTruncateNotExistentTable");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
long procId = ProcedureTestingUtility.submitAndWait(procExec,
new TruncateTableProcedure(procExec.getEnvironment(), tableName, true));
// Second delete should fail with TableNotFound
ProcedureResult result = procExec.getResult(procId);
assertTrue(result.isFailed());
LOG.debug("Truncate failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof TableNotFoundException);
}
@Test(timeout=60000)
public void testTruncateNotDisabledTable() throws Exception {
final TableName tableName = TableName.valueOf("testTruncateNotDisabledTable");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f");
long procId = ProcedureTestingUtility.submitAndWait(procExec,
new TruncateTableProcedure(procExec.getEnvironment(), tableName, false));
// Second delete should fail with TableNotDisabled
ProcedureResult result = procExec.getResult(procId);
assertTrue(result.isFailed());
LOG.debug("Truncate failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof TableNotDisabledException);
}
@Test(timeout=60000)
public void testSimpleTruncatePreserveSplits() throws Exception {
final TableName tableName = TableName.valueOf("testSimpleTruncatePreserveSplits");
testSimpleTruncate(tableName, true);
}
@Test(timeout=60000)
public void testSimpleTruncateNoPreserveSplits() throws Exception {
final TableName tableName = TableName.valueOf("testSimpleTruncateNoPreserveSplits");
testSimpleTruncate(tableName, false);
}
private void testSimpleTruncate(final TableName tableName, final boolean preserveSplits)
throws Exception {
final String[] families = new String[] { "f1", "f2" };
final byte[][] splitKeys = new byte[][] {
Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
};
HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
getMasterProcedureExecutor(), tableName, splitKeys, families);
// load and verify that there are rows in the table
MasterProcedureTestingUtility.loadData(
UTIL.getConnection(), tableName, 100, splitKeys, families);
assertEquals(100, UTIL.countRows(tableName));
// disable the table
UTIL.getHBaseAdmin().disableTable(tableName);
// truncate the table
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
long procId = ProcedureTestingUtility.submitAndWait(procExec,
new TruncateTableProcedure(procExec.getEnvironment(), tableName, preserveSplits));
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
UTIL.waitUntilAllRegionsAssigned(tableName);
// validate the table regions and layout
if (preserveSplits) {
assertEquals(1 + splitKeys.length, UTIL.getHBaseAdmin().getTableRegions(tableName).size());
} else {
regions = UTIL.getHBaseAdmin().getTableRegions(tableName).toArray(new HRegionInfo[1]);
assertEquals(1, regions.length);
}
MasterProcedureTestingUtility.validateTableCreation(
UTIL.getHBaseCluster().getMaster(), tableName, regions, families);
// verify that there are no rows in the table
assertEquals(0, UTIL.countRows(tableName));
// verify that the table is read/writable
MasterProcedureTestingUtility.loadData(
UTIL.getConnection(), tableName, 50, splitKeys, families);
assertEquals(50, UTIL.countRows(tableName));
}
@Test(timeout=60000)
public void testRecoveryAndDoubleExecutionPreserveSplits() throws Exception {
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecutionPreserveSplits");
testRecoveryAndDoubleExecution(tableName, true);
}
@Test(timeout=60000)
public void testRecoveryAndDoubleExecutionNoPreserveSplits() throws Exception {
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecutionNoPreserveSplits");
testRecoveryAndDoubleExecution(tableName, false);
}
private void testRecoveryAndDoubleExecution(final TableName tableName,
final boolean preserveSplits) throws Exception {
final String[] families = new String[] { "f1", "f2" };
// create the table
final byte[][] splitKeys = new byte[][] {
Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
};
HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
getMasterProcedureExecutor(), tableName, splitKeys, families);
// load and verify that there are rows in the table
MasterProcedureTestingUtility.loadData(
UTIL.getConnection(), tableName, 100, splitKeys, families);
assertEquals(100, UTIL.countRows(tableName));
// disable the table
UTIL.getHBaseAdmin().disableTable(tableName);
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the Truncate procedure && kill the executor
long procId = procExec.submitProcedure(
new TruncateTableProcedure(procExec.getEnvironment(), tableName, preserveSplits));
// Restart the executor and execute the step twice
// NOTE: the 7 (number of TruncateTableState steps) is hardcoded,
// so you have to look at this test at least once when you add a new step.
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(
procExec, procId, 7, TruncateTableState.values());
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
UTIL.waitUntilAllRegionsAssigned(tableName);
// validate the table regions and layout
if (preserveSplits) {
assertEquals(1 + splitKeys.length, UTIL.getHBaseAdmin().getTableRegions(tableName).size());
} else {
regions = UTIL.getHBaseAdmin().getTableRegions(tableName).toArray(new HRegionInfo[1]);
assertEquals(1, regions.length);
}
MasterProcedureTestingUtility.validateTableCreation(
UTIL.getHBaseCluster().getMaster(), tableName, regions, families);
// verify that there are no rows in the table
assertEquals(0, UTIL.countRows(tableName));
// verify that the table is read/writable
MasterProcedureTestingUtility.loadData(
UTIL.getConnection(), tableName, 50, splitKeys, families);
assertEquals(50, UTIL.countRows(tableName));
}
private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
}
}