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.impl;
019
020import java.io.IOException;
021import java.util.ArrayList;
022import java.util.Collections;
023import java.util.List;
024import java.util.Map;
025import org.apache.hadoop.conf.Configuration;
026import org.apache.hadoop.fs.FileStatus;
027import org.apache.hadoop.fs.FileSystem;
028import org.apache.hadoop.fs.Path;
029import org.apache.hadoop.fs.PathFilter;
030import org.apache.hadoop.hbase.HConstants;
031import org.apache.hadoop.hbase.backup.util.BackupUtils;
032import org.apache.hadoop.hbase.client.Connection;
033import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
034import org.apache.hadoop.hbase.util.CommonFSUtils;
035import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
036import org.apache.yetus.audience.InterfaceAudience;
037import org.slf4j.Logger;
038import org.slf4j.LoggerFactory;
039
040/**
041 * After a full backup was created, the incremental backup will only store the changes made after
042 * the last full or incremental backup. Creating the backup copies the logfiles in .logs and
043 * .oldlogs since the last backup timestamp.
044 */
045@InterfaceAudience.Private
046public class IncrementalBackupManager extends BackupManager {
047  public static final Logger LOG = LoggerFactory.getLogger(IncrementalBackupManager.class);
048
049  public IncrementalBackupManager(Connection conn, Configuration conf) throws IOException {
050    super(conn, conf);
051  }
052
053  /**
054   * Obtain the list of logs that need to be copied out for this incremental backup. The list is set
055   * in BackupInfo.
056   * @return The new HashMap of RS log time stamps after the log roll for this incremental backup.
057   * @throws IOException exception
058   */
059  public Map<String, Long> getIncrBackupLogFileMap() throws IOException {
060    List<String> logList;
061    Map<String, Long> newTimestamps;
062    Map<String, Long> previousTimestampMins =
063      BackupUtils.getRSLogTimestampMins(readLogTimestampMap());
064
065    // get all new log files from .logs and .oldlogs after last TS and before new timestamp
066    if (previousTimestampMins.isEmpty()) {
067      throw new IOException("Cannot read any previous back up timestamps from backup system table. "
068        + "In order to create an incremental backup, at least one full backup is needed.");
069    }
070
071    LOG.info("Execute roll log procedure for incremental backup ...");
072    BackupUtils.logRoll(conn, backupInfo.getBackupRootDir(), conf);
073
074    newTimestamps = readRegionServerLastLogRollResult();
075
076    logList = getLogFilesForNewBackup(previousTimestampMins, newTimestamps, conf);
077    logList = excludeProcV2WALs(logList);
078    backupInfo.setIncrBackupFileList(logList);
079
080    return newTimestamps;
081  }
082
083  private List<String> excludeProcV2WALs(List<String> logList) {
084    List<String> list = new ArrayList<>();
085    for (int i = 0; i < logList.size(); i++) {
086      Path p = new Path(logList.get(i));
087      String name = p.getName();
088
089      if (name.startsWith(WALProcedureStore.LOG_PREFIX)) {
090        continue;
091      }
092
093      list.add(logList.get(i));
094    }
095    return list;
096  }
097
098  /**
099   * For each region server: get all log files newer than the last timestamps but not newer than the
100   * newest timestamps.
101   * @param olderTimestamps  the timestamp for each region server of the last backup.
102   * @param newestTimestamps the timestamp for each region server that the backup should lead to.
103   * @param conf             the Hadoop and Hbase configuration
104   * @return a list of log files to be backed up
105   * @throws IOException exception
106   */
107  private List<String> getLogFilesForNewBackup(Map<String, Long> olderTimestamps,
108    Map<String, Long> newestTimestamps, Configuration conf) throws IOException {
109    LOG.debug("In getLogFilesForNewBackup()\n" + "olderTimestamps: " + olderTimestamps
110      + "\n newestTimestamps: " + newestTimestamps);
111
112    long prevBackupStartTs = Collections.min(olderTimestamps.values());
113    Path walRootDir = CommonFSUtils.getWALRootDir(conf);
114    Path logDir = new Path(walRootDir, HConstants.HREGION_LOGDIR_NAME);
115    Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
116    FileSystem fs = walRootDir.getFileSystem(conf);
117    NewestLogFilter pathFilter = new NewestLogFilter();
118
119    List<String> resultLogFiles = new ArrayList<>();
120    List<String> newestLogs = new ArrayList<>();
121
122    /*
123     * The old region servers and timestamps info we kept in backup system table may be out of sync
124     * if new region server is added or existing one lost. We'll deal with it here when processing
125     * the logs. If data in backup system table has more hosts, just ignore it. If the .logs
126     * directory includes more hosts, the additional hosts will not have old timestamps to compare
127     * with. We'll just use all the logs in that directory. We always write up-to-date region server
128     * and timestamp info to backup system table at the end of successful backup.
129     */
130    FileStatus[] rss;
131    Path p;
132    String host;
133    Long oldTimeStamp;
134    String currentLogFile;
135    long currentLogTS;
136
137    // Get the files in .logs.
138    rss = fs.listStatus(logDir);
139    for (FileStatus rs : rss) {
140      p = rs.getPath();
141      host = BackupUtils.parseHostNameFromLogFile(p);
142      if (host == null) {
143        continue;
144      }
145      FileStatus[] logs;
146      oldTimeStamp = olderTimestamps.get(host);
147      // It is possible that there is no old timestamp in backup system table for this host if
148      // this region server is newly added after our last backup.
149      if (oldTimeStamp == null) {
150        logs = fs.listStatus(p);
151      } else {
152        pathFilter.setLastBackupTS(oldTimeStamp);
153        logs = fs.listStatus(p, pathFilter);
154      }
155      for (FileStatus log : logs) {
156        LOG.debug("currentLogFile: " + log.getPath().toString());
157        if (AbstractFSWALProvider.isMetaFile(log.getPath())) {
158          if (LOG.isDebugEnabled()) {
159            LOG.debug("Skip hbase:meta log file: " + log.getPath().getName());
160          }
161          continue;
162        }
163        currentLogFile = log.getPath().toString();
164        resultLogFiles.add(currentLogFile);
165        currentLogTS = BackupUtils.getCreationTime(log.getPath());
166
167        // If newestTimestamps.get(host) is null, means that
168        // either RS (host) has been restarted recently with different port number
169        // or RS is down (was decommisioned). In any case, we treat this
170        // log file as eligible for inclusion into incremental backup log list
171        Long ts = newestTimestamps.get(host);
172        if (ts == null) {
173          LOG.warn("ORPHAN log found: " + log + " host=" + host);
174          LOG.debug("Known hosts (from newestTimestamps):");
175          for (String s : newestTimestamps.keySet()) {
176            LOG.debug(s);
177          }
178        }
179        if (ts == null || currentLogTS > ts) {
180          newestLogs.add(currentLogFile);
181        }
182      }
183    }
184
185    // Include the .oldlogs files too.
186    FileStatus[] oldlogs = fs.listStatus(oldLogDir);
187    for (FileStatus oldlog : oldlogs) {
188      p = oldlog.getPath();
189      currentLogFile = p.toString();
190      if (AbstractFSWALProvider.isMetaFile(p)) {
191        if (LOG.isDebugEnabled()) {
192          LOG.debug("Skip .meta log file: " + currentLogFile);
193        }
194        continue;
195      }
196      host = BackupUtils.parseHostFromOldLog(p);
197      if (host == null) {
198        continue;
199      }
200      currentLogTS = BackupUtils.getCreationTime(p);
201      oldTimeStamp = olderTimestamps.get(host);
202      /*
203       * It is possible that there is no old timestamp in backup system table for this host. At the
204       * time of our last backup operation, this rs did not exist. The reason can be one of the two:
205       * 1. The rs already left/crashed. Its logs were moved to .oldlogs. 2. The rs was added after
206       * our last backup.
207       */
208      if (oldTimeStamp == null) {
209        if (currentLogTS < prevBackupStartTs) {
210          // This log file is really old, its region server was before our last backup.
211          continue;
212        } else {
213          resultLogFiles.add(currentLogFile);
214        }
215      } else if (currentLogTS > oldTimeStamp) {
216        resultLogFiles.add(currentLogFile);
217      }
218
219      // It is possible that a host in .oldlogs is an obsolete region server
220      // so newestTimestamps.get(host) here can be null.
221      // Even if these logs belong to a obsolete region server, we still need
222      // to include they to avoid loss of edits for backup.
223      Long newTimestamp = newestTimestamps.get(host);
224      if (newTimestamp == null || currentLogTS > newTimestamp) {
225        newestLogs.add(currentLogFile);
226      }
227    }
228    // remove newest log per host because they are still in use
229    resultLogFiles.removeAll(newestLogs);
230    return resultLogFiles;
231  }
232
233  static class NewestLogFilter implements PathFilter {
234    private Long lastBackupTS = 0L;
235
236    public NewestLogFilter() {
237    }
238
239    protected void setLastBackupTS(Long ts) {
240      this.lastBackupTS = ts;
241    }
242
243    @Override
244    public boolean accept(Path path) {
245      // skip meta table log -- ts.meta file
246      if (AbstractFSWALProvider.isMetaFile(path)) {
247        if (LOG.isDebugEnabled()) {
248          LOG.debug("Skip .meta log file: " + path.getName());
249        }
250        return false;
251      }
252      long timestamp;
253      try {
254        timestamp = BackupUtils.getCreationTime(path);
255        return timestamp > lastBackupTS;
256      } catch (Exception e) {
257        LOG.warn("Cannot read timestamp of log file " + path);
258        return false;
259      }
260    }
261  }
262}