From 46303dfd751912371539aa41ee8698dfb5e8b304 Mon Sep 17 00:00:00 2001 From: stack Date: Tue, 5 Jan 2016 14:35:27 -0800 Subject: [PATCH] HBASE-14888 ClusterSchema: Add Namespace Operations --- .../org/apache/hadoop/hbase/client/Admin.java | 52 +- .../hbase/client/ClusterConnection.java | 1 + .../client/ConnectionImplementation.java | 55 +- .../hadoop/hbase/client/HBaseAdmin.java | 231 ++--- .../BypassCoprocessorException.java | 44 + .../java/org/apache/hadoop/hbase/Service.java | 50 ++ .../hbase/ServiceNotRunningException.java | 39 + .../apache/hadoop/hbase/ResourceChecker.java | 2 +- .../protobuf/generated/MasterProtos.java | 794 ++++++++++++------ hbase-protocol/src/main/protobuf/Master.proto | 3 + .../java/org/apache/hadoop/hbase/Server.java | 13 +- .../hadoop/hbase/master/ClusterSchema.java | 131 +++ .../hbase/master/ClusterSchemaException.java | 37 + .../hbase/master/ClusterSchemaService.java | 27 + .../master/ClusterSchemaServiceImpl.java | 131 +++ .../apache/hadoop/hbase/master/HMaster.java | 438 ++++------ .../hbase/master/MasterRpcServices.java | 139 +-- .../hadoop/hbase/master/MasterServices.java | 78 +- .../hadoop/hbase/master/ServerManager.java | 2 +- .../hbase/master/TableNamespaceManager.java | 99 ++- .../procedure/CreateNamespaceProcedure.java | 2 +- .../procedure/DeleteNamespaceProcedure.java | 2 +- .../procedure/DeleteTableProcedure.java | 4 +- .../procedure/ModifyNamespaceProcedure.java | 3 +- .../procedure/ServerCrashProcedure.java | 2 +- .../namespace/NamespaceStateManager.java | 6 +- .../hbase/regionserver/HRegionServer.java | 11 +- .../regionserver/ReplicationSyncUp.java | 6 + .../resources/hbase-webapps/master/table.jsp | 4 +- .../hbase/MockRegionServerServices.java | 6 + .../apache/hadoop/hbase/TestNamespace.java | 2 +- .../client/TestShortCircuitConnection.java | 2 +- .../hbase/coprocessor/TestMasterObserver.java | 24 +- .../hadoop/hbase/master/MockRegionServer.java | 6 + .../hbase/master/TestActiveMasterManager.java | 6 + .../hbase/master/TestCatalogJanitor.java | 73 +- .../hbase/master/TestClockSkewDetection.java | 6 + .../hbase/master/TestMasterNoCluster.java | 23 +- .../hbase/master/TestSplitLogManager.java | 6 + .../balancer/TestRegionLocationFinder.java | 5 +- .../master/cleaner/TestHFileCleaner.java | 6 + .../master/cleaner/TestHFileLinkCleaner.java | 6 + .../hbase/master/cleaner/TestLogsCleaner.java | 6 + .../cleaner/TestReplicationHFileCleaner.java | 6 + .../MasterProcedureTestingUtility.java | 4 +- .../regionserver/TestHeapMemoryManager.java | 6 + .../regionserver/TestSplitLogWorker.java | 6 + .../TestReplicationStateZKImpl.java | 6 + .../TestReplicationTrackerZKImpl.java | 7 + .../TestReplicationSourceManager.java | 6 + .../token/TestTokenAuthentication.java | 6 + .../apache/hadoop/hbase/util/MockServer.java | 6 + 52 files changed, 1628 insertions(+), 1008 deletions(-) create mode 100644 hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/BypassCoprocessorException.java create mode 100644 hbase-common/src/main/java/org/apache/hadoop/hbase/Service.java create mode 100644 hbase-common/src/main/java/org/apache/hadoop/hbase/ServiceNotRunningException.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaException.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaService.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java index b06902ae9cc..d7b52d57047 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.java @@ -964,50 +964,78 @@ public interface Admin extends Abortable, Closeable { */ 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 * * @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) - throws IOException; + Future 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; /** * Modify an existing 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) - throws IOException; + Future modifyNamespaceAsync(final NamespaceDescriptor descriptor) + 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. * * @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 deleteNamespaceAsync(final String name) throws IOException; /** * Get a namespace descriptor by name * * @param name name of namespace descriptor * @return A descriptor - * @throws IOException */ NamespaceDescriptor getNamespaceDescriptor(final String name) - throws IOException; + throws IOException; /** * List available namespace descriptors * * @return List of descriptors - * @throws IOException */ NamespaceDescriptor[] listNamespaceDescriptors() - throws IOException; + throws IOException; /** * Get list of table descriptors by namespace diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java index 99071fa49df..741989f16a8 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java @@ -1,5 +1,6 @@ /** * + * 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 diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java index 0ef2a173633..ecac792d48b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java @@ -150,8 +150,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable { // be waiting for the master lock => deadlock. private final Object masterAndZKLock = new Object(); - private long keepZooKeeperWatcherAliveUntil = Long.MAX_VALUE; - // thread executor shared by all HTableInterface instances created // by this connection private volatile ExecutorService batchPool = null; @@ -398,7 +396,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable { synchronized (this) { if (batchPool == null) { 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; } } @@ -482,7 +480,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable { /** * @return The cluster registry implementation to use. - * @throws java.io.IOException */ private Registry setupRegistry() throws IOException { return RegistryFactory.getRegistry(this); @@ -542,7 +539,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable { /** * @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.ZooKeeperConnectionException * @deprecated this has been deprecated without a replacement */ @Deprecated @@ -981,9 +977,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable { /** * Search the cache for a location that fits our table and row key. * Return null if no suitable region is located. - * - * @param tableName - * @param row * @return Null or region location found in cache. */ 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. - * @param channel */ protected abstract Object makeStub(final BlockingRpcChannel channel); /** * Once setup, check it works by doing isMasterRunning check. - * @throws com.google.protobuf.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 * protobuf services nor their interfaces. Let the caller do appropriate casting. * @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 { ZooKeeperKeepAliveConnection zkw; @@ -1370,10 +1358,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable { private ZooKeeperKeepAliveConnection keepAliveZookeeper; 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. @@ -1391,7 +1375,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable { keepAliveZookeeper = new ZooKeeperKeepAliveConnection(conf, this.toString(), this); } keepAliveZookeeperUserCount.addAndGet(1); - keepZooKeeperWatcherAliveUntil = Long.MAX_VALUE; return keepAliveZookeeper; } } @@ -1400,9 +1383,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable { if (zkw == null){ return; } - if (keepAliveZookeeperUserCount.addAndGet(-1) <= 0) { - keepZooKeeperWatcherAliveUntil = System.currentTimeMillis() + keepAlive; - } } private void closeZooKeeperWatcher() { @@ -1820,7 +1800,6 @@ class ConnectionImplementation implements ClusterConnection, Closeable { }; } - private static void release(MasterServiceState mss) { if (mss != null && mss.connection != null) { ((ConnectionImplementation)mss.connection).releaseMaster(mss); @@ -1893,7 +1872,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable { * or delete it from the cache. Does nothing if we can be sure from the exception that * the location is still accurate, or if the cache has already been updated. * @param exception an object (to simplify user code) on which we will try to find a nested - * or wrapped or both RegionMovedException + * or wrapped or both RegionMovedException * @param source server that is the source of the location update. */ @Override @@ -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 @Deprecated @@ -1999,8 +1978,8 @@ class ConnectionImplementation implements ClusterConnection, Closeable { * 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 results and exceptions. - * @deprecated since 0.96 - - * Use {@link org.apache.hadoop.hbase.client.HTable#processBatchCallback} instead + * @deprecated since 0.96 + * Use {@link org.apache.hadoop.hbase.client.HTable#processBatchCallback} instead */ @Override @Deprecated @@ -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 @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 @Override - public HTableDescriptor[] getHTableDescriptors( - List names) throws IOException { + public HTableDescriptor[] getHTableDescriptors(List names) throws IOException { List tableNames = new ArrayList(names.size()); for(String name : names) { tableNames.add(TableName.valueOf(name)); @@ -2269,7 +2250,9 @@ class ConnectionImplementation implements ClusterConnection, Closeable { * @param tableName table name * @throws java.io.IOException if the connection to master fails or if the table * 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 @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 @Override @@ -2306,10 +2291,8 @@ class ConnectionImplementation implements ClusterConnection, Closeable { @Override public TableState getTableState(TableName tableName) throws IOException { if (this.closed) throw new IOException(toString() + " closed"); - TableState tableState = MetaTableAccessor.getTableState(this, tableName); - if (tableState == null) - throw new TableNotFoundException(tableName); + if (tableState == null) throw new TableNotFoundException(tableName); return tableState; } @@ -2318,4 +2301,4 @@ class ConnectionImplementation implements ClusterConnection, Closeable { return RpcRetryingCallerFactory .instantiate(conf, this.interceptor, this.getStatisticsTracker()); } -} +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java index 7a50458e368..db94ff42749 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java @@ -28,12 +28,12 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; 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.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Pattern; 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.AssignRegionRequest; 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.DeleteTableRequest; 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.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; @@ -254,23 +257,10 @@ public class HBaseAdmin implements Admin { } @Override - public boolean abortProcedure( - final long procId, - final boolean mayInterruptIfRunning) throws IOException { - Future future = abortProcedureAsync(procId, mayInterruptIfRunning); - 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()); - } - } + public boolean abortProcedure(final long procId, final boolean mayInterruptIfRunning) + throws IOException { + return get(abortProcedureAsync(procId, mayInterruptIfRunning), this.syncWaitTimeout, + TimeUnit.MILLISECONDS); } @Override @@ -464,22 +454,7 @@ public class HBaseAdmin implements Admin { @Override public void createTable(final HTableDescriptor desc, byte [][] splitKeys) throws IOException { - Future future = createTableAsync(desc, splitKeys); - 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()); - } - } + get(createTableAsync(desc, splitKeys), syncWaitTimeout, TimeUnit.MILLISECONDS); } @Override @@ -550,20 +525,7 @@ public class HBaseAdmin implements Admin { @Override public void deleteTable(final TableName tableName) throws IOException { - Future future = deleteTableAsync(tableName); - 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()); - } - } + get(deleteTableAsync(tableName), syncWaitTimeout, TimeUnit.MILLISECONDS); } @Override @@ -641,21 +603,7 @@ public class HBaseAdmin implements Admin { @Override public void truncateTable(final TableName tableName, final boolean preserveSplits) throws IOException { - Future future = truncateTableAsync(tableName, preserveSplits); - 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()); - } - } + get(truncateTableAsync(tableName, preserveSplits), syncWaitTimeout, TimeUnit.MILLISECONDS); } @Override @@ -717,20 +665,7 @@ public class HBaseAdmin implements Admin { @Override public void enableTable(final TableName tableName) throws IOException { - Future future = enableTableAsync(tableName); - 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()); - } - } + get(enableTableAsync(tableName), syncWaitTimeout, TimeUnit.MILLISECONDS); } /** @@ -833,20 +768,7 @@ public class HBaseAdmin implements Admin { @Override public void disableTable(final TableName tableName) throws IOException { - Future future = disableTableAsync(tableName); - 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()); - } - } + get(disableTableAsync(tableName), syncWaitTimeout, TimeUnit.MILLISECONDS); } @Override @@ -1841,43 +1763,103 @@ public class HBaseAdmin implements Admin { return this.conf; } + /** + * Do a get with a timeout against the passed in future. + */ + private static T get(final Future 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 - public void createNamespace(final NamespaceDescriptor descriptor) throws IOException { - executeCallable(new MasterCallable(getConnection()) { + public void createNamespace(final NamespaceDescriptor descriptor) + throws IOException { + get(createNamespaceAsync(descriptor), this.syncWaitTimeout, TimeUnit.MILLISECONDS); + } + + @Override + public Future createNamespaceAsync(final NamespaceDescriptor descriptor) + throws IOException { + CreateNamespaceResponse response = + executeCallable(new MasterCallable(getConnection()) { @Override - public Void call(int callTimeout) throws Exception { - master.createNamespace(null, + public CreateNamespaceResponse call(int callTimeout) throws Exception { + return master.createNamespace(null, CreateNamespaceRequest.newBuilder() .setNamespaceDescriptor(ProtobufUtil .toProtoNamespaceDescriptor(descriptor)).build() ); - return null; } }); + return new NamespaceFuture(this, descriptor.getName(), response.getProcId()) { + @Override + public String getOperationType() { + return "CREATE_NAMESPACE"; + } + }; } @Override - public void modifyNamespace(final NamespaceDescriptor descriptor) throws IOException { - executeCallable(new MasterCallable(getConnection()) { + public void modifyNamespace(final NamespaceDescriptor descriptor) + throws IOException { + get(modifyNamespaceAsync(descriptor), this.syncWaitTimeout, TimeUnit.MILLISECONDS); + } + + @Override + public Future modifyNamespaceAsync(final NamespaceDescriptor descriptor) + throws IOException { + ModifyNamespaceResponse response = + executeCallable(new MasterCallable(getConnection()) { @Override - public Void call(int callTimeout) throws Exception { - master.modifyNamespace(null, ModifyNamespaceRequest.newBuilder(). + public ModifyNamespaceResponse call(int callTimeout) throws Exception { + return master.modifyNamespace(null, ModifyNamespaceRequest.newBuilder(). setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(descriptor)).build()); - return null; } }); + return new NamespaceFuture(this, descriptor.getName(), response.getProcId()) { + @Override + public String getOperationType() { + return "MODIFY_NAMESPACE"; + } + }; } @Override - public void deleteNamespace(final String name) throws IOException { - executeCallable(new MasterCallable(getConnection()) { + public void deleteNamespace(final String name) + throws IOException { + get(deleteNamespaceAsync(name), this.syncWaitTimeout, TimeUnit.MILLISECONDS); + } + + @Override + public Future deleteNamespaceAsync(final String name) + throws IOException { + DeleteNamespaceResponse response = + executeCallable(new MasterCallable(getConnection()) { @Override - public Void call(int callTimeout) throws Exception { - master.deleteNamespace(null, DeleteNamespaceRequest.newBuilder(). + public DeleteNamespaceResponse call(int callTimeout) throws Exception { + return master.deleteNamespace(null, DeleteNamespaceRequest.newBuilder(). setNamespaceName(name).build()); - return null; } }); + return new NamespaceFuture(this, name, response.getProcId()) { + @Override + public String getOperationType() { + return "DELETE_NAMESPACE"; + } + }; } @Override @@ -3184,6 +3166,11 @@ public class HBaseAdmin implements Admin { this.tableName = tableName; } + @Override + public String toString() { + return getDescription(); + } + /** * @return the table name */ @@ -3222,7 +3209,7 @@ public class HBaseAdmin implements Admin { @Override public void throwTimeoutException(long elapsedTime) throws TimeoutException { 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 { + 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 public List getSecurityCapabilities() throws IOException { try { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/BypassCoprocessorException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/BypassCoprocessorException.java new file mode 100644 index 00000000000..3b01a9e13b9 --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/BypassCoprocessorException.java @@ -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); + } +} \ No newline at end of file diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/Service.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/Service.java new file mode 100644 index 00000000000..97d93cc7586 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/Service.java @@ -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; +} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ServiceNotRunningException.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ServiceNotRunningException.java new file mode 100644 index 00000000000..f6325eaf4ec --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ServiceNotRunningException.java @@ -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); + } +} \ No newline at end of file diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java index 539aea3ab4e..ee0380a328e 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java @@ -41,7 +41,7 @@ public class ResourceChecker { /** * 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) { this.tagLine = tagLine; diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java index 6400887ee07..043d549b8d9 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java @@ -17439,6 +17439,16 @@ public final class MasterProtos { public interface CreateNamespaceResponseOrBuilder extends com.google.protobuf.MessageOrBuilder { + + // optional uint64 proc_id = 1; + /** + * optional uint64 proc_id = 1; + */ + boolean hasProcId(); + /** + * optional uint64 proc_id = 1; + */ + long getProcId(); } /** * Protobuf type {@code hbase.pb.CreateNamespaceResponse} @@ -17473,6 +17483,7 @@ public final class MasterProtos { com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { initFields(); + int mutable_bitField0_ = 0; com.google.protobuf.UnknownFieldSet.Builder unknownFields = com.google.protobuf.UnknownFieldSet.newBuilder(); try { @@ -17490,6 +17501,11 @@ public final class MasterProtos { } break; } + case 8: { + bitField0_ |= 0x00000001; + procId_ = input.readUInt64(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -17529,7 +17545,25 @@ public final class MasterProtos { return PARSER; } + private int bitField0_; + // optional uint64 proc_id = 1; + public static final int PROC_ID_FIELD_NUMBER = 1; + private long procId_; + /** + * optional uint64 proc_id = 1; + */ + public boolean hasProcId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional uint64 proc_id = 1; + */ + public long getProcId() { + return procId_; + } + private void initFields() { + procId_ = 0L; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -17543,6 +17577,9 @@ public final class MasterProtos { public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeUInt64(1, procId_); + } getUnknownFields().writeTo(output); } @@ -17552,6 +17589,10 @@ public final class MasterProtos { if (size != -1) return size; size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(1, procId_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -17575,6 +17616,11 @@ public final class MasterProtos { org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceResponse) obj; boolean result = true; + result = result && (hasProcId() == other.hasProcId()); + if (hasProcId()) { + result = result && (getProcId() + == other.getProcId()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -17588,6 +17634,10 @@ public final class MasterProtos { } int hash = 41; hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasProcId()) { + hash = (37 * hash) + PROC_ID_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getProcId()); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -17697,6 +17747,8 @@ public final class MasterProtos { public Builder clear() { super.clear(); + procId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); return this; } @@ -17723,6 +17775,13 @@ public final class MasterProtos { public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceResponse buildPartial() { org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceResponse(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.procId_ = procId_; + result.bitField0_ = to_bitField0_; onBuilt(); return result; } @@ -17738,6 +17797,9 @@ public final class MasterProtos { public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceResponse other) { if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceResponse.getDefaultInstance()) return this; + if (other.hasProcId()) { + setProcId(other.getProcId()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -17763,6 +17825,40 @@ public final class MasterProtos { } return this; } + private int bitField0_; + + // optional uint64 proc_id = 1; + private long procId_ ; + /** + * optional uint64 proc_id = 1; + */ + public boolean hasProcId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional uint64 proc_id = 1; + */ + public long getProcId() { + return procId_; + } + /** + * optional uint64 proc_id = 1; + */ + public Builder setProcId(long value) { + bitField0_ |= 0x00000001; + procId_ = value; + onChanged(); + return this; + } + /** + * optional uint64 proc_id = 1; + */ + public Builder clearProcId() { + bitField0_ = (bitField0_ & ~0x00000001); + procId_ = 0L; + onChanged(); + return this; + } // @@protoc_insertion_point(builder_scope:hbase.pb.CreateNamespaceResponse) } @@ -18474,6 +18570,16 @@ public final class MasterProtos { public interface DeleteNamespaceResponseOrBuilder extends com.google.protobuf.MessageOrBuilder { + + // optional uint64 proc_id = 1; + /** + * optional uint64 proc_id = 1; + */ + boolean hasProcId(); + /** + * optional uint64 proc_id = 1; + */ + long getProcId(); } /** * Protobuf type {@code hbase.pb.DeleteNamespaceResponse} @@ -18508,6 +18614,7 @@ public final class MasterProtos { com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { initFields(); + int mutable_bitField0_ = 0; com.google.protobuf.UnknownFieldSet.Builder unknownFields = com.google.protobuf.UnknownFieldSet.newBuilder(); try { @@ -18525,6 +18632,11 @@ public final class MasterProtos { } break; } + case 8: { + bitField0_ |= 0x00000001; + procId_ = input.readUInt64(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -18564,7 +18676,25 @@ public final class MasterProtos { return PARSER; } + private int bitField0_; + // optional uint64 proc_id = 1; + public static final int PROC_ID_FIELD_NUMBER = 1; + private long procId_; + /** + * optional uint64 proc_id = 1; + */ + public boolean hasProcId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional uint64 proc_id = 1; + */ + public long getProcId() { + return procId_; + } + private void initFields() { + procId_ = 0L; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -18578,6 +18708,9 @@ public final class MasterProtos { public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeUInt64(1, procId_); + } getUnknownFields().writeTo(output); } @@ -18587,6 +18720,10 @@ public final class MasterProtos { if (size != -1) return size; size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(1, procId_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -18610,6 +18747,11 @@ public final class MasterProtos { org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceResponse) obj; boolean result = true; + result = result && (hasProcId() == other.hasProcId()); + if (hasProcId()) { + result = result && (getProcId() + == other.getProcId()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -18623,6 +18765,10 @@ public final class MasterProtos { } int hash = 41; hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasProcId()) { + hash = (37 * hash) + PROC_ID_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getProcId()); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -18732,6 +18878,8 @@ public final class MasterProtos { public Builder clear() { super.clear(); + procId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); return this; } @@ -18758,6 +18906,13 @@ public final class MasterProtos { public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceResponse buildPartial() { org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceResponse(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.procId_ = procId_; + result.bitField0_ = to_bitField0_; onBuilt(); return result; } @@ -18773,6 +18928,9 @@ public final class MasterProtos { public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceResponse other) { if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceResponse.getDefaultInstance()) return this; + if (other.hasProcId()) { + setProcId(other.getProcId()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -18798,6 +18956,40 @@ public final class MasterProtos { } return this; } + private int bitField0_; + + // optional uint64 proc_id = 1; + private long procId_ ; + /** + * optional uint64 proc_id = 1; + */ + public boolean hasProcId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional uint64 proc_id = 1; + */ + public long getProcId() { + return procId_; + } + /** + * optional uint64 proc_id = 1; + */ + public Builder setProcId(long value) { + bitField0_ |= 0x00000001; + procId_ = value; + onChanged(); + return this; + } + /** + * optional uint64 proc_id = 1; + */ + public Builder clearProcId() { + bitField0_ = (bitField0_ & ~0x00000001); + procId_ = 0L; + onChanged(); + return this; + } // @@protoc_insertion_point(builder_scope:hbase.pb.DeleteNamespaceResponse) } @@ -19553,6 +19745,16 @@ public final class MasterProtos { public interface ModifyNamespaceResponseOrBuilder extends com.google.protobuf.MessageOrBuilder { + + // optional uint64 proc_id = 1; + /** + * optional uint64 proc_id = 1; + */ + boolean hasProcId(); + /** + * optional uint64 proc_id = 1; + */ + long getProcId(); } /** * Protobuf type {@code hbase.pb.ModifyNamespaceResponse} @@ -19587,6 +19789,7 @@ public final class MasterProtos { com.google.protobuf.ExtensionRegistryLite extensionRegistry) throws com.google.protobuf.InvalidProtocolBufferException { initFields(); + int mutable_bitField0_ = 0; com.google.protobuf.UnknownFieldSet.Builder unknownFields = com.google.protobuf.UnknownFieldSet.newBuilder(); try { @@ -19604,6 +19807,11 @@ public final class MasterProtos { } break; } + case 8: { + bitField0_ |= 0x00000001; + procId_ = input.readUInt64(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -19643,7 +19851,25 @@ public final class MasterProtos { return PARSER; } + private int bitField0_; + // optional uint64 proc_id = 1; + public static final int PROC_ID_FIELD_NUMBER = 1; + private long procId_; + /** + * optional uint64 proc_id = 1; + */ + public boolean hasProcId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional uint64 proc_id = 1; + */ + public long getProcId() { + return procId_; + } + private void initFields() { + procId_ = 0L; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -19657,6 +19883,9 @@ public final class MasterProtos { public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeUInt64(1, procId_); + } getUnknownFields().writeTo(output); } @@ -19666,6 +19895,10 @@ public final class MasterProtos { if (size != -1) return size; size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(1, procId_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -19689,6 +19922,11 @@ public final class MasterProtos { org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse) obj; boolean result = true; + result = result && (hasProcId() == other.hasProcId()); + if (hasProcId()) { + result = result && (getProcId() + == other.getProcId()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -19702,6 +19940,10 @@ public final class MasterProtos { } int hash = 41; hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasProcId()) { + hash = (37 * hash) + PROC_ID_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getProcId()); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -19811,6 +20053,8 @@ public final class MasterProtos { public Builder clear() { super.clear(); + procId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); return this; } @@ -19837,6 +20081,13 @@ public final class MasterProtos { public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse buildPartial() { org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.procId_ = procId_; + result.bitField0_ = to_bitField0_; onBuilt(); return result; } @@ -19852,6 +20103,9 @@ public final class MasterProtos { public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse other) { if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceResponse.getDefaultInstance()) return this; + if (other.hasProcId()) { + setProcId(other.getProcId()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -19877,6 +20131,40 @@ public final class MasterProtos { } return this; } + private int bitField0_; + + // optional uint64 proc_id = 1; + private long procId_ ; + /** + * optional uint64 proc_id = 1; + */ + public boolean hasProcId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional uint64 proc_id = 1; + */ + public long getProcId() { + return procId_; + } + /** + * optional uint64 proc_id = 1; + */ + public Builder setProcId(long value) { + bitField0_ |= 0x00000001; + procId_ = value; + onChanged(); + return this; + } + /** + * optional uint64 proc_id = 1; + */ + public Builder clearProcId() { + bitField0_ = (bitField0_ & ~0x00000001); + procId_ = 0L; + onChanged(); + return this; + } // @@protoc_insertion_point(builder_scope:hbase.pb.ModifyNamespaceResponse) } @@ -61891,255 +62179,257 @@ public final class MasterProtos { "\001(\004\"~\n\026CreateNamespaceRequest\022:\n\023namespa" + "ceDescriptor\030\001 \002(\0132\035.hbase.pb.NamespaceD" + "escriptor\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020\n\005non", - "ce\030\003 \001(\004:\0010\"\031\n\027CreateNamespaceResponse\"Y" + - "\n\026DeleteNamespaceRequest\022\025\n\rnamespaceNam" + - "e\030\001 \002(\t\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020\n\005nonce" + - "\030\003 \001(\004:\0010\"\031\n\027DeleteNamespaceResponse\"~\n\026" + - "ModifyNamespaceRequest\022:\n\023namespaceDescr" + - "iptor\030\001 \002(\0132\035.hbase.pb.NamespaceDescript" + - "or\022\026\n\013nonce_group\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(" + - "\004:\0010\"\031\n\027ModifyNamespaceResponse\"6\n\035GetNa" + - "mespaceDescriptorRequest\022\025\n\rnamespaceNam" + - "e\030\001 \002(\t\"\\\n\036GetNamespaceDescriptorRespons", - "e\022:\n\023namespaceDescriptor\030\001 \002(\0132\035.hbase.p" + - "b.NamespaceDescriptor\"!\n\037ListNamespaceDe" + - "scriptorsRequest\"^\n ListNamespaceDescrip" + - "torsResponse\022:\n\023namespaceDescriptor\030\001 \003(" + - "\0132\035.hbase.pb.NamespaceDescriptor\"?\n&List" + - "TableDescriptorsByNamespaceRequest\022\025\n\rna" + - "mespaceName\030\001 \002(\t\"U\n\'ListTableDescriptor" + - "sByNamespaceResponse\022*\n\013tableSchema\030\001 \003(" + - "\0132\025.hbase.pb.TableSchema\"9\n ListTableNam" + - "esByNamespaceRequest\022\025\n\rnamespaceName\030\001 ", - "\002(\t\"K\n!ListTableNamesByNamespaceResponse" + - "\022&\n\ttableName\030\001 \003(\0132\023.hbase.pb.TableName" + - "\"\021\n\017ShutdownRequest\"\022\n\020ShutdownResponse\"" + - "\023\n\021StopMasterRequest\"\024\n\022StopMasterRespon" + - "se\"\037\n\016BalanceRequest\022\r\n\005force\030\001 \001(\010\"\'\n\017B" + - "alanceResponse\022\024\n\014balancer_ran\030\001 \002(\010\"<\n\031" + - "SetBalancerRunningRequest\022\n\n\002on\030\001 \002(\010\022\023\n" + - "\013synchronous\030\002 \001(\010\"8\n\032SetBalancerRunning" + - "Response\022\032\n\022prev_balance_value\030\001 \001(\010\"\032\n\030" + - "IsBalancerEnabledRequest\",\n\031IsBalancerEn", - "abledResponse\022\017\n\007enabled\030\001 \002(\010\"\022\n\020Normal" + - "izeRequest\"+\n\021NormalizeResponse\022\026\n\016norma" + - "lizer_ran\030\001 \002(\010\")\n\033SetNormalizerRunningR" + - "equest\022\n\n\002on\030\001 \002(\010\"=\n\034SetNormalizerRunni" + - "ngResponse\022\035\n\025prev_normalizer_value\030\001 \001(" + - "\010\"\034\n\032IsNormalizerEnabledRequest\".\n\033IsNor" + - "malizerEnabledResponse\022\017\n\007enabled\030\001 \002(\010\"" + - "\027\n\025RunCatalogScanRequest\"-\n\026RunCatalogSc" + - "anResponse\022\023\n\013scan_result\030\001 \001(\005\"-\n\033Enabl" + - "eCatalogJanitorRequest\022\016\n\006enable\030\001 \002(\010\"2", - "\n\034EnableCatalogJanitorResponse\022\022\n\nprev_v" + - "alue\030\001 \001(\010\" \n\036IsCatalogJanitorEnabledReq" + - "uest\"0\n\037IsCatalogJanitorEnabledResponse\022" + - "\r\n\005value\030\001 \002(\010\"B\n\017SnapshotRequest\022/\n\010sna" + - "pshot\030\001 \002(\0132\035.hbase.pb.SnapshotDescripti" + - "on\",\n\020SnapshotResponse\022\030\n\020expected_timeo" + - "ut\030\001 \002(\003\"\036\n\034GetCompletedSnapshotsRequest" + - "\"Q\n\035GetCompletedSnapshotsResponse\0220\n\tsna" + - "pshots\030\001 \003(\0132\035.hbase.pb.SnapshotDescript" + - "ion\"H\n\025DeleteSnapshotRequest\022/\n\010snapshot", - "\030\001 \002(\0132\035.hbase.pb.SnapshotDescription\"\030\n" + - "\026DeleteSnapshotResponse\"I\n\026RestoreSnapsh" + - "otRequest\022/\n\010snapshot\030\001 \002(\0132\035.hbase.pb.S" + - "napshotDescription\"\031\n\027RestoreSnapshotRes" + - "ponse\"H\n\025IsSnapshotDoneRequest\022/\n\010snapsh" + - "ot\030\001 \001(\0132\035.hbase.pb.SnapshotDescription\"" + - "^\n\026IsSnapshotDoneResponse\022\023\n\004done\030\001 \001(\010:" + - "\005false\022/\n\010snapshot\030\002 \001(\0132\035.hbase.pb.Snap" + - "shotDescription\"O\n\034IsRestoreSnapshotDone" + - "Request\022/\n\010snapshot\030\001 \001(\0132\035.hbase.pb.Sna", - "pshotDescription\"4\n\035IsRestoreSnapshotDon" + - "eResponse\022\023\n\004done\030\001 \001(\010:\005false\"F\n\033GetSch" + - "emaAlterStatusRequest\022\'\n\ntable_name\030\001 \002(" + - "\0132\023.hbase.pb.TableName\"T\n\034GetSchemaAlter" + - "StatusResponse\022\035\n\025yet_to_update_regions\030" + - "\001 \001(\r\022\025\n\rtotal_regions\030\002 \001(\r\"\213\001\n\032GetTabl" + - "eDescriptorsRequest\022(\n\013table_names\030\001 \003(\013" + - "2\023.hbase.pb.TableName\022\r\n\005regex\030\002 \001(\t\022!\n\022" + - "include_sys_tables\030\003 \001(\010:\005false\022\021\n\tnames" + - "pace\030\004 \001(\t\"J\n\033GetTableDescriptorsRespons", - "e\022+\n\014table_schema\030\001 \003(\0132\025.hbase.pb.Table" + - "Schema\"[\n\024GetTableNamesRequest\022\r\n\005regex\030" + - "\001 \001(\t\022!\n\022include_sys_tables\030\002 \001(\010:\005false" + - "\022\021\n\tnamespace\030\003 \001(\t\"A\n\025GetTableNamesResp" + - "onse\022(\n\013table_names\030\001 \003(\0132\023.hbase.pb.Tab" + - "leName\"?\n\024GetTableStateRequest\022\'\n\ntable_" + - "name\030\001 \002(\0132\023.hbase.pb.TableName\"B\n\025GetTa" + - "bleStateResponse\022)\n\013table_state\030\001 \002(\0132\024." + - "hbase.pb.TableState\"\031\n\027GetClusterStatusR" + - "equest\"K\n\030GetClusterStatusResponse\022/\n\016cl", - "uster_status\030\001 \002(\0132\027.hbase.pb.ClusterSta" + - "tus\"\030\n\026IsMasterRunningRequest\"4\n\027IsMaste" + - "rRunningResponse\022\031\n\021is_master_running\030\001 " + - "\002(\010\"I\n\024ExecProcedureRequest\0221\n\tprocedure" + - "\030\001 \002(\0132\036.hbase.pb.ProcedureDescription\"F" + - "\n\025ExecProcedureResponse\022\030\n\020expected_time" + - "out\030\001 \001(\003\022\023\n\013return_data\030\002 \001(\014\"K\n\026IsProc" + - "edureDoneRequest\0221\n\tprocedure\030\001 \001(\0132\036.hb" + - "ase.pb.ProcedureDescription\"`\n\027IsProcedu" + - "reDoneResponse\022\023\n\004done\030\001 \001(\010:\005false\0220\n\010s", - "napshot\030\002 \001(\0132\036.hbase.pb.ProcedureDescri" + - "ption\",\n\031GetProcedureResultRequest\022\017\n\007pr" + - "oc_id\030\001 \002(\004\"\371\001\n\032GetProcedureResultRespon" + - "se\0229\n\005state\030\001 \002(\0162*.hbase.pb.GetProcedur" + - "eResultResponse.State\022\022\n\nstart_time\030\002 \001(" + - "\004\022\023\n\013last_update\030\003 \001(\004\022\016\n\006result\030\004 \001(\014\0224" + - "\n\texception\030\005 \001(\0132!.hbase.pb.ForeignExce" + - "ptionMessage\"1\n\005State\022\r\n\tNOT_FOUND\020\000\022\013\n\007" + - "RUNNING\020\001\022\014\n\010FINISHED\020\002\"M\n\025AbortProcedur" + - "eRequest\022\017\n\007proc_id\030\001 \002(\004\022#\n\025mayInterrup", - "tIfRunning\030\002 \001(\010:\004true\"6\n\026AbortProcedure" + - "Response\022\034\n\024is_procedure_aborted\030\001 \002(\010\"\027" + - "\n\025ListProceduresRequest\"@\n\026ListProcedure" + - "sResponse\022&\n\tprocedure\030\001 \003(\0132\023.hbase.pb." + - "Procedure\"\315\001\n\017SetQuotaRequest\022\021\n\tuser_na" + - "me\030\001 \001(\t\022\022\n\nuser_group\030\002 \001(\t\022\021\n\tnamespac" + - "e\030\003 \001(\t\022\'\n\ntable_name\030\004 \001(\0132\023.hbase.pb.T" + - "ableName\022\022\n\nremove_all\030\005 \001(\010\022\026\n\016bypass_g" + - "lobals\030\006 \001(\010\022+\n\010throttle\030\007 \001(\0132\031.hbase.p" + - "b.ThrottleRequest\"\022\n\020SetQuotaResponse\"J\n", - "\037MajorCompactionTimestampRequest\022\'\n\ntabl" + - "e_name\030\001 \002(\0132\023.hbase.pb.TableName\"U\n(Maj" + - "orCompactionTimestampForRegionRequest\022)\n" + - "\006region\030\001 \002(\0132\031.hbase.pb.RegionSpecifier" + - "\"@\n MajorCompactionTimestampResponse\022\034\n\024" + - "compaction_timestamp\030\001 \002(\003\"\035\n\033SecurityCa" + - "pabilitiesRequest\"\354\001\n\034SecurityCapabiliti" + - "esResponse\022G\n\014capabilities\030\001 \003(\01621.hbase" + - ".pb.SecurityCapabilitiesResponse.Capabil" + - "ity\"\202\001\n\nCapability\022\031\n\025SIMPLE_AUTHENTICAT", - "ION\020\000\022\031\n\025SECURE_AUTHENTICATION\020\001\022\021\n\rAUTH" + - "ORIZATION\020\002\022\026\n\022CELL_AUTHORIZATION\020\003\022\023\n\017C" + - "ELL_VISIBILITY\020\0042\374&\n\rMasterService\022e\n\024Ge" + - "tSchemaAlterStatus\022%.hbase.pb.GetSchemaA" + - "lterStatusRequest\032&.hbase.pb.GetSchemaAl" + - "terStatusResponse\022b\n\023GetTableDescriptors" + - "\022$.hbase.pb.GetTableDescriptorsRequest\032%" + - ".hbase.pb.GetTableDescriptorsResponse\022P\n" + - "\rGetTableNames\022\036.hbase.pb.GetTableNamesR" + - "equest\032\037.hbase.pb.GetTableNamesResponse\022", - "Y\n\020GetClusterStatus\022!.hbase.pb.GetCluste" + - "rStatusRequest\032\".hbase.pb.GetClusterStat" + - "usResponse\022V\n\017IsMasterRunning\022 .hbase.pb" + - ".IsMasterRunningRequest\032!.hbase.pb.IsMas" + - "terRunningResponse\022D\n\tAddColumn\022\032.hbase." + - "pb.AddColumnRequest\032\033.hbase.pb.AddColumn" + - "Response\022M\n\014DeleteColumn\022\035.hbase.pb.Dele" + - "teColumnRequest\032\036.hbase.pb.DeleteColumnR" + - "esponse\022M\n\014ModifyColumn\022\035.hbase.pb.Modif" + - "yColumnRequest\032\036.hbase.pb.ModifyColumnRe", - "sponse\022G\n\nMoveRegion\022\033.hbase.pb.MoveRegi" + - "onRequest\032\034.hbase.pb.MoveRegionResponse\022" + - "k\n\026DispatchMergingRegions\022\'.hbase.pb.Dis" + - "patchMergingRegionsRequest\032(.hbase.pb.Di" + - "spatchMergingRegionsResponse\022M\n\014AssignRe" + - "gion\022\035.hbase.pb.AssignRegionRequest\032\036.hb" + - "ase.pb.AssignRegionResponse\022S\n\016UnassignR" + - "egion\022\037.hbase.pb.UnassignRegionRequest\032 " + - ".hbase.pb.UnassignRegionResponse\022P\n\rOffl" + - "ineRegion\022\036.hbase.pb.OfflineRegionReques", - "t\032\037.hbase.pb.OfflineRegionResponse\022J\n\013De" + - "leteTable\022\034.hbase.pb.DeleteTableRequest\032" + - "\035.hbase.pb.DeleteTableResponse\022P\n\rtrunca" + - "teTable\022\036.hbase.pb.TruncateTableRequest\032" + - "\037.hbase.pb.TruncateTableResponse\022J\n\013Enab" + - "leTable\022\034.hbase.pb.EnableTableRequest\032\035." + - "hbase.pb.EnableTableResponse\022M\n\014DisableT" + - "able\022\035.hbase.pb.DisableTableRequest\032\036.hb" + - "ase.pb.DisableTableResponse\022J\n\013ModifyTab" + - "le\022\034.hbase.pb.ModifyTableRequest\032\035.hbase", - ".pb.ModifyTableResponse\022J\n\013CreateTable\022\034" + - ".hbase.pb.CreateTableRequest\032\035.hbase.pb." + - "CreateTableResponse\022A\n\010Shutdown\022\031.hbase." + - "pb.ShutdownRequest\032\032.hbase.pb.ShutdownRe" + - "sponse\022G\n\nStopMaster\022\033.hbase.pb.StopMast" + - "erRequest\032\034.hbase.pb.StopMasterResponse\022" + - ">\n\007Balance\022\030.hbase.pb.BalanceRequest\032\031.h" + - "base.pb.BalanceResponse\022_\n\022SetBalancerRu" + - "nning\022#.hbase.pb.SetBalancerRunningReque" + - "st\032$.hbase.pb.SetBalancerRunningResponse", - "\022\\\n\021IsBalancerEnabled\022\".hbase.pb.IsBalan" + - "cerEnabledRequest\032#.hbase.pb.IsBalancerE" + - "nabledResponse\022D\n\tNormalize\022\032.hbase.pb.N" + - "ormalizeRequest\032\033.hbase.pb.NormalizeResp" + - "onse\022e\n\024SetNormalizerRunning\022%.hbase.pb." + - "SetNormalizerRunningRequest\032&.hbase.pb.S" + - "etNormalizerRunningResponse\022b\n\023IsNormali" + - "zerEnabled\022$.hbase.pb.IsNormalizerEnable" + - "dRequest\032%.hbase.pb.IsNormalizerEnabledR" + - "esponse\022S\n\016RunCatalogScan\022\037.hbase.pb.Run", - "CatalogScanRequest\032 .hbase.pb.RunCatalog" + - "ScanResponse\022e\n\024EnableCatalogJanitor\022%.h" + - "base.pb.EnableCatalogJanitorRequest\032&.hb" + - "ase.pb.EnableCatalogJanitorResponse\022n\n\027I" + - "sCatalogJanitorEnabled\022(.hbase.pb.IsCata" + - "logJanitorEnabledRequest\032).hbase.pb.IsCa" + - "talogJanitorEnabledResponse\022^\n\021ExecMaste" + - "rService\022#.hbase.pb.CoprocessorServiceRe" + - "quest\032$.hbase.pb.CoprocessorServiceRespo" + - "nse\022A\n\010Snapshot\022\031.hbase.pb.SnapshotReque", - "st\032\032.hbase.pb.SnapshotResponse\022h\n\025GetCom" + - "pletedSnapshots\022&.hbase.pb.GetCompletedS" + - "napshotsRequest\032\'.hbase.pb.GetCompletedS" + - "napshotsResponse\022S\n\016DeleteSnapshot\022\037.hba" + - "se.pb.DeleteSnapshotRequest\032 .hbase.pb.D" + - "eleteSnapshotResponse\022S\n\016IsSnapshotDone\022" + - "\037.hbase.pb.IsSnapshotDoneRequest\032 .hbase" + - ".pb.IsSnapshotDoneResponse\022V\n\017RestoreSna" + - "pshot\022 .hbase.pb.RestoreSnapshotRequest\032" + - "!.hbase.pb.RestoreSnapshotResponse\022h\n\025Is", - "RestoreSnapshotDone\022&.hbase.pb.IsRestore" + - "SnapshotDoneRequest\032\'.hbase.pb.IsRestore" + - "SnapshotDoneResponse\022P\n\rExecProcedure\022\036." + - "hbase.pb.ExecProcedureRequest\032\037.hbase.pb" + - ".ExecProcedureResponse\022W\n\024ExecProcedureW" + - "ithRet\022\036.hbase.pb.ExecProcedureRequest\032\037" + - ".hbase.pb.ExecProcedureResponse\022V\n\017IsPro" + - "cedureDone\022 .hbase.pb.IsProcedureDoneReq" + - "uest\032!.hbase.pb.IsProcedureDoneResponse\022" + - "V\n\017ModifyNamespace\022 .hbase.pb.ModifyName", - "spaceRequest\032!.hbase.pb.ModifyNamespaceR" + - "esponse\022V\n\017CreateNamespace\022 .hbase.pb.Cr" + - "eateNamespaceRequest\032!.hbase.pb.CreateNa" + - "mespaceResponse\022V\n\017DeleteNamespace\022 .hba" + - "se.pb.DeleteNamespaceRequest\032!.hbase.pb." + - "DeleteNamespaceResponse\022k\n\026GetNamespaceD" + - "escriptor\022\'.hbase.pb.GetNamespaceDescrip" + - "torRequest\032(.hbase.pb.GetNamespaceDescri" + - "ptorResponse\022q\n\030ListNamespaceDescriptors" + - "\022).hbase.pb.ListNamespaceDescriptorsRequ", - "est\032*.hbase.pb.ListNamespaceDescriptorsR" + - "esponse\022\206\001\n\037ListTableDescriptorsByNamesp" + - "ace\0220.hbase.pb.ListTableDescriptorsByNam" + - "espaceRequest\0321.hbase.pb.ListTableDescri" + - "ptorsByNamespaceResponse\022t\n\031ListTableNam" + - "esByNamespace\022*.hbase.pb.ListTableNamesB" + - "yNamespaceRequest\032+.hbase.pb.ListTableNa" + - "mesByNamespaceResponse\022P\n\rGetTableState\022" + - "\036.hbase.pb.GetTableStateRequest\032\037.hbase." + - "pb.GetTableStateResponse\022A\n\010SetQuota\022\031.h", - "base.pb.SetQuotaRequest\032\032.hbase.pb.SetQu" + - "otaResponse\022x\n\037getLastMajorCompactionTim" + - "estamp\022).hbase.pb.MajorCompactionTimesta" + - "mpRequest\032*.hbase.pb.MajorCompactionTime" + - "stampResponse\022\212\001\n(getLastMajorCompaction" + - "TimestampForRegion\0222.hbase.pb.MajorCompa" + - "ctionTimestampForRegionRequest\032*.hbase.p" + - "b.MajorCompactionTimestampResponse\022_\n\022ge" + - "tProcedureResult\022#.hbase.pb.GetProcedure" + - "ResultRequest\032$.hbase.pb.GetProcedureRes", - "ultResponse\022h\n\027getSecurityCapabilities\022%" + - ".hbase.pb.SecurityCapabilitiesRequest\032&." + - "hbase.pb.SecurityCapabilitiesResponse\022S\n" + - "\016AbortProcedure\022\037.hbase.pb.AbortProcedur" + - "eRequest\032 .hbase.pb.AbortProcedureRespon" + - "se\022S\n\016ListProcedures\022\037.hbase.pb.ListProc" + - "eduresRequest\032 .hbase.pb.ListProceduresR" + - "esponseBB\n*org.apache.hadoop.hbase.proto" + - "buf.generatedB\014MasterProtosH\001\210\001\001\240\001\001" + "ce\030\003 \001(\004:\0010\"*\n\027CreateNamespaceResponse\022\017" + + "\n\007proc_id\030\001 \001(\004\"Y\n\026DeleteNamespaceReques" + + "t\022\025\n\rnamespaceName\030\001 \002(\t\022\026\n\013nonce_group\030" + + "\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004:\0010\"*\n\027DeleteName" + + "spaceResponse\022\017\n\007proc_id\030\001 \001(\004\"~\n\026Modify" + + "NamespaceRequest\022:\n\023namespaceDescriptor\030" + + "\001 \002(\0132\035.hbase.pb.NamespaceDescriptor\022\026\n\013" + + "nonce_group\030\002 \001(\004:\0010\022\020\n\005nonce\030\003 \001(\004:\0010\"*" + + "\n\027ModifyNamespaceResponse\022\017\n\007proc_id\030\001 \001" + + "(\004\"6\n\035GetNamespaceDescriptorRequest\022\025\n\rn", + "amespaceName\030\001 \002(\t\"\\\n\036GetNamespaceDescri" + + "ptorResponse\022:\n\023namespaceDescriptor\030\001 \002(" + + "\0132\035.hbase.pb.NamespaceDescriptor\"!\n\037List" + + "NamespaceDescriptorsRequest\"^\n ListNames" + + "paceDescriptorsResponse\022:\n\023namespaceDesc" + + "riptor\030\001 \003(\0132\035.hbase.pb.NamespaceDescrip" + + "tor\"?\n&ListTableDescriptorsByNamespaceRe" + + "quest\022\025\n\rnamespaceName\030\001 \002(\t\"U\n\'ListTabl" + + "eDescriptorsByNamespaceResponse\022*\n\013table" + + "Schema\030\001 \003(\0132\025.hbase.pb.TableSchema\"9\n L", + "istTableNamesByNamespaceRequest\022\025\n\rnames" + + "paceName\030\001 \002(\t\"K\n!ListTableNamesByNamesp" + + "aceResponse\022&\n\ttableName\030\001 \003(\0132\023.hbase.p" + + "b.TableName\"\021\n\017ShutdownRequest\"\022\n\020Shutdo" + + "wnResponse\"\023\n\021StopMasterRequest\"\024\n\022StopM" + + "asterResponse\"\037\n\016BalanceRequest\022\r\n\005force" + + "\030\001 \001(\010\"\'\n\017BalanceResponse\022\024\n\014balancer_ra" + + "n\030\001 \002(\010\"<\n\031SetBalancerRunningRequest\022\n\n\002" + + "on\030\001 \002(\010\022\023\n\013synchronous\030\002 \001(\010\"8\n\032SetBala" + + "ncerRunningResponse\022\032\n\022prev_balance_valu", + "e\030\001 \001(\010\"\032\n\030IsBalancerEnabledRequest\",\n\031I" + + "sBalancerEnabledResponse\022\017\n\007enabled\030\001 \002(" + + "\010\"\022\n\020NormalizeRequest\"+\n\021NormalizeRespon" + + "se\022\026\n\016normalizer_ran\030\001 \002(\010\")\n\033SetNormali" + + "zerRunningRequest\022\n\n\002on\030\001 \002(\010\"=\n\034SetNorm" + + "alizerRunningResponse\022\035\n\025prev_normalizer" + + "_value\030\001 \001(\010\"\034\n\032IsNormalizerEnabledReque" + + "st\".\n\033IsNormalizerEnabledResponse\022\017\n\007ena" + + "bled\030\001 \002(\010\"\027\n\025RunCatalogScanRequest\"-\n\026R" + + "unCatalogScanResponse\022\023\n\013scan_result\030\001 \001", + "(\005\"-\n\033EnableCatalogJanitorRequest\022\016\n\006ena" + + "ble\030\001 \002(\010\"2\n\034EnableCatalogJanitorRespons" + + "e\022\022\n\nprev_value\030\001 \001(\010\" \n\036IsCatalogJanito" + + "rEnabledRequest\"0\n\037IsCatalogJanitorEnabl" + + "edResponse\022\r\n\005value\030\001 \002(\010\"B\n\017SnapshotReq" + + "uest\022/\n\010snapshot\030\001 \002(\0132\035.hbase.pb.Snapsh" + + "otDescription\",\n\020SnapshotResponse\022\030\n\020exp" + + "ected_timeout\030\001 \002(\003\"\036\n\034GetCompletedSnaps" + + "hotsRequest\"Q\n\035GetCompletedSnapshotsResp" + + "onse\0220\n\tsnapshots\030\001 \003(\0132\035.hbase.pb.Snaps", + "hotDescription\"H\n\025DeleteSnapshotRequest\022" + + "/\n\010snapshot\030\001 \002(\0132\035.hbase.pb.SnapshotDes" + + "cription\"\030\n\026DeleteSnapshotResponse\"I\n\026Re" + + "storeSnapshotRequest\022/\n\010snapshot\030\001 \002(\0132\035" + + ".hbase.pb.SnapshotDescription\"\031\n\027Restore" + + "SnapshotResponse\"H\n\025IsSnapshotDoneReques" + + "t\022/\n\010snapshot\030\001 \001(\0132\035.hbase.pb.SnapshotD" + + "escription\"^\n\026IsSnapshotDoneResponse\022\023\n\004" + + "done\030\001 \001(\010:\005false\022/\n\010snapshot\030\002 \001(\0132\035.hb" + + "ase.pb.SnapshotDescription\"O\n\034IsRestoreS", + "napshotDoneRequest\022/\n\010snapshot\030\001 \001(\0132\035.h" + + "base.pb.SnapshotDescription\"4\n\035IsRestore" + + "SnapshotDoneResponse\022\023\n\004done\030\001 \001(\010:\005fals" + + "e\"F\n\033GetSchemaAlterStatusRequest\022\'\n\ntabl" + + "e_name\030\001 \002(\0132\023.hbase.pb.TableName\"T\n\034Get" + + "SchemaAlterStatusResponse\022\035\n\025yet_to_upda" + + "te_regions\030\001 \001(\r\022\025\n\rtotal_regions\030\002 \001(\r\"" + + "\213\001\n\032GetTableDescriptorsRequest\022(\n\013table_" + + "names\030\001 \003(\0132\023.hbase.pb.TableName\022\r\n\005rege" + + "x\030\002 \001(\t\022!\n\022include_sys_tables\030\003 \001(\010:\005fal", + "se\022\021\n\tnamespace\030\004 \001(\t\"J\n\033GetTableDescrip" + + "torsResponse\022+\n\014table_schema\030\001 \003(\0132\025.hba" + + "se.pb.TableSchema\"[\n\024GetTableNamesReques" + + "t\022\r\n\005regex\030\001 \001(\t\022!\n\022include_sys_tables\030\002" + + " \001(\010:\005false\022\021\n\tnamespace\030\003 \001(\t\"A\n\025GetTab" + + "leNamesResponse\022(\n\013table_names\030\001 \003(\0132\023.h" + + "base.pb.TableName\"?\n\024GetTableStateReques" + + "t\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb.TableNa" + + "me\"B\n\025GetTableStateResponse\022)\n\013table_sta" + + "te\030\001 \002(\0132\024.hbase.pb.TableState\"\031\n\027GetClu", + "sterStatusRequest\"K\n\030GetClusterStatusRes" + + "ponse\022/\n\016cluster_status\030\001 \002(\0132\027.hbase.pb" + + ".ClusterStatus\"\030\n\026IsMasterRunningRequest" + + "\"4\n\027IsMasterRunningResponse\022\031\n\021is_master" + + "_running\030\001 \002(\010\"I\n\024ExecProcedureRequest\0221" + + "\n\tprocedure\030\001 \002(\0132\036.hbase.pb.ProcedureDe" + + "scription\"F\n\025ExecProcedureResponse\022\030\n\020ex" + + "pected_timeout\030\001 \001(\003\022\023\n\013return_data\030\002 \001(" + + "\014\"K\n\026IsProcedureDoneRequest\0221\n\tprocedure" + + "\030\001 \001(\0132\036.hbase.pb.ProcedureDescription\"`", + "\n\027IsProcedureDoneResponse\022\023\n\004done\030\001 \001(\010:" + + "\005false\0220\n\010snapshot\030\002 \001(\0132\036.hbase.pb.Proc" + + "edureDescription\",\n\031GetProcedureResultRe" + + "quest\022\017\n\007proc_id\030\001 \002(\004\"\371\001\n\032GetProcedureR" + + "esultResponse\0229\n\005state\030\001 \002(\0162*.hbase.pb." + + "GetProcedureResultResponse.State\022\022\n\nstar" + + "t_time\030\002 \001(\004\022\023\n\013last_update\030\003 \001(\004\022\016\n\006res" + + "ult\030\004 \001(\014\0224\n\texception\030\005 \001(\0132!.hbase.pb." + + "ForeignExceptionMessage\"1\n\005State\022\r\n\tNOT_" + + "FOUND\020\000\022\013\n\007RUNNING\020\001\022\014\n\010FINISHED\020\002\"M\n\025Ab", + "ortProcedureRequest\022\017\n\007proc_id\030\001 \002(\004\022#\n\025" + + "mayInterruptIfRunning\030\002 \001(\010:\004true\"6\n\026Abo" + + "rtProcedureResponse\022\034\n\024is_procedure_abor" + + "ted\030\001 \002(\010\"\027\n\025ListProceduresRequest\"@\n\026Li" + + "stProceduresResponse\022&\n\tprocedure\030\001 \003(\0132" + + "\023.hbase.pb.Procedure\"\315\001\n\017SetQuotaRequest" + + "\022\021\n\tuser_name\030\001 \001(\t\022\022\n\nuser_group\030\002 \001(\t\022" + + "\021\n\tnamespace\030\003 \001(\t\022\'\n\ntable_name\030\004 \001(\0132\023" + + ".hbase.pb.TableName\022\022\n\nremove_all\030\005 \001(\010\022" + + "\026\n\016bypass_globals\030\006 \001(\010\022+\n\010throttle\030\007 \001(", + "\0132\031.hbase.pb.ThrottleRequest\"\022\n\020SetQuota" + + "Response\"J\n\037MajorCompactionTimestampRequ" + + "est\022\'\n\ntable_name\030\001 \002(\0132\023.hbase.pb.Table" + + "Name\"U\n(MajorCompactionTimestampForRegio" + + "nRequest\022)\n\006region\030\001 \002(\0132\031.hbase.pb.Regi" + + "onSpecifier\"@\n MajorCompactionTimestampR" + + "esponse\022\034\n\024compaction_timestamp\030\001 \002(\003\"\035\n" + + "\033SecurityCapabilitiesRequest\"\354\001\n\034Securit" + + "yCapabilitiesResponse\022G\n\014capabilities\030\001 " + + "\003(\01621.hbase.pb.SecurityCapabilitiesRespo", + "nse.Capability\"\202\001\n\nCapability\022\031\n\025SIMPLE_" + + "AUTHENTICATION\020\000\022\031\n\025SECURE_AUTHENTICATIO" + + "N\020\001\022\021\n\rAUTHORIZATION\020\002\022\026\n\022CELL_AUTHORIZA" + + "TION\020\003\022\023\n\017CELL_VISIBILITY\020\0042\374&\n\rMasterSe" + + "rvice\022e\n\024GetSchemaAlterStatus\022%.hbase.pb" + + ".GetSchemaAlterStatusRequest\032&.hbase.pb." + + "GetSchemaAlterStatusResponse\022b\n\023GetTable" + + "Descriptors\022$.hbase.pb.GetTableDescripto" + + "rsRequest\032%.hbase.pb.GetTableDescriptors" + + "Response\022P\n\rGetTableNames\022\036.hbase.pb.Get", + "TableNamesRequest\032\037.hbase.pb.GetTableNam" + + "esResponse\022Y\n\020GetClusterStatus\022!.hbase.p" + + "b.GetClusterStatusRequest\032\".hbase.pb.Get" + + "ClusterStatusResponse\022V\n\017IsMasterRunning" + + "\022 .hbase.pb.IsMasterRunningRequest\032!.hba" + + "se.pb.IsMasterRunningResponse\022D\n\tAddColu" + + "mn\022\032.hbase.pb.AddColumnRequest\032\033.hbase.p" + + "b.AddColumnResponse\022M\n\014DeleteColumn\022\035.hb" + + "ase.pb.DeleteColumnRequest\032\036.hbase.pb.De" + + "leteColumnResponse\022M\n\014ModifyColumn\022\035.hba", + "se.pb.ModifyColumnRequest\032\036.hbase.pb.Mod" + + "ifyColumnResponse\022G\n\nMoveRegion\022\033.hbase." + + "pb.MoveRegionRequest\032\034.hbase.pb.MoveRegi" + + "onResponse\022k\n\026DispatchMergingRegions\022\'.h" + + "base.pb.DispatchMergingRegionsRequest\032(." + + "hbase.pb.DispatchMergingRegionsResponse\022" + + "M\n\014AssignRegion\022\035.hbase.pb.AssignRegionR" + + "equest\032\036.hbase.pb.AssignRegionResponse\022S" + + "\n\016UnassignRegion\022\037.hbase.pb.UnassignRegi" + + "onRequest\032 .hbase.pb.UnassignRegionRespo", + "nse\022P\n\rOfflineRegion\022\036.hbase.pb.OfflineR" + + "egionRequest\032\037.hbase.pb.OfflineRegionRes" + + "ponse\022J\n\013DeleteTable\022\034.hbase.pb.DeleteTa" + + "bleRequest\032\035.hbase.pb.DeleteTableRespons" + + "e\022P\n\rtruncateTable\022\036.hbase.pb.TruncateTa" + + "bleRequest\032\037.hbase.pb.TruncateTableRespo" + + "nse\022J\n\013EnableTable\022\034.hbase.pb.EnableTabl" + + "eRequest\032\035.hbase.pb.EnableTableResponse\022" + + "M\n\014DisableTable\022\035.hbase.pb.DisableTableR" + + "equest\032\036.hbase.pb.DisableTableResponse\022J", + "\n\013ModifyTable\022\034.hbase.pb.ModifyTableRequ" + + "est\032\035.hbase.pb.ModifyTableResponse\022J\n\013Cr" + + "eateTable\022\034.hbase.pb.CreateTableRequest\032" + + "\035.hbase.pb.CreateTableResponse\022A\n\010Shutdo" + + "wn\022\031.hbase.pb.ShutdownRequest\032\032.hbase.pb" + + ".ShutdownResponse\022G\n\nStopMaster\022\033.hbase." + + "pb.StopMasterRequest\032\034.hbase.pb.StopMast" + + "erResponse\022>\n\007Balance\022\030.hbase.pb.Balance" + + "Request\032\031.hbase.pb.BalanceResponse\022_\n\022Se" + + "tBalancerRunning\022#.hbase.pb.SetBalancerR", + "unningRequest\032$.hbase.pb.SetBalancerRunn" + + "ingResponse\022\\\n\021IsBalancerEnabled\022\".hbase" + + ".pb.IsBalancerEnabledRequest\032#.hbase.pb." + + "IsBalancerEnabledResponse\022D\n\tNormalize\022\032" + + ".hbase.pb.NormalizeRequest\032\033.hbase.pb.No" + + "rmalizeResponse\022e\n\024SetNormalizerRunning\022" + + "%.hbase.pb.SetNormalizerRunningRequest\032&" + + ".hbase.pb.SetNormalizerRunningResponse\022b" + + "\n\023IsNormalizerEnabled\022$.hbase.pb.IsNorma" + + "lizerEnabledRequest\032%.hbase.pb.IsNormali", + "zerEnabledResponse\022S\n\016RunCatalogScan\022\037.h" + + "base.pb.RunCatalogScanRequest\032 .hbase.pb" + + ".RunCatalogScanResponse\022e\n\024EnableCatalog" + + "Janitor\022%.hbase.pb.EnableCatalogJanitorR" + + "equest\032&.hbase.pb.EnableCatalogJanitorRe" + + "sponse\022n\n\027IsCatalogJanitorEnabled\022(.hbas" + + "e.pb.IsCatalogJanitorEnabledRequest\032).hb" + + "ase.pb.IsCatalogJanitorEnabledResponse\022^" + + "\n\021ExecMasterService\022#.hbase.pb.Coprocess" + + "orServiceRequest\032$.hbase.pb.CoprocessorS", + "erviceResponse\022A\n\010Snapshot\022\031.hbase.pb.Sn" + + "apshotRequest\032\032.hbase.pb.SnapshotRespons" + + "e\022h\n\025GetCompletedSnapshots\022&.hbase.pb.Ge" + + "tCompletedSnapshotsRequest\032\'.hbase.pb.Ge" + + "tCompletedSnapshotsResponse\022S\n\016DeleteSna" + + "pshot\022\037.hbase.pb.DeleteSnapshotRequest\032 " + + ".hbase.pb.DeleteSnapshotResponse\022S\n\016IsSn" + + "apshotDone\022\037.hbase.pb.IsSnapshotDoneRequ" + + "est\032 .hbase.pb.IsSnapshotDoneResponse\022V\n" + + "\017RestoreSnapshot\022 .hbase.pb.RestoreSnaps", + "hotRequest\032!.hbase.pb.RestoreSnapshotRes" + + "ponse\022h\n\025IsRestoreSnapshotDone\022&.hbase.p" + + "b.IsRestoreSnapshotDoneRequest\032\'.hbase.p" + + "b.IsRestoreSnapshotDoneResponse\022P\n\rExecP" + + "rocedure\022\036.hbase.pb.ExecProcedureRequest" + + "\032\037.hbase.pb.ExecProcedureResponse\022W\n\024Exe" + + "cProcedureWithRet\022\036.hbase.pb.ExecProcedu" + + "reRequest\032\037.hbase.pb.ExecProcedureRespon" + + "se\022V\n\017IsProcedureDone\022 .hbase.pb.IsProce" + + "dureDoneRequest\032!.hbase.pb.IsProcedureDo", + "neResponse\022V\n\017ModifyNamespace\022 .hbase.pb" + + ".ModifyNamespaceRequest\032!.hbase.pb.Modif" + + "yNamespaceResponse\022V\n\017CreateNamespace\022 ." + + "hbase.pb.CreateNamespaceRequest\032!.hbase." + + "pb.CreateNamespaceResponse\022V\n\017DeleteName" + + "space\022 .hbase.pb.DeleteNamespaceRequest\032" + + "!.hbase.pb.DeleteNamespaceResponse\022k\n\026Ge" + + "tNamespaceDescriptor\022\'.hbase.pb.GetNames" + + "paceDescriptorRequest\032(.hbase.pb.GetName" + + "spaceDescriptorResponse\022q\n\030ListNamespace", + "Descriptors\022).hbase.pb.ListNamespaceDesc" + + "riptorsRequest\032*.hbase.pb.ListNamespaceD" + + "escriptorsResponse\022\206\001\n\037ListTableDescript" + + "orsByNamespace\0220.hbase.pb.ListTableDescr" + + "iptorsByNamespaceRequest\0321.hbase.pb.List" + + "TableDescriptorsByNamespaceResponse\022t\n\031L" + + "istTableNamesByNamespace\022*.hbase.pb.List" + + "TableNamesByNamespaceRequest\032+.hbase.pb." + + "ListTableNamesByNamespaceResponse\022P\n\rGet" + + "TableState\022\036.hbase.pb.GetTableStateReque", + "st\032\037.hbase.pb.GetTableStateResponse\022A\n\010S" + + "etQuota\022\031.hbase.pb.SetQuotaRequest\032\032.hba" + + "se.pb.SetQuotaResponse\022x\n\037getLastMajorCo" + + "mpactionTimestamp\022).hbase.pb.MajorCompac" + + "tionTimestampRequest\032*.hbase.pb.MajorCom" + + "pactionTimestampResponse\022\212\001\n(getLastMajo" + + "rCompactionTimestampForRegion\0222.hbase.pb" + + ".MajorCompactionTimestampForRegionReques" + + "t\032*.hbase.pb.MajorCompactionTimestampRes" + + "ponse\022_\n\022getProcedureResult\022#.hbase.pb.G", + "etProcedureResultRequest\032$.hbase.pb.GetP" + + "rocedureResultResponse\022h\n\027getSecurityCap" + + "abilities\022%.hbase.pb.SecurityCapabilitie" + + "sRequest\032&.hbase.pb.SecurityCapabilities" + + "Response\022S\n\016AbortProcedure\022\037.hbase.pb.Ab" + + "ortProcedureRequest\032 .hbase.pb.AbortProc" + + "edureResponse\022S\n\016ListProcedures\022\037.hbase." + + "pb.ListProceduresRequest\032 .hbase.pb.List" + + "ProceduresResponseBB\n*org.apache.hadoop." + + "hbase.protobuf.generatedB\014MasterProtosH\001", + "\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -62325,7 +62615,7 @@ public final class MasterProtos { internal_static_hbase_pb_CreateNamespaceResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_CreateNamespaceResponse_descriptor, - new java.lang.String[] { }); + new java.lang.String[] { "ProcId", }); internal_static_hbase_pb_DeleteNamespaceRequest_descriptor = getDescriptor().getMessageTypes().get(30); internal_static_hbase_pb_DeleteNamespaceRequest_fieldAccessorTable = new @@ -62337,7 +62627,7 @@ public final class MasterProtos { internal_static_hbase_pb_DeleteNamespaceResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_DeleteNamespaceResponse_descriptor, - new java.lang.String[] { }); + new java.lang.String[] { "ProcId", }); internal_static_hbase_pb_ModifyNamespaceRequest_descriptor = getDescriptor().getMessageTypes().get(32); internal_static_hbase_pb_ModifyNamespaceRequest_fieldAccessorTable = new @@ -62349,7 +62639,7 @@ public final class MasterProtos { internal_static_hbase_pb_ModifyNamespaceResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_ModifyNamespaceResponse_descriptor, - new java.lang.String[] { }); + new java.lang.String[] { "ProcId", }); internal_static_hbase_pb_GetNamespaceDescriptorRequest_descriptor = getDescriptor().getMessageTypes().get(34); internal_static_hbase_pb_GetNamespaceDescriptorRequest_fieldAccessorTable = new diff --git a/hbase-protocol/src/main/protobuf/Master.proto b/hbase-protocol/src/main/protobuf/Master.proto index 4d3a2e154fc..aa31a5e4668 100644 --- a/hbase-protocol/src/main/protobuf/Master.proto +++ b/hbase-protocol/src/main/protobuf/Master.proto @@ -186,6 +186,7 @@ message CreateNamespaceRequest { } message CreateNamespaceResponse { + optional uint64 proc_id = 1; } message DeleteNamespaceRequest { @@ -195,6 +196,7 @@ message DeleteNamespaceRequest { } message DeleteNamespaceResponse { + optional uint64 proc_id = 1; } message ModifyNamespaceRequest { @@ -204,6 +206,7 @@ message ModifyNamespaceRequest { } message ModifyNamespaceResponse { + optional uint64 proc_id = 1; } message GetNamespaceDescriptorRequest { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java index 365c0b845cb..26454f0db9b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/Server.java @@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; 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.ZooKeeperWatcher; @@ -43,12 +44,20 @@ public interface Server extends Abortable, Stoppable { 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 * 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} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java new file mode 100644 index 00000000000..cb3b684f521 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java @@ -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. + * + *

Implementation Notes

+ * 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 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 + * nonceGroup (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 + * nonceGroup (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 + * nonceGroup (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 name + * @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 getNamespaces() throws IOException; +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaException.java new file mode 100644 index 00000000000..62892b6256f --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaException.java @@ -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); + } +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaService.java new file mode 100644 index 00000000000..43353bafc80 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaService.java @@ -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 {} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java new file mode 100644 index 00000000000..0250f36e596 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java @@ -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 EMPTY_NAMESPACE_LIST = + Collections.unmodifiableList(new ArrayList(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 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 getNamespaces() throws IOException { + checkIsRunning(); + Set set = getTableNamespaceManager().list(); + if (set == null || set.isEmpty()) return EMPTY_NAMESPACE_LIST; + List list = new ArrayList(set.size()); + list.addAll(set); + return Collections.unmodifiableList(list); + } +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 8e51f25a6f6..4472b655d0c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -63,7 +63,6 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.NamespaceDescriptor; -import org.apache.hadoop.hbase.NamespaceNotFoundException; import org.apache.hadoop.hbase.PleaseHoldException; import org.apache.hadoop.hbase.ProcedureInfo; 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.Result; 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.exceptions.DeserializationException; 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.RegionNormalizerFactory; import org.apache.hadoop.hbase.master.procedure.AddColumnFamilyProcedure; -import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure; import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure; import org.apache.hadoop.hbase.master.procedure.DeleteColumnFamilyProcedure; -import org.apache.hadoop.hbase.master.procedure.DeleteNamespaceProcedure; import org.apache.hadoop.hbase.master.procedure.DeleteTableProcedure; import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure; import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure; import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; import org.apache.hadoop.hbase.master.procedure.ModifyColumnFamilyProcedure; -import org.apache.hadoop.hbase.master.procedure.ModifyNamespaceProcedure; import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure; import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch; import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait; @@ -185,7 +182,7 @@ import com.google.protobuf.Service; */ @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS) @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()); /** @@ -256,8 +253,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server { // Tracker for region normalizer state private RegionNormalizerTracker regionNormalizerTracker; - /** Namespace stuff */ - private TableNamespaceManager tableNamespaceManager; + private ClusterSchemaService clusterSchemaService; // Metrics for the HMaster final MetricsMaster metricsMaster; @@ -368,9 +364,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server { * Remaining steps of initialization occur in * #finishActiveMasterInitialization(MonitoredTask) after * the master becomes the active one. - * - * @throws KeeperException - * @throws IOException */ public HMaster(final Configuration conf, CoordinatedStateManager csm) throws IOException, KeeperException { @@ -570,10 +563,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server { /** * Initialize all ZK based system trackers. - * @throws IOException - * @throws InterruptedException - * @throws KeeperException - * @throws CoordinatedStateException */ void initializeZKBasedSystemTrackers() throws IOException, InterruptedException, KeeperException, CoordinatedStateException { @@ -588,12 +577,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server { this.balancer, this.service, this.metricsMaster, this.tableLockManager, tableStateManager); - this.regionServerTracker = new RegionServerTracker(zooKeeper, this, - this.serverManager); + this.regionServerTracker = new RegionServerTracker(zooKeeper, this, this.serverManager); this.regionServerTracker.start(); - this.drainingServerTracker = new DrainingServerTracker(zooKeeper, this, - this.serverManager); + this.drainingServerTracker = new DrainingServerTracker(zooKeeper, this, this.serverManager); this.drainingServerTracker.start(); // 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 { *
  • Ensure assignment of meta/namespace regions
  • *
  • Handle either fresh cluster start or master failover
  • * - * - * @throws IOException - * @throws InterruptedException - * @throws KeeperException - * @throws CoordinatedStateException */ private void finishActiveMasterInitialization(MonitoredTask status) throws IOException, InterruptedException, KeeperException, CoordinatedStateException { @@ -781,8 +763,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server { this.catalogJanitorChore = new CatalogJanitor(this, this); getChoreService().scheduleChore(catalogJanitorChore); - status.setStatus("Starting namespace manager"); - initNamespace(); + status.setStatus("Starting cluster schema service"); + initClusterSchemaService(); if (this.cpHost != null) { try { @@ -848,11 +830,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server { /** * 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, final MasterServices services) @@ -874,7 +851,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server { RegionState r = MetaTableLocator.getMetaRegionState(zkw, replicaId); LOG.info("Closing excess replica of meta region " + r.getRegion()); // send a close and wait for a max of 30 seconds - ServerManager.closeRegionSilentlyAndWait(getConnection(), r.getServerName(), + ServerManager.closeRegionSilentlyAndWait(getClusterConnection(), r.getServerName(), r.getRegion(), 30000); ZKUtil.deleteNode(zkw, zkw.getZNodeForReplica(replicaId)); } @@ -888,12 +865,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server { /** * Check hbase:meta is assigned. If not, assign it. - * @param status MonitoredTask - * @param previouslyFailedMetaRSs - * @param replicaId - * @throws InterruptedException - * @throws IOException - * @throws KeeperException */ void assignMeta(MonitoredTask status, Set previouslyFailedMetaRSs, int replicaId) throws InterruptedException, IOException, KeeperException { @@ -915,7 +886,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server { metaState.getServerName(), null); if (!metaState.isOpened() || !metaTableLocator.verifyMetaRegionLocation( - this.getConnection(), this.getZooKeeper(), timeout, replicaId)) { + this.getClusterConnection(), this.getZooKeeper(), timeout, replicaId)) { ServerName currentMetaServer = metaState.getServerName(); if (serverManager.isServerOnline(currentMetaServer)) { if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) { @@ -965,10 +936,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server { status.setStatus("META assigned."); } - void initNamespace() throws IOException { - //create namespace manager - tableNamespaceManager = new TableNamespaceManager(this); - tableNamespaceManager.start(); + void initClusterSchemaService() throws IOException, InterruptedException { + this.clusterSchemaService = new ClusterSchemaServiceImpl(this); + this.clusterSchemaService.startAndWait(); + if (!this.clusterSchemaService.isRunning()) throw new HBaseIOException("Failed start"); } 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 * @return Set of meta server names which were recorded in ZK - * @throws KeeperException */ private Set getPreviouselyFailedMetaServersFromZK() throws KeeperException { Set result = new HashSet(); @@ -1050,11 +1020,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server { return tableStateManager; } - @Override - public TableNamespaceManager getTableNamespaceManager() { - return tableNamespaceManager; - } - /* * Start up all services. If any of these threads gets an unhandled exception * then they just die with a logged message. This should be fine because @@ -1201,7 +1166,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server { /** * @return Get remote side's InetAddress - * @throws UnknownHostException */ InetAddress getRemoteInetAddress(final int port, final long serverStartCode) throws UnknownHostException { @@ -1336,9 +1300,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server { * Perform normalization of cluster (invoked by {@link RegionNormalizerChore}). * * @return true if normalization step was performed successfully, false otherwise - * (specifically, if HMaster hasn't been initialized properly or normalization - * is globally disabled) - * @throws IOException + * (specifically, if HMaster hasn't been initialized properly or normalization + * is globally disabled) */ public boolean normalizeRegions() throws IOException { if (!this.initialized) { @@ -1478,9 +1441,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server { if (isStopped()) { throw new MasterNotRunningException(); } - + checkInitialized(); String namespace = hTableDescriptor.getTableName().getNamespaceAsString(); - ensureNamespaceExists(namespace); + this.clusterSchemaService.getNamespace(namespace); HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(hTableDescriptor, splitKeys); 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 * statically allocated, it does not require that HMaster's cpHost be * initialized prior to accessing it. - * @return a String representation of the set of names of the loaded - * coprocessors. + * @return a String representation of the set of names of the loaded coprocessors. */ public static String getLoadedCoprocessors() { return CoprocessorHost.getLoadedCoprocessors().toString(); @@ -2305,18 +2267,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server { void checkInitialized() throws PleaseHoldException, ServerNotRunningYetException { checkServiceStarted(); - if (!this.initialized) { - throw new PleaseHoldException("Master is initializing"); - } + if (!isInitialized()) 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. * 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. * @param masterClass - * @param conf * @return HMaster instance. */ public static HMaster constructMaster(Class masterClass, @@ -2452,138 +2404,116 @@ public class HMaster extends HRegionServer implements MasterServices, Server { } @Override - public void createNamespace( - final NamespaceDescriptor descriptor, - 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); - } + public ClusterSchema getClusterSchema() { + return this.clusterSchemaService; } /** - * Ensure that the specified namespace exists, otherwise throws a NamespaceNotFoundException - * - * @param name the namespace to check - * @throws IOException if the namespace manager is not ready yet. - * @throws NamespaceNotFoundException if the namespace does not exists + * 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 + * nonceGroup (the source must ensure each operation gets a unique id). + * @return procedure id */ - private void ensureNamespaceExists(final String name) - throws IOException, NamespaceNotFoundException { - checkNamespaceManagerReady(); - NamespaceDescriptor nsd = tableNamespaceManager.get(name); - if (nsd == null) { - throw new NamespaceNotFoundException(name); + long createNamespace(final NamespaceDescriptor namespaceDescriptor, final long nonceGroup, + final long nonce) + throws IOException { + checkInitialized(); + TableName.isLegalNamespaceName(Bytes.toBytes(namespaceDescriptor.getName())); + 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 { - checkNamespaceManagerReady(); - - if (cpHost != null) { - cpHost.preGetNamespaceDescriptor(name); + /** + * 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 + * nonceGroup (the source must ensure each operation gets a unique id). + * @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) { - throw new NamespaceNotFoundException(name); - } - - if (cpHost != null) { - cpHost.postGetNamespaceDescriptor(nsd); + /** + * 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 + * nonceGroup (the source must ensure each operation gets a unique id). + * @return procedure id + */ + 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 name + */ + 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; } - @Override - public List listNamespaceDescriptors() throws IOException { - checkNamespaceManagerReady(); - - final List descriptors = new ArrayList(); + /** + * Get all Namespaces + * @return All Namespace descriptors + */ + List getNamespaces() throws IOException { + checkInitialized(); + final List nsds = new ArrayList(); boolean bypass = false; if (cpHost != null) { - bypass = cpHost.preListNamespaceDescriptors(descriptors); + bypass = cpHost.preListNamespaceDescriptors(nsds); } - if (!bypass) { - descriptors.addAll(tableNamespaceManager.list()); - - if (cpHost != null) { - cpHost.postListNamespaceDescriptors(descriptors); - } + nsds.addAll(this.clusterSchemaService.getNamespaces()); + if (this.cpHost != null) this.cpHost.postListNamespaceDescriptors(nsds); } - return descriptors; + return nsds; + } + + @Override + public List listTableNamesByNamespace(String name) throws IOException { + checkInitialized(); + return listTableNames(name, null, true); + } + + @Override + public List listTableDescriptorsByNamespace(String name) throws IOException { + checkInitialized(); + return listTableDescriptors(name, null, null, true); } @Override @@ -2617,21 +2547,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server { return procInfoList; } - @Override - public List listTableDescriptorsByNamespace(String name) throws IOException { - ensureNamespaceExists(name); - return listTableDescriptors(name, null, null, true); - } - - @Override - public List listTableNamesByNamespace(String name) throws IOException { - ensureNamespaceExists(name); - return listTableNames(name, null, true); - } - /** * Returns the list of table descriptors that match the specified request - * * @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 tableNameList the list of table names, or null if querying for all @@ -2640,51 +2557,17 @@ public class HMaster extends HRegionServer implements MasterServices, Server { */ public List listTableDescriptors(final String namespace, final String regex, final List tableNameList, final boolean includeSysTables) - throws IOException { - final List descriptors = new ArrayList(); - - boolean bypass = false; - if (cpHost != null) { - bypass = cpHost.preGetTableDescriptors(tableNameList, descriptors, regex); - } - + throws IOException { + List htds = new ArrayList(); + boolean bypass = cpHost != null? + cpHost.preGetTableDescriptors(tableNameList, htds, regex): false; if (!bypass) { - if (tableNameList == null || tableNameList.size() == 0) { - // request for all TableDescriptors - Collection 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)); - } - + htds = getTableDescriptors(htds, namespace, regex, tableNameList, includeSysTables); if (cpHost != null) { - cpHost.postGetTableDescriptors(tableNameList, descriptors, regex); + cpHost.postGetTableDescriptors(tableNameList, htds, regex); } } - return descriptors; + return htds; } /** @@ -2696,45 +2579,57 @@ public class HMaster extends HRegionServer implements MasterServices, Server { */ public List listTableNames(final String namespace, final String regex, final boolean includeSysTables) throws IOException { - final List descriptors = new ArrayList(); - - boolean bypass = false; - if (cpHost != null) { - bypass = cpHost.preGetTableNames(descriptors, regex); - } - + List htds = new ArrayList(); + boolean bypass = cpHost != null? cpHost.preGetTableNames(htds, regex): false; if (!bypass) { - // get all descriptors - Collection htds; - if (namespace != null && namespace.length() > 0) { - htds = tableDescriptors.getByNamespace(namespace).values(); - } else { - htds = tableDescriptors.getAll().values(); - } - - for (HTableDescriptor htd: htds) { - if (includeSysTables || !htd.getTableName().isSystemTable()) { - descriptors.add(htd); - } - } - - // Retains only those matched by regular expression. - if (regex != null) { - filterTablesByRegex(descriptors, Pattern.compile(regex)); - } - - if (cpHost != null) { - cpHost.postGetTableNames(descriptors, regex); - } - } - - List result = new ArrayList(descriptors.size()); - for (HTableDescriptor htd: descriptors) { - result.add(htd.getTableName()); + htds = getTableDescriptors(htds, namespace, regex, null, includeSysTables); + if (cpHost != null) cpHost.postGetTableNames(htds, regex); } + List result = new ArrayList(htds.size()); + for (HTableDescriptor htd: htds) result.add(htd.getTableName()); return result; } + /** + * @return list of table table descriptors after filtering by regex and whether to include system + * tables, etc. + * @throws IOException + */ + private List getTableDescriptors(final List htds, + final String namespace, final String regex, final List tableNameList, + final boolean includeSysTables) + throws IOException { + if (tableNameList == null || tableNameList.size() == 0) { + // request for all TableDescriptors + Collection 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. + if (regex != null) filterTablesByRegex(htds, Pattern.compile(regex)); + return htds; + } /** * Removes the table descriptors that don't match the pattern. @@ -2848,11 +2743,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server { * Queries the state of the {@link RegionNormalizerTracker}. If it's not initialized, * false is returned. */ - public boolean isNormalizerOn() { - if (null == regionNormalizerTracker) { - return false; - } - return regionNormalizerTracker.isNormalizerOn(); + public boolean isNormalizerOn() { + return null == regionNormalizerTracker? false: regionNormalizerTracker.isNormalizerOn(); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java index b269c3debd2..141fa88989d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java @@ -55,121 +55,21 @@ import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.RequestConverter; 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.CompactRegionResponse; 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.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.HBaseProtos; 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.RegionSpecifier.RegionSpecifierType; 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.AbortProcedureResponse; -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; +import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.*; 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.GetLastFlushedSequenceIdResponse; 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.snapshot.ClientSnapshotDescriptionUtils; 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.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.zookeeper.KeeperException; @@ -458,11 +358,11 @@ public class MasterRpcServices extends RSRpcServices public CreateNamespaceResponse createNamespace(RpcController controller, CreateNamespaceRequest request) throws ServiceException { try { - master.createNamespace( + long procId = master.createNamespace( ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()), request.getNonceGroup(), request.getNonce()); - return CreateNamespaceResponse.getDefaultInstance(); + return CreateNamespaceResponse.newBuilder().setProcId(procId).build(); } catch (IOException e) { throw new ServiceException(e); } @@ -506,11 +406,11 @@ public class MasterRpcServices extends RSRpcServices public DeleteNamespaceResponse deleteNamespace(RpcController controller, DeleteNamespaceRequest request) throws ServiceException { try { - master.deleteNamespace( + long procId = master.deleteNamespace( request.getNamespaceName(), request.getNonceGroup(), request.getNonce()); - return DeleteNamespaceResponse.getDefaultInstance(); + return DeleteNamespaceResponse.newBuilder().setProcId(procId).build(); } catch (IOException e) { throw new ServiceException(e); } @@ -832,7 +732,7 @@ public class MasterRpcServices extends RSRpcServices try { return GetNamespaceDescriptorResponse.newBuilder() .setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor( - master.getNamespaceDescriptor(request.getNamespaceName()))) + master.getNamespace(request.getNamespaceName()))) .build(); } catch (IOException e) { throw new ServiceException(e); @@ -977,10 +877,8 @@ public class MasterRpcServices extends RSRpcServices /** * Checks if the specified procedure is done. - * @return true if the procedure is done, - * false if the procedure is in the process of completing - * @throws ServiceException if invalid procedure, or - * a failed procedure with progress failure reason. + * @return true if the procedure is done, false if the procedure is in the process of completing + * @throws ServiceException if invalid procedure or failed procedure with progress failure reason. */ @Override public IsProcedureDoneResponse isProcedureDone(RpcController controller, @@ -1120,7 +1018,7 @@ public class MasterRpcServices extends RSRpcServices try { ListNamespaceDescriptorsResponse.Builder response = ListNamespaceDescriptorsResponse.newBuilder(); - for(NamespaceDescriptor ns: master.listNamespaceDescriptors()) { + for(NamespaceDescriptor ns: master.getNamespaces()) { response.addNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(ns)); } return response.build(); @@ -1200,11 +1098,11 @@ public class MasterRpcServices extends RSRpcServices public ModifyNamespaceResponse modifyNamespace(RpcController controller, ModifyNamespaceRequest request) throws ServiceException { try { - master.modifyNamespace( + long procId = master.modifyNamespace( ProtobufUtil.toNamespaceDescriptor(request.getNamespaceDescriptor()), request.getNonceGroup(), request.getNonce()); - return ModifyNamespaceResponse.getDefaultInstance(); + return ModifyNamespaceResponse.newBuilder().setProcId(procId).build(); } catch (IOException e) { throw new ServiceException(e); } @@ -1305,10 +1203,9 @@ public class MasterRpcServices extends RSRpcServices master.checkInitialized(); master.snapshotManager.checkSnapshotSupport(); - // ensure namespace exists + // Ensure namespace exists. Will throw exception if non-known NS. TableName dstTable = TableName.valueOf(request.getSnapshot().getTable()); - master.getNamespaceDescriptor(dstTable.getNamespaceAsString()); - + master.getNamespace(dstTable.getNamespaceAsString()); SnapshotDescription reqSnapshot = request.getSnapshot(); master.snapshotManager.restoreSnapshot(reqSnapshot); return RestoreSnapshotResponse.newBuilder().build(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java index af0e4907ec0..ec7db0c7c9e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java @@ -21,21 +21,20 @@ package org.apache.hadoop.hbase.master; import java.io.IOException; import java.util.List; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.ProcedureInfo; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotFoundException; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; -import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.quotas.MasterQuotaManager; import com.google.protobuf.Service; @@ -45,6 +44,11 @@ import com.google.protobuf.Service; */ @InterfaceAudience.Private public interface MasterServices extends Server { + /** + * @return Master's instance of {@link ClusterSchema} + */ + ClusterSchema getClusterSchema(); + /** * @return Master's instance of the {@link AssignmentManager} */ @@ -80,11 +84,6 @@ public interface MasterServices extends Server { */ MasterCoprocessorHost getMasterCoprocessorHost(); - /** - * @return Master's instance of {@link TableNamespaceManager} - */ - TableNamespaceManager getTableNamespaceManager(); - /** * @return Master's instance of {@link MasterQuotaManager} */ @@ -279,54 +278,6 @@ public interface MasterServices extends Server { */ 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. * @param procId ID of the procedure @@ -337,21 +288,6 @@ public interface MasterServices extends Server { public boolean abortProcedure(final long procId, final boolean mayInterruptIfRunning) 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 listNamespaceDescriptors() throws IOException; - /** * List procedures * @return procedure list diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java index 50f07c190ba..a95279c572f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java @@ -215,7 +215,7 @@ public class ServerManager { Configuration c = master.getConfiguration(); maxSkew = c.getLong("hbase.master.maxclockskew", 30000); 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( "hbase.master.maximum.ping.server.attempts", 10)); int pingSleepInterval = Math.max(1, master.getConfiguration().getInt( diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java index bbeaf769786..69d1280be47 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java @@ -27,17 +27,17 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; 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.Get; 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.TableState; 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.MasterProcedureEnv; +import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.Threads; import com.google.common.collect.Sets; /** - * This is a helper class used to manage the namespace - * metadata that is stored in TableName.NAMESPACE_TABLE_NAME - * It also mirrors updates to the ZK store by forwarding updates to - * {@link org.apache.hadoop.hbase.ZKNamespaceManager} + * This is a helper class used internally to manage the namespace metadata that is stored in + * TableName.NAMESPACE_TABLE_NAME. It also mirrors updates to the ZK store by forwarding updates to + * {@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 public class TableNamespaceManager { @@ -90,7 +95,7 @@ public class TableNamespaceManager { private long exclusiveLockTimeoutMs; private long sharedLockTimeoutMs; - public TableNamespaceManager(MasterServices masterServices) { + TableNamespaceManager(MasterServices masterServices) { this.masterServices = masterServices; this.conf = masterServices.getConfiguration(); @@ -104,7 +109,7 @@ public class TableNamespaceManager { public void start() throws IOException { if (!MetaTableAccessor.tableExists(masterServices.getConnection(), - TableName.NAMESPACE_TABLE_NAME)) { + TableName.NAMESPACE_TABLE_NAME)) { LOG.info("Namespace table not found. Creating..."); createNamespaceTable(masterServices); } @@ -113,7 +118,7 @@ public class TableNamespaceManager { // Wait for the namespace table to be initialized. long startTime = EnvironmentEdgeManager.currentTime(); int timeout = conf.getInt(NS_INIT_TIMEOUT, DEFAULT_NS_INIT_TIMEOUT); - while (!isTableAvailableAndInitialized(false)) { + while (!isTableAvailableAndInitialized()) { if (EnvironmentEdgeManager.currentTime() - startTime + 100 > timeout) { // We can't do anything if ns is not online. throw new IOException("Timedout " + timeout + "ms waiting for namespace table to " @@ -268,17 +273,49 @@ public class TableNamespaceManager { 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 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 - * 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. - * @return true if the namespace table manager is ready to serve, false - * otherwise - * @throws IOException + * @return true if the namespace table manager is ready to serve, false otherwise */ @SuppressWarnings("deprecation") - public synchronized boolean isTableAvailableAndInitialized( - final boolean createNamespaceAync) throws IOException { + public synchronized boolean isTableAvailableAndInitialized() + throws IOException { // Did we already get a table? If so, still make sure it's available if (isTableNamespaceManagerInitialized()) { return true; @@ -293,34 +330,10 @@ public class TableNamespaceManager { zkNamespaceManager.start(); if (get(nsTable, NamespaceDescriptor.DEFAULT_NAMESPACE.getName()) == null) { - if (createNamespaceAync) { - masterServices.getMasterProcedureExecutor().submitProcedure( - new CreateNamespaceProcedure( - masterServices.getMasterProcedureExecutor().getEnvironment(), - NamespaceDescriptor.DEFAULT_NAMESPACE)); - initGoodSofar = false; - } - else { - masterServices.createNamespaceSync( - NamespaceDescriptor.DEFAULT_NAMESPACE, - HConstants.NO_NONCE, - HConstants.NO_NONCE); - } + blockingCreateNamespace(NamespaceDescriptor.DEFAULT_NAMESPACE); } if (get(nsTable, NamespaceDescriptor.SYSTEM_NAMESPACE.getName()) == null) { - if (createNamespaceAync) { - masterServices.getMasterProcedureExecutor().submitProcedure( - new CreateNamespaceProcedure( - masterServices.getMasterProcedureExecutor().getEnvironment(), - NamespaceDescriptor.SYSTEM_NAMESPACE)); - initGoodSofar = false; - } - else { - masterServices.createNamespaceSync( - NamespaceDescriptor.SYSTEM_NAMESPACE, - HConstants.NO_NONCE, - HConstants.NO_NONCE); - } + blockingCreateNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE); } if (!initGoodSofar) { @@ -410,4 +423,4 @@ public class TableNamespaceManager { } return maxRegions; } -} +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java index 657bbfb34bf..f934737d970 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateNamespaceProcedure.java @@ -348,7 +348,7 @@ public class CreateNamespaceProcedure } private static TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) { - return env.getMasterServices().getTableNamespaceManager(); + return env.getMasterServices().getClusterSchema().getTableNamespaceManager(); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java index 5a4261451b3..2f991673a30 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteNamespaceProcedure.java @@ -383,7 +383,7 @@ public class DeleteNamespaceProcedure } 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 diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java index 1e86254a061..baef112fee7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java @@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.backup.HFileArchiver; 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.Result; import org.apache.hadoop.hbase.client.ResultScanner; @@ -374,7 +374,7 @@ public class DeleteTableProcedure */ private static void cleanAnyRemainingRows(final MasterProcedureEnv env, final TableName tableName) throws IOException { - ClusterConnection connection = env.getMasterServices().getConnection(); + Connection connection = env.getMasterServices().getConnection(); Scan tableScan = MetaTableAccessor.getScanForTableName(connection, tableName); try (Table metaTable = connection.getTable(TableName.META_TABLE_NAME)) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java index 30de25288c2..0f8c1724a0b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyNamespaceProcedure.java @@ -266,8 +266,9 @@ public class ModifyNamespaceProcedure } 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 * retrieve it. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java index 5c9f6f40b57..bdcd89cdc92 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java @@ -730,7 +730,7 @@ implements ServerProcedureInterface { boolean metaAssigned = false; // Is hbase:meta location available yet? if (mtl.isLocationAvailable(zkw)) { - ClusterConnection connection = env.getMasterServices().getConnection(); + ClusterConnection connection = env.getMasterServices().getClusterConnection(); // Is hbase:meta location good yet? long timeout = env.getMasterConfiguration().getLong(KEY_SHORT_WAIT_ON_META, DEFAULT_SHORT_WAIT_ON_META); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java index f24f8c08002..8035d32e325 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/namespace/NamespaceStateManager.java @@ -65,7 +65,7 @@ class NamespaceStateManager { /** * 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. */ public NamespaceTableAndRegionInfo getState(String name) { @@ -135,7 +135,7 @@ class NamespaceStateManager { private NamespaceDescriptor getNamespaceDescriptor(String namespaceAsString) { try { - return this.master.getNamespaceDescriptor(namespaceAsString); + return this.master.getClusterSchema().getNamespace(namespaceAsString); } catch (IOException e) { LOG.error("Error while fetching namespace descriptor for namespace : " + namespaceAsString); return null; @@ -212,7 +212,7 @@ class NamespaceStateManager { * Initialize namespace state cache by scanning meta table. */ private void initialize() throws IOException { - List namespaces = this.master.listNamespaceDescriptors(); + List namespaces = this.master.getClusterSchema().getNamespaces(); for (NamespaceDescriptor namespace : namespaces) { addNamespace(namespace.getName()); List tables = this.master.listTableNamesByNamespace(namespace.getName()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 211fed5ebfc..00046bafdb0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -79,6 +79,7 @@ import org.apache.hadoop.hbase.YouAreDeadException; import org.apache.hadoop.hbase.ZNodeClearer; 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.ConnectionUtils; import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory; import org.apache.hadoop.hbase.conf.ConfigurationManager; @@ -196,8 +197,7 @@ import sun.misc.SignalHandler; */ @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS) @SuppressWarnings("deprecation") -public class HRegionServer extends HasThread implements - RegionServerServices, LastSequenceId { +public class HRegionServer extends HasThread implements RegionServerServices, LastSequenceId { private static final Log LOG = LogFactory.getLog(HRegionServer.class); @@ -1867,7 +1867,12 @@ public class HRegionServer extends HasThread implements } @Override - public ClusterConnection getConnection() { + public Connection getConnection() { + return getClusterConnection(); + } + + @Override + public ClusterConnection getClusterConnection() { return this.clusterConnection; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java index 8d38b09253f..b86de122115 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSyncUp.java @@ -187,5 +187,11 @@ public class ReplicationSyncUp extends Configured implements Tool { public ChoreService getChoreService() { return null; } + + @Override + public ClusterConnection getClusterConnection() { + // TODO Auto-generated method stub + return null; + } } } diff --git a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp index f132b2b0975..878c5bfdb70 100644 --- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp +++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp @@ -124,7 +124,7 @@ <% if ( fqtn != null ) { - table = (HTable) master.getConnection().getTable(fqtn); + table = (HTable) master.getConnection().getTable(TableName.valueOf(fqtn)); if (table.getTableDescriptor().getRegionReplication() > 1) { tableHeader = "

    Table Regions

    "; withReplica = true; @@ -199,7 +199,7 @@ if ( fqtn != null ) {
    NameRegion ServerStart KeyEnd KeyLocalityRequestsReplicaID
    <%} else { Admin admin = master.getConnection().getAdmin(); - RegionLocator r = master.getConnection().getRegionLocator(table.getName()); + RegionLocator r = master.getClusterConnection().getRegionLocator(table.getName()); try { %>

    Table Attributes

    diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java index c126b19a764..a7fc75beffc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java @@ -306,4 +306,10 @@ public class MockRegionServerServices implements RegionServerServices { public double getCompactionPressure() { return 0; } + + @Override + public ClusterConnection getClusterConnection() { + // TODO Auto-generated method stub + return null; + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java index c24d8a3e598..f9e2a160c9a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java @@ -358,7 +358,7 @@ public class TestNamespace { runWithExpectedException(new Callable() { @Override public Void call() throws Exception { - admin.listTableDescriptorsByNamespace("non_existing_namespace"); + admin.listTableDescriptorsByNamespace("non_existant_namespace"); return null; } }, NamespaceNotFoundException.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java index e84d34c0707..618717bc843 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShortCircuitConnection.java @@ -65,7 +65,7 @@ public class TestShortCircuitConnection { htd.addFamily(hcd); UTIL.createTable(htd, null); HRegionServer regionServer = UTIL.getRSForFirstRegionInTable(tn); - ClusterConnection connection = regionServer.getConnection(); + ClusterConnection connection = regionServer.getClusterConnection(); HTableInterface tableIf = connection.getTable(tn); assertTrue(tableIf instanceof HTable); HTable table = (HTable) tableIf; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java index ef4a57942f7..638811a217e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java @@ -1586,7 +1586,13 @@ public class TestMasterObserver { cp.enableBypass(true); cp.resetStates(); - admin.modifyNamespace(NamespaceDescriptor.create(testNamespace).build()); + boolean expected = false; + try { + admin.modifyNamespace(NamespaceDescriptor.create(testNamespace).build()); + } catch (BypassCoprocessorException ce) { + expected = true; + } + assertTrue(expected); assertTrue("Test namespace should not have been modified", cp.preModifyNamespaceCalledOnly()); @@ -1594,7 +1600,13 @@ public class TestMasterObserver { assertTrue("Test namespace descriptor should have been called", cp.wasGetNamespaceDescriptorCalled()); - admin.deleteNamespace(testNamespace); + expected = false; + try { + admin.deleteNamespace(testNamespace); + } catch (BypassCoprocessorException ce) { + expected = true; + } + assertTrue(expected); assertTrue("Test namespace should not have been deleted", cp.preDeleteNamespaceCalledOnly()); assertNotNull(admin.getNamespaceDescriptor(testNamespace)); @@ -1614,7 +1626,13 @@ public class TestMasterObserver { cp.enableBypass(true); cp.resetStates(); - admin.createNamespace(NamespaceDescriptor.create(testNamespace).build()); + expected = false; + try { + admin.createNamespace(NamespaceDescriptor.create(testNamespace).build()); + } catch (BypassCoprocessorException ce) { + expected = true; + } + assertTrue(expected); assertTrue("Test namespace should not be created", cp.preCreateNamespaceCalledOnly()); // turn on bypass, run the test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java index eb8f8035e9f..234ad20aeda 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java @@ -651,4 +651,10 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices { public double getCompactionPressure() { return 0; } + + @Override + public ClusterConnection getClusterConnection() { + // TODO Auto-generated method stub + return null; + } } \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java index e3283e92e8c..e10ab2a804b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java @@ -326,5 +326,11 @@ public class TestActiveMasterManager { public ChoreService getChoreService() { return null; } + + @Override + public ClusterConnection getClusterConnection() { + // TODO Auto-generated method stub + return null; + } } } \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java index 8e35bbf5dd8..e26bd822b95 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java @@ -46,7 +46,6 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MetaMockingUtil; -import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException; import org.apache.hadoop.hbase.ProcedureInfo; import org.apache.hadoop.hbase.Server; @@ -143,11 +142,10 @@ public class TestCatalogJanitor { ServerName.valueOf("example.org,12345,6789"), HRegionInfo.FIRST_META_REGIONINFO); // Set hbase.rootdir into test dir. - FileSystem fs = FileSystem.get(this.c); + FileSystem.get(this.c); Path rootdir = FSUtils.getRootDir(this.c); FSUtils.setRootDir(this.c, rootdir); - AdminProtos.AdminService.BlockingInterface hri = - Mockito.mock(AdminProtos.AdminService.BlockingInterface.class); + Mockito.mock(AdminProtos.AdminService.BlockingInterface.class); } @Override @@ -208,6 +206,12 @@ public class TestCatalogJanitor { public ChoreService getChoreService() { return null; } + + @Override + public ClusterConnection getClusterConnection() { + // TODO Auto-generated method stub + return null; + } } /** @@ -401,48 +405,6 @@ public class TestCatalogJanitor { 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 listNamespaceDescriptors() throws IOException { - return null; //To change body of implemented methods use File | Settings | File Templates. - } - @Override public boolean abortProcedure(final long procId, final boolean mayInterruptIfRunning) throws IOException { @@ -535,11 +497,6 @@ public class TestCatalogJanitor { return null; } - @Override - public TableNamespaceManager getTableNamespaceManager() { - return null; - } - @Override public void dispatchMergingRegions(HRegionInfo region_a, HRegionInfo region_b, boolean forcible) throws IOException { @@ -547,21 +504,29 @@ public class TestCatalogJanitor { @Override public boolean isInitialized() { - // Auto-generated method stub return false; } @Override public long getLastMajorCompactionTimestamp(TableName table) throws IOException { - // Auto-generated method stub return 0; } @Override public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException { - // Auto-generated method stub return 0; } + + @Override + public ClusterSchema getClusterSchema() { + return null; + } + + @Override + public ClusterConnection getClusterConnection() { + // TODO Auto-generated method stub + return null; + } } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java index a19d5d86edd..142437c4d39 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java @@ -100,6 +100,12 @@ public class TestClockSkewDetection { public ChoreService getChoreService() { return null; } + + @Override + public ClusterConnection getClusterConnection() { + // TODO Auto-generated method stub + return null; + } }, null, false); LOG.debug("regionServerStartup 1"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java index 972834a39aa..398a89873e4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java @@ -32,6 +32,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.CategoryBasedTimeout; import org.apache.hadoop.hbase.CoordinatedStateException; import org.apache.hadoop.hbase.CoordinatedStateManager; import org.apache.hadoop.hbase.CoordinatedStateManagerFactory; @@ -62,8 +63,10 @@ import org.apache.zookeeper.KeeperException; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.junit.rules.TestRule; import org.mockito.Mockito; import com.google.protobuf.ServiceException; @@ -80,6 +83,8 @@ import com.google.protobuf.ServiceException; public class TestMasterNoCluster { private static final Log LOG = LogFactory.getLog(TestMasterNoCluster.class); private static final HBaseTestingUtility TESTUTIL = new HBaseTestingUtility(); + @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). + withTimeout(this.getClass()).withLookingForStuckThread(true).build(); @BeforeClass public static void setUpBeforeClass() throws Exception { @@ -122,7 +127,7 @@ public class TestMasterNoCluster { * @throws KeeperException * @throws InterruptedException */ - @Test (timeout=30000) + @Test public void testStopDuringStart() throws IOException, KeeperException, InterruptedException { CoordinatedStateManager cp = CoordinatedStateManagerFactory.getCoordinatedStateManager( @@ -141,7 +146,7 @@ public class TestMasterNoCluster { * @throws KeeperException * @throws InterruptedException */ - @Test (timeout=30000) + @Test public void testFailover() throws IOException, KeeperException, InterruptedException, ServiceException { final long now = System.currentTimeMillis(); @@ -192,6 +197,9 @@ public class TestMasterNoCluster { new byte [] {10, 0, 0, (byte)sn.getPort()}); } + @Override + void initClusterSchemaService() throws IOException, InterruptedException {} + @Override ServerManager createServerManager(Server master, MasterServices services) throws IOException { @@ -218,10 +226,6 @@ public class TestMasterNoCluster { return null; } } - - @Override - void initNamespace() { - } }; master.start(); @@ -265,6 +269,9 @@ public class TestMasterNoCluster { void assignMeta(MonitoredTask status, Set previouslyFailedMeatRSs, int replicaId) { } + @Override + void initClusterSchemaService() throws IOException, InterruptedException {} + @Override void initializeZKBasedSystemTrackers() throws IOException, InterruptedException, KeeperException, CoordinatedStateException { @@ -294,10 +301,6 @@ public class TestMasterNoCluster { return null; } } - - @Override - void initNamespace() { - } }; master.start(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java index 65c8649a513..c7707b7167d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitLogManager.java @@ -168,6 +168,12 @@ public class TestSplitLogManager { public ChoreService getChoreService() { return null; } + + @Override + public ClusterConnection getClusterConnection() { + // TODO Auto-generated method stub + return null; + } } static Stoppable stopper = new Stoppable() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java index 177adfdbb1a..0cccce15bd1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestRegionLocationFinder.java @@ -27,16 +27,15 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HDFSBlocksDistribution; -import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; 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.util.Bytes; +import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java index 078aaa65dba..60497016515 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java @@ -253,5 +253,11 @@ public class TestHFileCleaner { public ChoreService getChoreService() { return null; } + + @Override + public ClusterConnection getClusterConnection() { + // TODO Auto-generated method stub + return null; + } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java index d6f1606b2c0..0401ae878ca 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileLinkCleaner.java @@ -191,5 +191,11 @@ public class TestHFileLinkCleaner { public ChoreService getChoreService() { return null; } + + @Override + public ClusterConnection getClusterConnection() { + // TODO Auto-generated method stub + return null; + } } } \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java index f874523aac3..ebf3699a212 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java @@ -223,5 +223,11 @@ public class TestLogsCleaner { public ChoreService getChoreService() { return null; } + + @Override + public ClusterConnection getClusterConnection() { + // TODO Auto-generated method stub + return null; + } } } \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java index 87db3862425..b13f337ce4a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationHFileCleaner.java @@ -260,5 +260,11 @@ public class TestReplicationHFileCleaner { public ChoreService getChoreService() { return null; } + + @Override + public ClusterConnection getClusterConnection() { + // TODO Auto-generated method stub + return null; + } } } \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java index 77a603d0130..9731aa4405e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureTestingUtility.java @@ -438,11 +438,11 @@ public class MasterProcedureTestingUtility { } public static long generateNonceGroup(final HMaster master) { - return master.getConnection().getNonceGenerator().getNonceGroup(); + return master.getClusterConnection().getNonceGenerator().getNonceGroup(); } public static long generateNonce(final HMaster master) { - return master.getConnection().getNonceGenerator().newNonce(); + return master.getClusterConnection().getNonceGenerator().newNonce(); } public static class InjectAbortOnLoadListener diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java index e550c3ad60d..0e38afc42b6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java @@ -718,6 +718,12 @@ public class TestHeapMemoryManager { public ChoreService getChoreService() { return null; } + + @Override + public ClusterConnection getClusterConnection() { + // TODO Auto-generated method stub + return null; + } } static class CustomHeapMemoryTuner implements HeapMemoryTuner { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java index d62ccde26cd..96ec698278e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitLogWorker.java @@ -143,6 +143,12 @@ public class TestSplitLogWorker { public ChoreService getChoreService() { return null; } + + @Override + public ClusterConnection getClusterConnection() { + // TODO Auto-generated method stub + return null; + } } private void waitForCounter(AtomicLong ctr, long oldval, long newval, long timems) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java index 3b7402a5b97..94dbb2523b2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationStateZKImpl.java @@ -191,5 +191,11 @@ public class TestReplicationStateZKImpl extends TestReplicationStateBasic { public ChoreService getChoreService() { return null; } + + @Override + public ClusterConnection getClusterConnection() { + // TODO Auto-generated method stub + return null; + } } } \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java index a082b1978a5..9a878fd07df 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationTrackerZKImpl.java @@ -1,4 +1,5 @@ /** + * 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 @@ -292,5 +293,11 @@ public class TestReplicationTrackerZKImpl { public ChoreService getChoreService() { return null; } + + @Override + public ClusterConnection getClusterConnection() { + // TODO Auto-generated method stub + return null; + } } } \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java index a208120b676..f042a8d8722 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSourceManager.java @@ -595,5 +595,11 @@ public class TestReplicationSourceManager { public ChoreService getChoreService() { return null; } + + @Override + public ClusterConnection getClusterConnection() { + // TODO Auto-generated method stub + return null; + } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java index 69c6e63bf99..faac8eb48a6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestTokenAuthentication.java @@ -334,6 +334,12 @@ public class TestTokenAuthentication { public ChoreService getChoreService() { return null; } + + @Override + public ClusterConnection getClusterConnection() { + // TODO Auto-generated method stub + return null; + } } private static HBaseTestingUtility TEST_UTIL; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java index 1fcfcbb1f74..53e246790bd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/MockServer.java @@ -127,4 +127,10 @@ public class MockServer implements Server { public ChoreService getChoreService() { return null; } + + @Override + public ClusterConnection getClusterConnection() { + // TODO Auto-generated method stub + return null; + } } \ No newline at end of file