HBASE-13212: Procedure V2 - master Create/Modify/Delete namespace (Stephen Yuan Jiang)

This commit is contained in:
Stephen Yuan Jiang 2015-08-26 08:18:03 -07:00
parent 6661f2d025
commit dc79b3c5c9
17 changed files with 5495 additions and 407 deletions

View File

@ -177,6 +177,8 @@ message ModifyTableResponse {
message CreateNamespaceRequest {
required NamespaceDescriptor namespaceDescriptor = 1;
optional uint64 nonce_group = 2 [default = 0];
optional uint64 nonce = 3 [default = 0];
}
message CreateNamespaceResponse {
@ -184,6 +186,8 @@ message CreateNamespaceResponse {
message DeleteNamespaceRequest {
required string namespaceName = 1;
optional uint64 nonce_group = 2 [default = 0];
optional uint64 nonce = 3 [default = 0];
}
message DeleteNamespaceResponse {
@ -191,6 +195,8 @@ message DeleteNamespaceResponse {
message ModifyNamespaceRequest {
required NamespaceDescriptor namespaceDescriptor = 1;
optional uint64 nonce_group = 2 [default = 0];
optional uint64 nonce = 3 [default = 0];
}
message ModifyNamespaceResponse {

View File

@ -110,6 +110,42 @@ message DeleteTableStateData {
repeated RegionInfo region_info = 3;
}
enum CreateNamespaceState {
CREATE_NAMESPACE_PREPARE = 1;
CREATE_NAMESPACE_CREATE_DIRECTORY = 2;
CREATE_NAMESPACE_INSERT_INTO_NS_TABLE = 3;
CREATE_NAMESPACE_UPDATE_ZK = 4;
CREATE_NAMESPACE_SET_NAMESPACE_QUOTA = 5;
}
message CreateNamespaceStateData {
required NamespaceDescriptor namespace_descriptor = 1;
}
enum ModifyNamespaceState {
MODIFY_NAMESPACE_PREPARE = 1;
MODIFY_NAMESPACE_UPDATE_NS_TABLE = 2;
MODIFY_NAMESPACE_UPDATE_ZK = 3;
}
message ModifyNamespaceStateData {
required NamespaceDescriptor namespace_descriptor = 1;
optional NamespaceDescriptor unmodified_namespace_descriptor = 2;
}
enum DeleteNamespaceState {
DELETE_NAMESPACE_PREPARE = 1;
DELETE_NAMESPACE_DELETE_FROM_NS_TABLE = 2;
DELETE_NAMESPACE_REMOVE_FROM_ZK = 3;
DELETE_NAMESPACE_DELETE_DIRECTORIES = 4;
DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA = 5;
}
message DeleteNamespaceStateData {
required string namespace_name = 1;
optional NamespaceDescriptor namespace_descriptor = 2;
}
enum AddColumnFamilyState {
ADD_COLUMN_FAMILY_PREPARE = 1;
ADD_COLUMN_FAMILY_PRE_OPERATION = 2;

View File

@ -169,8 +169,13 @@ public class ZKNamespaceManager extends ZooKeeperListener {
try {
ZKUtil.deleteNode(watcher, zNode);
} catch (KeeperException e) {
LOG.error("Failed updating permissions for namespace "+name, e);
throw new IOException("Failed updating permissions for namespace "+name, e);
if (e instanceof KeeperException.NoNodeException) {
// If the node does not exist, it could be already deleted. Continue without fail.
LOG.warn("The ZNode " + zNode + " for namespace " + name + " does not exist.");
} else {
LOG.error("Failed updating permissions for namespace " + name, e);
throw new IOException("Failed updating permissions for namespace " + name, e);
}
}
}

View File

@ -91,14 +91,17 @@ import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
import org.apache.hadoop.hbase.master.handler.DispatchMergingRegionHandler;
import org.apache.hadoop.hbase.master.procedure.AddColumnFamilyProcedure;
import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure;
import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
import org.apache.hadoop.hbase.master.procedure.DeleteColumnFamilyProcedure;
import org.apache.hadoop.hbase.master.procedure.DeleteNamespaceProcedure;
import org.apache.hadoop.hbase.master.procedure.DeleteTableProcedure;
import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.ModifyColumnFamilyProcedure;
import org.apache.hadoop.hbase.master.procedure.ModifyNamespaceProcedure;
import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
@ -1037,6 +1040,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
return tableStateManager;
}
@Override
public TableNamespaceManager getTableNamespaceManager() {
return tableNamespaceManager;
}
/*
* Start up all services. If any of these threads gets an unhandled exception
* then they just die with a logged message. This should be fine because
@ -2191,7 +2199,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
void checkNamespaceManagerReady() throws IOException {
checkInitialized();
if (tableNamespaceManager == null ||
!tableNamespaceManager.isTableAvailableAndInitialized()) {
!tableNamespaceManager.isTableAvailableAndInitialized(true)) {
throw new IOException("Table Namespace Manager not ready yet, try again later");
}
}
@ -2332,7 +2340,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
@Override
public void createNamespace(NamespaceDescriptor descriptor) throws IOException {
public void createNamespace(
final NamespaceDescriptor descriptor,
final long nonceGroup,
final long nonce) throws IOException {
TableName.isLegalNamespaceName(Bytes.toBytes(descriptor.getName()));
checkNamespaceManagerReady();
if (cpHost != null) {
@ -2340,15 +2351,31 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
return;
}
}
LOG.info(getClientIdAuditPrefix() + " creating " + descriptor);
tableNamespaceManager.create(descriptor);
createNamespaceSync(descriptor, nonceGroup, nonce);
if (cpHost != null) {
cpHost.postCreateNamespace(descriptor);
}
}
@Override
public void modifyNamespace(NamespaceDescriptor descriptor) throws IOException {
public void createNamespaceSync(
final NamespaceDescriptor descriptor,
final long nonceGroup,
final long nonce) throws IOException {
LOG.info(getClientIdAuditPrefix() + " creating " + descriptor);
// Execute the operation synchronously - wait for the operation to complete before continuing.
long procId = this.procedureExecutor.submitProcedure(
new CreateNamespaceProcedure(procedureExecutor.getEnvironment(), descriptor),
nonceGroup,
nonce);
ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
}
@Override
public void modifyNamespace(
final NamespaceDescriptor descriptor,
final long nonceGroup,
final long nonce) throws IOException {
TableName.isLegalNamespaceName(Bytes.toBytes(descriptor.getName()));
checkNamespaceManagerReady();
if (cpHost != null) {
@ -2357,14 +2384,22 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
}
LOG.info(getClientIdAuditPrefix() + " modify " + descriptor);
tableNamespaceManager.update(descriptor);
// Execute the operation synchronously - wait for the operation to complete before continuing.
long procId = this.procedureExecutor.submitProcedure(
new ModifyNamespaceProcedure(procedureExecutor.getEnvironment(), descriptor),
nonceGroup,
nonce);
ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
if (cpHost != null) {
cpHost.postModifyNamespace(descriptor);
}
}
@Override
public void deleteNamespace(String name) throws IOException {
public void deleteNamespace(
final String name,
final long nonceGroup,
final long nonce) throws IOException {
checkNamespaceManagerReady();
if (cpHost != null) {
if (cpHost.preDeleteNamespace(name)) {
@ -2372,7 +2407,12 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
}
LOG.info(getClientIdAuditPrefix() + " delete " + name);
tableNamespaceManager.remove(name);
// Execute the operation synchronously - wait for the operation to complete before continuing.
long procId = this.procedureExecutor.submitProcedure(
new DeleteNamespaceProcedure(procedureExecutor.getEnvironment(), name),
nonceGroup,
nonce);
ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
if (cpHost != null) {
cpHost.postDeleteNamespace(name);
}

View File

@ -417,7 +417,10 @@ public class MasterRpcServices extends RSRpcServices
public CreateNamespaceResponse createNamespace(RpcController controller,
CreateNamespaceRequest request) throws ServiceException {
try {
master.createNamespace(ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()));
master.createNamespace(
ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()),
request.getNonceGroup(),
request.getNonce());
return CreateNamespaceResponse.getDefaultInstance();
} catch (IOException e) {
throw new ServiceException(e);
@ -457,7 +460,10 @@ public class MasterRpcServices extends RSRpcServices
public DeleteNamespaceResponse deleteNamespace(RpcController controller,
DeleteNamespaceRequest request) throws ServiceException {
try {
master.deleteNamespace(request.getNamespaceName());
master.deleteNamespace(
request.getNamespaceName(),
request.getNonceGroup(),
request.getNonce());
return DeleteNamespaceResponse.getDefaultInstance();
} catch (IOException e) {
throw new ServiceException(e);
@ -1113,7 +1119,9 @@ public class MasterRpcServices extends RSRpcServices
ModifyNamespaceRequest request) throws ServiceException {
try {
master.modifyNamespace(
ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()));
ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()),
request.getNonceGroup(),
request.getNonce());
return ModifyNamespaceResponse.getDefaultInstance();
} catch (IOException e) {
throw new ServiceException(e);

View File

@ -78,6 +78,11 @@ public interface MasterServices extends Server {
*/
MasterCoprocessorHost getMasterCoprocessorHost();
/**
* @return Master's instance of {@link TableNamespaceManager}
*/
TableNamespaceManager getTableNamespaceManager();
/**
* @return Master's instance of {@link MasterQuotaManager}
*/
@ -270,23 +275,50 @@ public interface MasterServices extends Server {
/**
* Create a new namespace
* @param descriptor descriptor which describes the new namespace
* @param nonceGroup
* @param nonce
* @throws IOException
*/
public void createNamespace(NamespaceDescriptor descriptor) throws IOException;
public void createNamespace(
final NamespaceDescriptor descriptor,
final long nonceGroup,
final long nonce) throws IOException;
/**
* Create a new namespace synchronously.
* @param descriptor descriptor which describes the new namespace
* @param nonceGroup
* @param nonce
* @throws IOException
*/
public void createNamespaceSync(
final NamespaceDescriptor descriptor,
final long nonceGroup,
final long nonce) throws IOException;
/**
* Modify an existing namespace
* @param descriptor descriptor which updates the existing namespace
* @param nonceGroup
* @param nonce
* @throws IOException
*/
public void modifyNamespace(NamespaceDescriptor descriptor) throws IOException;
public void modifyNamespace(
final NamespaceDescriptor descriptor,
final long nonceGroup,
final long nonce) throws IOException;
/**
* Delete an existing namespace. Only empty namespaces (no tables) can be removed.
* @param name namespace name
* @param nonceGroup
* @param nonce
* @throws IOException
*/
public void deleteNamespace(String name) throws IOException;
public void deleteNamespace(
final String name,
final long nonceGroup,
final long nonce) throws IOException;
/**
* Get a namespace descriptor by name

View File

@ -18,26 +18,23 @@
package org.apache.hadoop.hbase.master;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.NavigableSet;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NamespaceExistException;
import org.apache.hadoop.hbase.NamespaceNotFoundException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZKNamespaceManager;
import org.apache.hadoop.hbase.MetaTableAccessor;
@ -49,12 +46,12 @@ 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.procedure.CreateNamespaceProcedure;
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;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import com.google.common.collect.Sets;
@ -70,18 +67,39 @@ public class TableNamespaceManager {
private Configuration conf;
private MasterServices masterServices;
private Table nsTable;
private Table nsTable = null;
private ZKNamespaceManager zkNamespaceManager;
private boolean initialized;
private final ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock();
public static final String KEY_MAX_REGIONS = "hbase.namespace.quota.maxregions";
public static final String KEY_MAX_TABLES = "hbase.namespace.quota.maxtables";
static final String NS_INIT_TIMEOUT = "hbase.master.namespace.init.timeout";
static final int DEFAULT_NS_INIT_TIMEOUT = 300000;
/** Configuration key for time out for trying to acquire table locks */
private static final String TABLE_WRITE_LOCK_TIMEOUT_MS =
"hbase.table.write.lock.timeout.ms";
/** Configuration key for time out for trying to acquire table locks */
private static final String TABLE_READ_LOCK_TIMEOUT_MS =
"hbase.table.read.lock.timeout.ms";
private static final long DEFAULT_TABLE_WRITE_LOCK_TIMEOUT_MS = 600 * 1000; //10 min default
private static final long DEFAULT_TABLE_READ_LOCK_TIMEOUT_MS = 600 * 1000; //10 min default
private long exclusiveLockTimeoutMs;
private long sharedLockTimeoutMs;
public TableNamespaceManager(MasterServices masterServices) {
this.masterServices = masterServices;
this.conf = masterServices.getConfiguration();
this.exclusiveLockTimeoutMs = conf.getLong(
TABLE_WRITE_LOCK_TIMEOUT_MS,
DEFAULT_TABLE_WRITE_LOCK_TIMEOUT_MS);
this.sharedLockTimeoutMs = conf.getLong(
TABLE_READ_LOCK_TIMEOUT_MS,
DEFAULT_TABLE_READ_LOCK_TIMEOUT_MS);
}
public void start() throws IOException {
@ -95,7 +113,7 @@ public class TableNamespaceManager {
// Wait for the namespace table to be initialized.
long startTime = EnvironmentEdgeManager.currentTime();
int timeout = conf.getInt(NS_INIT_TIMEOUT, DEFAULT_NS_INIT_TIMEOUT);
while (!isTableAvailableAndInitialized()) {
while (!isTableAvailableAndInitialized(false)) {
if (EnvironmentEdgeManager.currentTime() - startTime + 100 > timeout) {
// We can't do anything if ns is not online.
throw new IOException("Timedout " + timeout + "ms waiting for namespace table to "
@ -109,28 +127,51 @@ public class TableNamespaceManager {
}
private synchronized Table getNamespaceTable() throws IOException {
if (!isTableAvailableAndInitialized()) {
if (!isTableNamespaceManagerInitialized()) {
throw new IOException(this.getClass().getName() + " isn't ready to serve");
}
return nsTable;
}
private synchronized boolean acquireSharedLock() throws IOException {
try {
return rwLock.readLock().tryLock(sharedLockTimeoutMs, TimeUnit.MILLISECONDS);
} catch (InterruptedException e) {
throw (InterruptedIOException) new InterruptedIOException().initCause(e);
}
}
public synchronized void releaseSharedLock() {
rwLock.readLock().unlock();
}
public synchronized boolean acquireExclusiveLock() {
try {
return rwLock.writeLock().tryLock(exclusiveLockTimeoutMs, TimeUnit.MILLISECONDS);
} catch (InterruptedException e) {
return false;
}
}
public synchronized void releaseExclusiveLock() {
rwLock.writeLock().unlock();
}
/*
* check whether a namespace has already existed.
*/
public boolean doesNamespaceExist(final String namespaceName) throws IOException {
if (nsTable == null) {
throw new IOException(this.getClass().getName() + " isn't ready to serve");
}
return (get(nsTable, namespaceName) != null);
}
public synchronized NamespaceDescriptor get(String name) throws IOException {
if (!isTableAvailableAndInitialized()) return null;
return zkNamespaceManager.get(name);
}
public synchronized void create(NamespaceDescriptor ns) throws IOException {
create(getNamespaceTable(), ns);
}
public synchronized void update(NamespaceDescriptor ns) throws IOException {
Table table = getNamespaceTable();
if (get(table, ns.getName()) == null) {
throw new NamespaceNotFoundException(ns.getName());
if (!isTableNamespaceManagerInitialized()) {
return null;
}
upsert(table, ns);
return zkNamespaceManager.get(name);
}
private NamespaceDescriptor get(Table table, String name) throws IOException {
@ -145,78 +186,51 @@ public class TableNamespaceManager {
HBaseProtos.NamespaceDescriptor.parseFrom(val));
}
private void create(Table table, NamespaceDescriptor ns) throws IOException {
if (get(table, ns.getName()) != null) {
throw new NamespaceExistException(ns.getName());
public void insertIntoNSTable(final NamespaceDescriptor ns) throws IOException {
if (nsTable == null) {
throw new IOException(this.getClass().getName() + " isn't ready to serve");
}
validateTableAndRegionCount(ns);
FileSystem fs = masterServices.getMasterFileSystem().getFileSystem();
fs.mkdirs(FSUtils.getNamespaceDir(
masterServices.getMasterFileSystem().getRootDir(), ns.getName()));
upsert(table, ns);
if (this.masterServices.isInitialized()) {
this.masterServices.getMasterQuotaManager().setNamespaceQuota(ns);
}
}
private void upsert(Table table, NamespaceDescriptor ns) throws IOException {
validateTableAndRegionCount(ns);
Put p = new Put(Bytes.toBytes(ns.getName()));
p.addImmutable(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES,
HTableDescriptor.NAMESPACE_COL_DESC_BYTES,
ProtobufUtil.toProtoNamespaceDescriptor(ns).toByteArray());
table.put(p);
nsTable.put(p);
}
public void updateZKNamespaceManager(final NamespaceDescriptor ns) throws IOException {
try {
zkNamespaceManager.update(ns);
} catch(IOException ex) {
String msg = "Failed to update namespace information in ZK. Aborting.";
LOG.fatal(msg, ex);
masterServices.abort(msg, ex);
} catch (IOException ex) {
String msg = "Failed to update namespace information in ZK.";
LOG.error(msg, ex);
throw new IOException(msg, ex);
}
}
public synchronized void remove(String name) throws IOException {
if (get(name) == null) {
throw new NamespaceNotFoundException(name);
public void removeFromNSTable(final String namespaceName) throws IOException {
if (nsTable == null) {
throw new IOException(this.getClass().getName() + " isn't ready to serve");
}
if (NamespaceDescriptor.RESERVED_NAMESPACES.contains(name)) {
throw new ConstraintException("Reserved namespace "+name+" cannot be removed.");
}
int tableCount;
try {
tableCount = masterServices.listTableDescriptorsByNamespace(name).size();
} catch (FileNotFoundException fnfe) {
throw new NamespaceNotFoundException(name);
}
if (tableCount > 0) {
throw new ConstraintException("Only empty namespaces can be removed. " +
"Namespace "+name+" has "+tableCount+" tables");
}
Delete d = new Delete(Bytes.toBytes(name));
getNamespaceTable().delete(d);
//don't abort if cleanup isn't complete
//it will be replaced on new namespace creation
zkNamespaceManager.remove(name);
FileSystem fs = masterServices.getMasterFileSystem().getFileSystem();
for(FileStatus status :
fs.listStatus(FSUtils.getNamespaceDir(
masterServices.getMasterFileSystem().getRootDir(), name))) {
if (!HConstants.HBASE_NON_TABLE_DIRS.contains(status.getPath().getName())) {
throw new IOException("Namespace directory contains table dir: "+status.getPath());
}
}
if (!fs.delete(FSUtils.getNamespaceDir(
masterServices.getMasterFileSystem().getRootDir(), name), true)) {
throw new IOException("Failed to remove namespace: "+name);
}
this.masterServices.getMasterQuotaManager().removeNamespaceQuota(name);
Delete d = new Delete(Bytes.toBytes(namespaceName));
nsTable.delete(d);
}
public void removeFromZKNamespaceManager(final String namespaceName) throws IOException {
zkNamespaceManager.remove(namespaceName);
}
public synchronized NavigableSet<NamespaceDescriptor> list() throws IOException {
NavigableSet<NamespaceDescriptor> ret =
Sets.newTreeSet(NamespaceDescriptor.NAMESPACE_DESCRIPTOR_COMPARATOR);
ResultScanner scanner = getNamespaceTable().getScanner(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES);
ResultScanner scanner =
getNamespaceTable().getScanner(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES);
boolean locked = false;
try {
locked = acquireSharedLock();
if (!locked) {
throw new IOException(
"Fail to acquire lock to scan namespace list. Some namespace DDL is in progress.");
}
for(Result r : scanner) {
byte[] val = CellUtil.cloneValue(r.getColumnLatestCell(
HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES,
@ -226,6 +240,9 @@ public class TableNamespaceManager {
}
} finally {
scanner.close();
if (locked) {
releaseSharedLock();
}
}
return ret;
}
@ -242,6 +259,15 @@ public class TableNamespaceManager {
newRegions));
}
@SuppressWarnings("deprecation")
private boolean isTableNamespaceManagerInitialized() throws IOException {
if (initialized) {
this.nsTable = this.masterServices.getConnection().getTable(TableName.NAMESPACE_TABLE_NAME);
return true;
}
return false;
}
/**
* This method checks if the namespace table is assigned and then
* tries to create its HTable. If it was already created before, it also makes
@ -251,25 +277,55 @@ public class TableNamespaceManager {
* @throws IOException
*/
@SuppressWarnings("deprecation")
public synchronized boolean isTableAvailableAndInitialized() throws IOException {
public synchronized boolean isTableAvailableAndInitialized(
final boolean createNamespaceAync) throws IOException {
// Did we already get a table? If so, still make sure it's available
if (initialized) {
this.nsTable = this.masterServices.getConnection().getTable(TableName.NAMESPACE_TABLE_NAME);
if (isTableNamespaceManagerInitialized()) {
return true;
}
// Now check if the table is assigned, if not then fail fast
if (isTableAssigned() && isTableEnabled()) {
try {
boolean initGoodSofar = true;
nsTable = this.masterServices.getConnection().getTable(TableName.NAMESPACE_TABLE_NAME);
zkNamespaceManager = new ZKNamespaceManager(masterServices.getZooKeeper());
zkNamespaceManager.start();
if (get(nsTable, NamespaceDescriptor.DEFAULT_NAMESPACE.getName()) == null) {
create(nsTable, NamespaceDescriptor.DEFAULT_NAMESPACE);
if (createNamespaceAync) {
masterServices.getMasterProcedureExecutor().submitProcedure(
new CreateNamespaceProcedure(
masterServices.getMasterProcedureExecutor().getEnvironment(),
NamespaceDescriptor.DEFAULT_NAMESPACE));
initGoodSofar = false;
}
else {
masterServices.createNamespaceSync(
NamespaceDescriptor.DEFAULT_NAMESPACE,
HConstants.NO_NONCE,
HConstants.NO_NONCE);
}
}
if (get(nsTable, NamespaceDescriptor.SYSTEM_NAMESPACE.getName()) == null) {
create(nsTable, NamespaceDescriptor.SYSTEM_NAMESPACE);
if (createNamespaceAync) {
masterServices.getMasterProcedureExecutor().submitProcedure(
new CreateNamespaceProcedure(
masterServices.getMasterProcedureExecutor().getEnvironment(),
NamespaceDescriptor.SYSTEM_NAMESPACE));
initGoodSofar = false;
}
else {
masterServices.createNamespaceSync(
NamespaceDescriptor.SYSTEM_NAMESPACE,
HConstants.NO_NONCE,
HConstants.NO_NONCE);
}
}
if (!initGoodSofar) {
// some required namespace is created asynchronized. We should complete init later.
return false;
}
ResultScanner scanner = nsTable.getScanner(HTableDescriptor.NAMESPACE_FAMILY_INFO_BYTES);
@ -312,7 +368,7 @@ public class TableNamespaceManager {
.getRegionStates().getRegionsOfTable(TableName.NAMESPACE_TABLE_NAME).isEmpty();
}
void validateTableAndRegionCount(NamespaceDescriptor desc) throws IOException {
public void validateTableAndRegionCount(NamespaceDescriptor desc) throws IOException {
if (getMaxRegions(desc) <= 0) {
throw new ConstraintException("The max region quota for " + desc.getName()
+ " is less than or equal to zero.");

View File

@ -0,0 +1,364 @@
/**
* 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.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NamespaceExistException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.TableNamespaceManager;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateNamespaceState;
import org.apache.hadoop.hbase.util.FSUtils;
/**
* The procedure to create a new namespace.
*/
@InterfaceAudience.Private
public class CreateNamespaceProcedure
extends StateMachineProcedure<MasterProcedureEnv, CreateNamespaceState>
implements TableProcedureInterface {
private static final Log LOG = LogFactory.getLog(CreateNamespaceProcedure.class);
private final AtomicBoolean aborted = new AtomicBoolean(false);
private NamespaceDescriptor nsDescriptor;
private Boolean traceEnabled;
public CreateNamespaceProcedure() {
this.traceEnabled = null;
}
public CreateNamespaceProcedure(
final MasterProcedureEnv env,
final NamespaceDescriptor nsDescriptor) throws IOException {
this.nsDescriptor = nsDescriptor;
this.traceEnabled = null;
}
@Override
protected Flow executeFromState(final MasterProcedureEnv env, final CreateNamespaceState state)
throws InterruptedException {
if (isTraceEnabled()) {
LOG.trace(this + " execute state=" + state);
}
try {
switch (state) {
case CREATE_NAMESPACE_PREPARE:
prepareCreate(env);
setNextState(CreateNamespaceState.CREATE_NAMESPACE_CREATE_DIRECTORY);
break;
case CREATE_NAMESPACE_CREATE_DIRECTORY:
createDirectory(env, nsDescriptor);
setNextState(CreateNamespaceState.CREATE_NAMESPACE_INSERT_INTO_NS_TABLE);
break;
case CREATE_NAMESPACE_INSERT_INTO_NS_TABLE:
insertIntoNSTable(env, nsDescriptor);
setNextState(CreateNamespaceState.CREATE_NAMESPACE_UPDATE_ZK);
break;
case CREATE_NAMESPACE_UPDATE_ZK:
updateZKNamespaceManager(env, nsDescriptor);
setNextState(CreateNamespaceState.CREATE_NAMESPACE_SET_NAMESPACE_QUOTA);
break;
case CREATE_NAMESPACE_SET_NAMESPACE_QUOTA:
setNamespaceQuota(env, nsDescriptor);
return Flow.NO_MORE_STATE;
default:
throw new UnsupportedOperationException(this + " unhandled state=" + state);
}
} catch (IOException e) {
LOG.warn("Error trying to create the namespace" + nsDescriptor.getName()
+ " (in state=" + state + ")", e);
setFailure("master-create-namespace", e);
}
return Flow.HAS_MORE_STATE;
}
@Override
protected void rollbackState(final MasterProcedureEnv env, final CreateNamespaceState state)
throws IOException {
if (isTraceEnabled()) {
LOG.trace(this + " rollback state=" + state);
}
try {
switch (state) {
case CREATE_NAMESPACE_SET_NAMESPACE_QUOTA:
rollbackSetNamespaceQuota(env);
break;
case CREATE_NAMESPACE_UPDATE_ZK:
rollbackZKNamespaceManagerChange(env);
break;
case CREATE_NAMESPACE_INSERT_INTO_NS_TABLE:
rollbackInsertIntoNSTable(env);
break;
case CREATE_NAMESPACE_CREATE_DIRECTORY:
rollbackCreateDirectory(env);
break;
case CREATE_NAMESPACE_PREPARE:
break; // nothing to do
default:
throw new UnsupportedOperationException(this + " 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 + " for creating the namespace "
+ nsDescriptor.getName(), e);
throw e;
}
}
@Override
protected CreateNamespaceState getState(final int stateId) {
return CreateNamespaceState.valueOf(stateId);
}
@Override
protected int getStateId(final CreateNamespaceState state) {
return state.getNumber();
}
@Override
protected CreateNamespaceState getInitialState() {
return CreateNamespaceState.CREATE_NAMESPACE_PREPARE;
}
@Override
protected void setNextState(CreateNamespaceState state) {
if (aborted.get()) {
setAbortFailure("create-namespace", "abort requested");
} else {
super.setNextState(state);
}
}
@Override
public boolean abort(final MasterProcedureEnv env) {
aborted.set(true);
return true;
}
@Override
public void serializeStateData(final OutputStream stream) throws IOException {
super.serializeStateData(stream);
MasterProcedureProtos.CreateNamespaceStateData.Builder createNamespaceMsg =
MasterProcedureProtos.CreateNamespaceStateData.newBuilder().setNamespaceDescriptor(
ProtobufUtil.toProtoNamespaceDescriptor(this.nsDescriptor));
createNamespaceMsg.build().writeDelimitedTo(stream);
}
@Override
public void deserializeStateData(final InputStream stream) throws IOException {
super.deserializeStateData(stream);
MasterProcedureProtos.CreateNamespaceStateData createNamespaceMsg =
MasterProcedureProtos.CreateNamespaceStateData.parseDelimitedFrom(stream);
nsDescriptor = ProtobufUtil.toNamespaceDescriptor(createNamespaceMsg.getNamespaceDescriptor());
}
@Override
public void toStringClassDetails(StringBuilder sb) {
sb.append(getClass().getSimpleName());
sb.append(" (Namespace=");
sb.append(nsDescriptor.getName());
sb.append(")");
}
@Override
protected boolean acquireLock(final MasterProcedureEnv env) {
if (!env.getMasterServices().isInitialized()) {
// Namespace manager might not be ready if master is not fully initialized,
// return false to reject user namespace creation; return true for default
// and system namespace creation (this is part of master initialization).
if (nsDescriptor.equals(NamespaceDescriptor.DEFAULT_NAMESPACE) ||
nsDescriptor.equals(NamespaceDescriptor.SYSTEM_NAMESPACE)) {
return true;
}
return false;
}
return getTableNamespaceManager(env).acquireExclusiveLock();
}
@Override
protected void releaseLock(final MasterProcedureEnv env) {
if (env.getMasterServices().isInitialized()) {
getTableNamespaceManager(env).releaseExclusiveLock();
}
}
@Override
public TableName getTableName() {
return TableName.NAMESPACE_TABLE_NAME;
}
@Override
public TableOperationType getTableOperationType() {
return TableOperationType.EDIT;
}
/**
* Action before any real action of creating namespace.
* @param env MasterProcedureEnv
* @throws IOException
*/
private void prepareCreate(final MasterProcedureEnv env) throws IOException {
if (getTableNamespaceManager(env).doesNamespaceExist(nsDescriptor.getName())) {
throw new NamespaceExistException(nsDescriptor.getName());
}
getTableNamespaceManager(env).validateTableAndRegionCount(nsDescriptor);
}
/**
* Create the namespace directory
* @param env MasterProcedureEnv
* @param nsDescriptor NamespaceDescriptor
* @throws IOException
*/
protected static void createDirectory(
final MasterProcedureEnv env,
final NamespaceDescriptor nsDescriptor) throws IOException {
MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
mfs.getFileSystem().mkdirs(
FSUtils.getNamespaceDir(mfs.getRootDir(), nsDescriptor.getName()));
}
/**
* undo create directory
* @param env MasterProcedureEnv
* @throws IOException
*/
private void rollbackCreateDirectory(final MasterProcedureEnv env) throws IOException {
try {
DeleteNamespaceProcedure.deleteDirectory(env, nsDescriptor.getName());
} catch (Exception e) {
// Ignore exception
LOG.debug("Rollback of createDirectory throws exception: " + e);
}
}
/**
* Insert the row into ns table
* @param env MasterProcedureEnv
* @param nsDescriptor NamespaceDescriptor
* @throws IOException
*/
protected static void insertIntoNSTable(
final MasterProcedureEnv env,
final NamespaceDescriptor nsDescriptor) throws IOException {
getTableNamespaceManager(env).insertIntoNSTable(nsDescriptor);
}
/**
* Undo the insert.
* @param env MasterProcedureEnv
* @throws IOException
*/
private void rollbackInsertIntoNSTable(final MasterProcedureEnv env) throws IOException {
try {
DeleteNamespaceProcedure.deleteFromNSTable(env, nsDescriptor.getName());
} catch (Exception e) {
// Ignore exception
LOG.debug("Rollback of insertIntoNSTable throws exception: " + e);
}
}
/**
* Update Zookeeper.
* @param env MasterProcedureEnv
* @param nsDescriptor NamespaceDescriptor
* @throws IOException
*/
protected static void updateZKNamespaceManager(
final MasterProcedureEnv env,
final NamespaceDescriptor nsDescriptor) throws IOException {
getTableNamespaceManager(env).updateZKNamespaceManager(nsDescriptor);
}
/**
* rollback Zookeeper update.
* @param env MasterProcedureEnv
* @throws IOException
*/
private void rollbackZKNamespaceManagerChange(final MasterProcedureEnv env) throws IOException {
try {
DeleteNamespaceProcedure.removeFromZKNamespaceManager(env, nsDescriptor.getName());
} catch (Exception e) {
// Ignore exception
LOG.debug("Rollback of updateZKNamespaceManager throws exception: " + e);
}
}
/**
* Set quota for the namespace
* @param env MasterProcedureEnv
* @param nsDescriptor NamespaceDescriptor
* @throws IOException
**/
protected static void setNamespaceQuota(
final MasterProcedureEnv env,
final NamespaceDescriptor nsDescriptor) throws IOException {
if (env.getMasterServices().isInitialized()) {
env.getMasterServices().getMasterQuotaManager().setNamespaceQuota(nsDescriptor);
}
}
/**
* remove quota for the namespace if exists
* @param env MasterProcedureEnv
* @throws IOException
**/
private void rollbackSetNamespaceQuota(final MasterProcedureEnv env) throws IOException {
try {
DeleteNamespaceProcedure.removeNamespaceQuota(env, nsDescriptor.getName());
} catch (Exception e) {
// Ignore exception
LOG.debug("Rollback of setNamespaceQuota throws exception: " + e);
}
}
private static TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
return env.getMasterServices().getTableNamespaceManager();
}
/**
* The procedure could be restarted from a different machine. If the variable is null, we need to
* retrieve it.
* @return traceEnabled
*/
private Boolean isTraceEnabled() {
if (traceEnabled == null) {
traceEnabled = LOG.isTraceEnabled();
}
return traceEnabled;
}
}

View File

@ -0,0 +1,398 @@
/**
* 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.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NamespaceNotFoundException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.TableNamespaceManager;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteNamespaceState;
import org.apache.hadoop.hbase.util.FSUtils;
/**
* The procedure to remove a namespace.
*/
@InterfaceAudience.Private
public class DeleteNamespaceProcedure
extends StateMachineProcedure<MasterProcedureEnv, DeleteNamespaceState>
implements TableProcedureInterface {
private static final Log LOG = LogFactory.getLog(DeleteNamespaceProcedure.class);
private final AtomicBoolean aborted = new AtomicBoolean(false);
private NamespaceDescriptor nsDescriptor;
private String namespaceName;
private Boolean traceEnabled;
public DeleteNamespaceProcedure() {
this.nsDescriptor = null;
this.traceEnabled = null;
}
public DeleteNamespaceProcedure(
final MasterProcedureEnv env,
final String namespaceName) throws IOException {
this.namespaceName = namespaceName;
this.nsDescriptor = null;
this.traceEnabled = null;
}
@Override
protected Flow executeFromState(final MasterProcedureEnv env, final DeleteNamespaceState state)
throws InterruptedException {
if (isTraceEnabled()) {
LOG.trace(this + " execute state=" + state);
}
try {
switch (state) {
case DELETE_NAMESPACE_PREPARE:
prepareDelete(env);
setNextState(DeleteNamespaceState.DELETE_NAMESPACE_DELETE_FROM_NS_TABLE);
break;
case DELETE_NAMESPACE_DELETE_FROM_NS_TABLE:
deleteFromNSTable(env, namespaceName);
setNextState(DeleteNamespaceState.DELETE_NAMESPACE_REMOVE_FROM_ZK);
break;
case DELETE_NAMESPACE_REMOVE_FROM_ZK:
removeFromZKNamespaceManager(env, namespaceName);
setNextState(DeleteNamespaceState.DELETE_NAMESPACE_DELETE_DIRECTORIES);
break;
case DELETE_NAMESPACE_DELETE_DIRECTORIES:
deleteDirectory(env, namespaceName);
setNextState(DeleteNamespaceState.DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA);
break;
case DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA:
removeNamespaceQuota(env, namespaceName);
return Flow.NO_MORE_STATE;
default:
throw new UnsupportedOperationException(this + " unhandled state=" + state);
}
} catch (IOException e) {
LOG.warn("Error trying to delete the namespace" + namespaceName
+ " (in state=" + state + ")", e);
setFailure("master-delete-namespace", e);
}
return Flow.HAS_MORE_STATE;
}
@Override
protected void rollbackState(final MasterProcedureEnv env, final DeleteNamespaceState state)
throws IOException {
if (isTraceEnabled()) {
LOG.trace(this + " rollback state=" + state);
}
try {
switch (state) {
case DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA:
rollbacRemoveNamespaceQuota(env);
break;
case DELETE_NAMESPACE_DELETE_DIRECTORIES:
rollbackDeleteDirectory(env);
break;
case DELETE_NAMESPACE_REMOVE_FROM_ZK:
undoRemoveFromZKNamespaceManager(env);
break;
case DELETE_NAMESPACE_DELETE_FROM_NS_TABLE:
undoDeleteFromNSTable(env);
break;
case DELETE_NAMESPACE_PREPARE:
break; // nothing to do
default:
throw new UnsupportedOperationException(this + " 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 + " for deleting the namespace "
+ namespaceName, e);
throw e;
}
}
@Override
protected DeleteNamespaceState getState(final int stateId) {
return DeleteNamespaceState.valueOf(stateId);
}
@Override
protected int getStateId(final DeleteNamespaceState state) {
return state.getNumber();
}
@Override
protected DeleteNamespaceState getInitialState() {
return DeleteNamespaceState.DELETE_NAMESPACE_PREPARE;
}
@Override
protected void setNextState(DeleteNamespaceState state) {
if (aborted.get()) {
setAbortFailure("delete-namespace", "abort requested");
} else {
super.setNextState(state);
}
}
@Override
public boolean abort(final MasterProcedureEnv env) {
aborted.set(true);
return true;
}
@Override
public void serializeStateData(final OutputStream stream) throws IOException {
super.serializeStateData(stream);
MasterProcedureProtos.DeleteNamespaceStateData.Builder deleteNamespaceMsg =
MasterProcedureProtos.DeleteNamespaceStateData.newBuilder().setNamespaceName(namespaceName);
if (this.nsDescriptor != null) {
deleteNamespaceMsg.setNamespaceDescriptor(
ProtobufUtil.toProtoNamespaceDescriptor(this.nsDescriptor));
}
deleteNamespaceMsg.build().writeDelimitedTo(stream);
}
@Override
public void deserializeStateData(final InputStream stream) throws IOException {
super.deserializeStateData(stream);
MasterProcedureProtos.DeleteNamespaceStateData deleteNamespaceMsg =
MasterProcedureProtos.DeleteNamespaceStateData.parseDelimitedFrom(stream);
namespaceName = deleteNamespaceMsg.getNamespaceName();
if (deleteNamespaceMsg.hasNamespaceDescriptor()) {
nsDescriptor =
ProtobufUtil.toNamespaceDescriptor(deleteNamespaceMsg.getNamespaceDescriptor());
}
}
@Override
public void toStringClassDetails(StringBuilder sb) {
sb.append(getClass().getSimpleName());
sb.append(" (Namespace=");
sb.append(namespaceName);
sb.append(")");
}
@Override
protected boolean acquireLock(final MasterProcedureEnv env) {
return getTableNamespaceManager(env).acquireExclusiveLock();
}
@Override
protected void releaseLock(final MasterProcedureEnv env) {
getTableNamespaceManager(env).releaseExclusiveLock();
}
@Override
public TableName getTableName() {
return TableName.NAMESPACE_TABLE_NAME;
}
@Override
public TableOperationType getTableOperationType() {
return TableOperationType.EDIT;
}
/**
* Action before any real action of deleting namespace.
* @param env MasterProcedureEnv
* @throws IOException
*/
private void prepareDelete(final MasterProcedureEnv env) throws IOException {
if (getTableNamespaceManager(env).doesNamespaceExist(namespaceName) == false) {
throw new NamespaceNotFoundException(namespaceName);
}
if (NamespaceDescriptor.RESERVED_NAMESPACES.contains(namespaceName)) {
throw new ConstraintException("Reserved namespace "+ namespaceName +" cannot be removed.");
}
int tableCount = 0;
try {
tableCount = env.getMasterServices().listTableDescriptorsByNamespace(namespaceName).size();
} catch (FileNotFoundException fnfe) {
throw new NamespaceNotFoundException(namespaceName);
}
if (tableCount > 0) {
throw new ConstraintException("Only empty namespaces can be removed. " +
"Namespace "+ namespaceName + " has "+ tableCount +" tables");
}
// This is used for rollback
nsDescriptor = getTableNamespaceManager(env).get(namespaceName);
}
/**
* delete the row from namespace table
* @param env MasterProcedureEnv
* @param namespaceName name of the namespace in string format
* @throws IOException
*/
protected static void deleteFromNSTable(
final MasterProcedureEnv env,
final String namespaceName) throws IOException {
getTableNamespaceManager(env).removeFromNSTable(namespaceName);
}
/**
* undo the delete
* @param env MasterProcedureEnv
* @throws IOException
*/
private void undoDeleteFromNSTable(final MasterProcedureEnv env) {
try {
if (nsDescriptor != null) {
CreateNamespaceProcedure.insertIntoNSTable(env, nsDescriptor);
}
} catch (Exception e) {
// Ignore
LOG.debug("Rollback of deleteFromNSTable throws exception: " + e);
}
}
/**
* remove from Zookeeper.
* @param env MasterProcedureEnv
* @param namespaceName name of the namespace in string format
* @throws IOException
*/
protected static void removeFromZKNamespaceManager(
final MasterProcedureEnv env,
final String namespaceName) throws IOException {
getTableNamespaceManager(env).removeFromZKNamespaceManager(namespaceName);
}
/**
* undo the remove from Zookeeper
* @param env MasterProcedureEnv
* @throws IOException
*/
private void undoRemoveFromZKNamespaceManager(final MasterProcedureEnv env) {
try {
if (nsDescriptor != null) {
CreateNamespaceProcedure.updateZKNamespaceManager(env, nsDescriptor);
}
} catch (Exception e) {
// Ignore
LOG.debug("Rollback of removeFromZKNamespaceManager throws exception: " + e);
}
}
/**
* Delete the namespace directories from the file system
* @param env MasterProcedureEnv
* @param namespaceName name of the namespace in string format
* @throws IOException
*/
protected static void deleteDirectory(
final MasterProcedureEnv env,
final String namespaceName) throws IOException {
MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
FileSystem fs = mfs.getFileSystem();
Path p = FSUtils.getNamespaceDir(mfs.getRootDir(), namespaceName);
try {
for(FileStatus status : fs.listStatus(p)) {
if (!HConstants.HBASE_NON_TABLE_DIRS.contains(status.getPath().getName())) {
throw new IOException("Namespace directory contains table dir: " + status.getPath());
}
}
if (!fs.delete(FSUtils.getNamespaceDir(mfs.getRootDir(), namespaceName), true)) {
throw new IOException("Failed to remove namespace: " + namespaceName);
}
} catch (FileNotFoundException e) {
// File already deleted, continue
LOG.debug("deleteDirectory throws exception: " + e);
}
}
/**
* undo delete directory
* @param env MasterProcedureEnv
* @throws IOException
*/
private void rollbackDeleteDirectory(final MasterProcedureEnv env) throws IOException {
try {
CreateNamespaceProcedure.createDirectory(env, nsDescriptor);
} catch (Exception e) {
// Ignore exception
LOG.debug("Rollback of deleteDirectory throws exception: " + e);
}
}
/**
* remove quota for the namespace
* @param env MasterProcedureEnv
* @param namespaceName name of the namespace in string format
* @throws IOException
**/
protected static void removeNamespaceQuota(
final MasterProcedureEnv env,
final String namespaceName) throws IOException {
env.getMasterServices().getMasterQuotaManager().removeNamespaceQuota(namespaceName);
}
/**
* undo remove quota for the namespace
* @param env MasterProcedureEnv
* @throws IOException
**/
private void rollbacRemoveNamespaceQuota(final MasterProcedureEnv env) throws IOException {
try {
CreateNamespaceProcedure.setNamespaceQuota(env, nsDescriptor);
} catch (Exception e) {
// Ignore exception
LOG.debug("Rollback of removeNamespaceQuota throws exception: " + e);
}
}
private static TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
return env.getMasterServices().getTableNamespaceManager();
}
/**
* The procedure could be restarted from a different machine. If the variable is null, we need to
* retrieve it.
* @return traceEnabled
*/
private Boolean isTraceEnabled() {
if (traceEnabled == null) {
traceEnabled = LOG.isTraceEnabled();
}
return traceEnabled;
}
}

View File

@ -0,0 +1,281 @@
/**
* 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.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NamespaceNotFoundException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.master.TableNamespaceManager;
import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ModifyNamespaceState;
/**
* The procedure to add a namespace to an existing table.
*/
@InterfaceAudience.Private
public class ModifyNamespaceProcedure
extends StateMachineProcedure<MasterProcedureEnv, ModifyNamespaceState>
implements TableProcedureInterface {
private static final Log LOG = LogFactory.getLog(ModifyNamespaceProcedure.class);
private final AtomicBoolean aborted = new AtomicBoolean(false);
private NamespaceDescriptor oldNsDescriptor;
private NamespaceDescriptor newNsDescriptor;
private Boolean traceEnabled;
public ModifyNamespaceProcedure() {
this.oldNsDescriptor = null;
this.traceEnabled = null;
}
public ModifyNamespaceProcedure(
final MasterProcedureEnv env,
final NamespaceDescriptor newNsDescriptor) throws IOException {
this.oldNsDescriptor = null;
this.newNsDescriptor = newNsDescriptor;
this.traceEnabled = null;
}
@Override
protected Flow executeFromState(final MasterProcedureEnv env, final ModifyNamespaceState state)
throws InterruptedException {
if (isTraceEnabled()) {
LOG.trace(this + " execute state=" + state);
}
try {
switch (state) {
case MODIFY_NAMESPACE_PREPARE:
prepareModify(env);
setNextState(ModifyNamespaceState.MODIFY_NAMESPACE_UPDATE_NS_TABLE);
break;
case MODIFY_NAMESPACE_UPDATE_NS_TABLE:
insertIntoNSTable(env);
setNextState(ModifyNamespaceState.MODIFY_NAMESPACE_UPDATE_ZK);
break;
case MODIFY_NAMESPACE_UPDATE_ZK:
updateZKNamespaceManager(env);
return Flow.NO_MORE_STATE;
default:
throw new UnsupportedOperationException(this + " unhandled state=" + state);
}
} catch (IOException e) {
LOG.warn("Error trying to modify the namespace" + newNsDescriptor.getName()
+ " (in state=" + state + ")", e);
setFailure("master-modify-namespace", e);
}
return Flow.HAS_MORE_STATE;
}
@Override
protected void rollbackState(final MasterProcedureEnv env, final ModifyNamespaceState state)
throws IOException {
if (isTraceEnabled()) {
LOG.trace(this + " rollback state=" + state);
}
try {
switch (state) {
case MODIFY_NAMESPACE_UPDATE_ZK:
rollbackZKNamespaceManagerChange(env);
break;
case MODIFY_NAMESPACE_UPDATE_NS_TABLE:
rollbackUpdateInNSTable(env);
break;
case MODIFY_NAMESPACE_PREPARE:
break; // nothing to do
default:
throw new UnsupportedOperationException(this + " 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 + " for creating the namespace "
+ newNsDescriptor.getName(), e);
throw e;
}
}
@Override
protected ModifyNamespaceState getState(final int stateId) {
return ModifyNamespaceState.valueOf(stateId);
}
@Override
protected int getStateId(final ModifyNamespaceState state) {
return state.getNumber();
}
@Override
protected ModifyNamespaceState getInitialState() {
return ModifyNamespaceState.MODIFY_NAMESPACE_PREPARE;
}
@Override
protected void setNextState(ModifyNamespaceState state) {
if (aborted.get()) {
setAbortFailure("modify-namespace", "abort requested");
} else {
super.setNextState(state);
}
}
@Override
public boolean abort(final MasterProcedureEnv env) {
aborted.set(true);
return true;
}
@Override
public void serializeStateData(final OutputStream stream) throws IOException {
super.serializeStateData(stream);
MasterProcedureProtos.ModifyNamespaceStateData.Builder modifyNamespaceMsg =
MasterProcedureProtos.ModifyNamespaceStateData.newBuilder().setNamespaceDescriptor(
ProtobufUtil.toProtoNamespaceDescriptor(this.newNsDescriptor));
if (this.oldNsDescriptor != null) {
modifyNamespaceMsg.setUnmodifiedNamespaceDescriptor(
ProtobufUtil.toProtoNamespaceDescriptor(this.oldNsDescriptor));
}
modifyNamespaceMsg.build().writeDelimitedTo(stream);
}
@Override
public void deserializeStateData(final InputStream stream) throws IOException {
super.deserializeStateData(stream);
MasterProcedureProtos.ModifyNamespaceStateData modifyNamespaceMsg =
MasterProcedureProtos.ModifyNamespaceStateData.parseDelimitedFrom(stream);
newNsDescriptor =
ProtobufUtil.toNamespaceDescriptor(modifyNamespaceMsg.getNamespaceDescriptor());
if (modifyNamespaceMsg.hasUnmodifiedNamespaceDescriptor()) {
oldNsDescriptor =
ProtobufUtil.toNamespaceDescriptor(modifyNamespaceMsg.getUnmodifiedNamespaceDescriptor());
}
}
@Override
public void toStringClassDetails(StringBuilder sb) {
sb.append(getClass().getSimpleName());
sb.append(" (Namespace=");
sb.append(newNsDescriptor.getName());
sb.append(")");
}
@Override
protected boolean acquireLock(final MasterProcedureEnv env) {
return getTableNamespaceManager(env).acquireExclusiveLock();
}
@Override
protected void releaseLock(final MasterProcedureEnv env) {
getTableNamespaceManager(env).releaseExclusiveLock();
}
@Override
public TableName getTableName() {
return TableName.NAMESPACE_TABLE_NAME;
}
@Override
public TableOperationType getTableOperationType() {
return TableOperationType.EDIT;
}
/**
* Action before any real action of adding namespace.
* @param env MasterProcedureEnv
* @throws IOException
*/
private void prepareModify(final MasterProcedureEnv env) throws IOException {
if (getTableNamespaceManager(env).doesNamespaceExist(newNsDescriptor.getName()) == false) {
throw new NamespaceNotFoundException(newNsDescriptor.getName());
}
getTableNamespaceManager(env).validateTableAndRegionCount(newNsDescriptor);
// This is used for rollback
oldNsDescriptor = getTableNamespaceManager(env).get(newNsDescriptor.getName());
}
/**
* Insert/update the row into namespace table
* @param env MasterProcedureEnv
* @throws IOException
*/
private void insertIntoNSTable(final MasterProcedureEnv env) throws IOException {
getTableNamespaceManager(env).insertIntoNSTable(newNsDescriptor);
}
/**
* rollback the row into namespace table
* @param env MasterProcedureEnv
* @throws IOException
*/
private void rollbackUpdateInNSTable(final MasterProcedureEnv env) throws IOException {
if (oldNsDescriptor != null) {
getTableNamespaceManager(env).insertIntoNSTable(oldNsDescriptor);
}
}
/**
* Update Zookeeper.
* @param env MasterProcedureEnv
* @throws IOException
*/
private void updateZKNamespaceManager(final MasterProcedureEnv env) throws IOException {
getTableNamespaceManager(env).updateZKNamespaceManager(newNsDescriptor);
}
/**
* Update Zookeeper during undo.
* @param env MasterProcedureEnv
* @throws IOException
*/
private void rollbackZKNamespaceManagerChange(final MasterProcedureEnv env) throws IOException {
if (oldNsDescriptor != null) {
getTableNamespaceManager(env).updateZKNamespaceManager(oldNsDescriptor);
}
}
private TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
return env.getMasterServices().getTableNamespaceManager();
}
/**
* The procedure could be restarted from a different machine. If the variable is null, we need to
* retrieve it.
* @return traceEnabled
*/
private Boolean isTraceEnabled() {
if (traceEnabled == null) {
traceEnabled = LOG.isTraceEnabled();
}
return traceEnabled;
}
}

View File

@ -395,17 +395,34 @@ public class TestCatalogJanitor {
}
@Override
public void createNamespace(NamespaceDescriptor descriptor) throws IOException {
public void createNamespace(
final NamespaceDescriptor descriptor,
final long nonceGroup,
final long nonce) throws IOException {
//To change body of implemented methods use File | Settings | File Templates.
}
@Override
public void modifyNamespace(NamespaceDescriptor descriptor) throws IOException {
public void createNamespaceSync(
final NamespaceDescriptor descriptor,
final long nonceGroup,
final long nonce) throws IOException {
//To change body of implemented methods use File | Settings | File Templates.
}
@Override
public void deleteNamespace(String name) throws IOException {
public void modifyNamespace(
final NamespaceDescriptor descriptor,
final long nonceGroup,
final long nonce) throws IOException {
//To change body of implemented methods use File | Settings | File Templates.
}
@Override
public void deleteNamespace(
final String name,
final long nonceGroup,
final long nonce) throws IOException {
//To change body of implemented methods use File | Settings | File Templates.
}
@ -503,6 +520,11 @@ public class TestCatalogJanitor {
return null;
}
@Override
public TableNamespaceManager getTableNamespaceManager() {
return null;
}
@Override
public void dispatchMergingRegions(HRegionInfo region_a, HRegionInfo region_b,
boolean forcible) throws IOException {

View File

@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.client.BufferedMutator;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.NonceGenerator;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.TableState;
@ -54,7 +53,6 @@ import org.apache.hadoop.hbase.util.MD5Hash;
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);

View File

@ -0,0 +1,292 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master.procedure;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
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.HConstants;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NamespaceExistException;
import org.apache.hadoop.hbase.NamespaceNotFoundException;
import org.apache.hadoop.hbase.constraint.ConstraintException;
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.CreateNamespaceState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
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;
@Category({MasterTests.class, MediumTests.class})
public class TestCreateNamespaceProcedure {
private static final Log LOG = LogFactory.getLog(TestCreateNamespaceProcedure.class);
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static long nonceGroup = HConstants.NO_NONCE;
private static long nonce = HConstants.NO_NONCE;
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 {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
nonceGroup =
MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
}
@After
public void tearDown() throws Exception {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
}
@Test(timeout = 60000)
public void testCreateNamespace() throws Exception {
final NamespaceDescriptor nsd = NamespaceDescriptor.create("testCreateNamespace").build();
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
long procId = procExec.submitProcedure(
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
validateNamespaceCreated(nsd);
}
@Test(timeout=60000)
public void testCreateSameNamespaceTwice() throws Exception {
final NamespaceDescriptor nsd =
NamespaceDescriptor.create("testCreateSameNamespaceTwice").build();
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
long procId1 = procExec.submitProcedure(
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
// Create the namespace that exists
long procId2 = procExec.submitProcedure(
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
nonceGroup + 1,
nonce + 1);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId2);
// Second create should fail with NamespaceExistException
ProcedureResult result = procExec.getResult(procId2);
assertTrue(result.isFailed());
LOG.debug("Create namespace failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof NamespaceExistException);
}
@Test(timeout=60000)
public void testCreateSystemNamespace() throws Exception {
final NamespaceDescriptor nsd =
UTIL.getHBaseAdmin().getNamespaceDescriptor(NamespaceDescriptor.SYSTEM_NAMESPACE.getName());
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
long procId = procExec.submitProcedure(
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureResult result = procExec.getResult(procId);
assertTrue(result.isFailed());
LOG.debug("Create namespace failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof NamespaceExistException);
}
@Test(timeout=60000)
public void testCreateNamespaceWithInvalidRegionCount() throws Exception {
final NamespaceDescriptor nsd =
NamespaceDescriptor.create("testCreateNamespaceWithInvalidRegionCount").build();
final String nsKey = "hbase.namespace.quota.maxregions";
final String nsValue = "-1";
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
nsd.setConfiguration(nsKey, nsValue);
long procId = procExec.submitProcedure(
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureResult result = procExec.getResult(procId);
assertTrue(result.isFailed());
LOG.debug("Create namespace failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof ConstraintException);
}
@Test(timeout=60000)
public void testCreateNamespaceWithInvalidTableCount() throws Exception {
final NamespaceDescriptor nsd =
NamespaceDescriptor.create("testCreateNamespaceWithInvalidTableCount").build();
final String nsKey = "hbase.namespace.quota.maxtables";
final String nsValue = "-1";
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
nsd.setConfiguration(nsKey, nsValue);
long procId = procExec.submitProcedure(
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureResult result = procExec.getResult(procId);
assertTrue(result.isFailed());
LOG.debug("Create namespace failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof ConstraintException);
}
@Test(timeout=60000)
public void testCreateSameNamespaceTwiceWithSameNonce() throws Exception {
final NamespaceDescriptor nsd =
NamespaceDescriptor.create("testCreateSameNamespaceTwiceWithSameNonce").build();
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
long procId1 = procExec.submitProcedure(
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
nonceGroup,
nonce);
long procId2 = procExec.submitProcedure(
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
validateNamespaceCreated(nsd);
// Wait the completion and expect not fail - because it is the same proc
ProcedureTestingUtility.waitProcedure(procExec, procId2);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
assertTrue(procId1 == procId2);
}
@Test(timeout = 60000)
public void testRecoveryAndDoubleExecution() throws Exception {
final NamespaceDescriptor nsd =
NamespaceDescriptor.create("testRecoveryAndDoubleExecution").build();
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the CreateNamespace procedure && kill the executor
long procId = procExec.submitProcedure(
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
nonceGroup,
nonce);
// Restart the executor and execute the step twice
int numberOfSteps = CreateNamespaceState.values().length;
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(
procExec,
procId,
numberOfSteps,
CreateNamespaceState.values());
// Validate the creation of namespace
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
validateNamespaceCreated(nsd);
}
@Test(timeout = 60000)
public void testRollbackAndDoubleExecution() throws Exception {
final NamespaceDescriptor nsd =
NamespaceDescriptor.create("testRollbackAndDoubleExecution").build();
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the CreateNamespace procedure && kill the executor
long procId = procExec.submitProcedure(
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
nonceGroup,
nonce);
int numberOfSteps = CreateNamespaceState.values().length - 2; // failing in the middle of proc
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(
procExec,
procId,
numberOfSteps,
CreateNamespaceState.values());
// Validate the non-existence of namespace
try {
NamespaceDescriptor nsDescriptor = UTIL.getHBaseAdmin().getNamespaceDescriptor(nsd.getName());
assertNull(nsDescriptor);
} catch (NamespaceNotFoundException nsnfe) {
// Expected
LOG.info("The namespace " + nsd.getName() + " is not created.");
}
}
private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
}
private void validateNamespaceCreated(NamespaceDescriptor nsd) throws IOException {
NamespaceDescriptor createdNsDescriptor =
UTIL.getHBaseAdmin().getNamespaceDescriptor(nsd.getName());
assertNotNull(createdNsDescriptor);
}
}

View File

@ -0,0 +1,282 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master.procedure;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
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.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NamespaceNotFoundException;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.constraint.ConstraintException;
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.DeleteNamespaceState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
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;
@Category({MasterTests.class, MediumTests.class})
public class TestDeleteNamespaceProcedure {
private static final Log LOG = LogFactory.getLog(TestDeleteNamespaceProcedure.class);
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static long nonceGroup = HConstants.NO_NONCE;
private static long nonce = HConstants.NO_NONCE;
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 {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
nonceGroup =
MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
}
@After
public void tearDown() throws Exception {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
for (HTableDescriptor htd: UTIL.getHBaseAdmin().listTables()) {
LOG.info("Tear down, remove table=" + htd.getTableName());
UTIL.deleteTable(htd.getTableName());
}
}
@Test(timeout = 60000)
public void testDeleteNamespace() throws Exception {
final String namespaceName = "testDeleteNamespace";
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
createNamespaceForTesting(namespaceName);
long procId = procExec.submitProcedure(
new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
validateNamespaceNotExist(namespaceName);
}
@Test(timeout=60000)
public void testDeleteNonExistNamespace() throws Exception {
final String namespaceName = "testDeleteNonExistNamespace";
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
validateNamespaceNotExist(namespaceName);
long procId = procExec.submitProcedure(
new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
// Expect fail with NamespaceNotFoundException
ProcedureResult result = procExec.getResult(procId);
assertTrue(result.isFailed());
LOG.debug("Delete namespace failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof NamespaceNotFoundException);
}
@Test(timeout=60000)
public void testDeleteSystemNamespace() throws Exception {
final String namespaceName = NamespaceDescriptor.SYSTEM_NAMESPACE.getName();
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
long procId = procExec.submitProcedure(
new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureResult result = procExec.getResult(procId);
assertTrue(result.isFailed());
LOG.debug("Delete namespace failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof ConstraintException);
}
@Test(timeout=60000)
public void testDeleteNonEmptyNamespace() throws Exception {
final String namespaceName = "testDeleteNonExistNamespace";
final TableName tableName = TableName.valueOf("testDeleteNonExistNamespace:t1");
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
// create namespace
createNamespaceForTesting(namespaceName);
// create the table under the new namespace
MasterProcedureTestingUtility.createTable(procExec, tableName, null, "f1");
long procId = procExec.submitProcedure(
new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureResult result = procExec.getResult(procId);
assertTrue(result.isFailed());
LOG.debug("Delete namespace failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof ConstraintException);
}
@Test(timeout=60000)
public void testDeleteSameNamespaceTwiceWithSameNonce() throws Exception {
final String namespaceName = "testDeleteSameNamespaceTwiceWithSameNonce";
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
createNamespaceForTesting(namespaceName);
long procId1 = procExec.submitProcedure(
new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName),
nonceGroup,
nonce);
long procId2 = procExec.submitProcedure(
new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
validateNamespaceNotExist(namespaceName);
// Wait the completion and expect not fail - because it is the same proc
ProcedureTestingUtility.waitProcedure(procExec, procId2);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId2);
assertTrue(procId1 == procId2);
}
@Test(timeout = 60000)
public void testRecoveryAndDoubleExecution() throws Exception {
final String namespaceName = "testRecoveryAndDoubleExecution";
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
createNamespaceForTesting(namespaceName);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the DeleteNamespace procedure && kill the executor
long procId = procExec.submitProcedure(
new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName),
nonceGroup,
nonce);
// Restart the executor and execute the step twice
int numberOfSteps = DeleteNamespaceState.values().length;
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(
procExec,
procId,
numberOfSteps,
DeleteNamespaceState.values());
// Validate the deletion of namespace
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
validateNamespaceNotExist(namespaceName);
}
@Test(timeout = 60000)
public void testRollbackAndDoubleExecution() throws Exception {
final String namespaceName = "testRollbackAndDoubleExecution";
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
createNamespaceForTesting(namespaceName);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Start the DeleteNamespace procedure && kill the executor
long procId = procExec.submitProcedure(
new DeleteNamespaceProcedure(procExec.getEnvironment(), namespaceName),
nonceGroup,
nonce);
int numberOfSteps = DeleteNamespaceState.values().length - 2; // failing in the middle of proc
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(
procExec,
procId,
numberOfSteps,
DeleteNamespaceState.values());
// Validate the namespace still exists
NamespaceDescriptor createdNsDescriptor=
UTIL.getHBaseAdmin().getNamespaceDescriptor(namespaceName);
assertNotNull(createdNsDescriptor);
}
private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
}
private void createNamespaceForTesting(final String namespaceName) throws Exception {
final NamespaceDescriptor nsd = NamespaceDescriptor.create(namespaceName).build();
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
long procId = procExec.submitProcedure(
new CreateNamespaceProcedure(procExec.getEnvironment(), nsd),
nonceGroup + 1,
nonce + 1);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
}
public static void validateNamespaceNotExist(final String nsName) throws IOException {
try {
NamespaceDescriptor nsDescriptor = UTIL.getHBaseAdmin().getNamespaceDescriptor(nsName);
assertNull(nsDescriptor);
} catch (NamespaceNotFoundException nsnfe) {
// Expected
}
}
}

View File

@ -0,0 +1,295 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master.procedure;
import static org.junit.Assert.*;
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.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.NamespaceNotFoundException;
import org.apache.hadoop.hbase.constraint.ConstraintException;
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.ModifyNamespaceState;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
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;
@Category({MasterTests.class, MediumTests.class})
public class TestModifyNamespaceProcedure {
private static final Log LOG = LogFactory.getLog(TestModifyNamespaceProcedure.class);
protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static long nonceGroup = HConstants.NO_NONCE;
private static long nonce = HConstants.NO_NONCE;
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 {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
nonceGroup =
MasterProcedureTestingUtility.generateNonceGroup(UTIL.getHBaseCluster().getMaster());
nonce = MasterProcedureTestingUtility.generateNonce(UTIL.getHBaseCluster().getMaster());
}
@After
public void tearDown() throws Exception {
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(getMasterProcedureExecutor(), false);
for (HTableDescriptor htd: UTIL.getHBaseAdmin().listTables()) {
LOG.info("Tear down, remove table=" + htd.getTableName());
UTIL.deleteTable(htd.getTableName());
}
}
@Test(timeout = 60000)
public void testModifyNamespace() throws Exception {
final NamespaceDescriptor nsd = NamespaceDescriptor.create("testModifyNamespace").build();
final String nsKey1 = "hbase.namespace.quota.maxregions";
final String nsValue1before = "1111";
final String nsValue1after = "9999";
final String nsKey2 = "hbase.namespace.quota.maxtables";
final String nsValue2 = "10";
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
nsd.setConfiguration(nsKey1, nsValue1before);
createNamespaceForTesting(nsd);
// Before modify
NamespaceDescriptor currentNsDescriptor =
UTIL.getHBaseAdmin().getNamespaceDescriptor(nsd.getName());
assertEquals(currentNsDescriptor.getConfigurationValue(nsKey1), nsValue1before);
assertNull(currentNsDescriptor.getConfigurationValue(nsKey2));
// Update
nsd.setConfiguration(nsKey1, nsValue1after);
nsd.setConfiguration(nsKey2, nsValue2);
long procId1 = procExec.submitProcedure(
new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId1);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId1);
// Verify the namespace is updated.
currentNsDescriptor =
UTIL.getHBaseAdmin().getNamespaceDescriptor(nsd.getName());
assertEquals(nsd.getConfigurationValue(nsKey1), nsValue1after);
assertEquals(currentNsDescriptor.getConfigurationValue(nsKey2), nsValue2);
}
@Test(timeout=60000)
public void testModifyNonExistNamespace() throws Exception {
final String namespaceName = "testModifyNonExistNamespace";
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
try {
NamespaceDescriptor nsDescriptor = UTIL.getHBaseAdmin().getNamespaceDescriptor(namespaceName);
assertNull(nsDescriptor);
} catch (NamespaceNotFoundException nsnfe) {
// Expected
LOG.debug("The namespace " + namespaceName + " does not exist. This is expected.");
}
final NamespaceDescriptor nsd = NamespaceDescriptor.create(namespaceName).build();
long procId = procExec.submitProcedure(
new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
// Expect fail with NamespaceNotFoundException
ProcedureResult result = procExec.getResult(procId);
assertTrue(result.isFailed());
LOG.debug("modify namespace failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof NamespaceNotFoundException);
}
@Test(timeout=60000)
public void testModifyNamespaceWithInvalidRegionCount() throws Exception {
final NamespaceDescriptor nsd =
NamespaceDescriptor.create("testModifyNamespaceWithInvalidRegionCount").build();
final String nsKey = "hbase.namespace.quota.maxregions";
final String nsValue = "-1";
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
createNamespaceForTesting(nsd);
// Modify
nsd.setConfiguration(nsKey, nsValue);
long procId = procExec.submitProcedure(
new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureResult result = procExec.getResult(procId);
assertTrue(result.isFailed());
LOG.debug("Modify namespace failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof ConstraintException);
}
@Test(timeout=60000)
public void testModifyNamespaceWithInvalidTableCount() throws Exception {
final NamespaceDescriptor nsd =
NamespaceDescriptor.create("testModifyNamespaceWithInvalidTableCount").build();
final String nsKey = "hbase.namespace.quota.maxtables";
final String nsValue = "-1";
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
createNamespaceForTesting(nsd);
// Modify
nsd.setConfiguration(nsKey, nsValue);
long procId = procExec.submitProcedure(
new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd),
nonceGroup,
nonce);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureResult result = procExec.getResult(procId);
assertTrue(result.isFailed());
LOG.debug("Modify namespace failed with exception: " + result.getException());
assertTrue(result.getException().getCause() instanceof ConstraintException);
}
@Test(timeout = 60000)
public void testRecoveryAndDoubleExecution() throws Exception {
final NamespaceDescriptor nsd =
NamespaceDescriptor.create("testRecoveryAndDoubleExecution").build();
final String nsKey = "foo";
final String nsValue = "bar";
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
createNamespaceForTesting(nsd);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Modify
nsd.setConfiguration(nsKey, nsValue);
// Start the Modify procedure && kill the executor
long procId = procExec.submitProcedure(
new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd),
nonceGroup,
nonce);
// Restart the executor and execute the step twice
int numberOfSteps = ModifyNamespaceState.values().length;
MasterProcedureTestingUtility.testRecoveryAndDoubleExecution(
procExec,
procId,
numberOfSteps,
ModifyNamespaceState.values());
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
// Validate
NamespaceDescriptor currentNsDescriptor =
UTIL.getHBaseAdmin().getNamespaceDescriptor(nsd.getName());
assertEquals(currentNsDescriptor.getConfigurationValue(nsKey), nsValue);
}
@Test(timeout = 60000)
public void testRollbackAndDoubleExecution() throws Exception {
final NamespaceDescriptor nsd =
NamespaceDescriptor.create("testRollbackAndDoubleExecution").build();
final String nsKey = "foo";
final String nsValue = "bar";
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
createNamespaceForTesting(nsd);
ProcedureTestingUtility.waitNoProcedureRunning(procExec);
ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
// Modify
nsd.setConfiguration(nsKey, nsValue);
// Start the Modify procedure && kill the executor
long procId = procExec.submitProcedure(
new ModifyNamespaceProcedure(procExec.getEnvironment(), nsd),
nonceGroup,
nonce);
// Failing in the middle of proc
int numberOfSteps = ModifyNamespaceState.values().length - 2;
MasterProcedureTestingUtility.testRollbackAndDoubleExecution(
procExec,
procId,
numberOfSteps,
ModifyNamespaceState.values());
// Validate
NamespaceDescriptor currentNsDescriptor =
UTIL.getHBaseAdmin().getNamespaceDescriptor(nsd.getName());
assertNull(currentNsDescriptor.getConfigurationValue(nsKey));
}
private ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
}
private void createNamespaceForTesting(NamespaceDescriptor nsDescriptor) throws Exception {
final ProcedureExecutor<MasterProcedureEnv> procExec = getMasterProcedureExecutor();
long procId = procExec.submitProcedure(
new CreateNamespaceProcedure(procExec.getEnvironment(), nsDescriptor),
nonceGroup + 1,
nonce + 1);
// Wait the completion
ProcedureTestingUtility.waitProcedure(procExec, procId);
ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
}
}