HBASE-13203 Procedure v2 - master create/delete table

This commit is contained in:
Matteo Bertozzi 2015-04-09 20:47:46 +01:00
parent 04246c6c3d
commit b5f1f98a25
29 changed files with 6280 additions and 54 deletions

View File

@ -249,8 +249,10 @@ public class MetaTableAccessor {
static Table getMetaHTable(final Connection connection)
throws IOException {
// We used to pass whole CatalogTracker in here, now we just pass in Connection
if (connection == null || connection.isClosed()) {
if (connection == null) {
throw new NullPointerException("No connection");
} else if (connection.isClosed()) {
throw new IOException("connection is closed");
}
return connection.getTable(TableName.META_TABLE_NAME);
}

View File

@ -0,0 +1,46 @@
/**
* 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.exceptions;
import java.io.IOException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
/**
* Exception thrown when a blocking operation times out.
*/
@SuppressWarnings("serial")
@InterfaceAudience.Private
public class TimeoutIOException extends IOException {
public TimeoutIOException() {
super();
}
public TimeoutIOException(final String message) {
super(message);
}
public TimeoutIOException(final String message, final Throwable t) {
super(message, t);
}
public TimeoutIOException(final Throwable t) {
super(t);
}
}

View File

@ -175,6 +175,7 @@
<include>LoadBalancer.proto</include>
<include>MapReduce.proto</include>
<include>Master.proto</include>
<include>MasterProcedure.proto</include>
<include>MultiRowMutation.proto</include>
<include>Procedure.proto</include>
<include>Quota.proto</include>

View File

@ -0,0 +1,74 @@
/**
* 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.
*/
option java_package = "org.apache.hadoop.hbase.protobuf.generated";
option java_outer_classname = "MasterProcedureProtos";
option java_generic_services = true;
option java_generate_equals_and_hash = true;
option optimize_for = SPEED;
import "HBase.proto";
import "RPC.proto";
// ============================================================================
// WARNING - Compatibility rules
// ============================================================================
// This .proto contains the data serialized by the master procedures.
// Each procedure has some state stored to know, which step were executed
// and what were the parameters or data created by the previous steps.
// new code should be able to handle the old format or at least fail cleanly
// triggering a rollback/cleanup.
//
// Procedures that are inheriting from a StateMachineProcedure have an enum:
// - Do not change the number of the 'State' enums.
// doing so, will cause executing the wrong 'step' on the pending
// procedures when they will be replayed.
// - Do not remove items from the enum, new code must be able to handle
// all the previous 'steps'. There may be pending procedure ready to be
// recovered replayed. alternative you can make sure that not-known state
// will result in a failure that will rollback the already executed steps.
// ============================================================================
enum CreateTableState {
CREATE_TABLE_PRE_OPERATION = 1;
CREATE_TABLE_WRITE_FS_LAYOUT = 2;
CREATE_TABLE_ADD_TO_META = 3;
CREATE_TABLE_ASSIGN_REGIONS = 4;
CREATE_TABLE_UPDATE_DESC_CACHE = 5;
CREATE_TABLE_POST_OPERATION = 6;
}
message CreateTableStateData {
required UserInformation user_info = 1;
required TableSchema table_schema = 2;
repeated RegionInfo region_info = 3;
}
enum DeleteTableState {
DELETE_TABLE_PRE_OPERATION = 1;
DELETE_TABLE_REMOVE_FROM_META = 2;
DELETE_TABLE_CLEAR_FS_LAYOUT = 3;
DELETE_TABLE_UPDATE_DESC_CACHE = 4;
DELETE_TABLE_UNASSIGN_REGIONS = 5;
DELETE_TABLE_POST_OPERATION = 6;
}
message DeleteTableStateData {
required UserInformation user_info = 1;
required TableName table_name = 2;
repeated RegionInfo region_info = 3;
}

View File

@ -314,6 +314,10 @@
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-protocol</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-procedure</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-client</artifactId>
@ -335,6 +339,12 @@
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-procedure</artifactId>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>commons-httpclient</groupId>
<artifactId>commons-httpclient</artifactId>

View File

@ -21,6 +21,7 @@ import java.net.InetAddress;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.VersionInfo;
public interface RpcCallContext extends Delayable {
/**
@ -57,4 +58,9 @@ public interface RpcCallContext extends Delayable {
* @return Address of remote client if a request is ongoing, else null
*/
InetAddress getRemoteAddress();
/**
* @return the client version info, or null if the information is not present
*/
VersionInfo getClientVersionInfo();
}

View File

@ -91,6 +91,7 @@ import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.VersionInfo;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.security.AccessDeniedException;
import org.apache.hadoop.hbase.security.AuthMethod;
@ -399,7 +400,7 @@ public class RpcServer implements RpcServerInterface {
// Set the exception as the result of the method invocation.
headerBuilder.setException(exceptionBuilder.build());
}
// Pass reservoir to buildCellBlock. Keep reference to returne so can add it back to the
// Pass reservoir to buildCellBlock. Keep reference to returne so can add it back to the
// reservoir when finished. This is hacky and the hack is not contained but benefits are
// high when we can avoid a big buffer allocation on each rpc.
this.cellBlock = ipcUtil.buildCellBlock(this.connection.codec,
@ -544,6 +545,11 @@ public class RpcServer implements RpcServerInterface {
public InetAddress getRemoteAddress() {
return remoteAddress;
}
@Override
public VersionInfo getClientVersionInfo() {
return connection.getVersionInfo();
}
}
/** Listens on the socket. Creates jobs for the handler threads*/
@ -1273,6 +1279,13 @@ public class RpcServer implements RpcServerInterface {
this.lastContact = lastContact;
}
public VersionInfo getVersionInfo() {
if (connectionHeader.hasVersionInfo()) {
return connectionHeader.getVersionInfo();
}
return null;
}
/* Return true if the connection has no outstanding rpc */
private boolean isIdle() {
return rpcCount.get() == 0;

View File

@ -90,8 +90,6 @@ import org.apache.hadoop.hbase.master.balancer.ClusterStatusChore;
import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
import org.apache.hadoop.hbase.master.handler.CreateTableHandler;
import org.apache.hadoop.hbase.master.handler.DeleteTableHandler;
import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
import org.apache.hadoop.hbase.master.handler.DispatchMergingRegionHandler;
import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
@ -100,11 +98,18 @@ import org.apache.hadoop.hbase.master.handler.TableAddFamilyHandler;
import org.apache.hadoop.hbase.master.handler.TableDeleteFamilyHandler;
import org.apache.hadoop.hbase.master.handler.TableModifyFamilyHandler;
import org.apache.hadoop.hbase.master.handler.TruncateTableHandler;
import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
import org.apache.hadoop.hbase.master.procedure.DeleteTableProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
import org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionServerInfo;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
@ -123,6 +128,7 @@ import org.apache.hadoop.hbase.util.EncryptionTest;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.VersionInfo;
@ -290,6 +296,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
// it is assigned after 'initialized' guard set to true, so should be volatile
private volatile MasterQuotaManager quotaManager;
private ProcedureExecutor<MasterProcedureEnv> procedureExecutor;
private WALProcedureStore procedureStore;
// handle table states
private TableStateManager tableStateManager;
@ -1002,6 +1011,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
// Any time changing this maxThreads to > 1, pls see the comment at
// AccessController#postCreateTableHandler
this.service.startExecutorService(ExecutorType.MASTER_TABLE_OPERATIONS, 1);
startProcedureExecutor();
// Start log cleaner thread
int cleanerInterval = conf.getInt("hbase.master.cleaner.interval", 60 * 1000);
@ -1022,6 +1032,12 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
}
@Override
protected void sendShutdownInterrupt() {
super.sendShutdownInterrupt();
stopProcedureExecutor();
}
@Override
protected void stopServiceThreads() {
if (masterJettyServer != null) {
@ -1034,6 +1050,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
super.stopServiceThreads();
stopChores();
// Wait for all the remaining region servers to report in IFF we were
// running a cluster shutdown AND we were NOT aborting.
if (!isAborted() && this.serverManager != null &&
@ -1054,6 +1071,34 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
if (this.mpmHost != null) this.mpmHost.stop("server shutting down.");
}
private void startProcedureExecutor() throws IOException {
final MasterProcedureEnv procEnv = new MasterProcedureEnv(this);
final Path logDir = new Path(fileSystemManager.getRootDir(),
MasterProcedureConstants.MASTER_PROCEDURE_LOGDIR);
procedureStore = new WALProcedureStore(conf, fileSystemManager.getFileSystem(), logDir,
new MasterProcedureEnv.WALStoreLeaseRecovery(this));
procedureStore.registerListener(new MasterProcedureEnv.MasterProcedureStoreListener(this));
procedureExecutor = new ProcedureExecutor(conf, procEnv, procedureStore,
procEnv.getProcedureQueue());
final int numThreads = conf.getInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS,
Math.max(Runtime.getRuntime().availableProcessors(),
MasterProcedureConstants.DEFAULT_MIN_MASTER_PROCEDURE_THREADS));
procedureStore.start(numThreads);
procedureExecutor.start(numThreads);
}
private void stopProcedureExecutor() {
if (procedureExecutor != null) {
procedureExecutor.stop();
}
if (procedureStore != null) {
procedureStore.stop(isAborted());
}
}
private void stopChores() {
if (this.balancerChore != null) {
this.balancerChore.cancel(true);
@ -1290,7 +1335,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
String namespace = hTableDescriptor.getTableName().getNamespaceAsString();
ensureNamespaceExists(namespace);
HRegionInfo[] newRegions = getHRegionInfos(hTableDescriptor, splitKeys);
HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(hTableDescriptor, splitKeys);
checkInitialized();
sanityCheckTableDescriptor(hTableDescriptor);
this.quotaManager.checkNamespaceTableAndRegionQuota(hTableDescriptor.getTableName(),
@ -1299,13 +1344,22 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
cpHost.preCreateTable(hTableDescriptor, newRegions);
}
LOG.info(getClientIdAuditPrefix() + " create " + hTableDescriptor);
this.service.submit(new CreateTableHandler(this,
this.fileSystemManager, hTableDescriptor, conf,
newRegions, this).prepare());
// TODO: We can handle/merge duplicate requests, and differentiate the case of
// TableExistsException by saying if the schema is the same or not.
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch();
long procId = this.procedureExecutor.submitProcedure(
new CreateTableProcedure(procedureExecutor.getEnvironment(),
hTableDescriptor, newRegions, latch));
latch.await();
if (cpHost != null) {
cpHost.postCreateTable(hTableDescriptor, newRegions);
}
// TODO: change the interface to return the procId,
// and add it to the response protobuf.
//return procId;
}
/**
@ -1512,29 +1566,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
RegionCoprocessorHost.testTableCoprocessorAttrs(conf, htd);
}
private HRegionInfo[] getHRegionInfos(HTableDescriptor hTableDescriptor,
byte[][] splitKeys) {
long regionId = System.currentTimeMillis();
HRegionInfo[] hRegionInfos = null;
if (splitKeys == null || splitKeys.length == 0) {
hRegionInfos = new HRegionInfo[]{new HRegionInfo(hTableDescriptor.getTableName(), null, null,
false, regionId)};
} else {
int numRegions = splitKeys.length + 1;
hRegionInfos = new HRegionInfo[numRegions];
byte[] startKey = null;
byte[] endKey = null;
for (int i = 0; i < numRegions; i++) {
endKey = (i == splitKeys.length) ? null : splitKeys[i];
hRegionInfos[i] =
new HRegionInfo(hTableDescriptor.getTableName(), startKey, endKey,
false, regionId);
startKey = endKey;
}
}
return hRegionInfos;
}
private static boolean isCatalogTable(final TableName tableName) {
return tableName.equals(TableName.META_TABLE_NAME);
}
@ -1546,10 +1577,20 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
cpHost.preDeleteTable(tableName);
}
LOG.info(getClientIdAuditPrefix() + " delete " + tableName);
this.service.submit(new DeleteTableHandler(tableName, this, this).prepare());
// TODO: We can handle/merge duplicate request
ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch();
long procId = this.procedureExecutor.submitProcedure(
new DeleteTableProcedure(procedureExecutor.getEnvironment(), tableName, latch));
latch.await();
if (cpHost != null) {
cpHost.postDeleteTable(tableName);
}
// TODO: change the interface to return the procId,
// and add it to the response protobuf.
//return procId;
}
@Override
@ -1850,6 +1891,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
return quotaManager;
}
@Override
public ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
return procedureExecutor;
}
@Override
public ServerName getServerName() {
return this.serverName;

View File

@ -31,6 +31,8 @@ import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
@ -81,6 +83,11 @@ public interface MasterServices extends Server {
*/
MasterQuotaManager getMasterQuotaManager();
/**
* @return Master's instance of {@link ProcedureExecutor}
*/
ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor();
/**
* Check table is modifiable; i.e. exists and is offline.
* @param tableName Name of table to check.

View File

@ -49,7 +49,7 @@ import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.master.handler.CreateTableHandler;
import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.util.Bytes;
@ -231,18 +231,15 @@ public class TableNamespaceManager {
}
private void createNamespaceTable(MasterServices masterServices) throws IOException {
HRegionInfo newRegions[] = new HRegionInfo[]{
HRegionInfo[] newRegions = new HRegionInfo[]{
new HRegionInfo(HTableDescriptor.NAMESPACE_TABLEDESC.getTableName(), null, null)};
//we need to create the table this way to bypass
//checkInitialized
masterServices.getExecutorService()
.submit(new CreateTableHandler(masterServices,
masterServices.getMasterFileSystem(),
HTableDescriptor.NAMESPACE_TABLEDESC,
masterServices.getConfiguration(),
newRegions,
masterServices).prepare());
// we need to create the table this way to bypass checkInitialized
masterServices.getMasterProcedureExecutor()
.submitProcedure(new CreateTableProcedure(
masterServices.getMasterProcedureExecutor().getEnvironment(),
HTableDescriptor.NAMESPACE_TABLEDESC,
newRegions));
}
/**

View File

@ -0,0 +1,442 @@
/**
* 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.OutputStream;
import java.io.IOException;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
import org.apache.hadoop.security.UserGroupInformation;
import com.google.common.collect.Lists;
@InterfaceAudience.Private
public class CreateTableProcedure
extends StateMachineProcedure<MasterProcedureEnv, CreateTableState>
implements TableProcedureInterface {
private static final Log LOG = LogFactory.getLog(CreateTableProcedure.class);
private final AtomicBoolean aborted = new AtomicBoolean(false);
// used for compatibility with old clients
private final ProcedurePrepareLatch syncLatch;
private HTableDescriptor hTableDescriptor;
private List<HRegionInfo> newRegions;
private UserGroupInformation user;
public CreateTableProcedure() {
// Required by the Procedure framework to create the procedure on replay
syncLatch = null;
}
public CreateTableProcedure(final MasterProcedureEnv env,
final HTableDescriptor hTableDescriptor, final HRegionInfo[] newRegions)
throws IOException {
this(env, hTableDescriptor, newRegions, null);
}
public CreateTableProcedure(final MasterProcedureEnv env,
final HTableDescriptor hTableDescriptor, final HRegionInfo[] newRegions,
final ProcedurePrepareLatch syncLatch)
throws IOException {
this.hTableDescriptor = hTableDescriptor;
this.newRegions = newRegions != null ? Lists.newArrayList(newRegions) : null;
this.user = env.getRequestUser().getUGI();
// used for compatibility with clients without procedures
// they need a sync TableExistsException
this.syncLatch = syncLatch;
}
@Override
protected Flow executeFromState(final MasterProcedureEnv env, final CreateTableState state) {
if (LOG.isTraceEnabled()) {
LOG.trace(this + " execute state=" + state);
}
try {
switch (state) {
case CREATE_TABLE_PRE_OPERATION:
// Verify if we can create the table
boolean exists = !prepareCreate(env);
ProcedurePrepareLatch.releaseLatch(syncLatch, this);
if (exists) {
assert isFailed() : "the delete should have an exception here";
return Flow.NO_MORE_STATE;
}
preCreate(env);
setNextState(CreateTableState.CREATE_TABLE_WRITE_FS_LAYOUT);
break;
case CREATE_TABLE_WRITE_FS_LAYOUT:
newRegions = createFsLayout(env, hTableDescriptor, newRegions);
setNextState(CreateTableState.CREATE_TABLE_ADD_TO_META);
break;
case CREATE_TABLE_ADD_TO_META:
newRegions = addTableToMeta(env, hTableDescriptor, newRegions);
setNextState(CreateTableState.CREATE_TABLE_ASSIGN_REGIONS);
break;
case CREATE_TABLE_ASSIGN_REGIONS:
assignRegions(env, getTableName(), newRegions);
setNextState(CreateTableState.CREATE_TABLE_UPDATE_DESC_CACHE);
break;
case CREATE_TABLE_UPDATE_DESC_CACHE:
updateTableDescCache(env, getTableName());
setNextState(CreateTableState.CREATE_TABLE_POST_OPERATION);
break;
case CREATE_TABLE_POST_OPERATION:
postCreate(env);
return Flow.NO_MORE_STATE;
default:
throw new UnsupportedOperationException("unhandled state=" + state);
}
} catch (InterruptedException|IOException e) {
LOG.error("Error trying to create table=" + getTableName() + " state=" + state, e);
setFailure("master-create-table", e);
}
return Flow.HAS_MORE_STATE;
}
@Override
protected void rollbackState(final MasterProcedureEnv env, final CreateTableState state)
throws IOException {
if (LOG.isTraceEnabled()) {
LOG.trace(this + " rollback state=" + state);
}
try {
switch (state) {
case CREATE_TABLE_POST_OPERATION:
break;
case CREATE_TABLE_UPDATE_DESC_CACHE:
DeleteTableProcedure.deleteTableDescriptorCache(env, getTableName());
break;
case CREATE_TABLE_ASSIGN_REGIONS:
DeleteTableProcedure.deleteAssignmentState(env, getTableName());
break;
case CREATE_TABLE_ADD_TO_META:
DeleteTableProcedure.deleteFromMeta(env, getTableName(), newRegions);
break;
case CREATE_TABLE_WRITE_FS_LAYOUT:
DeleteTableProcedure.deleteFromFs(env, getTableName(), newRegions, false);
break;
case CREATE_TABLE_PRE_OPERATION:
DeleteTableProcedure.deleteTableStates(env, getTableName());
// TODO-MAYBE: call the deleteTable coprocessor event?
ProcedurePrepareLatch.releaseLatch(syncLatch, this);
break;
default:
throw new UnsupportedOperationException("unhandled state=" + state);
}
} catch (IOException e) {
// This will be retried. Unless there is a bug in the code,
// this should be just a "temporary error" (e.g. network down)
LOG.warn("Failed rollback attempt step=" + state + " table=" + getTableName(), e);
throw e;
}
}
@Override
protected CreateTableState getState(final int stateId) {
return CreateTableState.valueOf(stateId);
}
@Override
protected int getStateId(final CreateTableState state) {
return state.getNumber();
}
@Override
protected CreateTableState getInitialState() {
return CreateTableState.CREATE_TABLE_PRE_OPERATION;
}
@Override
protected void setNextState(final CreateTableState state) {
if (aborted.get()) {
setAbortFailure("create-table", "abort requested");
} else {
super.setNextState(state);
}
}
@Override
public TableName getTableName() {
return hTableDescriptor.getTableName();
}
@Override
public TableOperationType getTableOperationType() {
return TableOperationType.CREATE;
}
@Override
public boolean abort(final MasterProcedureEnv env) {
aborted.set(true);
return true;
}
@Override
public void toStringClassDetails(StringBuilder sb) {
sb.append(getClass().getSimpleName());
sb.append(" (table=");
sb.append(getTableName());
sb.append(") user=");
sb.append(user);
}
@Override
public void serializeStateData(final OutputStream stream) throws IOException {
super.serializeStateData(stream);
MasterProcedureProtos.CreateTableStateData.Builder state =
MasterProcedureProtos.CreateTableStateData.newBuilder()
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(this.user))
.setTableSchema(hTableDescriptor.convert());
if (newRegions != null) {
for (HRegionInfo hri: newRegions) {
state.addRegionInfo(HRegionInfo.convert(hri));
}
}
state.build().writeDelimitedTo(stream);
}
@Override
public void deserializeStateData(final InputStream stream) throws IOException {
super.deserializeStateData(stream);
MasterProcedureProtos.CreateTableStateData state =
MasterProcedureProtos.CreateTableStateData.parseDelimitedFrom(stream);
user = MasterProcedureUtil.toUserInfo(state.getUserInfo());
hTableDescriptor = HTableDescriptor.convert(state.getTableSchema());
if (state.getRegionInfoCount() == 0) {
newRegions = null;
} else {
newRegions = new ArrayList<HRegionInfo>(state.getRegionInfoCount());
for (HBaseProtos.RegionInfo hri: state.getRegionInfoList()) {
newRegions.add(HRegionInfo.convert(hri));
}
}
}
@Override
protected boolean acquireLock(final MasterProcedureEnv env) {
return env.getProcedureQueue().tryAcquireTableWrite(getTableName(), "create table");
}
@Override
protected void releaseLock(final MasterProcedureEnv env) {
env.getProcedureQueue().releaseTableWrite(getTableName());
}
private boolean prepareCreate(final MasterProcedureEnv env) throws IOException {
final TableName tableName = getTableName();
if (MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), tableName)) {
setFailure("master-create-table", new TableExistsException(getTableName()));
return false;
}
return true;
}
private void preCreate(final MasterProcedureEnv env)
throws IOException, InterruptedException {
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
final HRegionInfo[] regions = newRegions == null ? null :
newRegions.toArray(new HRegionInfo[newRegions.size()]);
user.doAs(new PrivilegedExceptionAction<Void>() {
@Override
public Void run() throws Exception {
cpHost.preCreateTableHandler(hTableDescriptor, regions);
return null;
}
});
}
}
private void postCreate(final MasterProcedureEnv env)
throws IOException, InterruptedException {
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
final HRegionInfo[] regions = (newRegions == null) ? null :
newRegions.toArray(new HRegionInfo[newRegions.size()]);
user.doAs(new PrivilegedExceptionAction<Void>() {
@Override
public Void run() throws Exception {
cpHost.postCreateTableHandler(hTableDescriptor, regions);
return null;
}
});
}
}
protected interface CreateHdfsRegions {
List<HRegionInfo> createHdfsRegions(final MasterProcedureEnv env,
final Path tableRootDir, final TableName tableName,
final List<HRegionInfo> newRegions) throws IOException;
}
protected static List<HRegionInfo> createFsLayout(final MasterProcedureEnv env,
final HTableDescriptor hTableDescriptor, final List<HRegionInfo> newRegions)
throws IOException {
return createFsLayout(env, hTableDescriptor, newRegions, new CreateHdfsRegions() {
@Override
public List<HRegionInfo> createHdfsRegions(final MasterProcedureEnv env,
final Path tableRootDir, final TableName tableName,
final List<HRegionInfo> newRegions) throws IOException {
HRegionInfo[] regions = newRegions != null ?
newRegions.toArray(new HRegionInfo[newRegions.size()]) : null;
return ModifyRegionUtils.createRegions(env.getMasterConfiguration(),
tableRootDir, hTableDescriptor, regions, null);
}
});
}
protected static List<HRegionInfo> createFsLayout(final MasterProcedureEnv env,
final HTableDescriptor hTableDescriptor, List<HRegionInfo> newRegions,
final CreateHdfsRegions hdfsRegionHandler) throws IOException {
final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
final Path tempdir = mfs.getTempDir();
// 1. Create Table Descriptor
// using a copy of descriptor, table will be created enabling first
TableDescriptor underConstruction = new TableDescriptor(hTableDescriptor);
final Path tempTableDir = FSUtils.getTableDir(tempdir, hTableDescriptor.getTableName());
((FSTableDescriptors)(env.getMasterServices().getTableDescriptors()))
.createTableDescriptorForTableDirectory(
tempTableDir, underConstruction, false);
// 2. Create Regions
newRegions = hdfsRegionHandler.createHdfsRegions(env, tempdir,
hTableDescriptor.getTableName(), newRegions);
// 3. Move Table temp directory to the hbase root location
final Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), hTableDescriptor.getTableName());
FileSystem fs = mfs.getFileSystem();
if (!fs.delete(tableDir, true) && fs.exists(tableDir)) {
throw new IOException("Couldn't delete " + tableDir);
}
if (!fs.rename(tempTableDir, tableDir)) {
throw new IOException("Unable to move table from temp=" + tempTableDir +
" to hbase root=" + tableDir);
}
return newRegions;
}
protected static List<HRegionInfo> addTableToMeta(final MasterProcedureEnv env,
final HTableDescriptor hTableDescriptor,
final List<HRegionInfo> regions) throws IOException {
if (regions != null && regions.size() > 0) {
ProcedureSyncWait.waitMetaRegions(env);
// Add regions to META
addRegionsToMeta(env, hTableDescriptor, regions);
// Add replicas if needed
List<HRegionInfo> newRegions = addReplicas(env, hTableDescriptor, regions);
// Setup replication for region replicas if needed
if (hTableDescriptor.getRegionReplication() > 1) {
ServerRegionReplicaUtil.setupRegionReplicaReplication(env.getMasterConfiguration());
}
return newRegions;
}
return regions;
}
/**
* Create any replicas for the regions (the default replicas that was
* already created is passed to the method)
* @param hTableDescriptor descriptor to use
* @param regions default replicas
* @return the combined list of default and non-default replicas
*/
private static List<HRegionInfo> addReplicas(final MasterProcedureEnv env,
final HTableDescriptor hTableDescriptor,
final List<HRegionInfo> regions) {
int numRegionReplicas = hTableDescriptor.getRegionReplication() - 1;
if (numRegionReplicas <= 0) {
return regions;
}
List<HRegionInfo> hRegionInfos =
new ArrayList<HRegionInfo>((numRegionReplicas+1)*regions.size());
for (int i = 0; i < regions.size(); i++) {
for (int j = 1; j <= numRegionReplicas; j++) {
hRegionInfos.add(RegionReplicaUtil.getRegionInfoForReplica(regions.get(i), j));
}
}
hRegionInfos.addAll(regions);
return hRegionInfos;
}
protected static void assignRegions(final MasterProcedureEnv env,
final TableName tableName, final List<HRegionInfo> regions) throws IOException {
ProcedureSyncWait.waitRegionServers(env);
// Trigger immediate assignment of the regions in round-robin fashion
final AssignmentManager assignmentManager = env.getMasterServices().getAssignmentManager();
ModifyRegionUtils.assignRegions(assignmentManager, regions);
// Enable table
assignmentManager.getTableStateManager()
.setTableState(tableName, TableState.State.ENABLED);
}
/**
* Add the specified set of regions to the hbase:meta table.
*/
protected static void addRegionsToMeta(final MasterProcedureEnv env,
final HTableDescriptor hTableDescriptor,
final List<HRegionInfo> regionInfos) throws IOException {
MetaTableAccessor.addRegionsToMeta(env.getMasterServices().getConnection(),
regionInfos, hTableDescriptor.getRegionReplication());
}
protected static void updateTableDescCache(final MasterProcedureEnv env,
final TableName tableName) throws IOException {
env.getMasterServices().getTableDescriptors().get(tableName);
}
}

View File

@ -0,0 +1,420 @@
/**
* 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.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
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.backup.HFileArchiver;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.exceptions.HBaseException;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableState;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.security.UserGroupInformation;
@InterfaceAudience.Private
public class DeleteTableProcedure
extends StateMachineProcedure<MasterProcedureEnv, DeleteTableState>
implements TableProcedureInterface {
private static final Log LOG = LogFactory.getLog(DeleteTableProcedure.class);
private List<HRegionInfo> regions;
private UserGroupInformation user;
private TableName tableName;
// used for compatibility with old clients
private final ProcedurePrepareLatch syncLatch;
public DeleteTableProcedure() {
// Required by the Procedure framework to create the procedure on replay
syncLatch = null;
}
public DeleteTableProcedure(final MasterProcedureEnv env, final TableName tableName)
throws IOException {
this(env, tableName, null);
}
public DeleteTableProcedure(final MasterProcedureEnv env, final TableName tableName,
final ProcedurePrepareLatch syncLatch) throws IOException {
this.tableName = tableName;
this.user = env.getRequestUser().getUGI();
// used for compatibility with clients without procedures
// they need a sync TableNotFoundException, TableNotDisabledException, ...
this.syncLatch = syncLatch;
}
@Override
protected Flow executeFromState(final MasterProcedureEnv env, DeleteTableState state) {
if (LOG.isTraceEnabled()) {
LOG.trace(this + " execute state=" + state);
}
try {
switch (state) {
case DELETE_TABLE_PRE_OPERATION:
// Verify if we can delete the table
boolean deletable = prepareDelete(env);
ProcedurePrepareLatch.releaseLatch(syncLatch, this);
if (!deletable) {
assert isFailed() : "the delete should have an exception here";
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);
setNextState(DeleteTableState.DELETE_TABLE_REMOVE_FROM_META);
break;
case DELETE_TABLE_REMOVE_FROM_META:
LOG.debug("delete '" + getTableName() + "' regions from META");
DeleteTableProcedure.deleteFromMeta(env, getTableName(), regions);
setNextState(DeleteTableState.DELETE_TABLE_CLEAR_FS_LAYOUT);
break;
case DELETE_TABLE_CLEAR_FS_LAYOUT:
LOG.debug("delete '" + getTableName() + "' from filesystem");
DeleteTableProcedure.deleteFromFs(env, getTableName(), regions, true);
setNextState(DeleteTableState.DELETE_TABLE_UPDATE_DESC_CACHE);
break;
case DELETE_TABLE_UPDATE_DESC_CACHE:
LOG.debug("delete '" + getTableName() + "' descriptor");
DeleteTableProcedure.deleteTableDescriptorCache(env, getTableName());
setNextState(DeleteTableState.DELETE_TABLE_UNASSIGN_REGIONS);
break;
case DELETE_TABLE_UNASSIGN_REGIONS:
LOG.debug("delete '" + getTableName() + "' assignment state");
DeleteTableProcedure.deleteAssignmentState(env, getTableName());
setNextState(DeleteTableState.DELETE_TABLE_POST_OPERATION);
break;
case DELETE_TABLE_POST_OPERATION:
postDelete(env);
LOG.debug("delete '" + getTableName() + "' completed");
return Flow.NO_MORE_STATE;
default:
throw new UnsupportedOperationException("unhandled state=" + state);
}
} catch (HBaseException|IOException e) {
LOG.warn("Retriable error trying to delete table=" + getTableName() + " state=" + state, e);
} catch (InterruptedException e) {
// if the interrupt is real, the executor will be stopped.
LOG.warn("Interrupted trying to delete table=" + getTableName() + " state=" + state, e);
}
return Flow.HAS_MORE_STATE;
}
@Override
protected void rollbackState(final MasterProcedureEnv env, final DeleteTableState state) {
if (state == DeleteTableState.DELETE_TABLE_PRE_OPERATION) {
// nothing to rollback, pre-delete is just table-state checks.
// We can fail if the table does not exist or is not disabled.
ProcedurePrepareLatch.releaseLatch(syncLatch, this);
return;
}
// The delete doesn't have a rollback. The execution will succeed, at some point.
throw new UnsupportedOperationException("unhandled state=" + state);
}
@Override
protected DeleteTableState getState(final int stateId) {
return DeleteTableState.valueOf(stateId);
}
@Override
protected int getStateId(final DeleteTableState state) {
return state.getNumber();
}
@Override
protected DeleteTableState getInitialState() {
return DeleteTableState.DELETE_TABLE_PRE_OPERATION;
}
@Override
public TableName getTableName() {
return tableName;
}
@Override
public TableOperationType getTableOperationType() {
return TableOperationType.DELETE;
}
@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(), "delete 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(") user=");
sb.append(user);
}
@Override
public void serializeStateData(final OutputStream stream) throws IOException {
super.serializeStateData(stream);
MasterProcedureProtos.DeleteTableStateData.Builder state =
MasterProcedureProtos.DeleteTableStateData.newBuilder()
.setUserInfo(MasterProcedureUtil.toProtoUserInfo(this.user))
.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.DeleteTableStateData state =
MasterProcedureProtos.DeleteTableStateData.parseDelimitedFrom(stream);
user = MasterProcedureUtil.toUserInfo(state.getUserInfo());
tableName = ProtobufUtil.toTableName(state.getTableName());
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 prepareDelete(final MasterProcedureEnv env) throws IOException {
try {
env.getMasterServices().checkTableModifiable(tableName);
} catch (TableNotFoundException|TableNotDisabledException e) {
setFailure("master-delete-table", e);
return false;
}
return true;
}
private boolean preDelete(final MasterProcedureEnv env)
throws IOException, InterruptedException {
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
final TableName tableName = this.tableName;
user.doAs(new PrivilegedExceptionAction<Void>() {
@Override
public Void run() throws Exception {
cpHost.preDeleteTableHandler(tableName);
return null;
}
});
}
return true;
}
private void postDelete(final MasterProcedureEnv env)
throws IOException, InterruptedException {
deleteTableStates(env, tableName);
final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
if (cpHost != null) {
final TableName tableName = this.tableName;
user.doAs(new PrivilegedExceptionAction<Void>() {
@Override
public Void run() throws Exception {
cpHost.postDeleteTableHandler(tableName);
return null;
}
});
}
}
protected static void deleteFromFs(final MasterProcedureEnv env,
final TableName tableName, final List<HRegionInfo> regions,
final boolean archive) throws IOException {
final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
final FileSystem fs = mfs.getFileSystem();
final Path tempdir = mfs.getTempDir();
final Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), tableName);
final Path tempTableDir = FSUtils.getTableDir(tempdir, tableName);
if (fs.exists(tableDir)) {
// Ensure temp exists
if (!fs.exists(tempdir) && !fs.mkdirs(tempdir)) {
throw new IOException("HBase temp directory '" + tempdir + "' creation failure.");
}
// Ensure parent exists
if (!fs.exists(tempTableDir.getParent()) && !fs.mkdirs(tempTableDir.getParent())) {
throw new IOException("HBase temp directory '" + tempdir + "' creation failure.");
}
// Move the table in /hbase/.tmp
if (!fs.rename(tableDir, tempTableDir)) {
if (fs.exists(tempTableDir)) {
// TODO
// what's in this dir? something old? probably something manual from the user...
// let's get rid of this stuff...
FileStatus[] files = fs.listStatus(tempdir);
if (files != null && files.length > 0) {
for (int i = 0; i < files.length; ++i) {
if (!files[i].isDir()) continue;
HFileArchiver.archiveRegion(fs, mfs.getRootDir(), tempTableDir, files[i].getPath());
}
}
fs.delete(tempdir, true);
}
throw new IOException("Unable to move '" + tableDir + "' to temp '" + tempTableDir + "'");
}
}
// Archive regions from FS (temp directory)
if (archive) {
for (HRegionInfo hri : regions) {
LOG.debug("Archiving region " + hri.getRegionNameAsString() + " from FS");
HFileArchiver.archiveRegion(fs, mfs.getRootDir(),
tempTableDir, HRegion.getRegionDir(tempTableDir, hri.getEncodedName()));
}
LOG.debug("Table '" + tableName + "' archived!");
}
// Delete table directory from FS (temp directory)
if (!fs.delete(tempTableDir, true) && fs.exists(tempTableDir)) {
throw new IOException("Couldn't delete " + tempTableDir);
}
}
/**
* There may be items for this table still up in hbase:meta in the case where the
* info:regioninfo column was empty because of some write error. Remove ALL rows from hbase:meta
* that have to do with this table. See HBASE-12980.
* @throws IOException
*/
private static void cleanAnyRemainingRows(final MasterProcedureEnv env,
final TableName tableName) throws IOException {
ClusterConnection connection = env.getMasterServices().getConnection();
Scan tableScan = MetaTableAccessor.getScanForTableName(connection, tableName);
try (Table metaTable =
connection.getTable(TableName.META_TABLE_NAME)) {
List<Delete> deletes = new ArrayList<Delete>();
try (ResultScanner resScanner = metaTable.getScanner(tableScan)) {
for (Result result : resScanner) {
deletes.add(new Delete(result.getRow()));
}
}
if (!deletes.isEmpty()) {
LOG.warn("Deleting some vestigal " + deletes.size() + " rows of " + tableName +
" from " + TableName.META_TABLE_NAME);
metaTable.delete(deletes);
}
}
}
protected static void deleteFromMeta(final MasterProcedureEnv env,
final TableName tableName, List<HRegionInfo> regions) throws IOException {
MetaTableAccessor.deleteRegions(env.getMasterServices().getConnection(), regions);
// Clean any remaining rows for this table.
cleanAnyRemainingRows(env, tableName);
}
protected static void deleteAssignmentState(final MasterProcedureEnv env,
final TableName tableName) throws IOException {
AssignmentManager am = env.getMasterServices().getAssignmentManager();
// Clean up regions of the table in RegionStates.
LOG.debug("Removing '" + tableName + "' from region states.");
am.getRegionStates().tableDeleted(tableName);
// If entry for this table states, remove it.
LOG.debug("Marking '" + tableName + "' as deleted.");
am.getTableStateManager().setDeletedTable(tableName);
}
protected static void deleteTableDescriptorCache(final MasterProcedureEnv env,
final TableName tableName) throws IOException {
LOG.debug("Removing '" + tableName + "' descriptor.");
env.getMasterServices().getTableDescriptors().remove(tableName);
}
protected static void deleteTableStates(final MasterProcedureEnv env, final TableName tableName)
throws IOException {
getMasterQuotaManager(env).removeTableFromNamespaceQuota(tableName);
}
private static MasterQuotaManager getMasterQuotaManager(final MasterProcedureEnv env)
throws IOException {
return ProcedureSyncWait.waitFor(env, "quota manager to be available",
new ProcedureSyncWait.Predicate<MasterQuotaManager>() {
@Override
public MasterQuotaManager evaluate() throws IOException {
return env.getMasterServices().getMasterQuotaManager();
}
});
}
}

View File

@ -0,0 +1,31 @@
/**
* 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.hadoop.hbase.classification.InterfaceAudience;
@InterfaceAudience.Private
public final class MasterProcedureConstants {
private MasterProcedureConstants() {}
public static final String MASTER_PROCEDURE_LOGDIR = "MasterProcWALs";
public static final String MASTER_PROCEDURE_THREADS = "hbase.master.procedure.threads";
public static final int DEFAULT_MIN_MASTER_PROCEDURE_THREADS = 4;
}

View File

@ -0,0 +1,123 @@
/**
* 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.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.util.CancelableProgressable;
import org.apache.hadoop.hbase.util.FSUtils;
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class MasterProcedureEnv {
private static final Log LOG = LogFactory.getLog(MasterProcedureEnv.class);
@InterfaceAudience.Private
public static class WALStoreLeaseRecovery implements WALProcedureStore.LeaseRecovery {
private final HMaster master;
public WALStoreLeaseRecovery(final HMaster master) {
this.master = master;
}
@Override
public void recoverFileLease(final FileSystem fs, final Path path) throws IOException {
final Configuration conf = master.getConfiguration();
final FSUtils fsUtils = FSUtils.getInstance(fs, conf);
fsUtils.recoverFileLease(fs, path, conf, new CancelableProgressable() {
@Override
public boolean progress() {
LOG.debug("Recover Procedure Store log lease: " + path);
return master.isActiveMaster();
}
});
}
}
@InterfaceAudience.Private
public static class MasterProcedureStoreListener
implements ProcedureStore.ProcedureStoreListener {
private final HMaster master;
public MasterProcedureStoreListener(final HMaster master) {
this.master = master;
}
@Override
public void abortProcess() {
master.abort("The Procedure Store lost the lease");
}
}
private final MasterProcedureQueue procQueue;
private final MasterServices master;
public MasterProcedureEnv(final MasterServices master) {
this.master = master;
this.procQueue = new MasterProcedureQueue(master.getConfiguration(),
master.getTableLockManager());
}
public User getRequestUser() throws IOException {
User user = RpcServer.getRequestUser();
if (user == null) {
user = UserProvider.instantiate(getMasterConfiguration()).getCurrent();
}
return user;
}
public MasterServices getMasterServices() {
return master;
}
public Configuration getMasterConfiguration() {
return master.getConfiguration();
}
public MasterCoprocessorHost getMasterCoprocessorHost() {
return master.getMasterCoprocessorHost();
}
public MasterProcedureQueue getProcedureQueue() {
return procQueue;
}
public boolean isRunning() {
return master.getMasterProcedureExecutor().isRunning();
}
public boolean isInitialized() {
return master.isInitialized();
}
}

View File

@ -0,0 +1,448 @@
/**
* 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.ArrayDeque;
import java.util.Deque;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureFairRunQueues;
import org.apache.hadoop.hbase.procedure2.ProcedureRunnableSet;
import org.apache.hadoop.hbase.master.TableLockManager;
import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface.TableOperationType;
/**
* ProcedureRunnableSet for the Master Procedures.
* This RunnableSet tries to provide to the ProcedureExecutor procedures
* that can be executed without having to wait on a lock.
* Most of the master operations can be executed concurrently, if the they
* are operating on different tables (e.g. two create table can be performed
* at the same, time assuming table A and table B).
*
* Each procedure should implement an interface providing information for this queue.
* for example table related procedures should implement TableProcedureInterface.
* each procedure will be pushed in its own queue, and based on the operation type
* we may take smarter decision. e.g. we can abort all the operations preceding
* a delete table, or similar.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class MasterProcedureQueue implements ProcedureRunnableSet {
private static final Log LOG = LogFactory.getLog(MasterProcedureQueue.class);
private final ProcedureFairRunQueues<TableName, RunQueue> fairq;
private final ReentrantLock lock = new ReentrantLock();
private final Condition waitCond = lock.newCondition();
private final TableLockManager lockManager;
private final int metaTablePriority;
private final int userTablePriority;
private final int sysTablePriority;
private int queueSize;
public MasterProcedureQueue(final Configuration conf, final TableLockManager lockManager) {
this.fairq = new ProcedureFairRunQueues<TableName, RunQueue>(1);
this.lockManager = lockManager;
// TODO: should this be part of the HTD?
metaTablePriority = conf.getInt("hbase.master.procedure.queue.meta.table.priority", 3);
sysTablePriority = conf.getInt("hbase.master.procedure.queue.system.table.priority", 2);
userTablePriority = conf.getInt("hbase.master.procedure.queue.user.table.priority", 1);
}
@Override
public void addFront(final Procedure proc) {
lock.lock();
try {
getRunQueueOrCreate(proc).addFront(proc);
queueSize++;
waitCond.signal();
} finally {
lock.unlock();
}
}
@Override
public void addBack(final Procedure proc) {
lock.lock();
try {
getRunQueueOrCreate(proc).addBack(proc);
queueSize++;
waitCond.signal();
} finally {
lock.unlock();
}
}
@Override
public void yield(final Procedure proc) {
addFront(proc);
}
@Override
@edu.umd.cs.findbugs.annotations.SuppressWarnings("WA_AWAIT_NOT_IN_LOOP")
public Long poll() {
lock.lock();
try {
if (queueSize == 0) {
waitCond.await();
if (queueSize == 0) {
return null;
}
}
RunQueue queue = fairq.poll();
if (queue != null && queue.isAvailable()) {
queueSize--;
return queue.poll();
}
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
return null;
} finally {
lock.unlock();
}
return null;
}
@Override
public void signalAll() {
lock.lock();
try {
waitCond.signalAll();
} finally {
lock.unlock();
}
}
@Override
public void clear() {
lock.lock();
try {
fairq.clear();
queueSize = 0;
} finally {
lock.unlock();
}
}
@Override
public int size() {
lock.lock();
try {
return queueSize;
} finally {
lock.unlock();
}
}
@Override
public String toString() {
lock.lock();
try {
return "MasterProcedureQueue size=" + queueSize + ": " + fairq;
} finally {
lock.unlock();
}
}
@Override
public void completionCleanup(Procedure proc) {
if (proc instanceof TableProcedureInterface) {
TableProcedureInterface iProcTable = (TableProcedureInterface)proc;
boolean tableDeleted;
if (proc.hasException()) {
IOException procEx = proc.getException().unwrapRemoteException();
if (iProcTable.getTableOperationType() == TableOperationType.CREATE) {
// create failed because the table already exist
tableDeleted = !(procEx instanceof TableExistsException);
} else {
// the operation failed because the table does not exist
tableDeleted = (procEx instanceof TableNotFoundException);
}
} else {
// the table was deleted
tableDeleted = (iProcTable.getTableOperationType() == TableOperationType.DELETE);
}
if (tableDeleted) {
markTableAsDeleted(iProcTable.getTableName());
}
}
}
private RunQueue getRunQueueOrCreate(final Procedure proc) {
if (proc instanceof TableProcedureInterface) {
final TableName table = ((TableProcedureInterface)proc).getTableName();
return getRunQueueOrCreate(table);
}
// TODO: at the moment we only have Table procedures
// if you are implementing a non-table procedure, you have two option create
// a group for all the non-table procedures or try to find a key for your
// non-table procedure and implement something similar to the TableRunQueue.
throw new UnsupportedOperationException("RQs for non-table procedures are not implemented yet");
}
private TableRunQueue getRunQueueOrCreate(final TableName table) {
final TableRunQueue queue = getRunQueue(table);
if (queue != null) return queue;
return (TableRunQueue)fairq.add(table, createTableRunQueue(table));
}
private TableRunQueue createTableRunQueue(final TableName table) {
int priority = userTablePriority;
if (table.equals(TableName.META_TABLE_NAME)) {
priority = metaTablePriority;
} else if (table.isSystemTable()) {
priority = sysTablePriority;
}
return new TableRunQueue(priority);
}
private TableRunQueue getRunQueue(final TableName table) {
return (TableRunQueue)fairq.get(table);
}
/**
* Try to acquire the read lock on the specified table.
* other read operations in the table-queue may be executed concurrently,
* otherwise they have to wait until all the read-locks are released.
* @param table Table to lock
* @param purpose Human readable reason for locking the table
* @return true if we were able to acquire the lock on the table, otherwise false.
*/
public boolean tryAcquireTableRead(final TableName table, final String purpose) {
return getRunQueueOrCreate(table).tryRead(lockManager, table, purpose);
}
/**
* Release the read lock taken with tryAcquireTableRead()
* @param table the name of the table that has the read lock
*/
public void releaseTableRead(final TableName table) {
getRunQueue(table).releaseRead(lockManager, table);
}
/**
* Try to acquire the write lock on the specified table.
* other operations in the table-queue will be executed after the lock is released.
* @param table Table to lock
* @param purpose Human readable reason for locking the table
* @return true if we were able to acquire the lock on the table, otherwise false.
*/
public boolean tryAcquireTableWrite(final TableName table, final String purpose) {
return getRunQueueOrCreate(table).tryWrite(lockManager, table, purpose);
}
/**
* Release the write lock taken with tryAcquireTableWrite()
* @param table the name of the table that has the write lock
*/
public void releaseTableWrite(final TableName table) {
getRunQueue(table).releaseWrite(lockManager, table);
}
/**
* Tries to remove the queue and the table-lock of the specified table.
* If there are new operations pending (e.g. a new create),
* the remove will not be performed.
* @param table the name of the table that should be marked as deleted
* @return true if deletion succeeded, false otherwise meaning that there are
* other new operations pending for that table (e.g. a new create).
*/
protected boolean markTableAsDeleted(final TableName table) {
TableRunQueue queue = getRunQueue(table);
if (queue != null) {
lock.lock();
try {
if (queue.isEmpty() && !queue.isLocked()) {
fairq.remove(table);
// Remove the table lock
try {
lockManager.tableDeleted(table);
} catch (IOException e) {
LOG.warn("Received exception from TableLockManager.tableDeleted:", e); //not critical
}
} else {
// TODO: If there are no create, we can drop all the other ops
return false;
}
} finally {
lock.unlock();
}
}
return true;
}
private interface RunQueue extends ProcedureFairRunQueues.FairObject {
void addFront(Procedure proc);
void addBack(Procedure proc);
Long poll();
boolean isLocked();
}
/**
* Run Queue for a Table. It contains a read-write lock that is used by the
* MasterProcedureQueue to decide if we should fetch an item from this queue
* or skip to another one which will be able to run without waiting for locks.
*/
private static class TableRunQueue implements RunQueue {
private final Deque<Long> runnables = new ArrayDeque<Long>();
private final int priority;
private TableLock tableLock = null;
private boolean wlock = false;
private int rlock = 0;
public TableRunQueue(int priority) {
this.priority = priority;
}
@Override
public void addFront(final Procedure proc) {
runnables.addFirst(proc.getProcId());
}
// TODO: Improve run-queue push with TableProcedureInterface.getType()
// we can take smart decisions based on the type of the operation (e.g. create/delete)
@Override
public void addBack(final Procedure proc) {
runnables.addLast(proc.getProcId());
}
@Override
public Long poll() {
return runnables.poll();
}
@Override
public boolean isAvailable() {
synchronized (this) {
return !wlock && !runnables.isEmpty();
}
}
public boolean isEmpty() {
return runnables.isEmpty();
}
@Override
public boolean isLocked() {
synchronized (this) {
return wlock || rlock > 0;
}
}
public boolean tryRead(final TableLockManager lockManager,
final TableName tableName, final String purpose) {
synchronized (this) {
if (wlock) {
return false;
}
// Take zk-read-lock
tableLock = lockManager.readLock(tableName, purpose);
try {
tableLock.acquire();
} catch (IOException e) {
LOG.error("failed acquire read lock on " + tableName, e);
tableLock = null;
return false;
}
rlock++;
}
return true;
}
public void releaseRead(final TableLockManager lockManager,
final TableName tableName) {
synchronized (this) {
releaseTableLock(lockManager, rlock == 1);
rlock--;
}
}
public boolean tryWrite(final TableLockManager lockManager,
final TableName tableName, final String purpose) {
synchronized (this) {
if (wlock || rlock > 0) {
return false;
}
// Take zk-write-lock
tableLock = lockManager.writeLock(tableName, purpose);
try {
tableLock.acquire();
} catch (IOException e) {
LOG.error("failed acquire write lock on " + tableName, e);
tableLock = null;
return false;
}
wlock = true;
}
return true;
}
public void releaseWrite(final TableLockManager lockManager,
final TableName tableName) {
synchronized (this) {
releaseTableLock(lockManager, true);
wlock = false;
}
}
private void releaseTableLock(final TableLockManager lockManager, boolean reset) {
for (int i = 0; i < 3; ++i) {
try {
tableLock.release();
if (reset) {
tableLock = null;
}
break;
} catch (IOException e) {
LOG.warn("Could not release the table write-lock", e);
}
}
}
@Override
public int getPriority() {
return priority;
}
@Override
public String toString() {
return runnables.toString();
}
}
}

View File

@ -0,0 +1,56 @@
/**
* 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.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
import org.apache.hadoop.security.UserGroupInformation;
@InterfaceAudience.Private
@InterfaceStability.Evolving
public final class MasterProcedureUtil {
private static final Log LOG = LogFactory.getLog(MasterProcedureUtil.class);
private MasterProcedureUtil() {}
public static UserInformation toProtoUserInfo(UserGroupInformation ugi) {
UserInformation.Builder userInfoPB = UserInformation.newBuilder();
userInfoPB.setEffectiveUser(ugi.getUserName());
if (ugi.getRealUser() != null) {
userInfoPB.setRealUser(ugi.getRealUser().getUserName());
}
return userInfoPB.build();
}
public static UserGroupInformation toUserInfo(UserInformation userInfoProto) {
if (userInfoProto.hasEffectiveUser()) {
String effectiveUser = userInfoProto.getEffectiveUser();
if (userInfoProto.hasRealUser()) {
String realUser = userInfoProto.getRealUser();
UserGroupInformation realUserUgi = UserGroupInformation.createRemoteUser(realUser);
return UserGroupInformation.createProxyUser(effectiveUser, realUserUgi);
}
return UserGroupInformation.createRemoteUser(effectiveUser);
}
return null;
}
}

View File

@ -0,0 +1,105 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.concurrent.CountDownLatch;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.ipc.RpcCallContext;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.VersionInfo;
/**
* Latch used by the Master to have the prepare() sync behaviour for old
* clients, that can only get exceptions in a synchronous way.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public abstract class ProcedurePrepareLatch {
private static final NoopLatch noopLatch = new NoopLatch();
public static ProcedurePrepareLatch createLatch() {
// don't use the latch if we have procedure support
return hasProcedureSupport() ? noopLatch : new CompatibilityLatch();
}
public static boolean hasProcedureSupport() {
return currentClientHasMinimumVersion(1, 1);
}
private static boolean currentClientHasMinimumVersion(int major, int minor) {
RpcCallContext call = RpcServer.getCurrentCall();
VersionInfo versionInfo = call != null ? call.getClientVersionInfo() : null;
if (versionInfo != null) {
String[] components = versionInfo.getVersion().split("\\.");
int clientMajor = components.length > 0 ? Integer.parseInt(components[0]) : 0;
if (clientMajor != major) {
return clientMajor > major;
}
int clientMinor = components.length > 1 ? Integer.parseInt(components[1]) : 0;
return clientMinor >= minor;
}
return false;
}
protected abstract void countDown(final Procedure proc);
public abstract void await() throws IOException;
protected static void releaseLatch(final ProcedurePrepareLatch latch, final Procedure proc) {
if (latch != null) {
latch.countDown(proc);
}
}
private static class NoopLatch extends ProcedurePrepareLatch {
protected void countDown(final Procedure proc) {}
public void await() throws IOException {}
}
protected static class CompatibilityLatch extends ProcedurePrepareLatch {
private final CountDownLatch latch = new CountDownLatch(1);
private IOException exception = null;
protected void countDown(final Procedure proc) {
if (proc.hasException()) {
exception = proc.getException().unwrapRemoteException();
}
latch.countDown();
}
public void await() throws IOException {
try {
latch.await();
} catch (InterruptedException e) {
throw (InterruptedIOException)new InterruptedIOException().initCause(e);
}
if (exception != null) {
throw exception;
}
}
}
}

View File

@ -0,0 +1,179 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CoordinatedStateException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.RegionStates;
import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.master.ServerManager;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureResult;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
/**
* Helper to synchronously wait on conditions.
* This will be removed in the future (mainly when the AssignmentManager will be
* replaced with a Procedure version) by using ProcedureYieldException,
* and the queue will handle waiting and scheduling based on events.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public final class ProcedureSyncWait {
private static final Log LOG = LogFactory.getLog(ProcedureSyncWait.class);
private ProcedureSyncWait() {}
@InterfaceAudience.Private
public interface Predicate<T> {
T evaluate() throws IOException;
}
public static byte[] submitAndWaitProcedure(ProcedureExecutor<MasterProcedureEnv> procExec,
final Procedure proc) throws IOException {
long procId = procExec.submitProcedure(proc);
return waitForProcedureToComplete(procExec, procId);
}
public static byte[] waitForProcedureToComplete(ProcedureExecutor<MasterProcedureEnv> procExec,
final long procId) throws IOException {
while (!procExec.isFinished(procId) && procExec.isRunning()) {
// TODO: add a config to make it tunable
// Dev Consideration: are we waiting forever, or we can set up some timeout value?
Threads.sleepWithoutInterrupt(250);
}
ProcedureResult result = procExec.getResult(procId);
if (result != null) {
if (result.isFailed()) {
// If the procedure fails, we should always have an exception captured. Throw it.
throw result.getException().unwrapRemoteException();
}
return result.getResult();
} else {
if (procExec.isRunning()) {
throw new IOException("Procedure " + procId + "not found");
} else {
throw new IOException("The Master is Aborting");
}
}
}
public static <T> T waitFor(MasterProcedureEnv env, String purpose, Predicate<T> predicate)
throws IOException {
final Configuration conf = env.getMasterConfiguration();
final long waitTime = conf.getLong("hbase.master.wait.on.region", 5 * 60 * 1000);
final long waitingTimeForEvents = conf.getInt("hbase.master.event.waiting.time", 1000);
return waitFor(env, waitTime, waitingTimeForEvents, purpose, predicate);
}
public static <T> T waitFor(MasterProcedureEnv env, long waitTime, long waitingTimeForEvents,
String purpose, Predicate<T> predicate) throws IOException {
final long done = EnvironmentEdgeManager.currentTime() + waitTime;
do {
T result = predicate.evaluate();
if (result != null && !result.equals(Boolean.FALSE)) {
return result;
}
try {
Thread.sleep(waitingTimeForEvents);
} catch (InterruptedException e) {
LOG.warn("Interrupted while sleeping, waiting on " + purpose);
throw (InterruptedIOException)new InterruptedIOException().initCause(e);
}
LOG.debug("Waiting on " + purpose);
} while (EnvironmentEdgeManager.currentTime() < done && env.isRunning());
throw new TimeoutIOException("Timed out while waiting on " + purpose);
}
protected static void waitMetaRegions(final MasterProcedureEnv env) throws IOException {
int timeout = env.getMasterConfiguration().getInt("hbase.client.catalog.timeout", 10000);
try {
if (env.getMasterServices().getMetaTableLocator().waitMetaRegionLocation(
env.getMasterServices().getZooKeeper(), timeout) == null) {
throw new NotAllMetaRegionsOnlineException();
}
} catch (InterruptedException e) {
throw (InterruptedIOException)new InterruptedIOException().initCause(e);
}
}
protected static void waitRegionServers(final MasterProcedureEnv env) throws IOException {
final ServerManager sm = env.getMasterServices().getServerManager();
ProcedureSyncWait.waitFor(env, "server to assign region(s)",
new ProcedureSyncWait.Predicate<Boolean>() {
@Override
public Boolean evaluate() throws IOException {
List<ServerName> servers = sm.createDestinationServersList();
return servers != null && !servers.isEmpty();
}
});
}
protected static List<HRegionInfo> getRegionsFromMeta(final MasterProcedureEnv env,
final TableName tableName) throws IOException {
return ProcedureSyncWait.waitFor(env, "regions of table=" + tableName + " from meta",
new ProcedureSyncWait.Predicate<List<HRegionInfo>>() {
@Override
public List<HRegionInfo> evaluate() throws IOException {
if (TableName.META_TABLE_NAME.equals(tableName)) {
return new MetaTableLocator().getMetaRegions(env.getMasterServices().getZooKeeper());
}
return MetaTableAccessor.getTableRegions(env.getMasterServices().getConnection(),tableName);
}
});
}
protected static void waitRegionInTransition(final MasterProcedureEnv env,
final List<HRegionInfo> regions) throws IOException, CoordinatedStateException {
final AssignmentManager am = env.getMasterServices().getAssignmentManager();
final RegionStates states = am.getRegionStates();
for (final HRegionInfo region : regions) {
ProcedureSyncWait.waitFor(env, "regions " + region.getRegionNameAsString() + " in transition",
new ProcedureSyncWait.Predicate<Boolean>() {
@Override
public Boolean evaluate() throws IOException {
if (states.isRegionInState(region, State.FAILED_OPEN)) {
am.regionOffline(region);
}
return !states.isRegionInTransition(region);
}
});
}
}
}

View File

@ -0,0 +1,46 @@
/**
* 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.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.TableName;
/**
* Procedures that operates on a specific Table (e.g. create, delete, snapshot, ...)
* must implement this interface to allow the system handle the lock/concurrency problems.
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
public interface TableProcedureInterface {
public enum TableOperationType { CREATE, DELETE, EDIT, READ };
/**
* @return the name of the table the procedure is operating on
*/
TableName getTableName();
/**
* Given an operation type we can take decisions about what to do with pending operations.
* e.g. if we get a delete and we have some table operation pending (e.g. add column)
* we can abort those operations.
* @return the operation type that the procedure is executing.
*/
TableOperationType getTableOperationType();
}

View File

@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.handler.CreateTableHandler;
import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
import org.apache.hadoop.hbase.namespace.NamespaceAuditor;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetQuotaRequest;
@ -444,14 +444,11 @@ public class MasterQuotaManager implements RegionStateListener {
new HRegionInfo(QuotaUtil.QUOTA_TABLE_NAME)
};
masterServices.getExecutorService()
.submit(new CreateTableHandler(masterServices,
masterServices.getMasterFileSystem(),
QuotaUtil.QUOTA_TABLE_DESC,
masterServices.getConfiguration(),
newRegions,
masterServices)
.prepare());
masterServices.getMasterProcedureExecutor()
.submitProcedure(new CreateTableProcedure(
masterServices.getMasterProcedureExecutor().getEnvironment(),
QuotaUtil.QUOTA_TABLE_DESC,
newRegions));
}
private static class NamedLock<T> {

View File

@ -981,13 +981,14 @@ public class HRegionServer extends HasThread implements
// Send interrupts to wake up threads if sleeping so they notice shutdown.
// TODO: Should we check they are alive? If OOME could have exited already
if(this.hMemManager != null) this.hMemManager.stop();
if (this.hMemManager != null) this.hMemManager.stop();
if (this.cacheFlusher != null) this.cacheFlusher.interruptIfNecessary();
if (this.compactSplitThread != null) this.compactSplitThread.interruptIfNecessary();
if (this.compactionChecker != null) this.compactionChecker.cancel(true);
if (this.healthCheckChore != null) this.healthCheckChore.cancel(true);
if (this.nonceManagerChore != null) this.nonceManagerChore.cancel(true);
if (this.storefileRefresher != null) this.storefileRefresher.cancel(true);
sendShutdownInterrupt();
// Stop the quota manager
if (rsQuotaManager != null) {
@ -2072,6 +2073,12 @@ public class HRegionServer extends HasThread implements
abort("Simulated kill");
}
/**
* Called on stop/abort before closing the cluster connection and meta locator.
*/
protected void sendShutdownInterrupt() {
}
/**
* Wait on all threads to finish. Presumption is that all closes and stops
* have already been called.

View File

@ -67,6 +67,30 @@ public abstract class ModifyRegionUtils {
void editRegion(final HRegionInfo region) throws IOException;
}
public static HRegionInfo[] createHRegionInfos(HTableDescriptor hTableDescriptor,
byte[][] splitKeys) {
long regionId = System.currentTimeMillis();
HRegionInfo[] hRegionInfos = null;
if (splitKeys == null || splitKeys.length == 0) {
hRegionInfos = new HRegionInfo[]{
new HRegionInfo(hTableDescriptor.getTableName(), null, null, false, regionId)
};
} else {
int numRegions = splitKeys.length + 1;
hRegionInfos = new HRegionInfo[numRegions];
byte[] startKey = null;
byte[] endKey = null;
for (int i = 0; i < numRegions; i++) {
endKey = (i == splitKeys.length) ? null : splitKeys[i];
hRegionInfos[i] =
new HRegionInfo(hTableDescriptor.getTableName(), startKey, endKey,
false, regionId);
startKey = endKey;
}
}
return hRegionInfos;
}
/**
* Create new set of regions on the specified file-system.
* NOTE: that you should add the regions to hbase:meta after this operation.

View File

@ -63,6 +63,8 @@ import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination.SplitLog
import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.io.Reference;
import org.apache.hadoop.hbase.master.CatalogJanitor.SplitParentFirstComparator;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
@ -260,6 +262,11 @@ public class TestCatalogJanitor {
return null;
}
@Override
public ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
return null;
}
@Override
public ServerManager getServerManager() {
return null;
@ -912,7 +919,7 @@ public class TestCatalogJanitor {
MasterServices services = new MockMasterServices(server);
// create the janitor
CatalogJanitor janitor = new CatalogJanitor(server, services);
// Create regions.

View File

@ -0,0 +1,317 @@
/**
* 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.concurrent.atomic.AtomicInteger;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaTableAccessor;
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.Result;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
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 static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
public class MasterProcedureTestingUtility {
private static final Log LOG = LogFactory.getLog(MasterProcedureTestingUtility.class);
private MasterProcedureTestingUtility() {
}
public static HTableDescriptor createHTD(final TableName tableName, final String... family) {
HTableDescriptor htd = new HTableDescriptor(tableName);
for (int i = 0; i < family.length; ++i) {
htd.addFamily(new HColumnDescriptor(family[i]));
}
return htd;
}
public static HRegionInfo[] createTable(final ProcedureExecutor<MasterProcedureEnv> procExec,
final TableName tableName, final byte[][] splitKeys, String... family) throws IOException {
HTableDescriptor htd = createHTD(tableName, family);
HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
long procId = ProcedureTestingUtility.submitAndWait(procExec,
new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId));
return regions;
}
public static void validateTableCreation(final HMaster master, final TableName tableName,
final HRegionInfo[] regions, String... family) throws IOException {
validateTableCreation(master, tableName, regions, true, family);
}
public static void validateTableCreation(final HMaster master, final TableName tableName,
final HRegionInfo[] regions, boolean hasFamilyDirs, String... family) throws IOException {
// check filesystem
final FileSystem fs = master.getMasterFileSystem().getFileSystem();
final Path tableDir = FSUtils.getTableDir(master.getMasterFileSystem().getRootDir(), tableName);
assertTrue(fs.exists(tableDir));
List<Path> allRegionDirs = FSUtils.getRegionDirs(fs, tableDir);
for (int i = 0; i < regions.length; ++i) {
Path regionDir = new Path(tableDir, regions[i].getEncodedName());
assertTrue(regions[i] + " region dir does not exist", fs.exists(regionDir));
assertTrue(allRegionDirs.remove(regionDir));
List<Path> allFamilyDirs = FSUtils.getFamilyDirs(fs, regionDir);
for (int j = 0; j < family.length; ++j) {
final Path familyDir = new Path(regionDir, family[j]);
if (hasFamilyDirs) {
assertTrue(family[j] + " family dir does not exist", fs.exists(familyDir));
assertTrue(allFamilyDirs.remove(familyDir));
} else {
// TODO: WARN: Modify Table/Families does not create a family dir
if (!fs.exists(familyDir)) {
LOG.warn(family[j] + " family dir does not exist");
}
allFamilyDirs.remove(familyDir);
}
}
assertTrue("found extraneous families: " + allFamilyDirs, allFamilyDirs.isEmpty());
}
assertTrue("found extraneous regions: " + allRegionDirs, allRegionDirs.isEmpty());
// check meta
assertTrue(MetaTableAccessor.tableExists(master.getConnection(), tableName));
assertEquals(regions.length, countMetaRegions(master, tableName));
// check htd
TableDescriptor tableDesc = master.getTableDescriptors().getDescriptor(tableName);
assertTrue("table descriptor not found", tableDesc != null);
HTableDescriptor htd = tableDesc.getHTableDescriptor();
assertTrue("table descriptor not found", htd != null);
for (int i = 0; i < family.length; ++i) {
assertTrue("family not found " + family[i], htd.getFamily(Bytes.toBytes(family[i])) != null);
}
assertEquals(family.length, htd.getFamilies().size());
}
public static void validateTableDeletion(final HMaster master, final TableName tableName,
final HRegionInfo[] regions, String... family) throws IOException {
// check filesystem
final FileSystem fs = master.getMasterFileSystem().getFileSystem();
final Path tableDir = FSUtils.getTableDir(master.getMasterFileSystem().getRootDir(), tableName);
assertFalse(fs.exists(tableDir));
// check meta
assertFalse(MetaTableAccessor.tableExists(master.getConnection(), tableName));
assertEquals(0, countMetaRegions(master, tableName));
// check htd
assertTrue("found htd of deleted table",
master.getTableDescriptors().getDescriptor(tableName) == null);
}
private static int countMetaRegions(final HMaster master, final TableName tableName)
throws IOException {
final AtomicInteger actualRegCount = new AtomicInteger(0);
final MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
@Override
public boolean visit(Result rowResult) throws IOException {
RegionLocations list = MetaTableAccessor.getRegionLocations(rowResult);
if (list == null) {
LOG.warn("No serialized HRegionInfo in " + rowResult);
return true;
}
HRegionLocation l = list.getRegionLocation();
if (l == null) {
return true;
}
if (!l.getRegionInfo().getTable().equals(tableName)) {
return false;
}
if (l.getRegionInfo().isOffline() || l.getRegionInfo().isSplit()) return true;
HRegionLocation[] locations = list.getRegionLocations();
for (HRegionLocation location : locations) {
if (location == null) continue;
ServerName serverName = location.getServerName();
// Make sure that regions are assigned to server
if (serverName != null && serverName.getHostAndPort() != null) {
actualRegCount.incrementAndGet();
}
}
return true;
}
};
MetaTableAccessor.scanMetaForTableRegions(master.getConnection(), visitor, tableName);
return actualRegCount.get();
}
public static <TState> void testRecoveryAndDoubleExecution(
final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId,
final int numSteps, final TState[] states) throws Exception {
ProcedureTestingUtility.waitProcedure(procExec, procId);
assertEquals(false, procExec.isRunning());
// Restart the executor and execute the step twice
// execute step N - kill before store update
// restart executor/store
// execute step N - save on store
for (int i = 0; i < numSteps; ++i) {
LOG.info("Restart "+ i +" exec state: " + states[i]);
ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
ProcedureTestingUtility.restart(procExec);
ProcedureTestingUtility.waitProcedure(procExec, procId);
}
assertEquals(true, procExec.isRunning());
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
}
public static <TState> void testRollbackAndDoubleExecution(
final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId,
final int lastStep, final TState[] states) throws Exception {
ProcedureTestingUtility.waitProcedure(procExec, procId);
// Restart the executor and execute the step twice
// execute step N - kill before store update
// restart executor/store
// execute step N - save on store
for (int i = 0; i < lastStep; ++i) {
LOG.info("Restart "+ i +" exec state: " + states[i]);
ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
ProcedureTestingUtility.restart(procExec);
ProcedureTestingUtility.waitProcedure(procExec, procId);
}
// Restart the executor and rollback the step twice
// rollback step N - kill before store update
// restart executor/store
// rollback step N - save on store
MasterProcedureTestingUtility.InjectAbortOnLoadListener abortListener =
new MasterProcedureTestingUtility.InjectAbortOnLoadListener(procExec);
procExec.registerListener(abortListener);
try {
for (int i = lastStep + 1; i >= 0; --i) {
LOG.info("Restart " + i +" rollback state: "+ states[i]);
ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
ProcedureTestingUtility.restart(procExec);
ProcedureTestingUtility.waitProcedure(procExec, procId);
}
} finally {
assertTrue(procExec.unregisterListener(abortListener));
}
ProcedureTestingUtility.assertIsAbortException(procExec.getResult(procId));
}
public static <TState> void testRollbackAndDoubleExecutionAfterPONR(
final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId,
final int lastStep, final TState[] states) throws Exception {
ProcedureTestingUtility.waitProcedure(procExec, procId);
// Restart the executor and execute the step twice
// execute step N - kill before store update
// restart executor/store
// execute step N - save on store
for (int i = 0; i < lastStep; ++i) {
LOG.info("Restart "+ i +" exec state: " + states[i]);
ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
ProcedureTestingUtility.restart(procExec);
ProcedureTestingUtility.waitProcedure(procExec, procId);
}
// try to inject the abort
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
MasterProcedureTestingUtility.InjectAbortOnLoadListener abortListener =
new MasterProcedureTestingUtility.InjectAbortOnLoadListener(procExec);
procExec.registerListener(abortListener);
try {
ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
ProcedureTestingUtility.restart(procExec);
LOG.info("Restart and execute");
ProcedureTestingUtility.waitProcedure(procExec, procId);
} finally {
assertTrue(procExec.unregisterListener(abortListener));
}
assertEquals(true, procExec.isRunning());
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
}
public static <TState> void testRollbackRetriableFailure(
final ProcedureExecutor<MasterProcedureEnv> procExec, final long procId,
final int lastStep, final TState[] states) throws Exception {
ProcedureTestingUtility.waitProcedure(procExec, procId);
// Restart the executor and execute the step twice
// execute step N - kill before store update
// restart executor/store
// execute step N - save on store
for (int i = 0; i < lastStep; ++i) {
LOG.info("Restart "+ i +" exec state: " + states[i]);
ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
ProcedureTestingUtility.restart(procExec);
ProcedureTestingUtility.waitProcedure(procExec, procId);
}
// execute the rollback
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
MasterProcedureTestingUtility.InjectAbortOnLoadListener abortListener =
new MasterProcedureTestingUtility.InjectAbortOnLoadListener(procExec);
procExec.registerListener(abortListener);
try {
ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
ProcedureTestingUtility.restart(procExec);
LOG.info("Restart and rollback");
ProcedureTestingUtility.waitProcedure(procExec, procId);
} finally {
assertTrue(procExec.unregisterListener(abortListener));
}
ProcedureTestingUtility.assertIsAbortException(procExec.getResult(procId));
}
public static class InjectAbortOnLoadListener
implements ProcedureExecutor.ProcedureExecutorListener {
private final ProcedureExecutor<MasterProcedureEnv> procExec;
public InjectAbortOnLoadListener(final ProcedureExecutor<MasterProcedureEnv> procExec) {
this.procExec = procExec;
}
@Override
public void procedureLoaded(long procId) {
procExec.abort(procId);
}
@Override
public void procedureAdded(long procId) { /* no-op */ }
@Override
public void procedureFinished(long procId) { /* no-op */ }
}
}

