Return-Path: X-Original-To: apmail-hbase-commits-archive@www.apache.org Delivered-To: apmail-hbase-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 3217C1856E for ; Thu, 14 Jan 2016 17:08:22 +0000 (UTC) Received: (qmail 47548 invoked by uid 500); 14 Jan 2016 17:08:17 -0000 Delivered-To: apmail-hbase-commits-archive@hbase.apache.org Received: (qmail 46851 invoked by uid 500); 14 Jan 2016 17:08:17 -0000 Mailing-List: contact commits-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hbase.apache.org Delivered-To: mailing list commits@hbase.apache.org Received: (qmail 45148 invoked by uid 99); 14 Jan 2016 17:08:16 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 14 Jan 2016 17:08:16 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 559D7E3889; Thu, 14 Jan 2016 17:08:16 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: jmhsieh@apache.org To: commits@hbase.apache.org Date: Thu, 14 Jan 2016 17:08:36 -0000 Message-Id: <3ff9fff4511a4ee48c9c233b74bae4d8@git.apache.org> In-Reply-To: <905a3c78f4ba4d328e999299653daa05@git.apache.org> References: <905a3c78f4ba4d328e999299653daa05@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [22/50] [abbrv] hbase git commit: HBASE-14888 ClusterSchema: Add Namespace Operations HBASE-14888 ClusterSchema: Add Namespace Operations Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/46303dfd Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/46303dfd Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/46303dfd Branch: refs/heads/trunk Commit: 46303dfd751912371539aa41ee8698dfb5e8b304 Parents: 1c4edd2 Author: stack Authored: Tue Jan 5 14:35:27 2016 -0800 Committer: stack Committed: Tue Jan 5 14:35:27 2016 -0800 ---------------------------------------------------------------------- .../org/apache/hadoop/hbase/client/Admin.java | 50 +- .../hadoop/hbase/client/ClusterConnection.java | 1 + .../hbase/client/ConnectionImplementation.java | 55 +- .../apache/hadoop/hbase/client/HBaseAdmin.java | 231 +++--- .../coprocessor/BypassCoprocessorException.java | 44 + .../java/org/apache/hadoop/hbase/Service.java | 50 ++ .../hbase/ServiceNotRunningException.java | 39 + .../apache/hadoop/hbase/ResourceChecker.java | 2 +- .../hbase/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 + .../hbase/master/ClusterSchemaServiceImpl.java | 131 +++ .../org/apache/hadoop/hbase/master/HMaster.java | 414 ++++------ .../hadoop/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 +- .../master/procedure/DeleteTableProcedure.java | 4 +- .../procedure/ModifyNamespaceProcedure.java | 3 +- .../master/procedure/ServerCrashProcedure.java | 2 +- .../hbase/namespace/NamespaceStateManager.java | 6 +- .../hbase/regionserver/HRegionServer.java | 11 +- .../regionserver/ReplicationSyncUp.java | 6 + .../resources/hbase-webapps/master/table.jsp | 4 +- .../hadoop/hbase/MockRegionServerServices.java | 6 + .../org/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 + .../hadoop/hbase/master/TestCatalogJanitor.java | 73 +- .../hbase/master/TestClockSkewDetection.java | 6 + .../hbase/master/TestMasterNoCluster.java | 23 +- .../hbase/master/TestSplitLogManager.java | 6 + .../balancer/TestRegionLocationFinder.java | 5 +- .../hbase/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 + .../hbase/regionserver/TestSplitLogWorker.java | 6 + .../replication/TestReplicationStateZKImpl.java | 6 + .../TestReplicationTrackerZKImpl.java | 7 + .../TestReplicationSourceManager.java | 6 + .../security/token/TestTokenAuthentication.java | 6 + .../apache/hadoop/hbase/util/MockServer.java | 6 + 52 files changed, 1615 insertions(+), 995 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Admin.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 b06902a..d7b52d5 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 @@ -965,49 +965,77 @@ public interface Admin extends Abortable, Closeable { Configuration getConfiguration(); /** - * Create a new namespace + * Create a new namespace. Blocks until namespace has been successfully created or an exception + * is thrown. * * @param descriptor descriptor which describes the new namespace - * @throws IOException */ void createNamespace(final NamespaceDescriptor descriptor) - throws IOException; + throws IOException; /** - * Modify an existing namespace + * Create a new namespace + * + * @param descriptor descriptor which describes the new namespace + * @return the result of the async create namespace operation. Use Future.get(long, TimeUnit) to + * wait on the operation to complete. + */ + 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 - * @throws IOException */ void modifyNamespace(final NamespaceDescriptor descriptor) - throws IOException; + throws IOException; + + /** + * Modify an existing namespace + * + * @param descriptor descriptor which describes the new namespace + * @return the result of the async modify namespace operation. Use Future.get(long, TimeUnit) to + * wait on the operation to complete. + */ + 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 - * @throws IOException */ void deleteNamespace(final String name) throws IOException; /** + * Delete an existing namespace. Only empty namespaces (no tables) can be removed. + * + * @param name namespace name + * @return the result of the async delete namespace operation. Use Future.get(long, TimeUnit) to + * wait on the operation to complete. + */ + 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 http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java ---------------------------------------------------------------------- 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 99071fa..741989f 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 http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java ---------------------------------------------------------------------- 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 0ef2a17..ecac792 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 http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java ---------------------------------------------------------------------- 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 7a50458..db94ff4 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 { http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/BypassCoprocessorException.java ---------------------------------------------------------------------- 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 0000000..3b01a9e --- /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 http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-common/src/main/java/org/apache/hadoop/hbase/Service.java ---------------------------------------------------------------------- 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 0000000..97d93cc --- /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; +} http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-common/src/main/java/org/apache/hadoop/hbase/ServiceNotRunningException.java ---------------------------------------------------------------------- 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 0000000..f6325ea --- /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 http://git-wip-us.apache.org/repos/asf/hbase/blob/46303dfd/hbase-common/src/test/java/org/apache/hadoop/hbase/ResourceChecker.java ---------------------------------------------------------------------- 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 539aea3..ee0380a 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;