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.IOException;
021import org.apache.hadoop.hbase.HConstants;
022import org.apache.hadoop.hbase.TableName;
023import org.apache.hadoop.hbase.client.TableDescriptor;
024import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
025import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
026import org.apache.hadoop.hbase.snapshot.SnapshotDoesNotExistException;
027import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
028import org.apache.yetus.audience.InterfaceAudience;
029import org.slf4j.Logger;
030import org.slf4j.LoggerFactory;
031
032import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
033
034/**
035 * Utility class for recovery snapshot functionality, which automatically creates snapshots before
036 * dropping tables, truncating tables, or deleting column families.
037 */
038@InterfaceAudience.Private
039public final class RecoverySnapshotUtils {
040  private static final Logger LOG = LoggerFactory.getLogger(RecoverySnapshotUtils.class);
041
042  private RecoverySnapshotUtils() {
043
044  }
045
046  /**
047   * Checks if recovery snapshots are enabled for destructive table actions.
048   * @param env MasterProcedureEnv
049   * @return true if recovery snapshot functionality is enabled, false otherwise
050   */
051  public static boolean isRecoveryEnabled(final MasterProcedureEnv env) {
052    return env.getMasterConfiguration().getBoolean(
053      HConstants.SNAPSHOT_BEFORE_DESTRUCTIVE_ACTION_ENABLED_KEY,
054      HConstants.DEFAULT_SNAPSHOT_BEFORE_DESTRUCTIVE_ACTION_ENABLED)
055      && env.getMasterConfiguration().getBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
056  }
057
058  /**
059   * Gets the TTL that should be used for snapshots created before destructive schema actions.
060   * Checks for table-level override first, then falls back to site configuration.
061   * @param env             MasterProcedureEnv
062   * @param tableDescriptor the table descriptor to check for table-level TTL override
063   * @return TTL in seconds
064   */
065  public static long getRecoverySnapshotTtl(final MasterProcedureEnv env,
066    final TableDescriptor tableDescriptor) {
067    // Check table-level override first
068    if (tableDescriptor != null) {
069      String tableLevelTtl = tableDescriptor.getValue(HConstants.TABLE_RECOVERY_SNAPSHOT_TTL_KEY);
070      if (tableLevelTtl != null) {
071        try {
072          long ttl = Long.parseLong(tableLevelTtl);
073          LOG.debug("Using table-level recovery snapshot TTL {} seconds for table {}", ttl,
074            tableDescriptor.getTableName());
075          return ttl;
076        } catch (NumberFormatException e) {
077          LOG.warn("Invalid table-level recovery snapshot TTL '{}' for table {}, using default",
078            tableLevelTtl, tableDescriptor.getTableName());
079        }
080      }
081    }
082
083    // Fall back to site configuration
084    return env.getMasterConfiguration().getLong(
085      HConstants.SNAPSHOT_BEFORE_DESTRUCTIVE_ACTION_TTL_KEY,
086      HConstants.DEFAULT_SNAPSHOT_BEFORE_DESTRUCTIVE_ACTION_TTL);
087  }
088
089  /**
090   * Generates a recovery snapshot name.
091   * <p>
092   * The naming convention is: <tt>auto_{table}_{timestamp}</tt>
093   * @param tableName the table name
094   * @return the generated snapshot name
095   */
096  public static String generateSnapshotName(final TableName tableName) {
097    return generateSnapshotName(tableName, EnvironmentEdgeManager.currentTime());
098  }
099
100  /**
101   * Generates a recovery snapshot name.
102   * <p>
103   * The naming convention is: <tt>auto_{table}_{timestamp}</tt>
104   * @param tableName the table name
105   * @param timestamp the timestamp when the snapshot was initiated
106   * @return the generated snapshot name
107   */
108  public static String generateSnapshotName(final TableName tableName, final long timestamp) {
109    return "auto_" + tableName.getNameAsString() + "_" + timestamp;
110  }
111
112  /**
113   * Creates a SnapshotDescription for the recovery snapshot for a given operation.
114   * @param tableName    the table name
115   * @param snapshotName the snapshot name
116   * @return SnapshotDescription for the recovery snapshot
117   */
118  public static SnapshotProtos.SnapshotDescription
119    buildSnapshotDescription(final TableName tableName, final String snapshotName) {
120    return buildSnapshotDescription(tableName, snapshotName, 0,
121      SnapshotProtos.SnapshotDescription.Type.FLUSH);
122  }
123
124  /**
125   * Creates a SnapshotDescription for the recovery snapshot for a given operation.
126   * @param tableName    the table name
127   * @param snapshotName the snapshot name
128   * @param ttl          the TTL for the snapshot in seconds (0 means no TTL)
129   * @param type         the type of snapshot to create
130   * @return SnapshotDescription for the recovery snapshot
131   */
132  public static SnapshotProtos.SnapshotDescription buildSnapshotDescription(
133    final TableName tableName, final String snapshotName, final long ttl,
134    final SnapshotProtos.SnapshotDescription.Type type) {
135    SnapshotProtos.SnapshotDescription.Builder builder =
136      SnapshotProtos.SnapshotDescription.newBuilder();
137    builder.setVersion(SnapshotDescriptionUtils.SNAPSHOT_LAYOUT_VERSION);
138    builder.setName(snapshotName);
139    builder.setTable(tableName.getNameAsString());
140    builder.setType(type);
141    builder.setCreationTime(EnvironmentEdgeManager.currentTime());
142    builder.setTtl(ttl);
143    return builder.build();
144  }
145
146  /**
147   * Creates a SnapshotProcedure for soft drop functionality.
148   * <p>
149   * This method should be called from procedures that need to create a snapshot before performing
150   * destructive operations. It will check for table-level TTL overrides.
151   * @param env             MasterProcedureEnv
152   * @param tableName       the table name
153   * @param snapshotName    the name for the snapshot
154   * @param tableDescriptor the table descriptor to check for table-level TTL override
155   * @return SnapshotProcedure that can be added as a child procedure
156   * @throws IOException if snapshot creation fails
157   */
158  public static SnapshotProcedure createSnapshotProcedure(final MasterProcedureEnv env,
159    final TableName tableName, final String snapshotName, final TableDescriptor tableDescriptor)
160    throws IOException {
161    return new SnapshotProcedure(env,
162      buildSnapshotDescription(tableName, snapshotName,
163        getRecoverySnapshotTtl(env, tableDescriptor),
164        env.getMasterServices().getTableStateManager().isTableState(tableName,
165          org.apache.hadoop.hbase.client.TableState.State.DISABLED)
166            ? SnapshotProtos.SnapshotDescription.Type.SKIPFLUSH
167            : SnapshotProtos.SnapshotDescription.Type.FLUSH));
168  }
169
170  /**
171   * Deletes a recovery snapshot during rollback scenarios.
172   * <p>
173   * This method should be called during procedure rollback to clean up any snapshots that were
174   * created before the failure.
175   * @param env          MasterProcedureEnv
176   * @param snapshotName the name of the snapshot to delete
177   * @param tableName    the table name (for logging)
178   */
179  public static void deleteRecoverySnapshot(final MasterProcedureEnv env, final String snapshotName,
180    final TableName tableName) {
181    try {
182      LOG.debug("Deleting recovery snapshot {} for table {} during rollback", snapshotName,
183        tableName);
184      SnapshotManager snapshotManager = env.getMasterServices().getSnapshotManager();
185      if (snapshotManager == null) {
186        LOG.warn("SnapshotManager is not available, cannot delete recovery snapshot {}",
187          snapshotName);
188        return;
189      }
190      // Delete the snapshot using the snapshot manager. The SnapshotManager will handle existence
191      // checks.
192      snapshotManager.deleteSnapshot(buildSnapshotDescription(tableName, snapshotName));
193      LOG.info("Successfully deleted recovery snapshot {} for table {} during rollback",
194        snapshotName, tableName);
195    } catch (SnapshotDoesNotExistException e) {
196      // Expected during rollback if the snapshot was never created or already cleaned up.
197      LOG.debug("Recovery snapshot {} for table {} does not exist, skipping", snapshotName,
198        tableName);
199    } catch (Exception e) {
200      // During rollback, we don't want to fail the rollback process due to snapshot cleanup
201      // issues. Log the error and continue. The snapshot can be manually cleaned up later.
202      LOG.warn("Failed to delete recovery snapshot {} for table {} during rollback: {}. "
203        + "Manual cleanup may be required.", snapshotName, tableName, e.getMessage());
204    }
205  }
206}