View File

@ -0,0 +1,257 @@
/**
* 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.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableExistsException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
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 TestCreateTableProcedure {
private static final Log LOG = LogFactory.getLog(TestCreateTableProcedure.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 {
resetProcExecutorTestingKillFlag();
}
@After
public void tearDown() throws Exception {
resetProcExecutorTestingKillFlag();
for (HTableDescriptor htd: UTIL.getHBaseAdmin().listTables()) {
LOG.info("Tear down, remove table=" + htd.getTableName());
UTIL.deleteTable(htd.getTableName());
}
}
private void resetProcExecutorTestingKillFlag() {
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
assertTrue("expected executor to be running", procExec.isRunning());
}
@Test(timeout=60000)
public void testSimpleCreate() throws Exception {
final TableName tableName = TableName.valueOf("testSimpleCreate");
final byte[][] splitKeys = null;
testSimpleCreate(tableName, splitKeys);
}
@Test(timeout=60000)
public void testSimpleCreateWithSplits() throws Exception {
final TableName tableName = TableName.valueOf("testSimpleCreateWithSplits");
final byte[][] splitKeys = new byte[][] {
Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
};
testSimpleCreate(tableName, splitKeys);
}
private void testSimpleCreate(final TableName tableName, byte[][] splitKeys) throws Exception {
HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
MasterProcedureTestingUtility.validateTableCreation(
UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
}
@Test(timeout=60000, expected=TableExistsException.class)
public void testCreateExisting() throws Exception {
final TableName tableName = TableName.valueOf("testCreateExisting");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
final HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f");
final HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, null);
// create the table
long procId1 = procExec.submitProcedure(
new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
// create another with the same name
ProcedurePrepareLatch latch2 = new ProcedurePrepareLatch.CompatibilityLatch();
long procId2 = procExec.submitProcedure(
new CreateTableProcedure(procExec.getEnvironment(), htd, regions, latch2));
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.assertProcNotFailed(procExec.getResult(procId1));
ProcedureTestingUtility.waitProcedure(procExec, procId2);
latch2.await();
}
@Test(timeout=60000)
public void testRecoveryAndDoubleExecution() throws Exception {
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
// create the table
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the Create procedure && kill the executor
byte[][] splitKeys = null;
HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f1", "f2");
HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
long procId = procExec.submitProcedure(
new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
// Restart the executor and execute the step twice
// NOTE: the 6 (number of CreateTableState steps) is hardcoded,
// so you have to look at this test at least once when you add a new step.
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(
procExec, procId, 6, CreateTableState.values());
MasterProcedureTestingUtility.validateTableCreation(
UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
}
@Test(timeout=90000)
public void testRollbackAndDoubleExecution() throws Exception {
final TableName tableName = TableName.valueOf("testRollbackAndDoubleExecution");
// create the table
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the Create procedure && kill the executor
final byte[][] splitKeys = new byte[][] {
Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
};
HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f1", "f2");
htd.setRegionReplication(3);
HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
long procId = procExec.submitProcedure(
new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
// NOTE: the 4 (number of CreateTableState steps) is hardcoded,
// so you have to look at this test at least once when you add a new step.
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(
procExec, procId, 4, CreateTableState.values());
MasterProcedureTestingUtility.validateTableDeletion(
UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
// are we able to create the table after a rollback?
resetProcExecutorTestingKillFlag();
testSimpleCreate(tableName, splitKeys);
}
@Test(timeout=90000)
public void testRollbackRetriableFailure() throws Exception {
final TableName tableName = TableName.valueOf("testRollbackRetriableFailure");
// create the table
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the Create procedure && kill the executor
final byte[][] splitKeys = new byte[][] {
Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
};
HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f1", "f2");
HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
long procId = procExec.submitProcedure(
new FaultyCreateTableProcedure(procExec.getEnvironment(), htd, regions));
// NOTE: the 4 (number of CreateTableState steps) is hardcoded,
// so you have to look at this test at least once when you add a new step.
MasterProcedureTestingUtility.testRollbackRetriableFailure(
procExec, procId, 4, CreateTableState.values());
MasterProcedureTestingUtility.validateTableDeletion(
UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
// are we able to create the table after a rollback?
resetProcExecutorTestingKillFlag();
testSimpleCreate(tableName, splitKeys);
}
private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
}
public static class FaultyCreateTableProcedure extends CreateTableProcedure {
private int retries = 0;
public FaultyCreateTableProcedure() {
// Required by the Procedure framework to create the procedure on replay
}
public FaultyCreateTableProcedure(final MasterProcedureEnv env,
final HTableDescriptor hTableDescriptor, final HRegionInfo[] newRegions)
throws IOException {
super(env, hTableDescriptor, newRegions);
}
@Override
protected void rollbackState(final MasterProcedureEnv env, final CreateTableState state)
throws IOException {
if (retries++ < 3) {
LOG.info("inject rollback failure state=" + state);
throw new IOException("injected failure number " + retries);
} else {
super.rollbackState(env, state);
retries = 0;
}
}
}
}

View File

@ -0,0 +1,208 @@
/**
* 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.DeleteTableState;
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 TestDeleteTableProcedure {
private static final Log LOG = LogFactory.getLog(TestDeleteTableProcedure.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, expected=TableNotFoundException.class)
public void testDeleteNotExistentTable() throws Exception {
final TableName tableName = TableName.valueOf("testDeleteNotExistentTable");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
ProcedurePrepareLatch latch = new ProcedurePrepareLatch.CompatibilityLatch();
long procId = ProcedureTestingUtility.submitAndWait(procExec,
new DeleteTableProcedure(procExec.getEnvironment(), tableName, latch));
latch.await();
}
@Test(timeout=60000, expected=TableNotDisabledException.class)
public void testDeleteNotDisabledTable() throws Exception {
final TableName tableName = TableName.valueOf("testDeleteNotDisabledTable");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f");
ProcedurePrepareLatch latch = new ProcedurePrepareLatch.CompatibilityLatch();
long procId = ProcedureTestingUtility.submitAndWait(procExec,
new DeleteTableProcedure(procExec.getEnvironment(), tableName, latch));
latch.await();
}
@Test(timeout=60000)
public void testDeleteDeletedTable() throws Exception {
final TableName tableName = TableName.valueOf("testDeleteDeletedTable");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
procExec, tableName, null, "f");
UTIL.getHBaseAdmin().disableTable(tableName);
// delete the table (that exists)
long procId1 = procExec.submitProcedure(
new DeleteTableProcedure(procExec.getEnvironment(), tableName));
// delete the table (that will no longer exist)
long procId2 = procExec.submitProcedure(
new DeleteTableProcedure(procExec.getEnvironment(), tableName));
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.waitProcedure(procExec, procId2);
// First delete should succeed
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
MasterProcedureTestingUtility.validateTableDeletion(
UTIL.getHBaseCluster().getMaster(), tableName, regions, "f");
// Second delete should fail with TableNotFound
ProcedureResult result = procExec.getResult(procId2);
assertTrue(result.isFailed());
LOG.debug("Delete failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof TableNotFoundException);
}
@Test(timeout=60000)
public void testSimpleDelete() throws Exception {
final TableName tableName = TableName.valueOf("testSimpleDelete");
final byte[][] splitKeys = null;
testSimpleDelete(tableName, splitKeys);
}
@Test(timeout=60000)
public void testSimpleDeleteWithSplits() throws Exception {
final TableName tableName = TableName.valueOf("testSimpleDeleteWithSplits");
final byte[][] splitKeys = new byte[][] {
Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
};
testSimpleDelete(tableName, splitKeys);
}
private void testSimpleDelete(final TableName tableName, byte[][] splitKeys) throws Exception {
HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
UTIL.getHBaseAdmin().disableTable(tableName);
// delete the table
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
long procId = ProcedureTestingUtility.submitAndWait(procExec,
new DeleteTableProcedure(procExec.getEnvironment(), tableName));
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
MasterProcedureTestingUtility.validateTableDeletion(
UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
}
@Test(timeout=60000)
public void testRecoveryAndDoubleExecution() throws Exception {
final TableName tableName = TableName.valueOf("testRecoveryAndDoubleExecution");
// create the table
byte[][] splitKeys = null;
HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
UTIL.getHBaseAdmin().disableTable(tableName);
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the Delete procedure && kill the executor
long procId = procExec.submitProcedure(
new DeleteTableProcedure(procExec.getEnvironment(), tableName));
// Restart the executor and execute the step twice
// NOTE: the 6 (number of DeleteTableState steps) is hardcoded,
// so you have to look at this test at least once when you add a new step.
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(
procExec, procId, 6, DeleteTableState.values());
MasterProcedureTestingUtility.validateTableDeletion(
UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
}
private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
}
}

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.util.concurrent.CountDownLatch;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.mockito.Mockito;
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, LargeTests.class})
public class TestMasterFailoverWithProcedures {
private static final Log LOG = LogFactory.getLog(TestMasterFailoverWithProcedures.class);
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static void setupConf(Configuration conf) {
}
@Before
public void setup() throws Exception {
setupConf(UTIL.getConfiguration());
UTIL.startMiniCluster(2, 1);
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
ProcedureTestingUtility.setToggleKillBeforeStoreUpdate(procExec, false);
ProcedureTestingUtility.setKillBeforeStoreUpdate(procExec, false);
}
@After
public void tearDown() throws Exception {
try {
UTIL.shutdownMiniCluster();
} catch (Exception e) {
LOG.warn("failure shutting down cluster", e);
}
}
@Test(timeout=60000)
public void testWalRecoverLease() throws Exception {
final ProcedureStore masterStore = getMasterProcedureExecutor().getStore();
assertTrue("expected WALStore for this test", masterStore instanceof WALProcedureStore);
HMaster firstMaster = UTIL.getHBaseCluster().getMaster();
// Abort Latch for the master store
final CountDownLatch masterStoreAbort = new CountDownLatch(1);
masterStore.registerListener(new ProcedureStore.ProcedureStoreListener() {
@Override
public void abortProcess() {
LOG.debug("Abort store of Master");
masterStoreAbort.countDown();
}
});
// startup a fake master the new WAL store will take the lease
// and the active master should abort.
HMaster backupMaster3 = Mockito.mock(HMaster.class);
Mockito.doReturn(firstMaster.getConfiguration()).when(backupMaster3).getConfiguration();
Mockito.doReturn(true).when(backupMaster3).isActiveMaster();
final WALProcedureStore backupStore3 = new WALProcedureStore(firstMaster.getConfiguration(),
firstMaster.getMasterFileSystem().getFileSystem(),
((WALProcedureStore)masterStore).getLogDir(),
new MasterProcedureEnv.WALStoreLeaseRecovery(backupMaster3));
// Abort Latch for the test store
final CountDownLatch backupStore3Abort = new CountDownLatch(1);
backupStore3.registerListener(new ProcedureStore.ProcedureStoreListener() {
@Override
public void abortProcess() {
LOG.debug("Abort store of backupMaster3");
backupStore3Abort.countDown();
backupStore3.stop(true);
}
});
backupStore3.start(1);
backupStore3.recoverLease();
// Try to trigger a command on the master (WAL lease expired on the active one)
HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(TableName.valueOf("mtb"), "f");
HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, null);
LOG.debug("submit proc");
getMasterProcedureExecutor().submitProcedure(
new CreateTableProcedure(getMasterProcedureExecutor().getEnvironment(), htd, regions));
LOG.debug("wait master store abort");
masterStoreAbort.await();
// Now the real backup master should start up
LOG.debug("wait backup master to startup");
waitBackupMaster(UTIL, firstMaster);
assertEquals(true, firstMaster.isStopped());
// wait the store in here to abort (the test will fail due to timeout if it doesn't)
LOG.debug("wait the store to abort");
backupStore3.getStoreTracker().setDeleted(1, false);
backupStore3.delete(1);
backupStore3Abort.await();
}
// ==========================================================================
// Test Create Table
// ==========================================================================
@Test(timeout=60000)
public void testCreateWithFailover() throws Exception {
// TODO: Should we try every step? (master failover takes long time)
// It is already covered by TestCreateTableProcedure
// 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)
testCreateWithFailoverAtStep(CreateTableState.CREATE_TABLE_ASSIGN_REGIONS.ordinal());
}
private void testCreateWithFailoverAtStep(final int step) throws Exception {
final TableName tableName = TableName.valueOf("testCreateWithFailoverAtStep" + step);
// create the table
ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
ProcedureTestingUtility.setKillBeforeStoreUpdate(procExec, true);
ProcedureTestingUtility.setToggleKillBeforeStoreUpdate(procExec, true);
// Start the Create procedure && kill the executor
byte[][] splitKeys = null;
HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f1", "f2");
HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
long procId = procExec.submitProcedure(
new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
testRecoveryAndDoubleExecution(UTIL, procId, step, CreateTableState.values());
MasterProcedureTestingUtility.validateTableCreation(
UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
}
// ==========================================================================
// Test Delete Table
// ==========================================================================
@Test(timeout=60000)
public void testDeleteWithFailover() throws Exception {
// TODO: Should we try every step? (master failover takes long time)
// It is already covered by TestDeleteTableProcedure
// 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)
testDeleteWithFailoverAtStep(DeleteTableState.DELETE_TABLE_UNASSIGN_REGIONS.ordinal());
}
private void testDeleteWithFailoverAtStep(final int step) throws Exception {
final TableName tableName = TableName.valueOf("testDeleteWithFailoverAtStep" + step);
// create the table
byte[][] splitKeys = null;
HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
Path tableDir = FSUtils.getTableDir(getRootDir(), tableName);
MasterProcedureTestingUtility.validateTableCreation(
UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
UTIL.getHBaseAdmin().disableTable(tableName);
ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
ProcedureTestingUtility.setKillBeforeStoreUpdate(procExec, true);
ProcedureTestingUtility.setToggleKillBeforeStoreUpdate(procExec, true);
// Start the Delete procedure && kill the executor
long procId = procExec.submitProcedure(
new DeleteTableProcedure(procExec.getEnvironment(), tableName));
testRecoveryAndDoubleExecution(UTIL, procId, step, DeleteTableState.values());
MasterProcedureTestingUtility.validateTableDeletion(
UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
}
// ==========================================================================
// Test Helpers
// ==========================================================================
public static <TState> void testRecoveryAndDoubleExecution(final HBaseTestingUtility testUtil,
final long procId, final int lastStepBeforeFailover, TState[] states) throws Exception {
ProcedureExecutor<MasterProcedureEnv> procExec =
testUtil.getHBaseCluster().getMaster().getMasterProcedureExecutor();
ProcedureTestingUtility.waitProcedure(procExec, procId);
for (int i = 0; i < lastStepBeforeFailover; ++i) {
LOG.info("Restart "+ i +" exec state: " + states[i]);
ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
ProcedureTestingUtility.restart(procExec);
ProcedureTestingUtility.waitProcedure(procExec, procId);
}
ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
LOG.info("Trigger master failover");
masterFailover(testUtil);
procExec = testUtil.getHBaseCluster().getMaster().getMasterProcedureExecutor();
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
}
// ==========================================================================
// Master failover utils
// ==========================================================================
public static void masterFailover(final HBaseTestingUtility testUtil)
throws Exception {
MiniHBaseCluster cluster = testUtil.getMiniHBaseCluster();
// Kill the master
HMaster oldMaster = cluster.getMaster();
cluster.killMaster(cluster.getMaster().getServerName());
// Wait the secondary
waitBackupMaster(testUtil, oldMaster);
}
public static void waitBackupMaster(final HBaseTestingUtility testUtil,
final HMaster oldMaster) throws Exception {
MiniHBaseCluster cluster = testUtil.getMiniHBaseCluster();
HMaster newMaster = cluster.getMaster();
while (newMaster == null || newMaster == oldMaster) {
Thread.sleep(250);
newMaster = cluster.getMaster();
}
while (!(newMaster.isActiveMaster() && newMaster.isInitialized())) {
Thread.sleep(250);
}
}
// ==========================================================================
// Helpers
// ==========================================================================
private MasterProcedureEnv getMasterProcedureEnv() {
return getMasterProcedureExecutor().getEnvironment();
}
private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
}
private FileSystem getFileSystem() {
return UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
}
private Path getRootDir() {
return UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
}
private Path getTempDir() {
return UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getTempDir();
}
}

View File

@ -0,0 +1,433 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master.procedure;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.ConcurrentHashMap;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.master.TableLockManager;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
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, SmallTests.class})
public class TestMasterProcedureQueue {
private static final Log LOG = LogFactory.getLog(TestMasterProcedureQueue.class);
private MasterProcedureQueue queue;
private Configuration conf;
@Before
public void setUp() throws IOException {
conf = HBaseConfiguration.create();
queue = new MasterProcedureQueue(conf, new TableLockManager.NullTableLockManager());
}
@After
public void tearDown() throws IOException {
assertEquals(0, queue.size());
}
/**
* Verify simple create/insert/fetch/delete of the table queue.
*/
@Test
public void testSimpleTableOpsQueues() throws Exception {
final int NUM_TABLES = 10;
final int NUM_ITEMS = 10;
int count = 0;
for (int i = 1; i <= NUM_TABLES; ++i) {
TableName tableName = TableName.valueOf(String.format("test-%04d", i));
// insert items
for (int j = 1; j <= NUM_ITEMS; ++j) {
queue.addBack(new TestTableProcedure(i * 1000 + j, tableName,
TableProcedureInterface.TableOperationType.EDIT));
assertEquals(++count, queue.size());
}
}
assertEquals(NUM_TABLES * NUM_ITEMS, queue.size());
for (int j = 1; j <= NUM_ITEMS; ++j) {
for (int i = 1; i <= NUM_TABLES; ++i) {
Long procId = queue.poll();
assertEquals(--count, queue.size());
assertEquals(i * 1000 + j, procId.longValue());
}
}
assertEquals(0, queue.size());
for (int i = 1; i <= NUM_TABLES; ++i) {
TableName tableName = TableName.valueOf(String.format("test-%04d", i));
// complete the table deletion
assertTrue(queue.markTableAsDeleted(tableName));
}
}
/**
* Check that the table queue is not deletable until every procedure
* in-progress is completed (this is a special case for write-locks).
*/
@Test
public void testCreateDeleteTableOperationsWithWriteLock() throws Exception {
TableName tableName = TableName.valueOf("testtb");
queue.addBack(new TestTableProcedure(1, tableName,
TableProcedureInterface.TableOperationType.EDIT));
// table can't be deleted because one item is in the queue
assertFalse(queue.markTableAsDeleted(tableName));
// fetch item and take a lock
assertEquals(1, queue.poll().longValue());
// take the xlock
assertTrue(queue.tryAcquireTableWrite(tableName, "write"));
// table can't be deleted because we have the lock
assertEquals(0, queue.size());
assertFalse(queue.markTableAsDeleted(tableName));
// release the xlock
queue.releaseTableWrite(tableName);
// complete the table deletion
assertTrue(queue.markTableAsDeleted(tableName));
}
/**
* Check that the table queue is not deletable until every procedure
* in-progress is completed (this is a special case for read-locks).
*/
@Test
public void testCreateDeleteTableOperationsWithReadLock() throws Exception {
final TableName tableName = TableName.valueOf("testtb");
final int nitems = 2;
for (int i = 1; i <= nitems; ++i) {
queue.addBack(new TestTableProcedure(i, tableName,
TableProcedureInterface.TableOperationType.READ));
}
// table can't be deleted because one item is in the queue
assertFalse(queue.markTableAsDeleted(tableName));
for (int i = 1; i <= nitems; ++i) {
// fetch item and take a lock
assertEquals(i, queue.poll().longValue());
// take the rlock
assertTrue(queue.tryAcquireTableRead(tableName, "read " + i));
// table can't be deleted because we have locks and/or items in the queue
assertFalse(queue.markTableAsDeleted(tableName));
}
for (int i = 1; i <= nitems; ++i) {
// table can't be deleted because we have locks
assertFalse(queue.markTableAsDeleted(tableName));
// release the rlock
queue.releaseTableRead(tableName);
}
// there are no items and no lock in the queeu
assertEquals(0, queue.size());
// complete the table deletion
assertTrue(queue.markTableAsDeleted(tableName));
}
/**
* Verify the correct logic of RWLocks on the queue
*/
@Test
public void testVerifyRwLocks() throws Exception {
TableName tableName = TableName.valueOf("testtb");
queue.addBack(new TestTableProcedure(1, tableName,
TableProcedureInterface.TableOperationType.EDIT));
queue.addBack(new TestTableProcedure(2, tableName,
TableProcedureInterface.TableOperationType.READ));
queue.addBack(new TestTableProcedure(3, tableName,
TableProcedureInterface.TableOperationType.EDIT));
queue.addBack(new TestTableProcedure(4, tableName,
TableProcedureInterface.TableOperationType.READ));
queue.addBack(new TestTableProcedure(5, tableName,
TableProcedureInterface.TableOperationType.READ));
// Fetch the 1st item and take the write lock
Long procId = queue.poll();
assertEquals(1, procId.longValue());
assertEquals(true, queue.tryAcquireTableWrite(tableName, "write " + procId));
// Fetch the 2nd item and verify that the lock can't be acquired
assertEquals(null, queue.poll());
// Release the write lock and acquire the read lock
queue.releaseTableWrite(tableName);
// Fetch the 2nd item and take the read lock
procId = queue.poll();
assertEquals(2, procId.longValue());
assertEquals(true, queue.tryAcquireTableRead(tableName, "read " + procId));
// Fetch the 3rd item and verify that the lock can't be acquired
procId = queue.poll();
assertEquals(3, procId.longValue());
assertEquals(false, queue.tryAcquireTableWrite(tableName, "write " + procId));
// release the rdlock of item 2 and take the wrlock for the 3d item
queue.releaseTableRead(tableName);
assertEquals(true, queue.tryAcquireTableWrite(tableName, "write " + procId));
// Fetch 4th item and verify that the lock can't be acquired
assertEquals(null, queue.poll());
// Release the write lock and acquire the read lock
queue.releaseTableWrite(tableName);
// Fetch the 4th item and take the read lock
procId = queue.poll();
assertEquals(4, procId.longValue());
assertEquals(true, queue.tryAcquireTableRead(tableName, "read " + procId));
// Fetch the 4th item and take the read lock
procId = queue.poll();
assertEquals(5, procId.longValue());
assertEquals(true, queue.tryAcquireTableRead(tableName, "read " + procId));
// Release 4th and 5th read-lock
queue.releaseTableRead(tableName);
queue.releaseTableRead(tableName);
// remove table queue
assertEquals(0, queue.size());
assertTrue("queue should be deleted", queue.markTableAsDeleted(tableName));
}
/**
* Verify that "write" operations for a single table are serialized,
* but different tables can be executed in parallel.
*/
@Test(timeout=90000)
public void testConcurrentWriteOps() throws Exception {
final TestTableProcSet procSet = new TestTableProcSet(queue);
final int NUM_ITEMS = 10;
final int NUM_TABLES = 4;
final AtomicInteger opsCount = new AtomicInteger(0);
for (int i = 0; i < NUM_TABLES; ++i) {
TableName tableName = TableName.valueOf(String.format("testtb-%04d", i));
for (int j = 1; j < NUM_ITEMS; ++j) {
procSet.addBack(new TestTableProcedure(i * 100 + j, tableName,
TableProcedureInterface.TableOperationType.EDIT));
opsCount.incrementAndGet();
}
}
assertEquals(opsCount.get(), queue.size());
final Thread[] threads = new Thread[NUM_TABLES * 2];
final HashSet<TableName> concurrentTables = new HashSet<TableName>();
final ArrayList<String> failures = new ArrayList<String>();
final AtomicInteger concurrentCount = new AtomicInteger(0);
for (int i = 0; i < threads.length; ++i) {
threads[i] = new Thread() {
@Override
public void run() {
while (opsCount.get() > 0) {
try {
TableProcedureInterface proc = procSet.acquire();
if (proc == null) {
queue.signalAll();
if (opsCount.get() > 0) {
continue;
}
break;
}
synchronized (concurrentTables) {
assertTrue("unexpected concurrency on " + proc.getTableName(),
concurrentTables.add(proc.getTableName()));
}
assertTrue(opsCount.decrementAndGet() >= 0);
try {
long procId = ((Procedure)proc).getProcId();
TableName tableId = proc.getTableName();
int concurrent = concurrentCount.incrementAndGet();
assertTrue("inc-concurrent="+ concurrent +" 1 <= concurrent <= "+ NUM_TABLES,
concurrent >= 1 && concurrent <= NUM_TABLES);
LOG.debug("[S] tableId="+ tableId +" procId="+ procId +" concurrent="+ concurrent);
Thread.sleep(2000);
concurrent = concurrentCount.decrementAndGet();
LOG.debug("[E] tableId="+ tableId +" procId="+ procId +" concurrent="+ concurrent);
assertTrue("dec-concurrent=" + concurrent, concurrent < NUM_TABLES);
} finally {
synchronized (concurrentTables) {
assertTrue(concurrentTables.remove(proc.getTableName()));
}
procSet.release(proc);
}
} catch (Throwable e) {
LOG.error("Failed " + e.getMessage(), e);
synchronized (failures) {
failures.add(e.getMessage());
}
} finally {
queue.signalAll();
}
}
}
};
threads[i].start();
}
for (int i = 0; i < threads.length; ++i) {
threads[i].join();
}
assertTrue(failures.toString(), failures.isEmpty());
assertEquals(0, opsCount.get());
assertEquals(0, queue.size());
for (int i = 1; i <= NUM_TABLES; ++i) {
TableName table = TableName.valueOf(String.format("testtb-%04d", i));
assertTrue("queue should be deleted, table=" + table, queue.markTableAsDeleted(table));
}
}
public static class TestTableProcSet {
private final MasterProcedureQueue queue;
private Map<Long, TableProcedureInterface> procsMap =
new ConcurrentHashMap<Long, TableProcedureInterface>();
public TestTableProcSet(final MasterProcedureQueue queue) {
this.queue = queue;
}
public void addBack(TableProcedureInterface tableProc) {
Procedure proc = (Procedure)tableProc;
procsMap.put(proc.getProcId(), tableProc);
queue.addBack(proc);
}
public void addFront(TableProcedureInterface tableProc) {
Procedure proc = (Procedure)tableProc;
procsMap.put(proc.getProcId(), tableProc);
queue.addFront(proc);
}
public TableProcedureInterface acquire() {
TableProcedureInterface proc = null;
boolean avail = false;
while (!avail) {
Long procId = queue.poll();
proc = procId != null ? procsMap.remove(procId) : null;
if (proc == null) break;
switch (proc.getTableOperationType()) {
case CREATE:
case DELETE:
case EDIT:
avail = queue.tryAcquireTableWrite(proc.getTableName(),
"op="+ proc.getTableOperationType());
break;
case READ:
avail = queue.tryAcquireTableRead(proc.getTableName(),
"op="+ proc.getTableOperationType());
break;
}
if (!avail) {
addFront(proc);
LOG.debug("yield procId=" + procId);
}
}
return proc;
}
public void release(TableProcedureInterface proc) {
switch (proc.getTableOperationType()) {
case CREATE:
case DELETE:
case EDIT:
queue.releaseTableWrite(proc.getTableName());
break;
case READ:
queue.releaseTableRead(proc.getTableName());
break;
}
}
}
public static class TestTableProcedure extends Procedure<Void>
implements TableProcedureInterface {
private final TableOperationType opType;
private final TableName tableName;
public TestTableProcedure() {
throw new UnsupportedOperationException("recovery should not be triggered here");
}
public TestTableProcedure(long procId, TableName tableName, TableOperationType opType) {
this.tableName = tableName;
this.opType = opType;
setProcId(procId);
}
@Override
public TableName getTableName() {
return tableName;
}
@Override
public TableOperationType getTableOperationType() {
return opType;
}
@Override
protected Procedure[] execute(Void env) {
return null;
}
@Override
protected void rollback(Void env) {
throw new UnsupportedOperationException();
}
@Override
protected boolean abort(Void env) {
throw new UnsupportedOperationException();
}
@Override
protected void serializeStateData(final OutputStream stream) throws IOException {}
@Override
protected void deserializeStateData(final InputStream stream) throws IOException {}
}
}