HBASE-13203 Procedure v2 - master create/delete table
This commit is contained in:
parent
04246c6c3d
commit
b5f1f98a25
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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>
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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;
|
||||
}
|
|
@ -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>
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
|
@ -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> {
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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 */ }
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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 {}
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue