001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.master.procedure; 019 020import java.io.FileNotFoundException; 021import java.io.IOException; 022import org.apache.hadoop.fs.FileStatus; 023import org.apache.hadoop.fs.FileSystem; 024import org.apache.hadoop.fs.Path; 025import org.apache.hadoop.hbase.HConstants; 026import org.apache.hadoop.hbase.NamespaceDescriptor; 027import org.apache.hadoop.hbase.NamespaceNotFoundException; 028import org.apache.hadoop.hbase.constraint.ConstraintException; 029import org.apache.hadoop.hbase.master.MasterFileSystem; 030import org.apache.hadoop.hbase.master.TableNamespaceManager; 031import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; 032import org.apache.hadoop.hbase.util.CommonFSUtils; 033import org.apache.yetus.audience.InterfaceAudience; 034import org.slf4j.Logger; 035import org.slf4j.LoggerFactory; 036 037import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 038import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos; 039import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.DeleteNamespaceState; 040 041/** 042 * The procedure to remove a namespace. 043 */ 044@InterfaceAudience.Private 045public class DeleteNamespaceProcedure 046 extends AbstractStateMachineNamespaceProcedure<DeleteNamespaceState> { 047 private static final Logger LOG = LoggerFactory.getLogger(DeleteNamespaceProcedure.class); 048 049 private NamespaceDescriptor nsDescriptor; 050 private String namespaceName; 051 private Boolean traceEnabled; 052 053 public DeleteNamespaceProcedure() { 054 this.nsDescriptor = null; 055 this.traceEnabled = null; 056 } 057 058 public DeleteNamespaceProcedure(final MasterProcedureEnv env, final String namespaceName) { 059 this(env, namespaceName, null); 060 } 061 062 public DeleteNamespaceProcedure(final MasterProcedureEnv env, final String namespaceName, 063 final ProcedurePrepareLatch latch) { 064 super(env, latch); 065 this.namespaceName = namespaceName; 066 this.nsDescriptor = null; 067 this.traceEnabled = null; 068 } 069 070 @Override 071 protected Flow executeFromState(final MasterProcedureEnv env, final DeleteNamespaceState state) 072 throws InterruptedException { 073 LOG.info(this.toString()); 074 try { 075 switch (state) { 076 case DELETE_NAMESPACE_PREPARE: 077 boolean present = prepareDelete(env); 078 releaseSyncLatch(); 079 if (!present) { 080 assert isFailed() : "Delete namespace should have an exception here"; 081 return Flow.NO_MORE_STATE; 082 } 083 setNextState(DeleteNamespaceState.DELETE_NAMESPACE_DELETE_FROM_NS_TABLE); 084 break; 085 case DELETE_NAMESPACE_DELETE_FROM_NS_TABLE: 086 deleteFromNSTable(env, namespaceName); 087 setNextState(DeleteNamespaceState.DELETE_NAMESPACE_REMOVE_FROM_ZK); 088 break; 089 case DELETE_NAMESPACE_REMOVE_FROM_ZK: 090 removeFromZKNamespaceManager(env, namespaceName); 091 setNextState(DeleteNamespaceState.DELETE_NAMESPACE_DELETE_DIRECTORIES); 092 break; 093 case DELETE_NAMESPACE_DELETE_DIRECTORIES: 094 deleteDirectory(env, namespaceName); 095 setNextState(DeleteNamespaceState.DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA); 096 break; 097 case DELETE_NAMESPACE_REMOVE_NAMESPACE_QUOTA: 098 removeNamespaceQuota(env, namespaceName); 099 return Flow.NO_MORE_STATE; 100 default: 101 throw new UnsupportedOperationException(this + " unhandled state=" + state); 102 } 103 } catch (IOException e) { 104 if (isRollbackSupported(state)) { 105 setFailure("master-delete-namespace", e); 106 } else { 107 LOG.warn("Retriable error trying to delete namespace " + namespaceName + " (in state=" 108 + state + ")", e); 109 } 110 } 111 return Flow.HAS_MORE_STATE; 112 } 113 114 @Override 115 protected void rollbackState(final MasterProcedureEnv env, final DeleteNamespaceState state) 116 throws IOException { 117 if (state == DeleteNamespaceState.DELETE_NAMESPACE_PREPARE) { 118 // nothing to rollback, pre is just table-state checks. 119 // We can fail if the table does not exist or is not disabled. 120 // TODO: coprocessor rollback semantic is still undefined. 121 releaseSyncLatch(); 122 return; 123 } 124 125 // The procedure doesn't have a rollback. The execution will succeed, at some point. 126 throw new UnsupportedOperationException("unhandled state=" + state); 127 } 128 129 @Override 130 protected boolean isRollbackSupported(final DeleteNamespaceState state) { 131 switch (state) { 132 case DELETE_NAMESPACE_PREPARE: 133 return true; 134 default: 135 return false; 136 } 137 } 138 139 @Override 140 protected DeleteNamespaceState getState(final int stateId) { 141 return DeleteNamespaceState.valueOf(stateId); 142 } 143 144 @Override 145 protected int getStateId(final DeleteNamespaceState state) { 146 return state.getNumber(); 147 } 148 149 @Override 150 protected DeleteNamespaceState getInitialState() { 151 return DeleteNamespaceState.DELETE_NAMESPACE_PREPARE; 152 } 153 154 @Override 155 protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException { 156 super.serializeStateData(serializer); 157 158 MasterProcedureProtos.DeleteNamespaceStateData.Builder deleteNamespaceMsg = 159 MasterProcedureProtos.DeleteNamespaceStateData.newBuilder().setNamespaceName(namespaceName); 160 if (this.nsDescriptor != null) { 161 deleteNamespaceMsg 162 .setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(this.nsDescriptor)); 163 } 164 serializer.serialize(deleteNamespaceMsg.build()); 165 } 166 167 @Override 168 protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException { 169 super.deserializeStateData(serializer); 170 171 MasterProcedureProtos.DeleteNamespaceStateData deleteNamespaceMsg = 172 serializer.deserialize(MasterProcedureProtos.DeleteNamespaceStateData.class); 173 namespaceName = deleteNamespaceMsg.getNamespaceName(); 174 if (deleteNamespaceMsg.hasNamespaceDescriptor()) { 175 nsDescriptor = 176 ProtobufUtil.toNamespaceDescriptor(deleteNamespaceMsg.getNamespaceDescriptor()); 177 } 178 } 179 180 @Override 181 public TableOperationType getTableOperationType() { 182 return TableOperationType.EDIT; 183 } 184 185 @Override 186 protected String getNamespaceName() { 187 return namespaceName; 188 } 189 190 /** 191 * Action before any real action of deleting namespace. 192 * @param env MasterProcedureEnv 193 */ 194 private boolean prepareDelete(final MasterProcedureEnv env) throws IOException { 195 if (getTableNamespaceManager(env).doesNamespaceExist(namespaceName) == false) { 196 setFailure("master-delete-namespace", new NamespaceNotFoundException(namespaceName)); 197 return false; 198 } 199 if (NamespaceDescriptor.RESERVED_NAMESPACES.contains(namespaceName)) { 200 setFailure("master-delete-namespace", 201 new ConstraintException("Reserved namespace " + namespaceName + " cannot be removed.")); 202 return false; 203 } 204 205 int tableCount = 0; 206 try { 207 tableCount = env.getMasterServices().listTableDescriptorsByNamespace(namespaceName).size(); 208 } catch (FileNotFoundException fnfe) { 209 setFailure("master-delete-namespace", new NamespaceNotFoundException(namespaceName)); 210 return false; 211 } 212 if (tableCount > 0) { 213 setFailure("master-delete-namespace", 214 new ConstraintException("Only empty namespaces can be removed. Namespace " + namespaceName 215 + " has " + tableCount + " tables")); 216 return false; 217 } 218 219 // This is used for rollback 220 nsDescriptor = getTableNamespaceManager(env).get(namespaceName); 221 return true; 222 } 223 224 /** 225 * delete the row from namespace table 226 * @param env MasterProcedureEnv 227 * @param namespaceName name of the namespace in string format 228 */ 229 protected static void deleteFromNSTable(final MasterProcedureEnv env, final String namespaceName) 230 throws IOException { 231 getTableNamespaceManager(env).removeFromNSTable(namespaceName); 232 } 233 234 /** 235 * undo the delete 236 * @param env MasterProcedureEnv 237 */ 238 private void undoDeleteFromNSTable(final MasterProcedureEnv env) { 239 try { 240 if (nsDescriptor != null) { 241 CreateNamespaceProcedure.insertIntoNSTable(env, nsDescriptor); 242 } 243 } catch (Exception e) { 244 // Ignore 245 LOG.debug("Rollback of deleteFromNSTable throws exception: " + e); 246 } 247 } 248 249 /** 250 * remove from ZooKeeper. 251 * @param env MasterProcedureEnv 252 * @param namespaceName name of the namespace in string format 253 */ 254 protected static void removeFromZKNamespaceManager(final MasterProcedureEnv env, 255 final String namespaceName) throws IOException { 256 getTableNamespaceManager(env).removeFromZKNamespaceManager(namespaceName); 257 } 258 259 /** 260 * undo the remove from ZooKeeper 261 * @param env MasterProcedureEnv 262 */ 263 private void undoRemoveFromZKNamespaceManager(final MasterProcedureEnv env) { 264 try { 265 if (nsDescriptor != null) { 266 CreateNamespaceProcedure.updateZKNamespaceManager(env, nsDescriptor); 267 } 268 } catch (Exception e) { 269 // Ignore 270 LOG.debug("Rollback of removeFromZKNamespaceManager throws exception: " + e); 271 } 272 } 273 274 /** 275 * Delete the namespace directories from the file system 276 * @param env MasterProcedureEnv 277 * @param namespaceName name of the namespace in string format 278 */ 279 protected static void deleteDirectory(final MasterProcedureEnv env, final String namespaceName) 280 throws IOException { 281 MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem(); 282 FileSystem fs = mfs.getFileSystem(); 283 Path p = CommonFSUtils.getNamespaceDir(mfs.getRootDir(), namespaceName); 284 285 try { 286 for (FileStatus status : fs.listStatus(p)) { 287 if (!HConstants.HBASE_NON_TABLE_DIRS.contains(status.getPath().getName())) { 288 throw new IOException("Namespace directory contains table dir: " + status.getPath()); 289 } 290 } 291 if (!fs.delete(CommonFSUtils.getNamespaceDir(mfs.getRootDir(), namespaceName), true)) { 292 throw new IOException("Failed to remove namespace: " + namespaceName); 293 } 294 } catch (FileNotFoundException e) { 295 // File already deleted, continue 296 LOG.debug("deleteDirectory throws exception: " + e); 297 } 298 } 299 300 /** 301 * undo delete directory 302 * @param env MasterProcedureEnv 303 */ 304 private void rollbackDeleteDirectory(final MasterProcedureEnv env) throws IOException { 305 try { 306 CreateNamespaceProcedure.createDirectory(env, nsDescriptor); 307 } catch (Exception e) { 308 // Ignore exception 309 LOG.debug("Rollback of deleteDirectory throws exception: " + e); 310 } 311 } 312 313 /** 314 * remove quota for the namespace 315 * @param env MasterProcedureEnv 316 * @param namespaceName name of the namespace in string format 317 **/ 318 protected static void removeNamespaceQuota(final MasterProcedureEnv env, 319 final String namespaceName) throws IOException { 320 env.getMasterServices().getMasterQuotaManager().removeNamespaceQuota(namespaceName); 321 } 322 323 /** 324 * undo remove quota for the namespace 325 * @param env MasterProcedureEnv 326 **/ 327 private void rollbacRemoveNamespaceQuota(final MasterProcedureEnv env) throws IOException { 328 try { 329 CreateNamespaceProcedure.setNamespaceQuota(env, nsDescriptor); 330 } catch (Exception e) { 331 // Ignore exception 332 LOG.debug("Rollback of removeNamespaceQuota throws exception: " + e); 333 } 334 } 335 336 private static TableNamespaceManager getTableNamespaceManager(final MasterProcedureEnv env) { 337 return env.getMasterServices().getClusterSchema().getTableNamespaceManager(); 338 } 339 340 /** 341 * The procedure could be restarted from a different machine. If the variable is null, we need to 342 * retrieve it. 343 */ 344 private Boolean isTraceEnabled() { 345 if (traceEnabled == null) { 346 traceEnabled = LOG.isTraceEnabled(); 347 } 348 return traceEnabled; 349 } 350}