HBASE-23864 No need to submit SplitTableRegionProcedure/MergeTableRegionsProcedure...
[hbase.git] / hbase-server / src / main / java / org / apache / hadoop / hbase / master / ClusterSchemaServiceImpl.java
blob2188dc3d324c4d08780e56038c2656ddd656438d
1 /**
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
18 package org.apache.hadoop.hbase.master;
20 import java.io.IOException;
21 import java.util.List;
22 import org.apache.hadoop.hbase.NamespaceDescriptor;
23 import org.apache.hadoop.hbase.NamespaceNotFoundException;
24 import org.apache.hadoop.hbase.ServiceNotRunningException;
25 import org.apache.hadoop.hbase.master.procedure.CreateNamespaceProcedure;
26 import org.apache.hadoop.hbase.master.procedure.DeleteNamespaceProcedure;
27 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
28 import org.apache.hadoop.hbase.master.procedure.ModifyNamespaceProcedure;
29 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
30 import org.apache.hadoop.hbase.procedure2.Procedure;
31 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
32 import org.apache.hadoop.hbase.util.NonceKey;
33 import org.apache.yetus.audience.InterfaceAudience;
35 import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
36 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.AbstractService;
38 @InterfaceAudience.Private
39 class ClusterSchemaServiceImpl extends AbstractService implements ClusterSchemaService {
41 private final TableNamespaceManager tableNamespaceManager;
42 private final MasterServices masterServices;
44 ClusterSchemaServiceImpl(final MasterServices masterServices) {
45 this.masterServices = masterServices;
46 this.tableNamespaceManager = new TableNamespaceManager(masterServices);
49 // All below are synchronized so consistent view on whether running or not.
51 private synchronized void checkIsRunning() throws ServiceNotRunningException {
52 if (!isRunning()) {
53 throw new ServiceNotRunningException();
57 @Override
58 protected synchronized void doStart() {
59 try {
60 notifyStarted();
61 this.tableNamespaceManager.start();
62 } catch (IOException ioe) {
63 notifyFailed(ioe);
67 @Override
68 protected void doStop() {
69 // This is no stop for the table manager.
70 notifyStopped();
73 @Override
74 public TableNamespaceManager getTableNamespaceManager() {
75 return this.tableNamespaceManager;
78 private long submitProcedure(final Procedure<MasterProcedureEnv> procedure,
79 final NonceKey nonceKey) throws ServiceNotRunningException {
80 checkIsRunning();
81 ProcedureExecutor<MasterProcedureEnv> pe = this.masterServices.getMasterProcedureExecutor();
82 return pe.submitProcedure(procedure, nonceKey);
85 @Override
86 public long createNamespace(NamespaceDescriptor namespaceDescriptor, final NonceKey nonceKey,
87 final ProcedurePrepareLatch latch)
88 throws IOException {
89 return submitProcedure(new CreateNamespaceProcedure(
90 this.masterServices.getMasterProcedureExecutor().getEnvironment(), namespaceDescriptor, latch),
91 nonceKey);
94 @Override
95 public long modifyNamespace(NamespaceDescriptor namespaceDescriptor, final NonceKey nonceKey,
96 final ProcedurePrepareLatch latch) throws IOException {
97 return submitProcedure(new ModifyNamespaceProcedure(
98 this.masterServices.getMasterProcedureExecutor().getEnvironment(), namespaceDescriptor, latch),
99 nonceKey);
102 @Override
103 public long deleteNamespace(String name, final NonceKey nonceKey, final ProcedurePrepareLatch latch)
104 throws IOException {
105 return submitProcedure(new DeleteNamespaceProcedure(
106 this.masterServices.getMasterProcedureExecutor().getEnvironment(), name, latch),
107 nonceKey);
110 @Override
111 public NamespaceDescriptor getNamespace(String name) throws IOException {
112 NamespaceDescriptor nsd = getTableNamespaceManager().get(name);
113 if (nsd == null) throw new NamespaceNotFoundException(name);
114 return nsd;
117 @Override
118 public List<NamespaceDescriptor> getNamespaces() throws IOException {
119 checkIsRunning();
120 return getTableNamespaceManager().list().stream()
121 .sorted(NamespaceDescriptor.NAMESPACE_DESCRIPTOR_COMPARATOR)
122 .collect(ImmutableList.toImmutableList());