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