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..3498590caa7 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
@@ -398,7 +398,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;
}
}
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-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..e6322303a76
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchema.java
@@ -0,0 +1,153 @@
+/**
+ * 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 java.util.concurrent.Future;
+
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.ProcedureInfo;
+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;
+
+ /**
+ * Utility method that will wait {@link #HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT_KEY}
+ * timeout and if exceptions, THROWs the exception doing conversion so palatable outside Master:
+ * i.e. {@link InterruptedException} becomes {@link InterruptedIOException} and so on.
+ *
+ * <>
+ *
+ * @param future Future to wait on.
+ * @return On completion, info on the procedure that ran.
+ * @throws IOException
+ */
+ // TODO: Where to put this utility? It goes away?
+ ProcedureInfo get(final Future future) throws IOException;
+
+ /**
+ * 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 Operation Future.
+ * Use {@link Future#get(long, java.util.concurrent.TimeUnit)} to wait on completion.
+ * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
+ * as well as {@link IOException}
+ */
+ Future 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 Operation Future.
+ * Use {@link Future#get(long, java.util.concurrent.TimeUnit)} to wait on completion.
+ * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
+ * as well as {@link IOException}
+ */
+ Future 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 Operation Future.
+ * Use {@link Future#get(long, java.util.concurrent.TimeUnit)} to wait on completion.
+ * @throws IOException Throws {@link ClusterSchemaException} and {@link InterruptedIOException}
+ * as well as {@link IOException}
+ */
+ Future 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
+ * @throws IOException
+ */
+ 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..c7f4692979f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ClusterSchemaServiceImpl.java
@@ -0,0 +1,164 @@
+/**
+ * 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.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.NamespaceNotFoundException;
+import org.apache.hadoop.hbase.ProcedureInfo;
+import org.apache.hadoop.hbase.ServiceNotRunningException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
+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 int clusterSchemaOperationTimeoutInMillis;
+ private final static List EMPTY_NAMESPACE_LIST =
+ Collections.unmodifiableList(new ArrayList(0));
+
+ ClusterSchemaServiceImpl(final MasterServices masterServices) {
+ this.masterServices = masterServices;
+ this.tableNamespaceManager = new TableNamespaceManager(masterServices);
+ this.clusterSchemaOperationTimeoutInMillis = this.masterServices.getConfiguration().
+ getInt(HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT_KEY,
+ DEFAULT_HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT);
+ }
+
+ // 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 ProcedureInfo get(final Future future) throws IOException {
+ try {
+ ProcedureInfo pi =
+ future.get(this.clusterSchemaOperationTimeoutInMillis, TimeUnit.MILLISECONDS);
+ // If the procedure got an exception, throw it.
+ if (pi.getException() != null) throw pi.getException();
+ return pi;
+ } catch (ExecutionException ee) {
+ // No cleanup to do... just let the exception out.
+ if (ee.getCause() instanceof IOException) throw (IOException)ee.getCause();
+ else throw new ClusterSchemaException(ee.getCause());
+ } catch (InterruptedException e) {
+ IOException ioe = new InterruptedIOException();
+ ioe.initCause(e);
+ throw ioe;
+ } catch (TimeoutException e) {
+ throw new TimeoutIOException(future.toString());
+ }
+ }
+
+ @Override
+ public TableNamespaceManager getTableNamespaceManager() {
+ return this.tableNamespaceManager;
+ }
+
+ private Future submitProcedure(final Procedure> procedure, long nonceGroup,
+ long nonce)
+ throws ServiceNotRunningException {
+ checkIsRunning();
+ ProcedureExecutor pe = this.masterServices.getMasterProcedureExecutor();
+ long procId = pe.submitProcedure(procedure, nonceGroup, nonce);
+ return new ProcedureFuture(pe, procId);
+ }
+
+ @Override
+ public Future createNamespace(NamespaceDescriptor namespaceDescriptor,
+ long nonceGroup, long nonce)
+ throws IOException {
+ return submitProcedure(new CreateNamespaceProcedure(
+ this.masterServices.getMasterProcedureExecutor().getEnvironment(), namespaceDescriptor),
+ nonceGroup, nonce);
+ }
+
+ @Override
+ public Future modifyNamespace(NamespaceDescriptor namespaceDescriptor,
+ long nonceGroup, long nonce)
+ throws IOException {
+ return submitProcedure(new ModifyNamespaceProcedure(
+ this.masterServices.getMasterProcedureExecutor().getEnvironment(), namespaceDescriptor),
+ nonceGroup, nonce);
+ }
+
+ @Override
+ public Future 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..f29cad62572 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
@@ -36,6 +36,7 @@ import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
+import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
@@ -63,7 +64,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;
@@ -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 {
@@ -630,11 +619,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 +765,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 +832,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)
@@ -888,12 +867,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 {
@@ -965,10 +938,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 +987,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 +1022,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 +1168,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 {
@@ -1338,7 +1304,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
* @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
*/
public boolean normalizeRegions() throws IOException {
if (!this.initialized) {
@@ -1478,9 +1443,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();
@@ -2305,18 +2270,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 +2367,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 extends HMaster> masterClass,
@@ -2452,138 +2407,110 @@ 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).
*/
- private void ensureNamespaceExists(final String name)
- throws IOException, NamespaceNotFoundException {
- checkNamespaceManagerReady();
- NamespaceDescriptor nsd = tableNamespaceManager.get(name);
- if (nsd == null) {
- throw new NamespaceNotFoundException(name);
- }
+ void 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)) return;
+ LOG.info(getClientIdAuditPrefix() + " creating " + namespaceDescriptor);
+ // Execute the operation synchronously - wait for the operation to complete before continuing.
+ Future future =
+ getClusterSchema().createNamespace(namespaceDescriptor, nonceGroup, nonce);
+ getClusterSchema().get(future);
+ if (this.cpHost != null) this.cpHost.postCreateNamespace(namespaceDescriptor);
}
- @Override
- public NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException {
- checkNamespaceManagerReady();
+ /**
+ * 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).\
+ */
+ void 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)) return;
+ LOG.info(getClientIdAuditPrefix() + " modify " + namespaceDescriptor);
+ // Execute the operation synchronously - wait for the operation to complete before continuing.
+ Future future =
+ getClusterSchema().modifyNamespace(namespaceDescriptor, nonceGroup, nonce);
+ getClusterSchema().get(future);
+ if (this.cpHost != null) this.cpHost.postModifyNamespace(namespaceDescriptor);
+ }
- if (cpHost != null) {
- cpHost.preGetNamespaceDescriptor(name);
- }
-
- 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).
+ * @throws IOException
+ */
+ void deleteNamespace(final String name, final long nonceGroup, final long nonce)
+ throws IOException {
+ checkInitialized();
+ if (this.cpHost != null && this.cpHost.preDeleteNamespace(name)) return;
+ LOG.info(getClientIdAuditPrefix() + " delete " + name);
+ // Execute the operation synchronously - wait for the operation to complete before continuing.
+ Future future = getClusterSchema().deleteNamespace(name, nonceGroup, nonce);
+ getClusterSchema().get(future);
+ if (this.cpHost != null) this.cpHost.postDeleteNamespace(name);
+ }
+ /**
+ * 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,18 +2544,6 @@ 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
*
@@ -2653,6 +2568,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
// request for all TableDescriptors
Collection htds;
if (namespace != null && namespace.length() > 0) {
+ // Do a check on the namespace existence. Will fail if does not exist.
+ this.clusterSchemaService.getNamespace(namespace);
htds = tableDescriptors.getByNamespace(namespace).values();
} else {
htds = tableDescriptors.getAll().values();
@@ -2696,46 +2613,17 @@ 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.addAll(listTableDescriptors(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;
}
-
/**
* Removes the table descriptors that don't match the pattern.
* @param descriptors list of table descriptors to filter
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..25288144c4b 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
@@ -34,6 +34,7 @@ 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.NamespaceExistException;
import org.apache.hadoop.hbase.PleaseHoldException;
import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.ServerLoad;
@@ -832,7 +833,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);
@@ -1120,7 +1121,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();
@@ -1305,10 +1306,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/ProcedureFuture.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ProcedureFuture.java
new file mode 100644
index 00000000000..0a849fc518a
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ProcedureFuture.java
@@ -0,0 +1,132 @@
+/**
+ * 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.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.hadoop.hbase.ProcedureInfo;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.Pair;
+
+/**
+ * Map Future Interface on to Procedure result processing.
+ */
+// Has no extra methods as of now beyond Future. Use #toString if you want to log
+// procId of procedure.
+// TODO: This should be in Procedure? Have it in master package for now. Lets out ProcedureInfo.
+// Implementation informed by HBaseAdmin#ProcedureFuture.
+@InterfaceAudience.Private
+class ProcedureFuture implements Future {
+ // Save exception so we can rethrow if called again. Same for result.
+ private ExecutionException exception = null;
+ private ProcedureInfo result = null;
+ private boolean done = false;
+ private boolean cancelled = false;
+ private final Long procId;
+ private final ProcedureExecutor procedureExecutor;
+
+ ProcedureFuture(final ProcedureExecutor procedureExecutor,
+ final long procId) {
+ this.procedureExecutor = procedureExecutor;
+ this.procId = procId;
+ }
+
+ @Override
+ public String toString() {
+ return "procId=" + this.procId;
+ }
+
+ @Override
+ public boolean cancel(boolean mayInterruptIfRunning) {
+ if (!this.cancelled) {
+ this.cancelled = this.procedureExecutor.abort(this.procId, mayInterruptIfRunning);
+ }
+ return this.cancelled;
+ }
+
+ @Override
+ public boolean isCancelled() {
+ return this.cancelled;
+ }
+
+ @Override
+ public boolean isDone() {
+ return done;
+ }
+
+ /**
+ * This method is unsupported. We will throw an UnsupportedOperationException. Only the lazy
+ * would call this method because they can avoid thinking through implication of a Procedure that
+ * might never return so this is disallowed. Use {@link #get(long, TimeUnit)}.
+ */
+ @Override
+ public ProcedureInfo get() throws InterruptedException, ExecutionException {
+ // TODO: should we ever spin forever?
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public ProcedureInfo get(long timeout, TimeUnit unit)
+ throws InterruptedException, ExecutionException, TimeoutException {
+ if (!this.done) {
+ // TODO: add this sort of facility to EnvironmentEdgeManager
+ long deadlineTs = EnvironmentEdgeManager.currentTime() + unit.toMillis(timeout);
+ try {
+ this.result = waitProcedureResult(procId, deadlineTs);
+ } catch (IOException e) {
+ this.exception = new ExecutionException(e);
+ }
+ this.done = true;
+ }
+ if (exception != null) {
+ throw exception;
+ }
+ return result;
+ }
+
+ /**
+ * @param procId
+ * @param deadlineTs
+ * @return A ProcedureInfo instance or null if procedure not found.
+ * @throws IOException
+ * @throws TimeoutException
+ * @throws InterruptedException
+ */
+ private ProcedureInfo waitProcedureResult(long procId, long deadlineTs)
+ throws IOException, TimeoutException, InterruptedException {
+ while (EnvironmentEdgeManager.currentTime() < deadlineTs) {
+ Pair pair = this.procedureExecutor.getResultOrProcedure(procId);
+ if (pair.getFirst() != null) {
+ this.procedureExecutor.removeResult(procId);
+ return pair.getFirst();
+ } else {
+ if (pair.getSecond() == null) return null;
+ }
+ // TODO: Add a wait.
+ }
+ throw new TimeoutException("The procedure " + procId + " is still running");
+ }
+}
\ No newline at end of file
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..07822fd513c 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,7 +46,6 @@ import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.constraint.ConstraintException;
-import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure;
import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
@@ -56,10 +55,12 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
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 +91,7 @@ public class TableNamespaceManager {
private long exclusiveLockTimeoutMs;
private long sharedLockTimeoutMs;
- public TableNamespaceManager(MasterServices masterServices) {
+ TableNamespaceManager(MasterServices masterServices) {
this.masterServices = masterServices;
this.conf = masterServices.getConfiguration();
@@ -113,7 +114,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 +269,30 @@ public class TableNamespaceManager {
return false;
}
+ /**
+ * Create Namespace in a blocking manner; don't return till success.
+ * Note, by-passes notifying coprocessors and name checks. Use for system namespaces only.
+ * @throws IOException
+ * @throws InterruptedException
+ */
+ private void createNamespace(final NamespaceDescriptor namespaceDescriptor)
+ throws IOException {
+ ClusterSchema clusterSchema = this.masterServices.getClusterSchema();
+ clusterSchema.get(clusterSchema.
+ createNamespace(namespaceDescriptor, HConstants.NO_NONCE, HConstants.NO_NONCE));
+ }
+
/**
* 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
*/
@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 +307,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);
- }
+ createNamespace(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);
- }
+ createNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE);
}
if (!initGoodSofar) {
@@ -410,4 +400,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/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/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..1255fa4975f 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
@@ -196,8 +196,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);
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/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
index 8e35bbf5dd8..785a00600a0 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
@@ -401,48 +399,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 +491,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 +498,23 @@ 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;
+ }
}
@Test
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..20b492c8cdd 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
@@ -218,10 +218,6 @@ public class TestMasterNoCluster {
return null;
}
}
-
- @Override
- void initNamespace() {
- }
};
master.start();
@@ -294,10 +290,6 @@ public class TestMasterNoCluster {
return null;
}
}
-
- @Override
- void initNamespace() {
- }
};
master.start();