HBASE-14888 ClusterSchema: Add Namespace Operations
This commit is contained in:
parent
1c4edd2ab7
commit
46303dfd75
|
@ -964,38 +964,67 @@ public interface Admin extends Abortable, Closeable {
|
||||||
*/
|
*/
|
||||||
Configuration getConfiguration();
|
Configuration getConfiguration();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a new namespace. Blocks until namespace has been successfully created or an exception
|
||||||
|
* is thrown.
|
||||||
|
*
|
||||||
|
* @param descriptor descriptor which describes the new namespace
|
||||||
|
*/
|
||||||
|
void createNamespace(final NamespaceDescriptor descriptor)
|
||||||
|
throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a new namespace
|
* Create a new namespace
|
||||||
*
|
*
|
||||||
* @param descriptor descriptor which describes the new namespace
|
* @param descriptor descriptor which describes the new namespace
|
||||||
* @throws IOException
|
* @return the result of the async create namespace operation. Use Future.get(long, TimeUnit) to
|
||||||
|
* wait on the operation to complete.
|
||||||
*/
|
*/
|
||||||
void createNamespace(final NamespaceDescriptor descriptor)
|
Future<Void> createNamespaceAsync(final NamespaceDescriptor descriptor)
|
||||||
|
throws IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Modify an existing namespace. Blocks until namespace has been successfully modified or an
|
||||||
|
* exception is thrown.
|
||||||
|
*
|
||||||
|
* @param descriptor descriptor which describes the new namespace
|
||||||
|
*/
|
||||||
|
void modifyNamespace(final NamespaceDescriptor descriptor)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Modify an existing namespace
|
* Modify an existing namespace
|
||||||
*
|
*
|
||||||
* @param descriptor descriptor which describes the new namespace
|
* @param descriptor descriptor which describes the new namespace
|
||||||
* @throws IOException
|
* @return the result of the async modify namespace operation. Use Future.get(long, TimeUnit) to
|
||||||
|
* wait on the operation to complete.
|
||||||
*/
|
*/
|
||||||
void modifyNamespace(final NamespaceDescriptor descriptor)
|
Future<Void> modifyNamespaceAsync(final NamespaceDescriptor descriptor)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Delete an existing namespace. Only empty namespaces (no tables) can be removed.
|
||||||
|
* Blocks until namespace has been successfully deleted or an
|
||||||
|
* exception is thrown.
|
||||||
|
*
|
||||||
|
* @param name namespace name
|
||||||
|
*/
|
||||||
|
void deleteNamespace(final String name) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Delete an existing namespace. Only empty namespaces (no tables) can be removed.
|
* Delete an existing namespace. Only empty namespaces (no tables) can be removed.
|
||||||
*
|
*
|
||||||
* @param name namespace name
|
* @param name namespace name
|
||||||
* @throws IOException
|
* @return the result of the async delete namespace operation. Use Future.get(long, TimeUnit) to
|
||||||
|
* wait on the operation to complete.
|
||||||
*/
|
*/
|
||||||
void deleteNamespace(final String name) throws IOException;
|
Future<Void> deleteNamespaceAsync(final String name) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get a namespace descriptor by name
|
* Get a namespace descriptor by name
|
||||||
*
|
*
|
||||||
* @param name name of namespace descriptor
|
* @param name name of namespace descriptor
|
||||||
* @return A descriptor
|
* @return A descriptor
|
||||||
* @throws IOException
|
|
||||||
*/
|
*/
|
||||||
NamespaceDescriptor getNamespaceDescriptor(final String name)
|
NamespaceDescriptor getNamespaceDescriptor(final String name)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
@ -1004,7 +1033,6 @@ public interface Admin extends Abortable, Closeable {
|
||||||
* List available namespace descriptors
|
* List available namespace descriptors
|
||||||
*
|
*
|
||||||
* @return List of descriptors
|
* @return List of descriptors
|
||||||
* @throws IOException
|
|
||||||
*/
|
*/
|
||||||
NamespaceDescriptor[] listNamespaceDescriptors()
|
NamespaceDescriptor[] listNamespaceDescriptors()
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
|
@ -1,5 +1,6 @@
|
||||||
/**
|
/**
|
||||||
*
|
*
|
||||||
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
|
|
|
@ -150,8 +150,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
|
||||||
// be waiting for the master lock => deadlock.
|
// be waiting for the master lock => deadlock.
|
||||||
private final Object masterAndZKLock = new Object();
|
private final Object masterAndZKLock = new Object();
|
||||||
|
|
||||||
private long keepZooKeeperWatcherAliveUntil = Long.MAX_VALUE;
|
|
||||||
|
|
||||||
// thread executor shared by all HTableInterface instances created
|
// thread executor shared by all HTableInterface instances created
|
||||||
// by this connection
|
// by this connection
|
||||||
private volatile ExecutorService batchPool = null;
|
private volatile ExecutorService batchPool = null;
|
||||||
|
@ -398,7 +396,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
|
||||||
synchronized (this) {
|
synchronized (this) {
|
||||||
if (batchPool == null) {
|
if (batchPool == null) {
|
||||||
this.batchPool = getThreadPool(conf.getInt("hbase.hconnection.threads.max", 256),
|
this.batchPool = getThreadPool(conf.getInt("hbase.hconnection.threads.max", 256),
|
||||||
conf.getInt("hbase.hconnection.threads.core", 256), "-shared-", null);
|
conf.getInt("hbase.hconnection.threads.core", 256), "-shared", null);
|
||||||
this.cleanupPool = true;
|
this.cleanupPool = true;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -482,7 +480,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return The cluster registry implementation to use.
|
* @return The cluster registry implementation to use.
|
||||||
* @throws java.io.IOException
|
|
||||||
*/
|
*/
|
||||||
private Registry setupRegistry() throws IOException {
|
private Registry setupRegistry() throws IOException {
|
||||||
return RegistryFactory.getRegistry(this);
|
return RegistryFactory.getRegistry(this);
|
||||||
|
@ -542,7 +539,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
|
||||||
/**
|
/**
|
||||||
* @return true if the master is running, throws an exception otherwise
|
* @return true if the master is running, throws an exception otherwise
|
||||||
* @throws org.apache.hadoop.hbase.MasterNotRunningException - if the master is not running
|
* @throws org.apache.hadoop.hbase.MasterNotRunningException - if the master is not running
|
||||||
* @throws org.apache.hadoop.hbase.ZooKeeperConnectionException
|
|
||||||
* @deprecated this has been deprecated without a replacement
|
* @deprecated this has been deprecated without a replacement
|
||||||
*/
|
*/
|
||||||
@Deprecated
|
@Deprecated
|
||||||
|
@ -981,9 +977,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
|
||||||
/**
|
/**
|
||||||
* Search the cache for a location that fits our table and row key.
|
* Search the cache for a location that fits our table and row key.
|
||||||
* Return null if no suitable region is located.
|
* Return null if no suitable region is located.
|
||||||
*
|
|
||||||
* @param tableName
|
|
||||||
* @param row
|
|
||||||
* @return Null or region location found in cache.
|
* @return Null or region location found in cache.
|
||||||
*/
|
*/
|
||||||
RegionLocations getCachedLocation(final TableName tableName,
|
RegionLocations getCachedLocation(final TableName tableName,
|
||||||
|
@ -1181,13 +1174,11 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Make stub and cache it internal so can be used later doing the isMasterRunning call.
|
* Make stub and cache it internal so can be used later doing the isMasterRunning call.
|
||||||
* @param channel
|
|
||||||
*/
|
*/
|
||||||
protected abstract Object makeStub(final BlockingRpcChannel channel);
|
protected abstract Object makeStub(final BlockingRpcChannel channel);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Once setup, check it works by doing isMasterRunning check.
|
* Once setup, check it works by doing isMasterRunning check.
|
||||||
* @throws com.google.protobuf.ServiceException
|
|
||||||
*/
|
*/
|
||||||
protected abstract void isMasterRunning() throws ServiceException;
|
protected abstract void isMasterRunning() throws ServiceException;
|
||||||
|
|
||||||
|
@ -1195,9 +1186,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
|
||||||
* Create a stub. Try once only. It is not typed because there is no common type to
|
* Create a stub. Try once only. It is not typed because there is no common type to
|
||||||
* protobuf services nor their interfaces. Let the caller do appropriate casting.
|
* protobuf services nor their interfaces. Let the caller do appropriate casting.
|
||||||
* @return A stub for master services.
|
* @return A stub for master services.
|
||||||
* @throws java.io.IOException
|
|
||||||
* @throws org.apache.zookeeper.KeeperException
|
|
||||||
* @throws com.google.protobuf.ServiceException
|
|
||||||
*/
|
*/
|
||||||
private Object makeStubNoRetries() throws IOException, KeeperException, ServiceException {
|
private Object makeStubNoRetries() throws IOException, KeeperException, ServiceException {
|
||||||
ZooKeeperKeepAliveConnection zkw;
|
ZooKeeperKeepAliveConnection zkw;
|
||||||
|
@ -1370,10 +1358,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
|
||||||
|
|
||||||
private ZooKeeperKeepAliveConnection keepAliveZookeeper;
|
private ZooKeeperKeepAliveConnection keepAliveZookeeper;
|
||||||
private AtomicInteger keepAliveZookeeperUserCount = new AtomicInteger(0);
|
private AtomicInteger keepAliveZookeeperUserCount = new AtomicInteger(0);
|
||||||
private boolean canCloseZKW = true;
|
|
||||||
|
|
||||||
// keepAlive time, in ms. No reason to make it configurable.
|
|
||||||
private static final long keepAlive = 5 * 60 * 1000;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Retrieve a shared ZooKeeperWatcher. You must close it it once you've have finished with it.
|
* Retrieve a shared ZooKeeperWatcher. You must close it it once you've have finished with it.
|
||||||
|
@ -1391,7 +1375,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
|
||||||
keepAliveZookeeper = new ZooKeeperKeepAliveConnection(conf, this.toString(), this);
|
keepAliveZookeeper = new ZooKeeperKeepAliveConnection(conf, this.toString(), this);
|
||||||
}
|
}
|
||||||
keepAliveZookeeperUserCount.addAndGet(1);
|
keepAliveZookeeperUserCount.addAndGet(1);
|
||||||
keepZooKeeperWatcherAliveUntil = Long.MAX_VALUE;
|
|
||||||
return keepAliveZookeeper;
|
return keepAliveZookeeper;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1400,9 +1383,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
|
||||||
if (zkw == null){
|
if (zkw == null){
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
if (keepAliveZookeeperUserCount.addAndGet(-1) <= 0) {
|
|
||||||
keepZooKeeperWatcherAliveUntil = System.currentTimeMillis() + keepAlive;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void closeZooKeeperWatcher() {
|
private void closeZooKeeperWatcher() {
|
||||||
|
@ -1820,7 +1800,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
private static void release(MasterServiceState mss) {
|
private static void release(MasterServiceState mss) {
|
||||||
if (mss != null && mss.connection != null) {
|
if (mss != null && mss.connection != null) {
|
||||||
((ConnectionImplementation)mss.connection).releaseMaster(mss);
|
((ConnectionImplementation)mss.connection).releaseMaster(mss);
|
||||||
|
@ -1964,7 +1943,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @deprecated since 0.96 - Use {@link org.apache.hadoop.hbase.client.HTableInterface#batch} instead
|
* @deprecated since 0.96 Use {@link org.apache.hadoop.hbase.client.HTableInterface#batch} instead
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
@Deprecated
|
@Deprecated
|
||||||
|
@ -1999,7 +1978,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
|
||||||
* If the method returns it means that there is no error, and the 'results' array will
|
* If the method returns it means that there is no error, and the 'results' array will
|
||||||
* contain no exception. On error, an exception is thrown, and the 'results' array will
|
* contain no exception. On error, an exception is thrown, and the 'results' array will
|
||||||
* contain results and exceptions.
|
* contain results and exceptions.
|
||||||
* @deprecated since 0.96 -
|
* @deprecated since 0.96
|
||||||
* Use {@link org.apache.hadoop.hbase.client.HTable#processBatchCallback} instead
|
* Use {@link org.apache.hadoop.hbase.client.HTable#processBatchCallback} instead
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
|
@ -2225,7 +2204,8 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @deprecated Use {@link org.apache.hadoop.hbase.client.Admin#getTableDescriptorsByTableName(java.util.List)} instead
|
* @deprecated Use {@link
|
||||||
|
* org.apache.hadoop.hbase.client.Admin#getTableDescriptorsByTableName(java.util.List)} instead
|
||||||
*/
|
*/
|
||||||
@Deprecated
|
@Deprecated
|
||||||
@Override
|
@Override
|
||||||
|
@ -2245,12 +2225,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @deprecated Use {@link org.apache.hadoop.hbase.client.Admin#getTableDescriptorsByTableName(java.util.List)} instead
|
* @deprecated Use
|
||||||
|
* {@link org.apache.hadoop.hbase.client.Admin#getTableDescriptorsByTableName(java.util.List)}
|
||||||
|
* instead
|
||||||
*/
|
*/
|
||||||
@Deprecated
|
@Deprecated
|
||||||
@Override
|
@Override
|
||||||
public HTableDescriptor[] getHTableDescriptors(
|
public HTableDescriptor[] getHTableDescriptors(List<String> names) throws IOException {
|
||||||
List<String> names) throws IOException {
|
|
||||||
List<TableName> tableNames = new ArrayList<TableName>(names.size());
|
List<TableName> tableNames = new ArrayList<TableName>(names.size());
|
||||||
for(String name : names) {
|
for(String name : names) {
|
||||||
tableNames.add(TableName.valueOf(name));
|
tableNames.add(TableName.valueOf(name));
|
||||||
|
@ -2269,7 +2250,9 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
|
||||||
* @param tableName table name
|
* @param tableName table name
|
||||||
* @throws java.io.IOException if the connection to master fails or if the table
|
* @throws java.io.IOException if the connection to master fails or if the table
|
||||||
* is not found.
|
* is not found.
|
||||||
* @deprecated Use {@link org.apache.hadoop.hbase.client.Admin#getTableDescriptor(org.apache.hadoop.hbase.TableName)} instead
|
* @deprecated Use {@link
|
||||||
|
* org.apache.hadoop.hbase.client.Admin#getTableDescriptor(org.apache.hadoop.hbase.TableName)}
|
||||||
|
* instead
|
||||||
*/
|
*/
|
||||||
@Deprecated
|
@Deprecated
|
||||||
@Override
|
@Override
|
||||||
|
@ -2294,7 +2277,9 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @deprecated Use {@link org.apache.hadoop.hbase.client.Admin#getTableDescriptor(org.apache.hadoop.hbase.TableName)} instead
|
* @deprecated Use {@link
|
||||||
|
* org.apache.hadoop.hbase.client.Admin#getTableDescriptor(org.apache.hadoop.hbase.TableName)}
|
||||||
|
* instead
|
||||||
*/
|
*/
|
||||||
@Deprecated
|
@Deprecated
|
||||||
@Override
|
@Override
|
||||||
|
@ -2306,10 +2291,8 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
|
||||||
@Override
|
@Override
|
||||||
public TableState getTableState(TableName tableName) throws IOException {
|
public TableState getTableState(TableName tableName) throws IOException {
|
||||||
if (this.closed) throw new IOException(toString() + " closed");
|
if (this.closed) throw new IOException(toString() + " closed");
|
||||||
|
|
||||||
TableState tableState = MetaTableAccessor.getTableState(this, tableName);
|
TableState tableState = MetaTableAccessor.getTableState(this, tableName);
|
||||||
if (tableState == null)
|
if (tableState == null) throw new TableNotFoundException(tableName);
|
||||||
throw new TableNotFoundException(tableName);
|
|
||||||
return tableState;
|
return tableState;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -28,12 +28,12 @@ import java.util.LinkedList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Map.Entry;
|
import java.util.Map.Entry;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
|
||||||
import java.util.concurrent.atomic.AtomicReference;
|
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.Future;
|
import java.util.concurrent.Future;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.TimeoutException;
|
import java.util.concurrent.TimeoutException;
|
||||||
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
import java.util.regex.Pattern;
|
import java.util.regex.Pattern;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
|
@ -95,11 +95,13 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnResponse;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceRequest;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceResponse;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableResponse;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnResponse;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceRequest;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceResponse;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableResponse;
|
||||||
|
@ -135,6 +137,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MajorCompactionTi
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnResponse;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableResponse;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest;
|
||||||
|
@ -254,23 +257,10 @@ public class HBaseAdmin implements Admin {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean abortProcedure(
|
public boolean abortProcedure(final long procId, final boolean mayInterruptIfRunning)
|
||||||
final long procId,
|
throws IOException {
|
||||||
final boolean mayInterruptIfRunning) throws IOException {
|
return get(abortProcedureAsync(procId, mayInterruptIfRunning), this.syncWaitTimeout,
|
||||||
Future<Boolean> future = abortProcedureAsync(procId, mayInterruptIfRunning);
|
TimeUnit.MILLISECONDS);
|
||||||
try {
|
|
||||||
return future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
|
|
||||||
} catch (InterruptedException e) {
|
|
||||||
throw new InterruptedIOException("Interrupted when waiting for procedure to be cancelled");
|
|
||||||
} catch (TimeoutException e) {
|
|
||||||
throw new TimeoutIOException(e);
|
|
||||||
} catch (ExecutionException e) {
|
|
||||||
if (e.getCause() instanceof IOException) {
|
|
||||||
throw (IOException)e.getCause();
|
|
||||||
} else {
|
|
||||||
throw new IOException(e.getCause());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -464,22 +454,7 @@ public class HBaseAdmin implements Admin {
|
||||||
@Override
|
@Override
|
||||||
public void createTable(final HTableDescriptor desc, byte [][] splitKeys)
|
public void createTable(final HTableDescriptor desc, byte [][] splitKeys)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
Future<Void> future = createTableAsync(desc, splitKeys);
|
get(createTableAsync(desc, splitKeys), syncWaitTimeout, TimeUnit.MILLISECONDS);
|
||||||
try {
|
|
||||||
// TODO: how long should we wait? spin forever?
|
|
||||||
future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
|
|
||||||
} catch (InterruptedException e) {
|
|
||||||
throw new InterruptedIOException("Interrupted when waiting" +
|
|
||||||
" for table to be enabled; meta scan was done");
|
|
||||||
} catch (TimeoutException e) {
|
|
||||||
throw new TimeoutIOException(e);
|
|
||||||
} catch (ExecutionException e) {
|
|
||||||
if (e.getCause() instanceof IOException) {
|
|
||||||
throw (IOException)e.getCause();
|
|
||||||
} else {
|
|
||||||
throw new IOException(e.getCause());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -550,20 +525,7 @@ public class HBaseAdmin implements Admin {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void deleteTable(final TableName tableName) throws IOException {
|
public void deleteTable(final TableName tableName) throws IOException {
|
||||||
Future<Void> future = deleteTableAsync(tableName);
|
get(deleteTableAsync(tableName), syncWaitTimeout, TimeUnit.MILLISECONDS);
|
||||||
try {
|
|
||||||
future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
|
|
||||||
} catch (InterruptedException e) {
|
|
||||||
throw new InterruptedIOException("Interrupted when waiting for table to be deleted");
|
|
||||||
} catch (TimeoutException e) {
|
|
||||||
throw new TimeoutIOException(e);
|
|
||||||
} catch (ExecutionException e) {
|
|
||||||
if (e.getCause() instanceof IOException) {
|
|
||||||
throw (IOException)e.getCause();
|
|
||||||
} else {
|
|
||||||
throw new IOException(e.getCause());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -641,21 +603,7 @@ public class HBaseAdmin implements Admin {
|
||||||
@Override
|
@Override
|
||||||
public void truncateTable(final TableName tableName, final boolean preserveSplits)
|
public void truncateTable(final TableName tableName, final boolean preserveSplits)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
Future<Void> future = truncateTableAsync(tableName, preserveSplits);
|
get(truncateTableAsync(tableName, preserveSplits), syncWaitTimeout, TimeUnit.MILLISECONDS);
|
||||||
try {
|
|
||||||
future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
|
|
||||||
} catch (InterruptedException e) {
|
|
||||||
throw new InterruptedIOException("Interrupted when waiting for table " + tableName
|
|
||||||
+ " to be enabled.");
|
|
||||||
} catch (TimeoutException e) {
|
|
||||||
throw new TimeoutIOException(e);
|
|
||||||
} catch (ExecutionException e) {
|
|
||||||
if (e.getCause() instanceof IOException) {
|
|
||||||
throw (IOException) e.getCause();
|
|
||||||
} else {
|
|
||||||
throw new IOException(e.getCause());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -717,20 +665,7 @@ public class HBaseAdmin implements Admin {
|
||||||
@Override
|
@Override
|
||||||
public void enableTable(final TableName tableName)
|
public void enableTable(final TableName tableName)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
Future<Void> future = enableTableAsync(tableName);
|
get(enableTableAsync(tableName), syncWaitTimeout, TimeUnit.MILLISECONDS);
|
||||||
try {
|
|
||||||
future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
|
|
||||||
} catch (InterruptedException e) {
|
|
||||||
throw new InterruptedIOException("Interrupted when waiting for table to be disabled");
|
|
||||||
} catch (TimeoutException e) {
|
|
||||||
throw new TimeoutIOException(e);
|
|
||||||
} catch (ExecutionException e) {
|
|
||||||
if (e.getCause() instanceof IOException) {
|
|
||||||
throw (IOException)e.getCause();
|
|
||||||
} else {
|
|
||||||
throw new IOException(e.getCause());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -833,20 +768,7 @@ public class HBaseAdmin implements Admin {
|
||||||
@Override
|
@Override
|
||||||
public void disableTable(final TableName tableName)
|
public void disableTable(final TableName tableName)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
Future<Void> future = disableTableAsync(tableName);
|
get(disableTableAsync(tableName), syncWaitTimeout, TimeUnit.MILLISECONDS);
|
||||||
try {
|
|
||||||
future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
|
|
||||||
} catch (InterruptedException e) {
|
|
||||||
throw new InterruptedIOException("Interrupted when waiting for table to be disabled");
|
|
||||||
} catch (TimeoutException e) {
|
|
||||||
throw new TimeoutIOException(e);
|
|
||||||
} catch (ExecutionException e) {
|
|
||||||
if (e.getCause() instanceof IOException) {
|
|
||||||
throw (IOException)e.getCause();
|
|
||||||
} else {
|
|
||||||
throw new IOException(e.getCause());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -1841,43 +1763,103 @@ public class HBaseAdmin implements Admin {
|
||||||
return this.conf;
|
return this.conf;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Do a get with a timeout against the passed in <code>future<code>.
|
||||||
|
*/
|
||||||
|
private static <T> T get(final Future<T> future, final long timeout, final TimeUnit units)
|
||||||
|
throws IOException {
|
||||||
|
try {
|
||||||
|
// TODO: how long should we wait? Spin forever?
|
||||||
|
return future.get(timeout, units);
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
throw new InterruptedIOException("Interrupt while waiting on " + future);
|
||||||
|
} catch (TimeoutException e) {
|
||||||
|
throw new TimeoutIOException(e);
|
||||||
|
} catch (ExecutionException e) {
|
||||||
|
if (e.getCause() instanceof IOException) {
|
||||||
|
throw (IOException)e.getCause();
|
||||||
|
} else {
|
||||||
|
throw new IOException(e.getCause());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void createNamespace(final NamespaceDescriptor descriptor) throws IOException {
|
public void createNamespace(final NamespaceDescriptor descriptor)
|
||||||
executeCallable(new MasterCallable<Void>(getConnection()) {
|
throws IOException {
|
||||||
|
get(createNamespaceAsync(descriptor), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Void call(int callTimeout) throws Exception {
|
public Future<Void> createNamespaceAsync(final NamespaceDescriptor descriptor)
|
||||||
master.createNamespace(null,
|
throws IOException {
|
||||||
|
CreateNamespaceResponse response =
|
||||||
|
executeCallable(new MasterCallable<CreateNamespaceResponse>(getConnection()) {
|
||||||
|
@Override
|
||||||
|
public CreateNamespaceResponse call(int callTimeout) throws Exception {
|
||||||
|
return master.createNamespace(null,
|
||||||
CreateNamespaceRequest.newBuilder()
|
CreateNamespaceRequest.newBuilder()
|
||||||
.setNamespaceDescriptor(ProtobufUtil
|
.setNamespaceDescriptor(ProtobufUtil
|
||||||
.toProtoNamespaceDescriptor(descriptor)).build()
|
.toProtoNamespaceDescriptor(descriptor)).build()
|
||||||
);
|
);
|
||||||
return null;
|
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
return new NamespaceFuture(this, descriptor.getName(), response.getProcId()) {
|
||||||
|
@Override
|
||||||
|
public String getOperationType() {
|
||||||
|
return "CREATE_NAMESPACE";
|
||||||
|
}
|
||||||
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void modifyNamespace(final NamespaceDescriptor descriptor) throws IOException {
|
public void modifyNamespace(final NamespaceDescriptor descriptor)
|
||||||
executeCallable(new MasterCallable<Void>(getConnection()) {
|
throws IOException {
|
||||||
|
get(modifyNamespaceAsync(descriptor), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Void call(int callTimeout) throws Exception {
|
public Future<Void> modifyNamespaceAsync(final NamespaceDescriptor descriptor)
|
||||||
master.modifyNamespace(null, ModifyNamespaceRequest.newBuilder().
|
throws IOException {
|
||||||
|
ModifyNamespaceResponse response =
|
||||||
|
executeCallable(new MasterCallable<ModifyNamespaceResponse>(getConnection()) {
|
||||||
|
@Override
|
||||||
|
public ModifyNamespaceResponse call(int callTimeout) throws Exception {
|
||||||
|
return master.modifyNamespace(null, ModifyNamespaceRequest.newBuilder().
|
||||||
setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(descriptor)).build());
|
setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(descriptor)).build());
|
||||||
return null;
|
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
return new NamespaceFuture(this, descriptor.getName(), response.getProcId()) {
|
||||||
|
@Override
|
||||||
|
public String getOperationType() {
|
||||||
|
return "MODIFY_NAMESPACE";
|
||||||
|
}
|
||||||
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void deleteNamespace(final String name) throws IOException {
|
public void deleteNamespace(final String name)
|
||||||
executeCallable(new MasterCallable<Void>(getConnection()) {
|
throws IOException {
|
||||||
|
get(deleteNamespaceAsync(name), this.syncWaitTimeout, TimeUnit.MILLISECONDS);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Void call(int callTimeout) throws Exception {
|
public Future<Void> deleteNamespaceAsync(final String name)
|
||||||
master.deleteNamespace(null, DeleteNamespaceRequest.newBuilder().
|
throws IOException {
|
||||||
|
DeleteNamespaceResponse response =
|
||||||
|
executeCallable(new MasterCallable<DeleteNamespaceResponse>(getConnection()) {
|
||||||
|
@Override
|
||||||
|
public DeleteNamespaceResponse call(int callTimeout) throws Exception {
|
||||||
|
return master.deleteNamespace(null, DeleteNamespaceRequest.newBuilder().
|
||||||
setNamespaceName(name).build());
|
setNamespaceName(name).build());
|
||||||
return null;
|
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
return new NamespaceFuture(this, name, response.getProcId()) {
|
||||||
|
@Override
|
||||||
|
public String getOperationType() {
|
||||||
|
return "DELETE_NAMESPACE";
|
||||||
|
}
|
||||||
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -3184,6 +3166,11 @@ public class HBaseAdmin implements Admin {
|
||||||
this.tableName = tableName;
|
this.tableName = tableName;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return getDescription();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return the table name
|
* @return the table name
|
||||||
*/
|
*/
|
||||||
|
@ -3222,7 +3209,7 @@ public class HBaseAdmin implements Admin {
|
||||||
@Override
|
@Override
|
||||||
public void throwTimeoutException(long elapsedTime) throws TimeoutException {
|
public void throwTimeoutException(long elapsedTime) throws TimeoutException {
|
||||||
throw new TimeoutException("The operation: " + getOperationType() + " on table: " +
|
throw new TimeoutException("The operation: " + getOperationType() + " on table: " +
|
||||||
tableName.getNameAsString() + " not completed after " + elapsedTime + "msec");
|
tableName.getNameAsString() + " has not completed after " + elapsedTime + "ms");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -3344,6 +3331,34 @@ public class HBaseAdmin implements Admin {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
@InterfaceStability.Evolving
|
||||||
|
protected static abstract class NamespaceFuture extends ProcedureFuture<Void> {
|
||||||
|
private final String namespaceName;
|
||||||
|
|
||||||
|
public NamespaceFuture(final HBaseAdmin admin, final String namespaceName, final Long procId) {
|
||||||
|
super(admin, procId);
|
||||||
|
this.namespaceName = namespaceName;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the namespace name
|
||||||
|
*/
|
||||||
|
protected String getNamespaceName() {
|
||||||
|
return namespaceName;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the operation type like CREATE_NAMESPACE, DELETE_NAMESPACE, etc.
|
||||||
|
*/
|
||||||
|
public abstract String getOperationType();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return "Operation: " + getOperationType() + ", Namespace: " + getNamespaceName();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<SecurityCapability> getSecurityCapabilities() throws IOException {
|
public List<SecurityCapability> getSecurityCapabilities() throws IOException {
|
||||||
try {
|
try {
|
||||||
|
|
|
@ -0,0 +1,44 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* 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.coprocessor;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Thrown if a coprocessor rules we should bypass an operation
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Public
|
||||||
|
@InterfaceStability.Evolving
|
||||||
|
public class BypassCoprocessorException extends CoprocessorException {
|
||||||
|
private static final long serialVersionUID = 5943889011582357043L;
|
||||||
|
|
||||||
|
/** Default Constructor */
|
||||||
|
public BypassCoprocessorException() {
|
||||||
|
super();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Constructs the exception and supplies a string as the message
|
||||||
|
* @param s - message
|
||||||
|
*/
|
||||||
|
public BypassCoprocessorException(String s) {
|
||||||
|
super(s);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,50 @@
|
||||||
|
/**
|
||||||
|
* 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;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Simple Service.
|
||||||
|
*/
|
||||||
|
// This is a WIP. We have Services throughout hbase. Either have all implement what is here or
|
||||||
|
// just remove this as an experiment that did not work out.
|
||||||
|
// TODO: Move on to guava Service after we update our guava version; later guava has nicer
|
||||||
|
// Service implmentation.
|
||||||
|
// TODO: Move all Services on to this one Interface.
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public interface Service {
|
||||||
|
/**
|
||||||
|
* Initiates service startup (if necessary), returning once the service has finished starting.
|
||||||
|
* @throws IOException Throws exception if already running and if we fail to start successfully.
|
||||||
|
*/
|
||||||
|
void startAndWait() throws IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return True if this Service is running.
|
||||||
|
*/
|
||||||
|
boolean isRunning();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Initiates service shutdown (if necessary), returning once the service has finished stopping.
|
||||||
|
* @throws IOException Throws exception if not running of if we fail to stop successfully.
|
||||||
|
*/
|
||||||
|
void stopAndWait() throws IOException;
|
||||||
|
}
|
|
@ -0,0 +1,39 @@
|
||||||
|
package org.apache.hadoop.hbase;
|
||||||
|
/**
|
||||||
|
* 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.
|
||||||
|
*/
|
||||||
|
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
|
||||||
|
@SuppressWarnings("serial")
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public class ServiceNotRunningException extends HBaseIOException {
|
||||||
|
public ServiceNotRunningException() {
|
||||||
|
}
|
||||||
|
|
||||||
|
public ServiceNotRunningException(String message) {
|
||||||
|
super(message);
|
||||||
|
}
|
||||||
|
|
||||||
|
public ServiceNotRunningException(String message, Throwable cause) {
|
||||||
|
super(message, cause);
|
||||||
|
}
|
||||||
|
|
||||||
|
public ServiceNotRunningException(Throwable cause) {
|
||||||
|
super(cause);
|
||||||
|
}
|
||||||
|
}
|
|
@ -41,7 +41,7 @@ public class ResourceChecker {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructor
|
* Constructor
|
||||||
* @param tagLine - the tagLine is added to the logs. Must be be null.
|
* @param tagLine The tagLine is added to the logs. Must not be null.
|
||||||
*/
|
*/
|
||||||
public ResourceChecker(final String tagLine) {
|
public ResourceChecker(final String tagLine) {
|
||||||
this.tagLine = tagLine;
|
this.tagLine = tagLine;
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -186,6 +186,7 @@ message CreateNamespaceRequest {
|
||||||
}
|
}
|
||||||
|
|
||||||
message CreateNamespaceResponse {
|
message CreateNamespaceResponse {
|
||||||
|
optional uint64 proc_id = 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
message DeleteNamespaceRequest {
|
message DeleteNamespaceRequest {
|
||||||
|
@ -195,6 +196,7 @@ message DeleteNamespaceRequest {
|
||||||
}
|
}
|
||||||
|
|
||||||
message DeleteNamespaceResponse {
|
message DeleteNamespaceResponse {
|
||||||
|
optional uint64 proc_id = 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
message ModifyNamespaceRequest {
|
message ModifyNamespaceRequest {
|
||||||
|
@ -204,6 +206,7 @@ message ModifyNamespaceRequest {
|
||||||
}
|
}
|
||||||
|
|
||||||
message ModifyNamespaceResponse {
|
message ModifyNamespaceResponse {
|
||||||
|
optional uint64 proc_id = 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
message GetNamespaceDescriptorRequest {
|
message GetNamespaceDescriptorRequest {
|
||||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||||
|
import org.apache.hadoop.hbase.client.Connection;
|
||||||
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
||||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||||
|
|
||||||
|
@ -43,12 +44,20 @@ public interface Server extends Abortable, Stoppable {
|
||||||
ZooKeeperWatcher getZooKeeper();
|
ZooKeeperWatcher getZooKeeper();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns a reference to the servers' cluster connection.
|
* Returns a reference to the servers' connection.
|
||||||
*
|
*
|
||||||
* Important note: this method returns a reference to Connection which is managed
|
* Important note: this method returns a reference to Connection which is managed
|
||||||
* by Server itself, so callers must NOT attempt to close connection obtained.
|
* by Server itself, so callers must NOT attempt to close connection obtained.
|
||||||
*/
|
*/
|
||||||
ClusterConnection getConnection();
|
Connection getConnection();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a reference to the servers' cluster connection. Prefer {@link #getConnection()}.
|
||||||
|
*
|
||||||
|
* Important note: this method returns a reference to Connection which is managed
|
||||||
|
* by Server itself, so callers must NOT attempt to close connection obtained.
|
||||||
|
*/
|
||||||
|
ClusterConnection getClusterConnection();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns instance of {@link org.apache.hadoop.hbase.zookeeper.MetaTableLocator}
|
* Returns instance of {@link org.apache.hadoop.hbase.zookeeper.MetaTableLocator}
|
||||||
|
|
|
@ -0,0 +1,131 @@
|
||||||
|
/**
|
||||||
|
* 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;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.InterruptedIOException;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* View and edit the current cluster schema. Use this API making any modification to
|
||||||
|
* namespaces, tables, etc.
|
||||||
|
*
|
||||||
|
* <h2>Implementation Notes</h2>
|
||||||
|
* Nonces are for when operation is non-idempotent to ensure once-only semantic, even
|
||||||
|
* across process failures.
|
||||||
|
*/
|
||||||
|
// ClusterSchema is introduced to encapsulate schema modification. Currently the different aspects
|
||||||
|
// are spread about the code base. This effort is about cleanup, shutting down access, and
|
||||||
|
// coalescing common code. In particular, we'd contain filesystem modification. Other
|
||||||
|
// benefits are to make all schema modification work the same way (one way to do an operation only
|
||||||
|
// rather than the current approach where how an operation is done varies with context) and to make
|
||||||
|
// it so clusterschema modification can stand apart from Master to faciliate standalone
|
||||||
|
// testing. It is part of the filesystem refactor project that undoes the dependency on a
|
||||||
|
// layout in HDFS that mimics our model of tables have regions have column families have files.
|
||||||
|
// With this Interface in place, with all modifications going via this route where no filesystem
|
||||||
|
// particulars are exposed, redoing our internals will take less effort.
|
||||||
|
//
|
||||||
|
// Currently ClusterSchema Interface will include namespace and table manipulation. Ideally a
|
||||||
|
// form of this Interface will go all the ways down to the file manipulation level but currently
|
||||||
|
// TBD.
|
||||||
|
//
|
||||||
|
// ClusterSchema is private to the Master; only the Master knows current cluster state and has
|
||||||
|
// means of editing/altering it.
|
||||||
|
//
|
||||||
|
// TODO: Remove Server argument when MasterServices are passed.
|
||||||
|
// TODO: We return Future<ProcedureInfo> in the below from most methods. It may change to return
|
||||||
|
// a ProcedureFuture subsequently.
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public interface ClusterSchema {
|
||||||
|
/**
|
||||||
|
* Timeout for cluster operations in milliseconds.
|
||||||
|
*/
|
||||||
|
public static final String HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT_KEY =
|
||||||
|
"hbase.master.cluster.schema.operation.timeout";
|
||||||
|
/**
|
||||||
|
* Default operation timeout in milliseconds.
|
||||||
|
*/
|
||||||
|
public static final int DEFAULT_HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT =
|
||||||
|
5 * 60 * 1000;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* For internals use only. Do not use! Provisionally part of this Interface.
|
||||||
|
* Prefer the high-level APIs available elsewhere in this API.
|
||||||
|
* @return Instance of {@link TableNamespaceManager}
|
||||||
|
*/
|
||||||
|
// TODO: Remove from here. Keep internal. This Interface is too high-level to host this accessor.
|
||||||
|
TableNamespaceManager getTableNamespaceManager();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a new Namespace.
|
||||||
|
* @param namespaceDescriptor descriptor for new Namespace
|
||||||
|
* @param nonceGroup Identifier for the source of the request, a client or process.
|
||||||
|
* @param nonce A unique identifier for this operation from the client or process identified by
|
||||||
|
* <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
|
||||||
|
* @return procedure id
|
||||||
|
* @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
|
||||||
|
* as well as {@link IOException}
|
||||||
|
*/
|
||||||
|
long createNamespace(NamespaceDescriptor namespaceDescriptor, long nonceGroup, long nonce)
|
||||||
|
throws IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Modify an existing Namespace.
|
||||||
|
* @param nonceGroup Identifier for the source of the request, a client or process.
|
||||||
|
* @param nonce A unique identifier for this operation from the client or process identified by
|
||||||
|
* <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
|
||||||
|
* @return procedure id
|
||||||
|
* @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
|
||||||
|
* as well as {@link IOException}
|
||||||
|
*/
|
||||||
|
long modifyNamespace(NamespaceDescriptor descriptor, long nonceGroup, long nonce)
|
||||||
|
throws IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Delete an existing Namespace.
|
||||||
|
* Only empty Namespaces (no tables) can be removed.
|
||||||
|
* @param nonceGroup Identifier for the source of the request, a client or process.
|
||||||
|
* @param nonce A unique identifier for this operation from the client or process identified by
|
||||||
|
* <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
|
||||||
|
* @return procedure id
|
||||||
|
* @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
|
||||||
|
* as well as {@link IOException}
|
||||||
|
*/
|
||||||
|
long deleteNamespace(String name, long nonceGroup, long nonce)
|
||||||
|
throws IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get a Namespace
|
||||||
|
* @param name Name of the Namespace
|
||||||
|
* @return Namespace descriptor for <code>name</code>
|
||||||
|
* @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
|
||||||
|
* as well as {@link IOException}
|
||||||
|
*/
|
||||||
|
// No Future here because presumption is that the request will go against cached metadata so
|
||||||
|
// return immediately -- no need of running a Procedure.
|
||||||
|
NamespaceDescriptor getNamespace(String name) throws IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get all Namespaces
|
||||||
|
* @return All Namespace descriptors
|
||||||
|
*/
|
||||||
|
List<NamespaceDescriptor> getNamespaces() throws IOException;
|
||||||
|
}
|
|
@ -0,0 +1,37 @@
|
||||||
|
/**
|
||||||
|
* 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;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hbase.HBaseIOException;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
|
||||||
|
@SuppressWarnings("serial")
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public class ClusterSchemaException extends HBaseIOException {
|
||||||
|
public ClusterSchemaException(String message) {
|
||||||
|
super(message);
|
||||||
|
}
|
||||||
|
|
||||||
|
public ClusterSchemaException(String message, Throwable cause) {
|
||||||
|
super(message, cause);
|
||||||
|
}
|
||||||
|
|
||||||
|
public ClusterSchemaException(Throwable cause) {
|
||||||
|
super(cause);
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,27 @@
|
||||||
|
/**
|
||||||
|
* 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;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hbase.Service;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Mixes in ClusterSchema and Service
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public interface ClusterSchemaService extends ClusterSchema, Service {}
|
|
@ -0,0 +1,131 @@
|
||||||
|
/**
|
||||||
|
* 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;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.NamespaceNotFoundException;
|
||||||
|
import org.apache.hadoop.hbase.ServiceNotRunningException;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure;
|
||||||
|
import org.apache.hadoop.hbase.master.procedure.DeleteNamespaceProcedure;
|
||||||
|
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
||||||
|
import org.apache.hadoop.hbase.master.procedure.ModifyNamespaceProcedure;
|
||||||
|
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||||
|
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
||||||
|
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
class ClusterSchemaServiceImpl implements ClusterSchemaService {
|
||||||
|
private boolean running = false;
|
||||||
|
private final TableNamespaceManager tableNamespaceManager;
|
||||||
|
private final MasterServices masterServices;
|
||||||
|
private final static List<NamespaceDescriptor> EMPTY_NAMESPACE_LIST =
|
||||||
|
Collections.unmodifiableList(new ArrayList<NamespaceDescriptor>(0));
|
||||||
|
|
||||||
|
ClusterSchemaServiceImpl(final MasterServices masterServices) {
|
||||||
|
this.masterServices = masterServices;
|
||||||
|
this.tableNamespaceManager = new TableNamespaceManager(masterServices);
|
||||||
|
}
|
||||||
|
|
||||||
|
// All below are synchronized so consistent view on whether running or not.
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized boolean isRunning() {
|
||||||
|
return this.running;
|
||||||
|
}
|
||||||
|
|
||||||
|
private synchronized void checkIsRunning() throws ServiceNotRunningException {
|
||||||
|
if (!isRunning()) throw new ServiceNotRunningException();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized void startAndWait() throws IOException {
|
||||||
|
if (isRunning()) throw new IllegalStateException("Already running; cannot double-start.");
|
||||||
|
// Set to running FIRST because tableNamespaceManager start uses this class to do namespace ops
|
||||||
|
this.running = true;
|
||||||
|
this.tableNamespaceManager.start();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public synchronized void stopAndWait() throws IOException {
|
||||||
|
checkIsRunning();
|
||||||
|
// You can't stop tableNamespaceManager.
|
||||||
|
this.running = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public TableNamespaceManager getTableNamespaceManager() {
|
||||||
|
return this.tableNamespaceManager;
|
||||||
|
}
|
||||||
|
|
||||||
|
private long submitProcedure(final Procedure<?> procedure, long nonceGroup,
|
||||||
|
long nonce)
|
||||||
|
throws ServiceNotRunningException {
|
||||||
|
checkIsRunning();
|
||||||
|
ProcedureExecutor<MasterProcedureEnv> pe = this.masterServices.getMasterProcedureExecutor();
|
||||||
|
return pe.submitProcedure(procedure, nonceGroup, nonce);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long createNamespace(NamespaceDescriptor namespaceDescriptor,
|
||||||
|
long nonceGroup, long nonce)
|
||||||
|
throws IOException {
|
||||||
|
return submitProcedure(new CreateNamespaceProcedure(
|
||||||
|
this.masterServices.getMasterProcedureExecutor().getEnvironment(), namespaceDescriptor),
|
||||||
|
nonceGroup, nonce);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long modifyNamespace(NamespaceDescriptor namespaceDescriptor,
|
||||||
|
long nonceGroup, long nonce)
|
||||||
|
throws IOException {
|
||||||
|
return submitProcedure(new ModifyNamespaceProcedure(
|
||||||
|
this.masterServices.getMasterProcedureExecutor().getEnvironment(), namespaceDescriptor),
|
||||||
|
nonceGroup, nonce);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long deleteNamespace(String name, long nonceGroup, long nonce)
|
||||||
|
throws IOException {
|
||||||
|
return submitProcedure(new DeleteNamespaceProcedure(
|
||||||
|
this.masterServices.getMasterProcedureExecutor().getEnvironment(), name),
|
||||||
|
nonceGroup, nonce);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public NamespaceDescriptor getNamespace(String name) throws IOException {
|
||||||
|
NamespaceDescriptor nsd = getTableNamespaceManager().get(name);
|
||||||
|
if (nsd == null) throw new NamespaceNotFoundException(name);
|
||||||
|
return nsd;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<NamespaceDescriptor> getNamespaces() throws IOException {
|
||||||
|
checkIsRunning();
|
||||||
|
Set<NamespaceDescriptor> set = getTableNamespaceManager().list();
|
||||||
|
if (set == null || set.isEmpty()) return EMPTY_NAMESPACE_LIST;
|
||||||
|
List<NamespaceDescriptor> list = new ArrayList<NamespaceDescriptor>(set.size());
|
||||||
|
list.addAll(set);
|
||||||
|
return Collections.unmodifiableList(list);
|
||||||
|
}
|
||||||
|
}
|
|
@ -63,7 +63,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
import org.apache.hadoop.hbase.MasterNotRunningException;
|
import org.apache.hadoop.hbase.MasterNotRunningException;
|
||||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||||
import org.apache.hadoop.hbase.NamespaceNotFoundException;
|
|
||||||
import org.apache.hadoop.hbase.PleaseHoldException;
|
import org.apache.hadoop.hbase.PleaseHoldException;
|
||||||
import org.apache.hadoop.hbase.ProcedureInfo;
|
import org.apache.hadoop.hbase.ProcedureInfo;
|
||||||
import org.apache.hadoop.hbase.Server;
|
import org.apache.hadoop.hbase.Server;
|
||||||
|
@ -79,6 +78,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||||
import org.apache.hadoop.hbase.client.Result;
|
import org.apache.hadoop.hbase.client.Result;
|
||||||
import org.apache.hadoop.hbase.client.TableState;
|
import org.apache.hadoop.hbase.client.TableState;
|
||||||
|
import org.apache.hadoop.hbase.coprocessor.BypassCoprocessorException;
|
||||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||||
import org.apache.hadoop.hbase.executor.ExecutorType;
|
import org.apache.hadoop.hbase.executor.ExecutorType;
|
||||||
|
@ -97,17 +97,14 @@ import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
|
||||||
import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerChore;
|
import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerChore;
|
||||||
import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerFactory;
|
import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerFactory;
|
||||||
import org.apache.hadoop.hbase.master.procedure.AddColumnFamilyProcedure;
|
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.CreateTableProcedure;
|
||||||
import org.apache.hadoop.hbase.master.procedure.DeleteColumnFamilyProcedure;
|
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.DeleteTableProcedure;
|
||||||
import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
|
import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
|
||||||
import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure;
|
import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure;
|
||||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
|
import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
|
||||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
||||||
import org.apache.hadoop.hbase.master.procedure.ModifyColumnFamilyProcedure;
|
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.ModifyTableProcedure;
|
||||||
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
|
import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
|
||||||
import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
|
import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
|
||||||
|
@ -185,7 +182,7 @@ import com.google.protobuf.Service;
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
|
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
|
||||||
@SuppressWarnings("deprecation")
|
@SuppressWarnings("deprecation")
|
||||||
public class HMaster extends HRegionServer implements MasterServices, Server {
|
public class HMaster extends HRegionServer implements MasterServices {
|
||||||
private static final Log LOG = LogFactory.getLog(HMaster.class.getName());
|
private static final Log LOG = LogFactory.getLog(HMaster.class.getName());
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -256,8 +253,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
// Tracker for region normalizer state
|
// Tracker for region normalizer state
|
||||||
private RegionNormalizerTracker regionNormalizerTracker;
|
private RegionNormalizerTracker regionNormalizerTracker;
|
||||||
|
|
||||||
/** Namespace stuff */
|
private ClusterSchemaService clusterSchemaService;
|
||||||
private TableNamespaceManager tableNamespaceManager;
|
|
||||||
|
|
||||||
// Metrics for the HMaster
|
// Metrics for the HMaster
|
||||||
final MetricsMaster metricsMaster;
|
final MetricsMaster metricsMaster;
|
||||||
|
@ -368,9 +364,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
* Remaining steps of initialization occur in
|
* Remaining steps of initialization occur in
|
||||||
* #finishActiveMasterInitialization(MonitoredTask) after
|
* #finishActiveMasterInitialization(MonitoredTask) after
|
||||||
* the master becomes the active one.
|
* the master becomes the active one.
|
||||||
*
|
|
||||||
* @throws KeeperException
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
*/
|
||||||
public HMaster(final Configuration conf, CoordinatedStateManager csm)
|
public HMaster(final Configuration conf, CoordinatedStateManager csm)
|
||||||
throws IOException, KeeperException {
|
throws IOException, KeeperException {
|
||||||
|
@ -570,10 +563,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Initialize all ZK based system trackers.
|
* Initialize all ZK based system trackers.
|
||||||
* @throws IOException
|
|
||||||
* @throws InterruptedException
|
|
||||||
* @throws KeeperException
|
|
||||||
* @throws CoordinatedStateException
|
|
||||||
*/
|
*/
|
||||||
void initializeZKBasedSystemTrackers() throws IOException,
|
void initializeZKBasedSystemTrackers() throws IOException,
|
||||||
InterruptedException, KeeperException, CoordinatedStateException {
|
InterruptedException, KeeperException, CoordinatedStateException {
|
||||||
|
@ -588,12 +577,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
this.balancer, this.service, this.metricsMaster,
|
this.balancer, this.service, this.metricsMaster,
|
||||||
this.tableLockManager, tableStateManager);
|
this.tableLockManager, tableStateManager);
|
||||||
|
|
||||||
this.regionServerTracker = new RegionServerTracker(zooKeeper, this,
|
this.regionServerTracker = new RegionServerTracker(zooKeeper, this, this.serverManager);
|
||||||
this.serverManager);
|
|
||||||
this.regionServerTracker.start();
|
this.regionServerTracker.start();
|
||||||
|
|
||||||
this.drainingServerTracker = new DrainingServerTracker(zooKeeper, this,
|
this.drainingServerTracker = new DrainingServerTracker(zooKeeper, this, this.serverManager);
|
||||||
this.serverManager);
|
|
||||||
this.drainingServerTracker.start();
|
this.drainingServerTracker.start();
|
||||||
|
|
||||||
// Set the cluster as up. If new RSs, they'll be waiting on this before
|
// Set the cluster as up. If new RSs, they'll be waiting on this before
|
||||||
|
@ -630,11 +617,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
* <li>Ensure assignment of meta/namespace regions<li>
|
* <li>Ensure assignment of meta/namespace regions<li>
|
||||||
* <li>Handle either fresh cluster start or master failover</li>
|
* <li>Handle either fresh cluster start or master failover</li>
|
||||||
* </ol>
|
* </ol>
|
||||||
*
|
|
||||||
* @throws IOException
|
|
||||||
* @throws InterruptedException
|
|
||||||
* @throws KeeperException
|
|
||||||
* @throws CoordinatedStateException
|
|
||||||
*/
|
*/
|
||||||
private void finishActiveMasterInitialization(MonitoredTask status)
|
private void finishActiveMasterInitialization(MonitoredTask status)
|
||||||
throws IOException, InterruptedException, KeeperException, CoordinatedStateException {
|
throws IOException, InterruptedException, KeeperException, CoordinatedStateException {
|
||||||
|
@ -781,8 +763,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
this.catalogJanitorChore = new CatalogJanitor(this, this);
|
this.catalogJanitorChore = new CatalogJanitor(this, this);
|
||||||
getChoreService().scheduleChore(catalogJanitorChore);
|
getChoreService().scheduleChore(catalogJanitorChore);
|
||||||
|
|
||||||
status.setStatus("Starting namespace manager");
|
status.setStatus("Starting cluster schema service");
|
||||||
initNamespace();
|
initClusterSchemaService();
|
||||||
|
|
||||||
if (this.cpHost != null) {
|
if (this.cpHost != null) {
|
||||||
try {
|
try {
|
||||||
|
@ -848,11 +830,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a {@link ServerManager} instance.
|
* Create a {@link ServerManager} instance.
|
||||||
* @param master
|
|
||||||
* @param services
|
|
||||||
* @return An instance of {@link ServerManager}
|
|
||||||
* @throws org.apache.hadoop.hbase.ZooKeeperConnectionException
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
*/
|
||||||
ServerManager createServerManager(final Server master,
|
ServerManager createServerManager(final Server master,
|
||||||
final MasterServices services)
|
final MasterServices services)
|
||||||
|
@ -874,7 +851,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
RegionState r = MetaTableLocator.getMetaRegionState(zkw, replicaId);
|
RegionState r = MetaTableLocator.getMetaRegionState(zkw, replicaId);
|
||||||
LOG.info("Closing excess replica of meta region " + r.getRegion());
|
LOG.info("Closing excess replica of meta region " + r.getRegion());
|
||||||
// send a close and wait for a max of 30 seconds
|
// send a close and wait for a max of 30 seconds
|
||||||
ServerManager.closeRegionSilentlyAndWait(getConnection(), r.getServerName(),
|
ServerManager.closeRegionSilentlyAndWait(getClusterConnection(), r.getServerName(),
|
||||||
r.getRegion(), 30000);
|
r.getRegion(), 30000);
|
||||||
ZKUtil.deleteNode(zkw, zkw.getZNodeForReplica(replicaId));
|
ZKUtil.deleteNode(zkw, zkw.getZNodeForReplica(replicaId));
|
||||||
}
|
}
|
||||||
|
@ -888,12 +865,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Check <code>hbase:meta</code> is assigned. If not, assign it.
|
* Check <code>hbase:meta</code> is assigned. If not, assign it.
|
||||||
* @param status MonitoredTask
|
|
||||||
* @param previouslyFailedMetaRSs
|
|
||||||
* @param replicaId
|
|
||||||
* @throws InterruptedException
|
|
||||||
* @throws IOException
|
|
||||||
* @throws KeeperException
|
|
||||||
*/
|
*/
|
||||||
void assignMeta(MonitoredTask status, Set<ServerName> previouslyFailedMetaRSs, int replicaId)
|
void assignMeta(MonitoredTask status, Set<ServerName> previouslyFailedMetaRSs, int replicaId)
|
||||||
throws InterruptedException, IOException, KeeperException {
|
throws InterruptedException, IOException, KeeperException {
|
||||||
|
@ -915,7 +886,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
metaState.getServerName(), null);
|
metaState.getServerName(), null);
|
||||||
|
|
||||||
if (!metaState.isOpened() || !metaTableLocator.verifyMetaRegionLocation(
|
if (!metaState.isOpened() || !metaTableLocator.verifyMetaRegionLocation(
|
||||||
this.getConnection(), this.getZooKeeper(), timeout, replicaId)) {
|
this.getClusterConnection(), this.getZooKeeper(), timeout, replicaId)) {
|
||||||
ServerName currentMetaServer = metaState.getServerName();
|
ServerName currentMetaServer = metaState.getServerName();
|
||||||
if (serverManager.isServerOnline(currentMetaServer)) {
|
if (serverManager.isServerOnline(currentMetaServer)) {
|
||||||
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
|
if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
|
||||||
|
@ -965,10 +936,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
status.setStatus("META assigned.");
|
status.setStatus("META assigned.");
|
||||||
}
|
}
|
||||||
|
|
||||||
void initNamespace() throws IOException {
|
void initClusterSchemaService() throws IOException, InterruptedException {
|
||||||
//create namespace manager
|
this.clusterSchemaService = new ClusterSchemaServiceImpl(this);
|
||||||
tableNamespaceManager = new TableNamespaceManager(this);
|
this.clusterSchemaService.startAndWait();
|
||||||
tableNamespaceManager.start();
|
if (!this.clusterSchemaService.isRunning()) throw new HBaseIOException("Failed start");
|
||||||
}
|
}
|
||||||
|
|
||||||
void initQuotaManager() throws IOException {
|
void initQuotaManager() throws IOException {
|
||||||
|
@ -1014,7 +985,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
/**
|
/**
|
||||||
* This function returns a set of region server names under hbase:meta recovering region ZK node
|
* This function returns a set of region server names under hbase:meta recovering region ZK node
|
||||||
* @return Set of meta server names which were recorded in ZK
|
* @return Set of meta server names which were recorded in ZK
|
||||||
* @throws KeeperException
|
|
||||||
*/
|
*/
|
||||||
private Set<ServerName> getPreviouselyFailedMetaServersFromZK() throws KeeperException {
|
private Set<ServerName> getPreviouselyFailedMetaServersFromZK() throws KeeperException {
|
||||||
Set<ServerName> result = new HashSet<ServerName>();
|
Set<ServerName> result = new HashSet<ServerName>();
|
||||||
|
@ -1050,11 +1020,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
return tableStateManager;
|
return tableStateManager;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public TableNamespaceManager getTableNamespaceManager() {
|
|
||||||
return tableNamespaceManager;
|
|
||||||
}
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Start up all services. If any of these threads gets an unhandled exception
|
* 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
|
* then they just die with a logged message. This should be fine because
|
||||||
|
@ -1201,7 +1166,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return Get remote side's InetAddress
|
* @return Get remote side's InetAddress
|
||||||
* @throws UnknownHostException
|
|
||||||
*/
|
*/
|
||||||
InetAddress getRemoteInetAddress(final int port,
|
InetAddress getRemoteInetAddress(final int port,
|
||||||
final long serverStartCode) throws UnknownHostException {
|
final long serverStartCode) throws UnknownHostException {
|
||||||
|
@ -1338,7 +1302,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
* @return true if normalization step was performed successfully, false otherwise
|
* @return true if normalization step was performed successfully, false otherwise
|
||||||
* (specifically, if HMaster hasn't been initialized properly or normalization
|
* (specifically, if HMaster hasn't been initialized properly or normalization
|
||||||
* is globally disabled)
|
* is globally disabled)
|
||||||
* @throws IOException
|
|
||||||
*/
|
*/
|
||||||
public boolean normalizeRegions() throws IOException {
|
public boolean normalizeRegions() throws IOException {
|
||||||
if (!this.initialized) {
|
if (!this.initialized) {
|
||||||
|
@ -1478,9 +1441,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
if (isStopped()) {
|
if (isStopped()) {
|
||||||
throw new MasterNotRunningException();
|
throw new MasterNotRunningException();
|
||||||
}
|
}
|
||||||
|
checkInitialized();
|
||||||
String namespace = hTableDescriptor.getTableName().getNamespaceAsString();
|
String namespace = hTableDescriptor.getTableName().getNamespaceAsString();
|
||||||
ensureNamespaceExists(namespace);
|
this.clusterSchemaService.getNamespace(namespace);
|
||||||
|
|
||||||
HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(hTableDescriptor, splitKeys);
|
HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(hTableDescriptor, splitKeys);
|
||||||
checkInitialized();
|
checkInitialized();
|
||||||
|
@ -2167,8 +2130,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
* The set of loaded coprocessors is stored in a static set. Since it's
|
* The set of loaded coprocessors is stored in a static set. Since it's
|
||||||
* statically allocated, it does not require that HMaster's cpHost be
|
* statically allocated, it does not require that HMaster's cpHost be
|
||||||
* initialized prior to accessing it.
|
* initialized prior to accessing it.
|
||||||
* @return a String representation of the set of names of the loaded
|
* @return a String representation of the set of names of the loaded coprocessors.
|
||||||
* coprocessors.
|
|
||||||
*/
|
*/
|
||||||
public static String getLoadedCoprocessors() {
|
public static String getLoadedCoprocessors() {
|
||||||
return CoprocessorHost.getLoadedCoprocessors().toString();
|
return CoprocessorHost.getLoadedCoprocessors().toString();
|
||||||
|
@ -2305,18 +2267,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
|
|
||||||
void checkInitialized() throws PleaseHoldException, ServerNotRunningYetException {
|
void checkInitialized() throws PleaseHoldException, ServerNotRunningYetException {
|
||||||
checkServiceStarted();
|
checkServiceStarted();
|
||||||
if (!this.initialized) {
|
if (!isInitialized()) throw new PleaseHoldException("Master is initializing");
|
||||||
throw new PleaseHoldException("Master is initializing");
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
void checkNamespaceManagerReady() throws IOException {
|
|
||||||
checkInitialized();
|
|
||||||
if (tableNamespaceManager == null ||
|
|
||||||
!tableNamespaceManager.isTableAvailableAndInitialized(true)) {
|
|
||||||
throw new IOException("Table Namespace Manager not ready yet, try again later");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
/**
|
/**
|
||||||
* Report whether this master is currently the active master or not.
|
* Report whether this master is currently the active master or not.
|
||||||
* If not active master, we are parked on ZK waiting to become active.
|
* If not active master, we are parked on ZK waiting to become active.
|
||||||
|
@ -2411,7 +2364,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
/**
|
/**
|
||||||
* Utility for constructing an instance of the passed HMaster class.
|
* Utility for constructing an instance of the passed HMaster class.
|
||||||
* @param masterClass
|
* @param masterClass
|
||||||
* @param conf
|
|
||||||
* @return HMaster instance.
|
* @return HMaster instance.
|
||||||
*/
|
*/
|
||||||
public static HMaster constructMaster(Class<? extends HMaster> masterClass,
|
public static HMaster constructMaster(Class<? extends HMaster> masterClass,
|
||||||
|
@ -2452,138 +2404,116 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void createNamespace(
|
public ClusterSchema getClusterSchema() {
|
||||||
final NamespaceDescriptor descriptor,
|
return this.clusterSchemaService;
|
||||||
final long nonceGroup,
|
|
||||||
final long nonce) throws IOException {
|
|
||||||
TableName.isLegalNamespaceName(Bytes.toBytes(descriptor.getName()));
|
|
||||||
checkNamespaceManagerReady();
|
|
||||||
if (cpHost != null) {
|
|
||||||
if (cpHost.preCreateNamespace(descriptor)) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
createNamespaceSync(descriptor, nonceGroup, nonce);
|
|
||||||
if (cpHost != null) {
|
|
||||||
cpHost.postCreateNamespace(descriptor);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
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) {
|
|
||||||
if (cpHost.preModifyNamespace(descriptor)) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
LOG.info(getClientIdAuditPrefix() + " modify " + 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(
|
|
||||||
final String name,
|
|
||||||
final long nonceGroup,
|
|
||||||
final long nonce) throws IOException {
|
|
||||||
checkNamespaceManagerReady();
|
|
||||||
if (cpHost != null) {
|
|
||||||
if (cpHost.preDeleteNamespace(name)) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
LOG.info(getClientIdAuditPrefix() + " delete " + 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);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Ensure that the specified namespace exists, otherwise throws a NamespaceNotFoundException
|
* Create a new Namespace.
|
||||||
*
|
* @param namespaceDescriptor descriptor for new Namespace
|
||||||
* @param name the namespace to check
|
* @param nonceGroup Identifier for the source of the request, a client or process.
|
||||||
* @throws IOException if the namespace manager is not ready yet.
|
* @param nonce A unique identifier for this operation from the client or process identified by
|
||||||
* @throws NamespaceNotFoundException if the namespace does not exists
|
* <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
|
||||||
|
* @return procedure id
|
||||||
*/
|
*/
|
||||||
private void ensureNamespaceExists(final String name)
|
long createNamespace(final NamespaceDescriptor namespaceDescriptor, final long nonceGroup,
|
||||||
throws IOException, NamespaceNotFoundException {
|
final long nonce)
|
||||||
checkNamespaceManagerReady();
|
throws IOException {
|
||||||
NamespaceDescriptor nsd = tableNamespaceManager.get(name);
|
checkInitialized();
|
||||||
if (nsd == null) {
|
TableName.isLegalNamespaceName(Bytes.toBytes(namespaceDescriptor.getName()));
|
||||||
throw new NamespaceNotFoundException(name);
|
if (this.cpHost != null && this.cpHost.preCreateNamespace(namespaceDescriptor)) {
|
||||||
|
throw new BypassCoprocessorException();
|
||||||
}
|
}
|
||||||
|
LOG.info(getClientIdAuditPrefix() + " creating " + namespaceDescriptor);
|
||||||
|
// Execute the operation synchronously - wait for the operation to complete before continuing.
|
||||||
|
long procId = getClusterSchema().createNamespace(namespaceDescriptor, nonceGroup, nonce);
|
||||||
|
if (this.cpHost != null) this.cpHost.postCreateNamespace(namespaceDescriptor);
|
||||||
|
return procId;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
/**
|
||||||
public NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException {
|
* Modify an existing Namespace.
|
||||||
checkNamespaceManagerReady();
|
* @param nonceGroup Identifier for the source of the request, a client or process.
|
||||||
|
* @param nonce A unique identifier for this operation from the client or process identified by
|
||||||
if (cpHost != null) {
|
* <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
|
||||||
cpHost.preGetNamespaceDescriptor(name);
|
* @return procedure id
|
||||||
|
*/
|
||||||
|
long modifyNamespace(final NamespaceDescriptor namespaceDescriptor, final long nonceGroup,
|
||||||
|
final long nonce)
|
||||||
|
throws IOException {
|
||||||
|
checkInitialized();
|
||||||
|
TableName.isLegalNamespaceName(Bytes.toBytes(namespaceDescriptor.getName()));
|
||||||
|
if (this.cpHost != null && this.cpHost.preModifyNamespace(namespaceDescriptor)) {
|
||||||
|
throw new BypassCoprocessorException();
|
||||||
|
}
|
||||||
|
LOG.info(getClientIdAuditPrefix() + " modify " + namespaceDescriptor);
|
||||||
|
// Execute the operation synchronously - wait for the operation to complete before continuing.
|
||||||
|
long procId = getClusterSchema().modifyNamespace(namespaceDescriptor, nonceGroup, nonce);
|
||||||
|
if (this.cpHost != null) this.cpHost.postModifyNamespace(namespaceDescriptor);
|
||||||
|
return procId;
|
||||||
}
|
}
|
||||||
|
|
||||||
NamespaceDescriptor nsd = tableNamespaceManager.get(name);
|
/**
|
||||||
if (nsd == null) {
|
* Delete an existing Namespace. Only empty Namespaces (no tables) can be removed.
|
||||||
throw new NamespaceNotFoundException(name);
|
* @param nonceGroup Identifier for the source of the request, a client or process.
|
||||||
}
|
* @param nonce A unique identifier for this operation from the client or process identified by
|
||||||
|
* <code>nonceGroup</code> (the source must ensure each operation gets a unique id).
|
||||||
if (cpHost != null) {
|
* @return procedure id
|
||||||
cpHost.postGetNamespaceDescriptor(nsd);
|
*/
|
||||||
|
long deleteNamespace(final String name, final long nonceGroup, final long nonce)
|
||||||
|
throws IOException {
|
||||||
|
checkInitialized();
|
||||||
|
if (this.cpHost != null && this.cpHost.preDeleteNamespace(name)) {
|
||||||
|
throw new BypassCoprocessorException();
|
||||||
|
}
|
||||||
|
LOG.info(getClientIdAuditPrefix() + " delete " + name);
|
||||||
|
// Execute the operation synchronously - wait for the operation to complete before continuing.
|
||||||
|
long procId = getClusterSchema().deleteNamespace(name, nonceGroup, nonce);
|
||||||
|
if (this.cpHost != null) this.cpHost.postDeleteNamespace(name);
|
||||||
|
return procId;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get a Namespace
|
||||||
|
* @param name Name of the Namespace
|
||||||
|
* @return Namespace descriptor for <code>name</code>
|
||||||
|
*/
|
||||||
|
NamespaceDescriptor getNamespace(String name) throws IOException {
|
||||||
|
checkInitialized();
|
||||||
|
if (this.cpHost != null) this.cpHost.preGetNamespaceDescriptor(name);
|
||||||
|
NamespaceDescriptor nsd = this.clusterSchemaService.getNamespace(name);
|
||||||
|
if (this.cpHost != null) this.cpHost.postGetNamespaceDescriptor(nsd);
|
||||||
return nsd;
|
return nsd;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
/**
|
||||||
public List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException {
|
* Get all Namespaces
|
||||||
checkNamespaceManagerReady();
|
* @return All Namespace descriptors
|
||||||
|
*/
|
||||||
final List<NamespaceDescriptor> descriptors = new ArrayList<NamespaceDescriptor>();
|
List<NamespaceDescriptor> getNamespaces() throws IOException {
|
||||||
|
checkInitialized();
|
||||||
|
final List<NamespaceDescriptor> nsds = new ArrayList<NamespaceDescriptor>();
|
||||||
boolean bypass = false;
|
boolean bypass = false;
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
bypass = cpHost.preListNamespaceDescriptors(descriptors);
|
bypass = cpHost.preListNamespaceDescriptors(nsds);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!bypass) {
|
if (!bypass) {
|
||||||
descriptors.addAll(tableNamespaceManager.list());
|
nsds.addAll(this.clusterSchemaService.getNamespaces());
|
||||||
|
if (this.cpHost != null) this.cpHost.postListNamespaceDescriptors(nsds);
|
||||||
|
}
|
||||||
|
return nsds;
|
||||||
|
}
|
||||||
|
|
||||||
if (cpHost != null) {
|
@Override
|
||||||
cpHost.postListNamespaceDescriptors(descriptors);
|
public List<TableName> listTableNamesByNamespace(String name) throws IOException {
|
||||||
|
checkInitialized();
|
||||||
|
return listTableNames(name, null, true);
|
||||||
}
|
}
|
||||||
}
|
|
||||||
return descriptors;
|
@Override
|
||||||
|
public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
|
||||||
|
checkInitialized();
|
||||||
|
return listTableDescriptors(name, null, null, true);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -2617,21 +2547,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
return procInfoList;
|
return procInfoList;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
|
|
||||||
ensureNamespaceExists(name);
|
|
||||||
return listTableDescriptors(name, null, null, true);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public List<TableName> listTableNamesByNamespace(String name) throws IOException {
|
|
||||||
ensureNamespaceExists(name);
|
|
||||||
return listTableNames(name, null, true);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the list of table descriptors that match the specified request
|
* Returns the list of table descriptors that match the specified request
|
||||||
*
|
|
||||||
* @param namespace the namespace to query, or null if querying for all
|
* @param namespace the namespace to query, or null if querying for all
|
||||||
* @param regex The regular expression to match against, or null if querying for all
|
* @param regex The regular expression to match against, or null if querying for all
|
||||||
* @param tableNameList the list of table names, or null if querying for all
|
* @param tableNameList the list of table names, or null if querying for all
|
||||||
|
@ -2641,50 +2558,16 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
public List<HTableDescriptor> listTableDescriptors(final String namespace, final String regex,
|
public List<HTableDescriptor> listTableDescriptors(final String namespace, final String regex,
|
||||||
final List<TableName> tableNameList, final boolean includeSysTables)
|
final List<TableName> tableNameList, final boolean includeSysTables)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
final List<HTableDescriptor> descriptors = new ArrayList<HTableDescriptor>();
|
List<HTableDescriptor> htds = new ArrayList<HTableDescriptor>();
|
||||||
|
boolean bypass = cpHost != null?
|
||||||
boolean bypass = false;
|
cpHost.preGetTableDescriptors(tableNameList, htds, regex): false;
|
||||||
if (cpHost != null) {
|
|
||||||
bypass = cpHost.preGetTableDescriptors(tableNameList, descriptors, regex);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (!bypass) {
|
if (!bypass) {
|
||||||
if (tableNameList == null || tableNameList.size() == 0) {
|
htds = getTableDescriptors(htds, namespace, regex, tableNameList, includeSysTables);
|
||||||
// request for all TableDescriptors
|
|
||||||
Collection<HTableDescriptor> htds;
|
|
||||||
if (namespace != null && namespace.length() > 0) {
|
|
||||||
htds = tableDescriptors.getByNamespace(namespace).values();
|
|
||||||
} else {
|
|
||||||
htds = tableDescriptors.getAll().values();
|
|
||||||
}
|
|
||||||
|
|
||||||
for (HTableDescriptor desc: htds) {
|
|
||||||
if (tableStateManager.isTablePresent(desc.getTableName())
|
|
||||||
&& (includeSysTables || !desc.getTableName().isSystemTable())) {
|
|
||||||
descriptors.add(desc);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
for (TableName s: tableNameList) {
|
|
||||||
if (tableStateManager.isTablePresent(s)) {
|
|
||||||
HTableDescriptor desc = tableDescriptors.get(s);
|
|
||||||
if (desc != null) {
|
|
||||||
descriptors.add(desc);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Retains only those matched by regular expression.
|
|
||||||
if (regex != null) {
|
|
||||||
filterTablesByRegex(descriptors, Pattern.compile(regex));
|
|
||||||
}
|
|
||||||
|
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
cpHost.postGetTableDescriptors(tableNameList, descriptors, regex);
|
cpHost.postGetTableDescriptors(tableNameList, htds, regex);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return descriptors;
|
return htds;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -2696,46 +2579,58 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
*/
|
*/
|
||||||
public List<TableName> listTableNames(final String namespace, final String regex,
|
public List<TableName> listTableNames(final String namespace, final String regex,
|
||||||
final boolean includeSysTables) throws IOException {
|
final boolean includeSysTables) throws IOException {
|
||||||
final List<HTableDescriptor> descriptors = new ArrayList<HTableDescriptor>();
|
List<HTableDescriptor> htds = new ArrayList<HTableDescriptor>();
|
||||||
|
boolean bypass = cpHost != null? cpHost.preGetTableNames(htds, regex): false;
|
||||||
boolean bypass = false;
|
|
||||||
if (cpHost != null) {
|
|
||||||
bypass = cpHost.preGetTableNames(descriptors, regex);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (!bypass) {
|
if (!bypass) {
|
||||||
// get all descriptors
|
htds = getTableDescriptors(htds, namespace, regex, null, includeSysTables);
|
||||||
Collection<HTableDescriptor> htds;
|
if (cpHost != null) cpHost.postGetTableNames(htds, regex);
|
||||||
if (namespace != null && namespace.length() > 0) {
|
}
|
||||||
htds = tableDescriptors.getByNamespace(namespace).values();
|
List<TableName> result = new ArrayList<TableName>(htds.size());
|
||||||
} else {
|
for (HTableDescriptor htd: htds) result.add(htd.getTableName());
|
||||||
htds = tableDescriptors.getAll().values();
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
for (HTableDescriptor htd: htds) {
|
/**
|
||||||
if (includeSysTables || !htd.getTableName().isSystemTable()) {
|
* @return list of table table descriptors after filtering by regex and whether to include system
|
||||||
descriptors.add(htd);
|
* tables, etc.
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
private List<HTableDescriptor> getTableDescriptors(final List<HTableDescriptor> htds,
|
||||||
|
final String namespace, final String regex, final List<TableName> tableNameList,
|
||||||
|
final boolean includeSysTables)
|
||||||
|
throws IOException {
|
||||||
|
if (tableNameList == null || tableNameList.size() == 0) {
|
||||||
|
// request for all TableDescriptors
|
||||||
|
Collection<HTableDescriptor> allHtds;
|
||||||
|
if (namespace != null && namespace.length() > 0) {
|
||||||
|
// Do a check on the namespace existence. Will fail if does not exist.
|
||||||
|
this.clusterSchemaService.getNamespace(namespace);
|
||||||
|
allHtds = tableDescriptors.getByNamespace(namespace).values();
|
||||||
|
} else {
|
||||||
|
allHtds = tableDescriptors.getAll().values();
|
||||||
|
}
|
||||||
|
for (HTableDescriptor desc: allHtds) {
|
||||||
|
if (tableStateManager.isTablePresent(desc.getTableName())
|
||||||
|
&& (includeSysTables || !desc.getTableName().isSystemTable())) {
|
||||||
|
htds.add(desc);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
for (TableName s: tableNameList) {
|
||||||
|
if (tableStateManager.isTablePresent(s)) {
|
||||||
|
HTableDescriptor desc = tableDescriptors.get(s);
|
||||||
|
if (desc != null) {
|
||||||
|
htds.add(desc);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Retains only those matched by regular expression.
|
// Retains only those matched by regular expression.
|
||||||
if (regex != null) {
|
if (regex != null) filterTablesByRegex(htds, Pattern.compile(regex));
|
||||||
filterTablesByRegex(descriptors, Pattern.compile(regex));
|
return htds;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (cpHost != null) {
|
|
||||||
cpHost.postGetTableNames(descriptors, regex);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
List<TableName> result = new ArrayList<TableName>(descriptors.size());
|
|
||||||
for (HTableDescriptor htd: descriptors) {
|
|
||||||
result.add(htd.getTableName());
|
|
||||||
}
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Removes the table descriptors that don't match the pattern.
|
* Removes the table descriptors that don't match the pattern.
|
||||||
* @param descriptors list of table descriptors to filter
|
* @param descriptors list of table descriptors to filter
|
||||||
|
@ -2849,10 +2744,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
|
||||||
* false is returned.
|
* false is returned.
|
||||||
*/
|
*/
|
||||||
public boolean isNormalizerOn() {
|
public boolean isNormalizerOn() {
|
||||||
if (null == regionNormalizerTracker) {
|
return null == regionNormalizerTracker? false: regionNormalizerTracker.isNormalizerOn();
|
||||||
return false;
|
|
||||||
}
|
|
||||||
return regionNormalizerTracker.isNormalizerOn();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -55,121 +55,21 @@ import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
||||||
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
|
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.*;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionResponse;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
|
import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
|
||||||
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription;
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AbortProcedureRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AbortProcedureResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.*;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableCatalogJanitorRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableCatalogJanitorResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsBalancerEnabledResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsNormalizerEnabledRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsNormalizerEnabledResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListNamespaceDescriptorsResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListProceduresRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListProceduresResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MajorCompactionTimestampResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.NormalizeResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability;
|
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SecurityCapabilitiesResponse.Capability;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetQuotaRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetQuotaResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionResponse;
|
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
|
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
|
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
|
||||||
|
@ -188,8 +88,8 @@ import org.apache.hadoop.hbase.security.access.AccessController;
|
||||||
import org.apache.hadoop.hbase.security.visibility.VisibilityController;
|
import org.apache.hadoop.hbase.security.visibility.VisibilityController;
|
||||||
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
|
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
|
||||||
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
|
||||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||||
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
import org.apache.zookeeper.KeeperException;
|
import org.apache.zookeeper.KeeperException;
|
||||||
|
|
||||||
|
@ -458,11 +358,11 @@ public class MasterRpcServices extends RSRpcServices
|
||||||
public CreateNamespaceResponse createNamespace(RpcController controller,
|
public CreateNamespaceResponse createNamespace(RpcController controller,
|
||||||
CreateNamespaceRequest request) throws ServiceException {
|
CreateNamespaceRequest request) throws ServiceException {
|
||||||
try {
|
try {
|
||||||
master.createNamespace(
|
long procId = master.createNamespace(
|
||||||
ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()),
|
ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()),
|
||||||
request.getNonceGroup(),
|
request.getNonceGroup(),
|
||||||
request.getNonce());
|
request.getNonce());
|
||||||
return CreateNamespaceResponse.getDefaultInstance();
|
return CreateNamespaceResponse.newBuilder().setProcId(procId).build();
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new ServiceException(e);
|
throw new ServiceException(e);
|
||||||
}
|
}
|
||||||
|
@ -506,11 +406,11 @@ public class MasterRpcServices extends RSRpcServices
|
||||||
public DeleteNamespaceResponse deleteNamespace(RpcController controller,
|
public DeleteNamespaceResponse deleteNamespace(RpcController controller,
|
||||||
DeleteNamespaceRequest request) throws ServiceException {
|
DeleteNamespaceRequest request) throws ServiceException {
|
||||||
try {
|
try {
|
||||||
master.deleteNamespace(
|
long procId = master.deleteNamespace(
|
||||||
request.getNamespaceName(),
|
request.getNamespaceName(),
|
||||||
request.getNonceGroup(),
|
request.getNonceGroup(),
|
||||||
request.getNonce());
|
request.getNonce());
|
||||||
return DeleteNamespaceResponse.getDefaultInstance();
|
return DeleteNamespaceResponse.newBuilder().setProcId(procId).build();
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new ServiceException(e);
|
throw new ServiceException(e);
|
||||||
}
|
}
|
||||||
|
@ -832,7 +732,7 @@ public class MasterRpcServices extends RSRpcServices
|
||||||
try {
|
try {
|
||||||
return GetNamespaceDescriptorResponse.newBuilder()
|
return GetNamespaceDescriptorResponse.newBuilder()
|
||||||
.setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(
|
.setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(
|
||||||
master.getNamespaceDescriptor(request.getNamespaceName())))
|
master.getNamespace(request.getNamespaceName())))
|
||||||
.build();
|
.build();
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new ServiceException(e);
|
throw new ServiceException(e);
|
||||||
|
@ -977,10 +877,8 @@ public class MasterRpcServices extends RSRpcServices
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Checks if the specified procedure is done.
|
* Checks if the specified procedure is done.
|
||||||
* @return true if the procedure is done,
|
* @return true if the procedure is done, false if the procedure is in the process of completing
|
||||||
* false if the procedure is in the process of completing
|
* @throws ServiceException if invalid procedure or failed procedure with progress failure reason.
|
||||||
* @throws ServiceException if invalid procedure, or
|
|
||||||
* a failed procedure with progress failure reason.
|
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public IsProcedureDoneResponse isProcedureDone(RpcController controller,
|
public IsProcedureDoneResponse isProcedureDone(RpcController controller,
|
||||||
|
@ -1120,7 +1018,7 @@ public class MasterRpcServices extends RSRpcServices
|
||||||
try {
|
try {
|
||||||
ListNamespaceDescriptorsResponse.Builder response =
|
ListNamespaceDescriptorsResponse.Builder response =
|
||||||
ListNamespaceDescriptorsResponse.newBuilder();
|
ListNamespaceDescriptorsResponse.newBuilder();
|
||||||
for(NamespaceDescriptor ns: master.listNamespaceDescriptors()) {
|
for(NamespaceDescriptor ns: master.getNamespaces()) {
|
||||||
response.addNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(ns));
|
response.addNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(ns));
|
||||||
}
|
}
|
||||||
return response.build();
|
return response.build();
|
||||||
|
@ -1200,11 +1098,11 @@ public class MasterRpcServices extends RSRpcServices
|
||||||
public ModifyNamespaceResponse modifyNamespace(RpcController controller,
|
public ModifyNamespaceResponse modifyNamespace(RpcController controller,
|
||||||
ModifyNamespaceRequest request) throws ServiceException {
|
ModifyNamespaceRequest request) throws ServiceException {
|
||||||
try {
|
try {
|
||||||
master.modifyNamespace(
|
long procId = master.modifyNamespace(
|
||||||
ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()),
|
ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()),
|
||||||
request.getNonceGroup(),
|
request.getNonceGroup(),
|
||||||
request.getNonce());
|
request.getNonce());
|
||||||
return ModifyNamespaceResponse.getDefaultInstance();
|
return ModifyNamespaceResponse.newBuilder().setProcId(procId).build();
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new ServiceException(e);
|
throw new ServiceException(e);
|
||||||
}
|
}
|
||||||
|
@ -1305,10 +1203,9 @@ public class MasterRpcServices extends RSRpcServices
|
||||||
master.checkInitialized();
|
master.checkInitialized();
|
||||||
master.snapshotManager.checkSnapshotSupport();
|
master.snapshotManager.checkSnapshotSupport();
|
||||||
|
|
||||||
// ensure namespace exists
|
// Ensure namespace exists. Will throw exception if non-known NS.
|
||||||
TableName dstTable = TableName.valueOf(request.getSnapshot().getTable());
|
TableName dstTable = TableName.valueOf(request.getSnapshot().getTable());
|
||||||
master.getNamespaceDescriptor(dstTable.getNamespaceAsString());
|
master.getNamespace(dstTable.getNamespaceAsString());
|
||||||
|
|
||||||
SnapshotDescription reqSnapshot = request.getSnapshot();
|
SnapshotDescription reqSnapshot = request.getSnapshot();
|
||||||
master.snapshotManager.restoreSnapshot(reqSnapshot);
|
master.snapshotManager.restoreSnapshot(reqSnapshot);
|
||||||
return RestoreSnapshotResponse.newBuilder().build();
|
return RestoreSnapshotResponse.newBuilder().build();
|
||||||
|
|
|
@ -21,21 +21,20 @@ package org.apache.hadoop.hbase.master;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|
||||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||||
import org.apache.hadoop.hbase.HRegionInfo;
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
|
||||||
import org.apache.hadoop.hbase.ProcedureInfo;
|
import org.apache.hadoop.hbase.ProcedureInfo;
|
||||||
import org.apache.hadoop.hbase.Server;
|
import org.apache.hadoop.hbase.Server;
|
||||||
import org.apache.hadoop.hbase.TableDescriptors;
|
import org.apache.hadoop.hbase.TableDescriptors;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.TableNotDisabledException;
|
import org.apache.hadoop.hbase.TableNotDisabledException;
|
||||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
|
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||||
import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
|
import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
|
||||||
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
|
||||||
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
||||||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
|
||||||
import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
|
import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
|
||||||
|
|
||||||
import com.google.protobuf.Service;
|
import com.google.protobuf.Service;
|
||||||
|
@ -45,6 +44,11 @@ import com.google.protobuf.Service;
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public interface MasterServices extends Server {
|
public interface MasterServices extends Server {
|
||||||
|
/**
|
||||||
|
* @return Master's instance of {@link ClusterSchema}
|
||||||
|
*/
|
||||||
|
ClusterSchema getClusterSchema();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return Master's instance of the {@link AssignmentManager}
|
* @return Master's instance of the {@link AssignmentManager}
|
||||||
*/
|
*/
|
||||||
|
@ -80,11 +84,6 @@ public interface MasterServices extends Server {
|
||||||
*/
|
*/
|
||||||
MasterCoprocessorHost getMasterCoprocessorHost();
|
MasterCoprocessorHost getMasterCoprocessorHost();
|
||||||
|
|
||||||
/**
|
|
||||||
* @return Master's instance of {@link TableNamespaceManager}
|
|
||||||
*/
|
|
||||||
TableNamespaceManager getTableNamespaceManager();
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return Master's instance of {@link MasterQuotaManager}
|
* @return Master's instance of {@link MasterQuotaManager}
|
||||||
*/
|
*/
|
||||||
|
@ -279,54 +278,6 @@ public interface MasterServices extends Server {
|
||||||
*/
|
*/
|
||||||
boolean isInitialized();
|
boolean isInitialized();
|
||||||
|
|
||||||
/**
|
|
||||||
* Create a new namespace
|
|
||||||
* @param descriptor descriptor which describes the new namespace
|
|
||||||
* @param nonceGroup
|
|
||||||
* @param nonce
|
|
||||||
* @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(
|
|
||||||
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(
|
|
||||||
final String name,
|
|
||||||
final long nonceGroup,
|
|
||||||
final long nonce) throws IOException;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Abort a procedure.
|
* Abort a procedure.
|
||||||
* @param procId ID of the procedure
|
* @param procId ID of the procedure
|
||||||
|
@ -337,21 +288,6 @@ public interface MasterServices extends Server {
|
||||||
public boolean abortProcedure(final long procId, final boolean mayInterruptIfRunning)
|
public boolean abortProcedure(final long procId, final boolean mayInterruptIfRunning)
|
||||||
throws IOException;
|
throws IOException;
|
||||||
|
|
||||||
/**
|
|
||||||
* Get a namespace descriptor by name
|
|
||||||
* @param name name of namespace descriptor
|
|
||||||
* @return A descriptor
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
public NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* List available namespace descriptors
|
|
||||||
* @return A descriptor
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
public List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* List procedures
|
* List procedures
|
||||||
* @return procedure list
|
* @return procedure list
|
||||||
|
|
|
@ -215,7 +215,7 @@ public class ServerManager {
|
||||||
Configuration c = master.getConfiguration();
|
Configuration c = master.getConfiguration();
|
||||||
maxSkew = c.getLong("hbase.master.maxclockskew", 30000);
|
maxSkew = c.getLong("hbase.master.maxclockskew", 30000);
|
||||||
warningSkew = c.getLong("hbase.master.warningclockskew", 10000);
|
warningSkew = c.getLong("hbase.master.warningclockskew", 10000);
|
||||||
this.connection = connect ? master.getConnection() : null;
|
this.connection = connect ? master.getClusterConnection() : null;
|
||||||
int pingMaxAttempts = Math.max(1, master.getConfiguration().getInt(
|
int pingMaxAttempts = Math.max(1, master.getConfiguration().getInt(
|
||||||
"hbase.master.maximum.ping.server.attempts", 10));
|
"hbase.master.maximum.ping.server.attempts", 10));
|
||||||
int pingSleepInterval = Math.max(1, master.getConfiguration().getInt(
|
int pingSleepInterval = Math.max(1, master.getConfiguration().getInt(
|
||||||
|
|
|
@ -27,17 +27,17 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||||
import org.apache.commons.lang.StringUtils;
|
import org.apache.commons.lang.StringUtils;
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.CellUtil;
|
import org.apache.hadoop.hbase.CellUtil;
|
||||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.HRegionInfo;
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.ZKNamespaceManager;
|
import org.apache.hadoop.hbase.ZKNamespaceManager;
|
||||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.client.Delete;
|
import org.apache.hadoop.hbase.client.Delete;
|
||||||
import org.apache.hadoop.hbase.client.Get;
|
import org.apache.hadoop.hbase.client.Get;
|
||||||
import org.apache.hadoop.hbase.client.Put;
|
import org.apache.hadoop.hbase.client.Put;
|
||||||
|
@ -46,20 +46,25 @@ import org.apache.hadoop.hbase.client.ResultScanner;
|
||||||
import org.apache.hadoop.hbase.client.Table;
|
import org.apache.hadoop.hbase.client.Table;
|
||||||
import org.apache.hadoop.hbase.client.TableState;
|
import org.apache.hadoop.hbase.client.TableState;
|
||||||
import org.apache.hadoop.hbase.constraint.ConstraintException;
|
import org.apache.hadoop.hbase.constraint.ConstraintException;
|
||||||
import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure;
|
import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
|
||||||
import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
|
import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
|
||||||
|
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.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
|
import org.apache.hadoop.hbase.util.Threads;
|
||||||
|
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This is a helper class used to manage the namespace
|
* This is a helper class used internally to manage the namespace metadata that is stored in
|
||||||
* metadata that is stored in TableName.NAMESPACE_TABLE_NAME
|
* TableName.NAMESPACE_TABLE_NAME. It also mirrors updates to the ZK store by forwarding updates to
|
||||||
* It also mirrors updates to the ZK store by forwarding updates to
|
* {@link org.apache.hadoop.hbase.ZKNamespaceManager}.
|
||||||
* {@link org.apache.hadoop.hbase.ZKNamespaceManager}
|
*
|
||||||
|
* WARNING: Do not use. Go via the higher-level {@link ClusterSchema} API instead. This manager
|
||||||
|
* is likely to go aways anyways.
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class TableNamespaceManager {
|
public class TableNamespaceManager {
|
||||||
|
@ -90,7 +95,7 @@ public class TableNamespaceManager {
|
||||||
private long exclusiveLockTimeoutMs;
|
private long exclusiveLockTimeoutMs;
|
||||||
private long sharedLockTimeoutMs;
|
private long sharedLockTimeoutMs;
|
||||||
|
|
||||||
public TableNamespaceManager(MasterServices masterServices) {
|
TableNamespaceManager(MasterServices masterServices) {
|
||||||
this.masterServices = masterServices;
|
this.masterServices = masterServices;
|
||||||
this.conf = masterServices.getConfiguration();
|
this.conf = masterServices.getConfiguration();
|
||||||
|
|
||||||
|
@ -113,7 +118,7 @@ public class TableNamespaceManager {
|
||||||
// Wait for the namespace table to be initialized.
|
// Wait for the namespace table to be initialized.
|
||||||
long startTime = EnvironmentEdgeManager.currentTime();
|
long startTime = EnvironmentEdgeManager.currentTime();
|
||||||
int timeout = conf.getInt(NS_INIT_TIMEOUT, DEFAULT_NS_INIT_TIMEOUT);
|
int timeout = conf.getInt(NS_INIT_TIMEOUT, DEFAULT_NS_INIT_TIMEOUT);
|
||||||
while (!isTableAvailableAndInitialized(false)) {
|
while (!isTableAvailableAndInitialized()) {
|
||||||
if (EnvironmentEdgeManager.currentTime() - startTime + 100 > timeout) {
|
if (EnvironmentEdgeManager.currentTime() - startTime + 100 > timeout) {
|
||||||
// We can't do anything if ns is not online.
|
// We can't do anything if ns is not online.
|
||||||
throw new IOException("Timedout " + timeout + "ms waiting for namespace table to "
|
throw new IOException("Timedout " + timeout + "ms waiting for namespace table to "
|
||||||
|
@ -268,17 +273,49 @@ public class TableNamespaceManager {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create Namespace in a blocking manner. Keeps trying until
|
||||||
|
* {@link ClusterSchema.HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT_KEY} expires.
|
||||||
|
* Note, by-passes notifying coprocessors and name checks. Use for system namespaces only.
|
||||||
|
*/
|
||||||
|
private void blockingCreateNamespace(final NamespaceDescriptor namespaceDescriptor)
|
||||||
|
throws IOException {
|
||||||
|
ClusterSchema clusterSchema = this.masterServices.getClusterSchema();
|
||||||
|
long procId =
|
||||||
|
clusterSchema.createNamespace(namespaceDescriptor, HConstants.NO_NONCE, HConstants.NO_NONCE);
|
||||||
|
block(this.masterServices, procId);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An ugly utility to be removed when refactor TableNamespaceManager.
|
||||||
|
* @throws TimeoutIOException
|
||||||
|
*/
|
||||||
|
private static void block(final MasterServices services, final long procId)
|
||||||
|
throws TimeoutIOException {
|
||||||
|
int timeoutInMillis = services.getConfiguration().
|
||||||
|
getInt(ClusterSchema.HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT_KEY,
|
||||||
|
ClusterSchema.DEFAULT_HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT);
|
||||||
|
long deadlineTs = EnvironmentEdgeManager.currentTime() + timeoutInMillis;
|
||||||
|
ProcedureExecutor<MasterProcedureEnv> procedureExecutor =
|
||||||
|
services.getMasterProcedureExecutor();
|
||||||
|
while(EnvironmentEdgeManager.currentTime() < deadlineTs) {
|
||||||
|
if (procedureExecutor.isFinished(procId)) return;
|
||||||
|
// Sleep some
|
||||||
|
Threads.sleep(10);
|
||||||
|
}
|
||||||
|
throw new TimeoutIOException("Procedure " + procId + " is still running");
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This method checks if the namespace table is assigned and then
|
* 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
|
* tries to create its Table reference. If it was already created before, it also makes
|
||||||
* sure that the connection isn't closed.
|
* sure that the connection isn't closed.
|
||||||
* @return true if the namespace table manager is ready to serve, false
|
* @return true if the namespace table manager is ready to serve, false otherwise
|
||||||
* otherwise
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
*/
|
||||||
@SuppressWarnings("deprecation")
|
@SuppressWarnings("deprecation")
|
||||||
public synchronized boolean isTableAvailableAndInitialized(
|
public synchronized boolean isTableAvailableAndInitialized()
|
||||||
final boolean createNamespaceAync) throws IOException {
|
throws IOException {
|
||||||
// Did we already get a table? If so, still make sure it's available
|
// Did we already get a table? If so, still make sure it's available
|
||||||
if (isTableNamespaceManagerInitialized()) {
|
if (isTableNamespaceManagerInitialized()) {
|
||||||
return true;
|
return true;
|
||||||
|
@ -293,34 +330,10 @@ public class TableNamespaceManager {
|
||||||
zkNamespaceManager.start();
|
zkNamespaceManager.start();
|
||||||
|
|
||||||
if (get(nsTable, NamespaceDescriptor.DEFAULT_NAMESPACE.getName()) == null) {
|
if (get(nsTable, NamespaceDescriptor.DEFAULT_NAMESPACE.getName()) == null) {
|
||||||
if (createNamespaceAync) {
|
blockingCreateNamespace(NamespaceDescriptor.DEFAULT_NAMESPACE);
|
||||||
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) {
|
if (get(nsTable, NamespaceDescriptor.SYSTEM_NAMESPACE.getName()) == null) {
|
||||||
if (createNamespaceAync) {
|
blockingCreateNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE);
|
||||||
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) {
|
if (!initGoodSofar) {
|
||||||
|
|
|
@ -348,7 +348,7 @@ public class CreateNamespaceProcedure
|
||||||
}
|
}
|
||||||
|
|
||||||
private static TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
|
private static TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
|
||||||
return env.getMasterServices().getTableNamespaceManager();
|
return env.getMasterServices().getClusterSchema().getTableNamespaceManager();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -383,7 +383,7 @@ public class DeleteNamespaceProcedure
|
||||||
}
|
}
|
||||||
|
|
||||||
private static TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
|
private static TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
|
||||||
return env.getMasterServices().getTableNamespaceManager();
|
return env.getMasterServices().getClusterSchema().getTableNamespaceManager();
|
||||||
}
|
}
|
||||||
/**
|
/**
|
||||||
* The procedure could be restarted from a different machine. If the variable is null, we need to
|
* The procedure could be restarted from a different machine. If the variable is null, we need to
|
||||||
|
|
|
@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.TableNotDisabledException;
|
||||||
import org.apache.hadoop.hbase.TableNotFoundException;
|
import org.apache.hadoop.hbase.TableNotFoundException;
|
||||||
import org.apache.hadoop.hbase.backup.HFileArchiver;
|
import org.apache.hadoop.hbase.backup.HFileArchiver;
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
import org.apache.hadoop.hbase.client.Connection;
|
||||||
import org.apache.hadoop.hbase.client.Delete;
|
import org.apache.hadoop.hbase.client.Delete;
|
||||||
import org.apache.hadoop.hbase.client.Result;
|
import org.apache.hadoop.hbase.client.Result;
|
||||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||||
|
@ -374,7 +374,7 @@ public class DeleteTableProcedure
|
||||||
*/
|
*/
|
||||||
private static void cleanAnyRemainingRows(final MasterProcedureEnv env,
|
private static void cleanAnyRemainingRows(final MasterProcedureEnv env,
|
||||||
final TableName tableName) throws IOException {
|
final TableName tableName) throws IOException {
|
||||||
ClusterConnection connection = env.getMasterServices().getConnection();
|
Connection connection = env.getMasterServices().getConnection();
|
||||||
Scan tableScan = MetaTableAccessor.getScanForTableName(connection, tableName);
|
Scan tableScan = MetaTableAccessor.getScanForTableName(connection, tableName);
|
||||||
try (Table metaTable =
|
try (Table metaTable =
|
||||||
connection.getTable(TableName.META_TABLE_NAME)) {
|
connection.getTable(TableName.META_TABLE_NAME)) {
|
||||||
|
|
|
@ -266,8 +266,9 @@ public class ModifyNamespaceProcedure
|
||||||
}
|
}
|
||||||
|
|
||||||
private TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
|
private TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) {
|
||||||
return env.getMasterServices().getTableNamespaceManager();
|
return env.getMasterServices().getClusterSchema().getTableNamespaceManager();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The procedure could be restarted from a different machine. If the variable is null, we need to
|
* The procedure could be restarted from a different machine. If the variable is null, we need to
|
||||||
* retrieve it.
|
* retrieve it.
|
||||||
|
|
|
@ -730,7 +730,7 @@ implements ServerProcedureInterface {
|
||||||
boolean metaAssigned = false;
|
boolean metaAssigned = false;
|
||||||
// Is hbase:meta location available yet?
|
// Is hbase:meta location available yet?
|
||||||
if (mtl.isLocationAvailable(zkw)) {
|
if (mtl.isLocationAvailable(zkw)) {
|
||||||
ClusterConnection connection = env.getMasterServices().getConnection();
|
ClusterConnection connection = env.getMasterServices().getClusterConnection();
|
||||||
// Is hbase:meta location good yet?
|
// Is hbase:meta location good yet?
|
||||||
long timeout =
|
long timeout =
|
||||||
env.getMasterConfiguration().getLong(KEY_SHORT_WAIT_ON_META, DEFAULT_SHORT_WAIT_ON_META);
|
env.getMasterConfiguration().getLong(KEY_SHORT_WAIT_ON_META, DEFAULT_SHORT_WAIT_ON_META);
|
||||||
|
|
|
@ -65,7 +65,7 @@ class NamespaceStateManager {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Gets an instance of NamespaceTableAndRegionInfo associated with namespace.
|
* Gets an instance of NamespaceTableAndRegionInfo associated with namespace.
|
||||||
* @param The name of the namespace
|
* @param name The name of the namespace
|
||||||
* @return An instance of NamespaceTableAndRegionInfo.
|
* @return An instance of NamespaceTableAndRegionInfo.
|
||||||
*/
|
*/
|
||||||
public NamespaceTableAndRegionInfo getState(String name) {
|
public NamespaceTableAndRegionInfo getState(String name) {
|
||||||
|
@ -135,7 +135,7 @@ class NamespaceStateManager {
|
||||||
|
|
||||||
private NamespaceDescriptor getNamespaceDescriptor(String namespaceAsString) {
|
private NamespaceDescriptor getNamespaceDescriptor(String namespaceAsString) {
|
||||||
try {
|
try {
|
||||||
return this.master.getNamespaceDescriptor(namespaceAsString);
|
return this.master.getClusterSchema().getNamespace(namespaceAsString);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
LOG.error("Error while fetching namespace descriptor for namespace : " + namespaceAsString);
|
LOG.error("Error while fetching namespace descriptor for namespace : " + namespaceAsString);
|
||||||
return null;
|
return null;
|
||||||
|
@ -212,7 +212,7 @@ class NamespaceStateManager {
|
||||||
* Initialize namespace state cache by scanning meta table.
|
* Initialize namespace state cache by scanning meta table.
|
||||||
*/
|
*/
|
||||||
private void initialize() throws IOException {
|
private void initialize() throws IOException {
|
||||||
List<NamespaceDescriptor> namespaces = this.master.listNamespaceDescriptors();
|
List<NamespaceDescriptor> namespaces = this.master.getClusterSchema().getNamespaces();
|
||||||
for (NamespaceDescriptor namespace : namespaces) {
|
for (NamespaceDescriptor namespace : namespaces) {
|
||||||
addNamespace(namespace.getName());
|
addNamespace(namespace.getName());
|
||||||
List<TableName> tables = this.master.listTableNamesByNamespace(namespace.getName());
|
List<TableName> tables = this.master.listTableNamesByNamespace(namespace.getName());
|
||||||
|
|
|
@ -79,6 +79,7 @@ import org.apache.hadoop.hbase.YouAreDeadException;
|
||||||
import org.apache.hadoop.hbase.ZNodeClearer;
|
import org.apache.hadoop.hbase.ZNodeClearer;
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||||
|
import org.apache.hadoop.hbase.client.Connection;
|
||||||
import org.apache.hadoop.hbase.client.ConnectionUtils;
|
import org.apache.hadoop.hbase.client.ConnectionUtils;
|
||||||
import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
|
import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
|
||||||
import org.apache.hadoop.hbase.conf.ConfigurationManager;
|
import org.apache.hadoop.hbase.conf.ConfigurationManager;
|
||||||
|
@ -196,8 +197,7 @@ import sun.misc.SignalHandler;
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
|
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
|
||||||
@SuppressWarnings("deprecation")
|
@SuppressWarnings("deprecation")
|
||||||
public class HRegionServer extends HasThread implements
|
public class HRegionServer extends HasThread implements RegionServerServices, LastSequenceId {
|
||||||
RegionServerServices, LastSequenceId {
|
|
||||||
|
|
||||||
private static final Log LOG = LogFactory.getLog(HRegionServer.class);
|
private static final Log LOG = LogFactory.getLog(HRegionServer.class);
|
||||||
|
|
||||||
|
@ -1867,7 +1867,12 @@ public class HRegionServer extends HasThread implements
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ClusterConnection getConnection() {
|
public Connection getConnection() {
|
||||||
|
return getClusterConnection();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ClusterConnection getClusterConnection() {
|
||||||
return this.clusterConnection;
|
return this.clusterConnection;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -187,5 +187,11 @@ public class ReplicationSyncUp extends Configured implements Tool {
|
||||||
public ChoreService getChoreService() {
|
public ChoreService getChoreService() {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ClusterConnection getClusterConnection() {
|
||||||
|
// TODO Auto-generated method stub
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -124,7 +124,7 @@
|
||||||
</div>
|
</div>
|
||||||
<%
|
<%
|
||||||
if ( fqtn != null ) {
|
if ( fqtn != null ) {
|
||||||
table = (HTable) master.getConnection().getTable(fqtn);
|
table = (HTable) master.getConnection().getTable(TableName.valueOf(fqtn));
|
||||||
if (table.getTableDescriptor().getRegionReplication() > 1) {
|
if (table.getTableDescriptor().getRegionReplication() > 1) {
|
||||||
tableHeader = "<h2>Table Regions</h2><table class=\"table table-striped\" style=\"table-layout: fixed; word-wrap: break-word;\"><tr><th style=\"width:22%\">Name</th><th>Region Server</th><th style=\"width:22%\">Start Key</th><th style=\"width:22%\">End Key</th><th>Locality</th><th>Requests</th><th>ReplicaID</th></tr>";
|
tableHeader = "<h2>Table Regions</h2><table class=\"table table-striped\" style=\"table-layout: fixed; word-wrap: break-word;\"><tr><th style=\"width:22%\">Name</th><th>Region Server</th><th style=\"width:22%\">Start Key</th><th style=\"width:22%\">End Key</th><th>Locality</th><th>Requests</th><th>ReplicaID</th></tr>";
|
||||||
withReplica = true;
|
withReplica = true;
|
||||||
|
@ -199,7 +199,7 @@ if ( fqtn != null ) {
|
||||||
</table>
|
</table>
|
||||||
<%} else {
|
<%} else {
|
||||||
Admin admin = master.getConnection().getAdmin();
|
Admin admin = master.getConnection().getAdmin();
|
||||||
RegionLocator r = master.getConnection().getRegionLocator(table.getName());
|
RegionLocator r = master.getClusterConnection().getRegionLocator(table.getName());
|
||||||
try { %>
|
try { %>
|
||||||
<h2>Table Attributes</h2>
|
<h2>Table Attributes</h2>
|
||||||
<table class="table table-striped">
|
<table class="table table-striped">
|
||||||
|
|
|
@ -306,4 +306,10 @@ public class MockRegionServerServices implements RegionServerServices {
|
||||||
public double getCompactionPressure() {
|
public double getCompactionPressure() {
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ClusterConnection getClusterConnection() {
|
||||||
|
// TODO Auto-generated method stub
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -358,7 +358,7 @@ public class TestNamespace {
|
||||||
runWithExpectedException(new Callable<Void>() {
|
runWithExpectedException(new Callable<Void>() {
|
||||||
@Override
|
@Override
|
||||||
public Void call() throws Exception {
|
public Void call() throws Exception {
|
||||||
admin.listTableDescriptorsByNamespace("non_existing_namespace");
|
admin.listTableDescriptorsByNamespace("non_existant_namespace");
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
}, NamespaceNotFoundException.class);
|
}, NamespaceNotFoundException.class);
|
||||||
|
|
|
@ -65,7 +65,7 @@ public class TestShortCircuitConnection {
|
||||||
htd.addFamily(hcd);
|
htd.addFamily(hcd);
|
||||||
UTIL.createTable(htd, null);
|
UTIL.createTable(htd, null);
|
||||||
HRegionServer regionServer = UTIL.getRSForFirstRegionInTable(tn);
|
HRegionServer regionServer = UTIL.getRSForFirstRegionInTable(tn);
|
||||||
ClusterConnection connection = regionServer.getConnection();
|
ClusterConnection connection = regionServer.getClusterConnection();
|
||||||
HTableInterface tableIf = connection.getTable(tn);
|
HTableInterface tableIf = connection.getTable(tn);
|
||||||
assertTrue(tableIf instanceof HTable);
|
assertTrue(tableIf instanceof HTable);
|
||||||
HTable table = (HTable) tableIf;
|
HTable table = (HTable) tableIf;
|
||||||
|
|
|
@ -1586,7 +1586,13 @@ public class TestMasterObserver {
|
||||||
cp.enableBypass(true);
|
cp.enableBypass(true);
|
||||||
cp.resetStates();
|
cp.resetStates();
|
||||||
|
|
||||||
|
boolean expected = false;
|
||||||
|
try {
|
||||||
admin.modifyNamespace(NamespaceDescriptor.create(testNamespace).build());
|
admin.modifyNamespace(NamespaceDescriptor.create(testNamespace).build());
|
||||||
|
} catch (BypassCoprocessorException ce) {
|
||||||
|
expected = true;
|
||||||
|
}
|
||||||
|
assertTrue(expected);
|
||||||
assertTrue("Test namespace should not have been modified",
|
assertTrue("Test namespace should not have been modified",
|
||||||
cp.preModifyNamespaceCalledOnly());
|
cp.preModifyNamespaceCalledOnly());
|
||||||
|
|
||||||
|
@ -1594,7 +1600,13 @@ public class TestMasterObserver {
|
||||||
assertTrue("Test namespace descriptor should have been called",
|
assertTrue("Test namespace descriptor should have been called",
|
||||||
cp.wasGetNamespaceDescriptorCalled());
|
cp.wasGetNamespaceDescriptorCalled());
|
||||||
|
|
||||||
|
expected = false;
|
||||||
|
try {
|
||||||
admin.deleteNamespace(testNamespace);
|
admin.deleteNamespace(testNamespace);
|
||||||
|
} catch (BypassCoprocessorException ce) {
|
||||||
|
expected = true;
|
||||||
|
}
|
||||||
|
assertTrue(expected);
|
||||||
assertTrue("Test namespace should not have been deleted", cp.preDeleteNamespaceCalledOnly());
|
assertTrue("Test namespace should not have been deleted", cp.preDeleteNamespaceCalledOnly());
|
||||||
|
|
||||||
assertNotNull(admin.getNamespaceDescriptor(testNamespace));
|
assertNotNull(admin.getNamespaceDescriptor(testNamespace));
|
||||||
|
@ -1614,7 +1626,13 @@ public class TestMasterObserver {
|
||||||
cp.enableBypass(true);
|
cp.enableBypass(true);
|
||||||
cp.resetStates();
|
cp.resetStates();
|
||||||
|
|
||||||
|
expected = false;
|
||||||
|
try {
|
||||||
admin.createNamespace(NamespaceDescriptor.create(testNamespace).build());
|
admin.createNamespace(NamespaceDescriptor.create(testNamespace).build());
|
||||||
|
} catch (BypassCoprocessorException ce) {
|
||||||
|
expected = true;
|
||||||
|
}
|
||||||
|
assertTrue(expected);
|
||||||
assertTrue("Test namespace should not be created", cp.preCreateNamespaceCalledOnly());
|
assertTrue("Test namespace should not be created", cp.preCreateNamespaceCalledOnly());
|
||||||
|
|
||||||
// turn on bypass, run the test
|
// turn on bypass, run the test
|
||||||
|
|
|
@ -651,4 +651,10 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
|
||||||
public double getCompactionPressure() {
|
public double getCompactionPressure() {
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ClusterConnection getClusterConnection() {
|
||||||
|
// TODO Auto-generated method stub
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
|
@ -326,5 +326,11 @@ public class TestActiveMasterManager {
|
||||||
public ChoreService getChoreService() {
|
public ChoreService getChoreService() {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ClusterConnection getClusterConnection() {
|
||||||
|
// TODO Auto-generated method stub
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -46,7 +46,6 @@ import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.HRegionInfo;
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
import org.apache.hadoop.hbase.MetaMockingUtil;
|
import org.apache.hadoop.hbase.MetaMockingUtil;
|
||||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
|
||||||
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
|
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
|
||||||
import org.apache.hadoop.hbase.ProcedureInfo;
|
import org.apache.hadoop.hbase.ProcedureInfo;
|
||||||
import org.apache.hadoop.hbase.Server;
|
import org.apache.hadoop.hbase.Server;
|
||||||
|
@ -143,10 +142,9 @@ public class TestCatalogJanitor {
|
||||||
ServerName.valueOf("example.org,12345,6789"),
|
ServerName.valueOf("example.org,12345,6789"),
|
||||||
HRegionInfo.FIRST_META_REGIONINFO);
|
HRegionInfo.FIRST_META_REGIONINFO);
|
||||||
// Set hbase.rootdir into test dir.
|
// Set hbase.rootdir into test dir.
|
||||||
FileSystem fs = FileSystem.get(this.c);
|
FileSystem.get(this.c);
|
||||||
Path rootdir = FSUtils.getRootDir(this.c);
|
Path rootdir = FSUtils.getRootDir(this.c);
|
||||||
FSUtils.setRootDir(this.c, rootdir);
|
FSUtils.setRootDir(this.c, rootdir);
|
||||||
AdminProtos.AdminService.BlockingInterface hri =
|
|
||||||
Mockito.mock(AdminProtos.AdminService.BlockingInterface.class);
|
Mockito.mock(AdminProtos.AdminService.BlockingInterface.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -208,6 +206,12 @@ public class TestCatalogJanitor {
|
||||||
public ChoreService getChoreService() {
|
public ChoreService getChoreService() {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ClusterConnection getClusterConnection() {
|
||||||
|
// TODO Auto-generated method stub
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -401,48 +405,6 @@ public class TestCatalogJanitor {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
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 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 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.
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException {
|
|
||||||
return null; //To change body of implemented methods use File | Settings | File Templates.
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException {
|
|
||||||
return null; //To change body of implemented methods use File | Settings | File Templates.
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean abortProcedure(final long procId, final boolean mayInterruptIfRunning)
|
public boolean abortProcedure(final long procId, final boolean mayInterruptIfRunning)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
@ -535,11 +497,6 @@ public class TestCatalogJanitor {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public TableNamespaceManager getTableNamespaceManager() {
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void dispatchMergingRegions(HRegionInfo region_a, HRegionInfo region_b,
|
public void dispatchMergingRegions(HRegionInfo region_a, HRegionInfo region_b,
|
||||||
boolean forcible) throws IOException {
|
boolean forcible) throws IOException {
|
||||||
|
@ -547,21 +504,29 @@ public class TestCatalogJanitor {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean isInitialized() {
|
public boolean isInitialized() {
|
||||||
// Auto-generated method stub
|
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getLastMajorCompactionTimestamp(TableName table) throws IOException {
|
public long getLastMajorCompactionTimestamp(TableName table) throws IOException {
|
||||||
// Auto-generated method stub
|
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException {
|
public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException {
|
||||||
// Auto-generated method stub
|
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ClusterSchema getClusterSchema() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ClusterConnection getClusterConnection() {
|
||||||
|
// TODO Auto-generated method stub
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
@ -100,6 +100,12 @@ public class TestClockSkewDetection {
|
||||||
public ChoreService getChoreService() {
|
public ChoreService getChoreService() {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ClusterConnection getClusterConnection() {
|
||||||
|
// TODO Auto-generated method stub
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}, null, false);
|
}, null, false);
|
||||||
|
|
||||||
LOG.debug("regionServerStartup 1");
|
LOG.debug("regionServerStartup 1");
|
||||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.Abortable;
|
import org.apache.hadoop.hbase.Abortable;
|
||||||
|
import org.apache.hadoop.hbase.CategoryBasedTimeout;
|
||||||
import org.apache.hadoop.hbase.CoordinatedStateException;
|
import org.apache.hadoop.hbase.CoordinatedStateException;
|
||||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||||
import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
|
import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
|
||||||
|
@ -62,8 +63,10 @@ import org.apache.zookeeper.KeeperException;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.AfterClass;
|
import org.junit.AfterClass;
|
||||||
import org.junit.BeforeClass;
|
import org.junit.BeforeClass;
|
||||||
|
import org.junit.Rule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.experimental.categories.Category;
|
import org.junit.experimental.categories.Category;
|
||||||
|
import org.junit.rules.TestRule;
|
||||||
import org.mockito.Mockito;
|
import org.mockito.Mockito;
|
||||||
|
|
||||||
import com.google.protobuf.ServiceException;
|
import com.google.protobuf.ServiceException;
|
||||||
|
@ -80,6 +83,8 @@ import com.google.protobuf.ServiceException;
|
||||||
public class TestMasterNoCluster {
|
public class TestMasterNoCluster {
|
||||||
private static final Log LOG = LogFactory.getLog(TestMasterNoCluster.class);
|
private static final Log LOG = LogFactory.getLog(TestMasterNoCluster.class);
|
||||||
private static final HBaseTestingUtility TESTUTIL = new HBaseTestingUtility();
|
private static final HBaseTestingUtility TESTUTIL = new HBaseTestingUtility();
|
||||||
|
@Rule public final TestRule timeout = CategoryBasedTimeout.builder().
|
||||||
|
withTimeout(this.getClass()).withLookingForStuckThread(true).build();
|
||||||
|
|
||||||
@BeforeClass
|
@BeforeClass
|
||||||
public static void setUpBeforeClass() throws Exception {
|
public static void setUpBeforeClass() throws Exception {
|
||||||
|
@ -122,7 +127,7 @@ public class TestMasterNoCluster {
|
||||||
* @throws KeeperException
|
* @throws KeeperException
|
||||||
* @throws InterruptedException
|
* @throws InterruptedException
|
||||||
*/
|
*/
|
||||||
@Test (timeout=30000)
|
@Test
|
||||||
public void testStopDuringStart()
|
public void testStopDuringStart()
|
||||||
throws IOException, KeeperException, InterruptedException {
|
throws IOException, KeeperException, InterruptedException {
|
||||||
CoordinatedStateManager cp = CoordinatedStateManagerFactory.getCoordinatedStateManager(
|
CoordinatedStateManager cp = CoordinatedStateManagerFactory.getCoordinatedStateManager(
|
||||||
|
@ -141,7 +146,7 @@ public class TestMasterNoCluster {
|
||||||
* @throws KeeperException
|
* @throws KeeperException
|
||||||
* @throws InterruptedException
|
* @throws InterruptedException
|
||||||
*/
|
*/
|
||||||
@Test (timeout=30000)
|
@Test
|
||||||
public void testFailover()
|
public void testFailover()
|
||||||
throws IOException, KeeperException, InterruptedException, ServiceException {
|
throws IOException, KeeperException, InterruptedException, ServiceException {
|
||||||
final long now = System.currentTimeMillis();
|
final long now = System.currentTimeMillis();
|
||||||
|
@ -192,6 +197,9 @@ public class TestMasterNoCluster {
|
||||||
new byte [] {10, 0, 0, (byte)sn.getPort()});
|
new byte [] {10, 0, 0, (byte)sn.getPort()});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
void initClusterSchemaService() throws IOException, InterruptedException {}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
ServerManager createServerManager(Server master, MasterServices services)
|
ServerManager createServerManager(Server master, MasterServices services)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
@ -218,10 +226,6 @@ public class TestMasterNoCluster {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
void initNamespace() {
|
|
||||||
}
|
|
||||||
};
|
};
|
||||||
master.start();
|
master.start();
|
||||||
|
|
||||||
|
@ -265,6 +269,9 @@ public class TestMasterNoCluster {
|
||||||
void assignMeta(MonitoredTask status, Set<ServerName> previouslyFailedMeatRSs, int replicaId)
|
void assignMeta(MonitoredTask status, Set<ServerName> previouslyFailedMeatRSs, int replicaId)
|
||||||
{ }
|
{ }
|
||||||
|
|
||||||
|
@Override
|
||||||
|
void initClusterSchemaService() throws IOException, InterruptedException {}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
void initializeZKBasedSystemTrackers() throws IOException,
|
void initializeZKBasedSystemTrackers() throws IOException,
|
||||||
InterruptedException, KeeperException, CoordinatedStateException {
|
InterruptedException, KeeperException, CoordinatedStateException {
|
||||||
|
@ -294,10 +301,6 @@ public class TestMasterNoCluster {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
void initNamespace() {
|
|
||||||
}
|
|
||||||
};
|
};
|
||||||
master.start();
|
master.start();
|
||||||
|
|
||||||
|
|
|
@ -168,6 +168,12 @@ public class TestSplitLogManager {
|
||||||
public ChoreService getChoreService() {
|
public ChoreService getChoreService() {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ClusterConnection getClusterConnection() {
|
||||||
|
// TODO Auto-generated method stub
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
static Stoppable stopper = new Stoppable() {
|
static Stoppable stopper = new Stoppable() {
|
||||||
|
|
|
@ -27,16 +27,15 @@ import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||||
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
||||||
import org.apache.hadoop.hbase.HRegionInfo;
|
|
||||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.client.Table;
|
import org.apache.hadoop.hbase.client.Table;
|
||||||
import org.apache.hadoop.hbase.regionserver.Region;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.regionserver.Region;
|
||||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||||
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.junit.AfterClass;
|
import org.junit.AfterClass;
|
||||||
import org.junit.BeforeClass;
|
import org.junit.BeforeClass;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
|
@ -253,5 +253,11 @@ public class TestHFileCleaner {
|
||||||
public ChoreService getChoreService() {
|
public ChoreService getChoreService() {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ClusterConnection getClusterConnection() {
|
||||||
|
// TODO Auto-generated method stub
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -191,5 +191,11 @@ public class TestHFileLinkCleaner {
|
||||||
public ChoreService getChoreService() {
|
public ChoreService getChoreService() {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ClusterConnection getClusterConnection() {
|
||||||
|
// TODO Auto-generated method stub
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -223,5 +223,11 @@ public class TestLogsCleaner {
|
||||||
public ChoreService getChoreService() {
|
public ChoreService getChoreService() {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ClusterConnection getClusterConnection() {
|
||||||
|
// TODO Auto-generated method stub
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -260,5 +260,11 @@ public class TestReplicationHFileCleaner {
|
||||||
public ChoreService getChoreService() {
|
public ChoreService getChoreService() {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ClusterConnection getClusterConnection() {
|
||||||
|
// TODO Auto-generated method stub
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -438,11 +438,11 @@ public class MasterProcedureTestingUtility {
|
||||||
}
|
}
|
||||||
|
|
||||||
public static long generateNonceGroup(final HMaster master) {
|
public static long generateNonceGroup(final HMaster master) {
|
||||||
return master.getConnection().getNonceGenerator().getNonceGroup();
|
return master.getClusterConnection().getNonceGenerator().getNonceGroup();
|
||||||
}
|
}
|
||||||
|
|
||||||
public static long generateNonce(final HMaster master) {
|
public static long generateNonce(final HMaster master) {
|
||||||
return master.getConnection().getNonceGenerator().newNonce();
|
return master.getClusterConnection().getNonceGenerator().newNonce();
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class InjectAbortOnLoadListener
|
public static class InjectAbortOnLoadListener
|
||||||
|
|
|
@ -718,6 +718,12 @@ public class TestHeapMemoryManager {
|
||||||
public ChoreService getChoreService() {
|
public ChoreService getChoreService() {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ClusterConnection getClusterConnection() {
|
||||||
|
// TODO Auto-generated method stub
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
static class CustomHeapMemoryTuner implements HeapMemoryTuner {
|
static class CustomHeapMemoryTuner implements HeapMemoryTuner {
|
||||||
|
|
|
@ -143,6 +143,12 @@ public class TestSplitLogWorker {
|
||||||
public ChoreService getChoreService() {
|
public ChoreService getChoreService() {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ClusterConnection getClusterConnection() {
|
||||||
|
// TODO Auto-generated method stub
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void waitForCounter(AtomicLong ctr, long oldval, long newval, long timems)
|
private void waitForCounter(AtomicLong ctr, long oldval, long newval, long timems)
|
||||||
|
|
|
@ -191,5 +191,11 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic {
|
||||||
public ChoreService getChoreService() {
|
public ChoreService getChoreService() {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ClusterConnection getClusterConnection() {
|
||||||
|
// TODO Auto-generated method stub
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -1,4 +1,5 @@
|
||||||
/**
|
/**
|
||||||
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
|
@ -292,5 +293,11 @@ public class TestReplicationTrackerZKImpl {
|
||||||
public ChoreService getChoreService() {
|
public ChoreService getChoreService() {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ClusterConnection getClusterConnection() {
|
||||||
|
// TODO Auto-generated method stub
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -595,5 +595,11 @@ public class TestReplicationSourceManager {
|
||||||
public ChoreService getChoreService() {
|
public ChoreService getChoreService() {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ClusterConnection getClusterConnection() {
|
||||||
|
// TODO Auto-generated method stub
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -334,6 +334,12 @@ public class TestTokenAuthentication {
|
||||||
public ChoreService getChoreService() {
|
public ChoreService getChoreService() {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ClusterConnection getClusterConnection() {
|
||||||
|
// TODO Auto-generated method stub
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static HBaseTestingUtility TEST_UTIL;
|
private static HBaseTestingUtility TEST_UTIL;
|
||||||
|
|
|
@ -127,4 +127,10 @@ public class MockServer implements Server {
|
||||||
public ChoreService getChoreService() {
|
public ChoreService getChoreService() {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ClusterConnection getClusterConnection() {
|
||||||
|
// TODO Auto-generated method stub
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
Loading…
Reference in New Issue