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.backup.util;
019
020import java.util.Collections;
021import java.util.HashMap;
022import java.util.Map;
023import org.apache.hadoop.fs.Path;
024import org.apache.hadoop.hbase.TableName;
025import org.apache.hadoop.hbase.backup.BackupInfo;
026import org.apache.hadoop.hbase.net.Address;
027import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
028import org.apache.yetus.audience.InterfaceAudience;
029import org.slf4j.Logger;
030import org.slf4j.LoggerFactory;
031
032/**
033 * Tracks time boundaries for WAL file cleanup during backup operations. Maintains the oldest
034 * timestamp per RegionServer included in any backup, enabling safe determination of which WAL files
035 * can be deleted without compromising backup integrity.
036 */
037@InterfaceAudience.Private
038public class BackupBoundaries {
039  private static final Logger LOG = LoggerFactory.getLogger(BackupBoundaries.class);
040
041  // This map tracks, for every RegionServer, the least recent (= oldest / lowest timestamp)
042  // inclusion in any backup. In other words, it is the timestamp boundary up to which all backup
043  // roots have included the WAL in their backup.
044  private final Map<Address, Long> boundaries;
045
046  // The fallback cleanup boundary for RegionServers without explicit backup boundaries
047  // (e.g., servers that joined after backups began can be checked against this boundary)
048  private final long defaultBoundary;
049
050  private BackupBoundaries(Map<Address, Long> boundaries, long defaultBoundary) {
051    this.boundaries = boundaries;
052    this.defaultBoundary = defaultBoundary;
053  }
054
055  public boolean isDeletable(Path walLogPath) {
056    try {
057      String hostname = BackupUtils.parseHostNameFromLogFile(walLogPath);
058
059      if (hostname == null) {
060        LOG.warn(
061          "Cannot parse hostname from RegionServer WAL file: {}. Ignoring cleanup of this log",
062          walLogPath);
063        return false;
064      }
065
066      Address address = Address.fromString(hostname);
067      long pathTs = AbstractFSWALProvider.getTimestamp(walLogPath.getName());
068
069      if (!boundaries.containsKey(address)) {
070        boolean isDeletable = pathTs <= defaultBoundary;
071        if (LOG.isDebugEnabled()) {
072          LOG.debug(
073            "Boundary for {} not found. isDeletable = {} based on defaultBoundary = {} and WAL ts of {}",
074            walLogPath, isDeletable, defaultBoundary, pathTs);
075        }
076        return isDeletable;
077      }
078
079      long backupTs = boundaries.get(address);
080      if (pathTs <= backupTs) {
081        if (LOG.isDebugEnabled()) {
082          LOG.debug(
083            "WAL cleanup time-boundary found for server {}: {}. Ok to delete older file: {}",
084            address.getHostName(), pathTs, walLogPath);
085        }
086        return true;
087      }
088
089      if (LOG.isDebugEnabled()) {
090        LOG.debug("WAL cleanup time-boundary found for server {}: {}. Keeping younger file: {}",
091          address.getHostName(), backupTs, walLogPath);
092      }
093
094      return false;
095    } catch (Exception e) {
096      LOG.warn("Error occurred while filtering file: {}. Ignoring cleanup of this log", walLogPath,
097        e);
098      return false;
099    }
100  }
101
102  public Map<Address, Long> getBoundaries() {
103    return boundaries;
104  }
105
106  public long getDefaultBoundary() {
107    return defaultBoundary;
108  }
109
110  public static BackupBoundariesBuilder builder(long tsCleanupBuffer) {
111    return new BackupBoundariesBuilder(tsCleanupBuffer);
112  }
113
114  public static class BackupBoundariesBuilder {
115    private final Map<Address, Long> boundaries = new HashMap<>();
116    private final long tsCleanupBuffer;
117
118    private long oldestStartTs = Long.MAX_VALUE;
119
120    private BackupBoundariesBuilder(long tsCleanupBuffer) {
121      this.tsCleanupBuffer = tsCleanupBuffer;
122    }
123
124    /**
125     * Updates the boundaries based on the provided backup info.
126     * @param backupInfo the most recent completed backup info for a backup root, or if there is no
127     *                   such completed backup, the currently running backup.
128     */
129    public void update(BackupInfo backupInfo) {
130      switch (backupInfo.getState()) {
131        case COMPLETE:
132          // If a completed backup exists in the backup root, we want to protect all logs that
133          // have been created since the log-roll that happened for that backup.
134          for (TableName table : backupInfo.getTableSetTimestampMap().keySet()) {
135            for (Map.Entry<String, Long> entry : backupInfo.getTableSetTimestampMap().get(table)
136              .entrySet()) {
137              Address regionServerAddress = Address.fromString(entry.getKey());
138              Long logRollTs = entry.getValue();
139
140              Long storedTs = boundaries.get(regionServerAddress);
141              if (storedTs == null || logRollTs < storedTs) {
142                boundaries.put(regionServerAddress, logRollTs);
143              }
144            }
145          }
146          break;
147        case RUNNING:
148          // If there is NO completed backup in the backup root, there are no persisted log-roll
149          // timestamps available yet. But, we still want to protect all files that have been
150          // created since the start of the currently running backup.
151          oldestStartTs = Math.min(oldestStartTs, backupInfo.getStartTs());
152          break;
153        default:
154          throw new IllegalStateException("Unexpected backupInfo state: " + backupInfo.getState());
155      }
156    }
157
158    public BackupBoundaries build() {
159      if (boundaries.isEmpty()) {
160        long defaultBoundary = oldestStartTs - tsCleanupBuffer;
161        return new BackupBoundaries(Collections.emptyMap(), defaultBoundary);
162      }
163
164      long oldestRollTs = Collections.min(boundaries.values());
165      long defaultBoundary = Math.min(oldestRollTs, oldestStartTs) - tsCleanupBuffer;
166      return new BackupBoundaries(boundaries, defaultBoundary);
167    }
168  }
169}