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 n
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 n
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 n
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 n
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 n
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 n
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 n
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 n
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 n
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. n
343   */
344  private Boolean isTraceEnabled() {
345    if (traceEnabled == null) {
346      traceEnabled = LOG.isTraceEnabled();
347    }
348    return traceEnabled;
349  }
350}