HBASE-13455 Procedure V2 - master truncate table
This commit is contained in:
parent
7c5ae63850
commit
ec472fb18a
File diff suppressed because it is too large
Load Diff
|
@ -75,6 +75,24 @@ message ModifyTableStateData {
|
||||||
required bool delete_column_family_in_modify = 4;
|
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 {
|
enum DeleteTableState {
|
||||||
DELETE_TABLE_PRE_OPERATION = 1;
|
DELETE_TABLE_PRE_OPERATION = 1;
|
||||||
DELETE_TABLE_REMOVE_FROM_META = 2;
|
DELETE_TABLE_REMOVE_FROM_META = 2;
|
||||||
|
|
|
@ -91,7 +91,6 @@ import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
|
||||||
import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
|
import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
|
||||||
import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
|
import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
|
||||||
import org.apache.hadoop.hbase.master.handler.DispatchMergingRegionHandler;
|
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.AddColumnFamilyProcedure;
|
||||||
import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
|
import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
|
||||||
import org.apache.hadoop.hbase.master.procedure.DeleteColumnFamilyProcedure;
|
import org.apache.hadoop.hbase.master.procedure.DeleteColumnFamilyProcedure;
|
||||||
|
@ -104,6 +103,7 @@ import org.apache.hadoop.hbase.master.procedure.ModifyColumnFamilyProcedure;
|
||||||
import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
|
import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
|
||||||
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
|
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
|
||||||
import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
|
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.master.snapshot.SnapshotManager;
|
||||||
import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
|
import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
|
||||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||||
|
@ -1630,9 +1630,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
cpHost.preTruncateTable(tableName);
|
cpHost.preTruncateTable(tableName);
|
||||||
}
|
}
|
||||||
LOG.info(getClientIdAuditPrefix() + " truncate " + tableName);
|
LOG.info(getClientIdAuditPrefix() + " truncate " + tableName);
|
||||||
TruncateTableHandler handler = new TruncateTableHandler(tableName, this, this, preserveSplits);
|
|
||||||
handler.prepare();
|
long procId = this.procedureExecutor.submitProcedure(
|
||||||
handler.process();
|
new TruncateTableProcedure(procedureExecutor.getEnvironment(), tableName, preserveSplits));
|
||||||
|
ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
|
||||||
|
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
cpHost.postTruncateTable(tableName);
|
cpHost.postTruncateTable(tableName);
|
||||||
}
|
}
|
||||||
|
|
|
@ -106,14 +106,15 @@ public class DeleteTableProcedure
|
||||||
return Flow.NO_MORE_STATE;
|
return Flow.NO_MORE_STATE;
|
||||||
}
|
}
|
||||||
|
|
||||||
preDelete(env);
|
|
||||||
|
|
||||||
// TODO: Move out... in the acquireLock()
|
// TODO: Move out... in the acquireLock()
|
||||||
LOG.debug("waiting for '" + getTableName() + "' regions in transition");
|
LOG.debug("waiting for '" + getTableName() + "' regions in transition");
|
||||||
regions = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
|
regions = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
|
||||||
assert regions != null && !regions.isEmpty() : "unexpected 0 regions";
|
assert regions != null && !regions.isEmpty() : "unexpected 0 regions";
|
||||||
ProcedureSyncWait.waitRegionInTransition(env, regions);
|
ProcedureSyncWait.waitRegionInTransition(env, regions);
|
||||||
|
|
||||||
|
// Call coprocessors
|
||||||
|
preDelete(env);
|
||||||
|
|
||||||
setNextState(DeleteTableState.DELETE_TABLE_REMOVE_FROM_META);
|
setNextState(DeleteTableState.DELETE_TABLE_REMOVE_FROM_META);
|
||||||
break;
|
break;
|
||||||
case DELETE_TABLE_REMOVE_FROM_META:
|
case DELETE_TABLE_REMOVE_FROM_META:
|
||||||
|
|
|
@ -0,0 +1,290 @@
|
||||||
|
/**
|
||||||
|
* 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().get(tableName);
|
||||||
|
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;
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -2222,6 +2222,18 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
|
||||||
return count;
|
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.
|
* Return an md5 digest of the entire contents of a table.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -35,6 +35,10 @@ import org.apache.hadoop.hbase.RegionLocations;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.TableStateManager;
|
import org.apache.hadoop.hbase.TableStateManager;
|
||||||
|
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.MetaScanner;
|
import org.apache.hadoop.hbase.client.MetaScanner;
|
||||||
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
|
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
|
||||||
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
|
import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
|
||||||
|
@ -46,6 +50,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||||
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
|
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
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.assertEquals;
|
||||||
import static org.junit.Assert.assertFalse;
|
import static org.junit.Assert.assertFalse;
|
||||||
|
@ -87,6 +92,7 @@ public class MasterProcedureTestingUtility {
|
||||||
final FileSystem fs = master.getMasterFileSystem().getFileSystem();
|
final FileSystem fs = master.getMasterFileSystem().getFileSystem();
|
||||||
final Path tableDir = FSUtils.getTableDir(master.getMasterFileSystem().getRootDir(), tableName);
|
final Path tableDir = FSUtils.getTableDir(master.getMasterFileSystem().getRootDir(), tableName);
|
||||||
assertTrue(fs.exists(tableDir));
|
assertTrue(fs.exists(tableDir));
|
||||||
|
FSUtils.logFileSystemState(fs, tableDir, LOG);
|
||||||
List<Path> allRegionDirs = FSUtils.getRegionDirs(fs, tableDir);
|
List<Path> allRegionDirs = FSUtils.getRegionDirs(fs, tableDir);
|
||||||
for (int i = 0; i < regions.length; ++i) {
|
for (int i = 0; i < regions.length; ++i) {
|
||||||
Path regionDir = new Path(tableDir, regions[i].getEncodedName());
|
Path regionDir = new Path(tableDir, regions[i].getEncodedName());
|
||||||
|
@ -342,6 +348,43 @@ public class MasterProcedureTestingUtility {
|
||||||
assertTrue(hcfd.equals(columnDescriptor));
|
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
|
public static class InjectAbortOnLoadListener
|
||||||
implements ProcedureExecutor.ProcedureExecutorListener {
|
implements ProcedureExecutor.ProcedureExecutorListener {
|
||||||
private final ProcedureExecutor<MasterProcedureEnv> procExec;
|
private final ProcedureExecutor<MasterProcedureEnv> procExec;
|
||||||
|
|
|
@ -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.DeleteTableState;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableState;
|
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.EnableTableState;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableState;
|
||||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
|
@ -213,6 +214,67 @@ public class TestMasterFailoverWithProcedures {
|
||||||
UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
|
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
|
// Test Disable Table
|
||||||
// ==========================================================================
|
// ==========================================================================
|
||||||
|
|
|
@ -0,0 +1,245 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.hbase.master.procedure;
|
||||||
|
|
||||||
|
import 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.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(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();
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue