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.HashMap;
024import java.util.List;
025import java.util.Map;
026import java.util.Map.Entry;
027import java.util.TreeMap;
028import org.apache.hadoop.conf.Configuration;
029import org.apache.hadoop.fs.FSDataInputStream;
030import org.apache.hadoop.fs.FSDataOutputStream;
031import org.apache.hadoop.fs.FileStatus;
032import org.apache.hadoop.fs.FileSystem;
033import org.apache.hadoop.fs.Path;
034import org.apache.hadoop.hbase.ServerName;
035import org.apache.hadoop.hbase.TableName;
036import org.apache.hadoop.hbase.backup.BackupInfo;
037import org.apache.hadoop.hbase.backup.BackupType;
038import org.apache.hadoop.hbase.backup.HBackupFileSystem;
039import org.apache.hadoop.hbase.backup.util.BackupUtils;
040import org.apache.yetus.audience.InterfaceAudience;
041import org.slf4j.Logger;
042import org.slf4j.LoggerFactory;
043
044import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
045import org.apache.hadoop.hbase.shaded.protobuf.generated.BackupProtos;
046import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
047
048/**
049 * Backup manifest contains all the meta data of a backup image. The manifest info will be bundled
050 * as manifest file together with data. So that each backup image will contain all the info needed
051 * for restore. BackupManifest is a storage container for BackupImage. It is responsible for
052 * storing/reading backup image data and has some additional utility methods.
053 */
054@InterfaceAudience.Private
055public class BackupManifest {
056  private static final Logger LOG = LoggerFactory.getLogger(BackupManifest.class);
057
058  // manifest file name
059  public static final String MANIFEST_FILE_NAME = ".backup.manifest";
060
061  /**
062   * Backup image, the dependency graph is made up by series of backup images BackupImage contains
063   * all the relevant information to restore the backup and is used during restore operation
064   */
065  public static class BackupImage implements Comparable<BackupImage> {
066    static class Builder {
067      BackupImage image;
068
069      Builder() {
070        image = new BackupImage();
071      }
072
073      Builder withBackupId(String backupId) {
074        image.setBackupId(backupId);
075        return this;
076      }
077
078      Builder withType(BackupType type) {
079        image.setType(type);
080        return this;
081      }
082
083      Builder withRootDir(String rootDir) {
084        image.setRootDir(rootDir);
085        return this;
086      }
087
088      Builder withTableList(List<TableName> tableList) {
089        image.setTableList(tableList);
090        return this;
091      }
092
093      Builder withStartTime(long startTime) {
094        image.setStartTs(startTime);
095        return this;
096      }
097
098      Builder withCompleteTime(long completeTime) {
099        image.setCompleteTs(completeTime);
100        return this;
101      }
102
103      BackupImage build() {
104        return image;
105      }
106
107    }
108
109    private String backupId;
110    private BackupType type;
111    private String rootDir;
112    private List<TableName> tableList;
113    private long startTs;
114    private long completeTs;
115    private ArrayList<BackupImage> ancestors;
116    private Map<TableName, Map<String, Long>> incrTimeRanges;
117
118    static Builder newBuilder() {
119      return new Builder();
120    }
121
122    public BackupImage() {
123      super();
124    }
125
126    private BackupImage(String backupId, BackupType type, String rootDir, List<TableName> tableList,
127      long startTs, long completeTs) {
128      this.backupId = backupId;
129      this.type = type;
130      this.rootDir = rootDir;
131      this.tableList = tableList;
132      this.startTs = startTs;
133      this.completeTs = completeTs;
134    }
135
136    static BackupImage fromProto(BackupProtos.BackupImage im) {
137      String backupId = im.getBackupId();
138      String rootDir = im.getBackupRootDir();
139      long startTs = im.getStartTs();
140      long completeTs = im.getCompleteTs();
141      List<HBaseProtos.TableName> tableListList = im.getTableListList();
142      List<TableName> tableList = new ArrayList<>();
143      for (HBaseProtos.TableName tn : tableListList) {
144        tableList.add(ProtobufUtil.toTableName(tn));
145      }
146
147      List<BackupProtos.BackupImage> ancestorList = im.getAncestorsList();
148
149      BackupType type = im.getBackupType() == BackupProtos.BackupType.FULL
150        ? BackupType.FULL
151        : BackupType.INCREMENTAL;
152
153      BackupImage image = new BackupImage(backupId, type, rootDir, tableList, startTs, completeTs);
154      for (BackupProtos.BackupImage img : ancestorList) {
155        image.addAncestor(fromProto(img));
156      }
157      image.setIncrTimeRanges(loadIncrementalTimestampMap(im));
158      return image;
159    }
160
161    BackupProtos.BackupImage toProto() {
162      BackupProtos.BackupImage.Builder builder = BackupProtos.BackupImage.newBuilder();
163      builder.setBackupId(backupId);
164      builder.setCompleteTs(completeTs);
165      builder.setStartTs(startTs);
166      builder.setBackupRootDir(rootDir);
167      if (type == BackupType.FULL) {
168        builder.setBackupType(BackupProtos.BackupType.FULL);
169      } else {
170        builder.setBackupType(BackupProtos.BackupType.INCREMENTAL);
171      }
172
173      for (TableName name : tableList) {
174        builder.addTableList(ProtobufUtil.toProtoTableName(name));
175      }
176
177      if (ancestors != null) {
178        for (BackupImage im : ancestors) {
179          builder.addAncestors(im.toProto());
180        }
181      }
182
183      setIncrementalTimestampMap(builder);
184      return builder.build();
185    }
186
187    private static Map<TableName, Map<String, Long>>
188      loadIncrementalTimestampMap(BackupProtos.BackupImage proto) {
189      List<BackupProtos.TableServerTimestamp> list = proto.getTstMapList();
190
191      Map<TableName, Map<String, Long>> incrTimeRanges = new HashMap<>();
192
193      if (list == null || list.size() == 0) {
194        return incrTimeRanges;
195      }
196
197      for (BackupProtos.TableServerTimestamp tst : list) {
198        TableName tn = ProtobufUtil.toTableName(tst.getTableName());
199        Map<String, Long> map = incrTimeRanges.get(tn);
200        if (map == null) {
201          map = new HashMap<>();
202          incrTimeRanges.put(tn, map);
203        }
204        List<BackupProtos.ServerTimestamp> listSt = tst.getServerTimestampList();
205        for (BackupProtos.ServerTimestamp stm : listSt) {
206          ServerName sn = ProtobufUtil.toServerName(stm.getServerName());
207          map.put(sn.getHostname() + ":" + sn.getPort(), stm.getTimestamp());
208        }
209      }
210      return incrTimeRanges;
211    }
212
213    private void setIncrementalTimestampMap(BackupProtos.BackupImage.Builder builder) {
214      if (this.incrTimeRanges == null) {
215        return;
216      }
217      for (Entry<TableName, Map<String, Long>> entry : this.incrTimeRanges.entrySet()) {
218        TableName key = entry.getKey();
219        Map<String, Long> value = entry.getValue();
220        BackupProtos.TableServerTimestamp.Builder tstBuilder =
221          BackupProtos.TableServerTimestamp.newBuilder();
222        tstBuilder.setTableName(ProtobufUtil.toProtoTableName(key));
223
224        for (Map.Entry<String, Long> entry2 : value.entrySet()) {
225          String s = entry2.getKey();
226          BackupProtos.ServerTimestamp.Builder stBuilder =
227            BackupProtos.ServerTimestamp.newBuilder();
228          HBaseProtos.ServerName.Builder snBuilder = HBaseProtos.ServerName.newBuilder();
229          ServerName sn = ServerName.parseServerName(s);
230          snBuilder.setHostName(sn.getHostname());
231          snBuilder.setPort(sn.getPort());
232          stBuilder.setServerName(snBuilder.build());
233          stBuilder.setTimestamp(entry2.getValue());
234          tstBuilder.addServerTimestamp(stBuilder.build());
235        }
236        builder.addTstMap(tstBuilder.build());
237      }
238    }
239
240    public String getBackupId() {
241      return backupId;
242    }
243
244    private void setBackupId(String backupId) {
245      this.backupId = backupId;
246    }
247
248    public BackupType getType() {
249      return type;
250    }
251
252    private void setType(BackupType type) {
253      this.type = type;
254    }
255
256    public String getRootDir() {
257      return rootDir;
258    }
259
260    private void setRootDir(String rootDir) {
261      this.rootDir = rootDir;
262    }
263
264    public List<TableName> getTableNames() {
265      return tableList;
266    }
267
268    private void setTableList(List<TableName> tableList) {
269      this.tableList = tableList;
270    }
271
272    public long getStartTs() {
273      return startTs;
274    }
275
276    private void setStartTs(long startTs) {
277      this.startTs = startTs;
278    }
279
280    public long getCompleteTs() {
281      return completeTs;
282    }
283
284    private void setCompleteTs(long completeTs) {
285      this.completeTs = completeTs;
286    }
287
288    public ArrayList<BackupImage> getAncestors() {
289      if (this.ancestors == null) {
290        this.ancestors = new ArrayList<>();
291      }
292      return this.ancestors;
293    }
294
295    public void removeAncestors(List<String> backupIds) {
296      List<BackupImage> toRemove = new ArrayList<>();
297      for (BackupImage im : this.ancestors) {
298        if (backupIds.contains(im.getBackupId())) {
299          toRemove.add(im);
300        }
301      }
302      this.ancestors.removeAll(toRemove);
303    }
304
305    private void addAncestor(BackupImage backupImage) {
306      this.getAncestors().add(backupImage);
307    }
308
309    public boolean hasAncestor(String token) {
310      for (BackupImage image : this.getAncestors()) {
311        if (image.getBackupId().equals(token)) {
312          return true;
313        }
314      }
315      return false;
316    }
317
318    public boolean hasTable(TableName table) {
319      return tableList.contains(table);
320    }
321
322    @Override
323    public int compareTo(BackupImage other) {
324      String thisBackupId = this.getBackupId();
325      String otherBackupId = other.getBackupId();
326      int index1 = thisBackupId.lastIndexOf("_");
327      int index2 = otherBackupId.lastIndexOf("_");
328      String name1 = thisBackupId.substring(0, index1);
329      String name2 = otherBackupId.substring(0, index2);
330      if (name1.equals(name2)) {
331        Long thisTS = Long.valueOf(thisBackupId.substring(index1 + 1));
332        Long otherTS = Long.valueOf(otherBackupId.substring(index2 + 1));
333        return thisTS.compareTo(otherTS);
334      } else {
335        return name1.compareTo(name2);
336      }
337    }
338
339    @Override
340    public boolean equals(Object obj) {
341      if (obj instanceof BackupImage) {
342        return this.compareTo((BackupImage) obj) == 0;
343      }
344      return false;
345    }
346
347    @Override
348    public int hashCode() {
349      int hash = 33 * this.getBackupId().hashCode() + type.hashCode();
350      hash = 33 * hash + rootDir.hashCode();
351      hash = 33 * hash + Long.valueOf(startTs).hashCode();
352      hash = 33 * hash + Long.valueOf(completeTs).hashCode();
353      for (TableName table : tableList) {
354        hash = 33 * hash + table.hashCode();
355      }
356      return hash;
357    }
358
359    public Map<TableName, Map<String, Long>> getIncrTimeRanges() {
360      return incrTimeRanges;
361    }
362
363    private void setIncrTimeRanges(Map<TableName, Map<String, Long>> incrTimeRanges) {
364      this.incrTimeRanges = incrTimeRanges;
365    }
366  }
367
368  // backup image directory
369  private BackupImage backupImage;
370
371  /**
372   * Construct manifest for a ongoing backup.
373   * @param backup The ongoing backup info
374   */
375  public BackupManifest(BackupInfo backup) {
376    BackupImage.Builder builder = BackupImage.newBuilder();
377    this.backupImage = builder.withBackupId(backup.getBackupId()).withType(backup.getType())
378      .withRootDir(backup.getBackupRootDir()).withTableList(backup.getTableNames())
379      .withStartTime(backup.getStartTs()).withCompleteTime(backup.getCompleteTs()).build();
380  }
381
382  /**
383   * Construct a table level manifest for a backup of the named table.
384   * @param backup The ongoing backup session info
385   */
386  public BackupManifest(BackupInfo backup, TableName table) {
387    List<TableName> tables = new ArrayList<TableName>();
388    tables.add(table);
389    BackupImage.Builder builder = BackupImage.newBuilder();
390    this.backupImage = builder.withBackupId(backup.getBackupId()).withType(backup.getType())
391      .withRootDir(backup.getBackupRootDir()).withTableList(tables)
392      .withStartTime(backup.getStartTs()).withCompleteTime(backup.getCompleteTs()).build();
393  }
394
395  /**
396   * Construct manifest from a backup directory.
397   * @param conf       configuration
398   * @param backupPath backup path
399   * @throws IOException if constructing the manifest from the backup directory fails
400   */
401  public BackupManifest(Configuration conf, Path backupPath) throws IOException {
402    this(backupPath.getFileSystem(conf), backupPath);
403  }
404
405  /**
406   * Construct manifest from a backup directory.
407   * @param fs         the FileSystem
408   * @param backupPath backup path
409   * @throws BackupException exception
410   */
411  public BackupManifest(FileSystem fs, Path backupPath) throws BackupException {
412    if (LOG.isDebugEnabled()) {
413      LOG.debug("Loading manifest from: " + backupPath.toString());
414    }
415    // The input backupDir may not exactly be the backup table dir.
416    // It could be the backup log dir where there is also a manifest file stored.
417    // This variable's purpose is to keep the correct and original location so
418    // that we can store/persist it.
419    try {
420      FileStatus[] subFiles = BackupUtils.listStatus(fs, backupPath, null);
421      if (subFiles == null) {
422        String errorMsg = backupPath.toString() + " does not exist";
423        LOG.error(errorMsg);
424        throw new IOException(errorMsg);
425      }
426      for (FileStatus subFile : subFiles) {
427        if (subFile.getPath().getName().equals(MANIFEST_FILE_NAME)) {
428          // load and set manifest field from file content
429          long len = subFile.getLen();
430          byte[] pbBytes = new byte[(int) len];
431          try (FSDataInputStream in = fs.open(subFile.getPath())) {
432            in.readFully(pbBytes);
433          } catch (IOException e) {
434            throw new BackupException(e.getMessage());
435          }
436          BackupProtos.BackupImage proto = null;
437          try {
438            proto = BackupProtos.BackupImage.parseFrom(pbBytes);
439          } catch (Exception e) {
440            throw new BackupException(e);
441          }
442          this.backupImage = BackupImage.fromProto(proto);
443          LOG.debug("Loaded manifest instance from manifest file: "
444            + BackupUtils.getPath(subFile.getPath()));
445          return;
446        }
447      }
448      String errorMsg = "No manifest file found in: " + backupPath.toString();
449      throw new IOException(errorMsg);
450    } catch (IOException e) {
451      throw new BackupException(e.getMessage());
452    }
453  }
454
455  public BackupType getType() {
456    return backupImage.getType();
457  }
458
459  /**
460   * Get the table set of this image.
461   * @return The table set list
462   */
463  public List<TableName> getTableList() {
464    return backupImage.getTableNames();
465  }
466
467  /**
468   * Persist the manifest file.
469   * @throws BackupException if an error occurred while storing the manifest file.
470   */
471  public void store(Configuration conf) throws BackupException {
472    byte[] data = backupImage.toProto().toByteArray();
473    // write the file, overwrite if already exist
474    Path manifestFilePath =
475      new Path(HBackupFileSystem.getBackupPath(backupImage.getRootDir(), backupImage.getBackupId()),
476        MANIFEST_FILE_NAME);
477    try (FSDataOutputStream out =
478      manifestFilePath.getFileSystem(conf).create(manifestFilePath, true)) {
479      out.write(data);
480    } catch (IOException e) {
481      throw new BackupException(e.getMessage());
482    }
483
484    LOG.info("Manifest file stored to " + manifestFilePath);
485  }
486
487  /**
488   * Get this backup image.
489   * @return the backup image.
490   */
491  public BackupImage getBackupImage() {
492    return backupImage;
493  }
494
495  /**
496   * Add dependent backup image for this backup.
497   * @param image The direct dependent backup image
498   */
499  public void addDependentImage(BackupImage image) {
500    this.backupImage.addAncestor(image);
501  }
502
503  /**
504   * Set the incremental timestamp map directly.
505   * @param incrTimestampMap timestamp map
506   */
507  public void setIncrTimestampMap(Map<TableName, Map<String, Long>> incrTimestampMap) {
508    this.backupImage.setIncrTimeRanges(incrTimestampMap);
509  }
510
511  public Map<TableName, Map<String, Long>> getIncrTimestampMap() {
512    return backupImage.getIncrTimeRanges();
513  }
514
515  /**
516   * Get the image list of this backup for restore in time order.
517   * @param reverse If true, then output in reverse order, otherwise in time order from old to new
518   * @return the backup image list for restore in time order
519   */
520  public ArrayList<BackupImage> getRestoreDependentList(boolean reverse) {
521    TreeMap<Long, BackupImage> restoreImages = new TreeMap<>();
522    restoreImages.put(backupImage.startTs, backupImage);
523    for (BackupImage image : backupImage.getAncestors()) {
524      restoreImages.put(Long.valueOf(image.startTs), image);
525    }
526    return new ArrayList<>(
527      reverse ? restoreImages.descendingMap().values() : restoreImages.values());
528  }
529
530  /**
531   * Get the dependent image list for a specific table of this backup in time order from old to new
532   * if want to restore to this backup image level.
533   * @param table table
534   * @return the backup image list for a table in time order
535   */
536  public ArrayList<BackupImage> getDependentListByTable(TableName table) {
537    ArrayList<BackupImage> tableImageList = new ArrayList<>();
538    ArrayList<BackupImage> imageList = getRestoreDependentList(true);
539    for (BackupImage image : imageList) {
540      if (image.hasTable(table)) {
541        tableImageList.add(image);
542        if (image.getType() == BackupType.FULL) {
543          break;
544        }
545      }
546    }
547    Collections.reverse(tableImageList);
548    return tableImageList;
549  }
550
551  public BackupInfo toBackupInfo() {
552    BackupInfo info = new BackupInfo();
553    info.setType(backupImage.getType());
554    List<TableName> list = backupImage.getTableNames();
555    TableName[] tables = new TableName[list.size()];
556    info.addTables(list.toArray(tables));
557    info.setBackupId(backupImage.getBackupId());
558    info.setStartTs(backupImage.getStartTs());
559    info.setBackupRootDir(backupImage.getRootDir());
560    if (backupImage.getType() == BackupType.INCREMENTAL) {
561      info.setHLogTargetDir(
562        BackupUtils.getLogBackupDir(backupImage.getRootDir(), backupImage.getBackupId()));
563    }
564    return info;
565  }
566}