View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.backup.impl;
20  
21  import java.io.IOException;
22  import java.util.ArrayList;
23  import java.util.Collections;
24  import java.util.HashMap;
25  import java.util.List;
26  import java.util.Map;
27  import java.util.Map.Entry;
28  import java.util.TreeMap;
29  
30  import org.apache.commons.logging.Log;
31  import org.apache.commons.logging.LogFactory;
32  import org.apache.hadoop.conf.Configuration;
33  import org.apache.hadoop.fs.FSDataInputStream;
34  import org.apache.hadoop.fs.FSDataOutputStream;
35  import org.apache.hadoop.fs.FileStatus;
36  import org.apache.hadoop.fs.FileSystem;
37  import org.apache.hadoop.fs.Path;
38  import org.apache.hadoop.hbase.HConstants;
39  import org.apache.hadoop.hbase.TableName;
40  import org.apache.hadoop.hbase.backup.BackupInfo;
41  import org.apache.hadoop.hbase.backup.BackupType;
42  import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
43  import org.apache.hadoop.hbase.classification.InterfaceAudience;
44  import org.apache.hadoop.hbase.classification.InterfaceStability;
45  import org.apache.hadoop.hbase.exceptions.DeserializationException;
46  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
47  import org.apache.hadoop.hbase.protobuf.generated.BackupProtos;
48  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
49  
50  import com.google.protobuf.InvalidProtocolBufferException;
51  
52  
53  /**
54   * Backup manifest Contains all the meta data of a backup image. The manifest info will be bundled
55   * as manifest file together with data. So that each backup image will contain all the info needed
56   * for restore.
57   */
58  @InterfaceAudience.Private
59  @InterfaceStability.Evolving
60  public class BackupManifest {
61  
62    private static final Log LOG = LogFactory.getLog(BackupManifest.class);
63  
64    // manifest file name
65    public static final String MANIFEST_FILE_NAME = ".backup.manifest";
66  
67    // manifest file version, current is 1.0
68    public static final String MANIFEST_VERSION = "1.0";
69  
70    // backup image, the dependency graph is made up by series of backup images
71  
72    public static class BackupImage implements Comparable<BackupImage> {
73  
74      private String backupId;
75      private BackupType type;
76      private String rootDir;
77      private List<TableName> tableList;
78      private long startTs;
79      private long completeTs;
80      private ArrayList<BackupImage> ancestors;
81  
82      public BackupImage() {
83        super();
84      }
85  
86      public BackupImage(String backupId, BackupType type, String rootDir,
87          List<TableName> tableList, long startTs, long completeTs) {
88        this.backupId = backupId;
89        this.type = type;
90        this.rootDir = rootDir;
91        this.tableList = tableList;
92        this.startTs = startTs;
93        this.completeTs = completeTs;
94      }
95  
96      static BackupImage fromProto(BackupProtos.BackupImage im) {
97        String backupId = im.getBackupId();
98        String rootDir = im.getRootDir();
99        long startTs = im.getStartTs();
100       long completeTs = im.getCompleteTs();
101       List<HBaseProtos.TableName> tableListList = im.getTableListList();
102       List<TableName> tableList = new ArrayList<TableName>();
103       for(HBaseProtos.TableName tn : tableListList) {
104         tableList.add(ProtobufUtil.toTableName(tn));
105       }
106       
107       List<BackupProtos.BackupImage> ancestorList = im.getAncestorsList();
108       
109       BackupType type =
110           im.getBackupType() == BackupProtos.BackupType.FULL ? BackupType.FULL:
111             BackupType.INCREMENTAL;
112 
113       BackupImage image = new BackupImage(backupId, type, rootDir, tableList, startTs, completeTs);
114       for(BackupProtos.BackupImage img: ancestorList) {
115         image.addAncestor(fromProto(img));
116       }
117       return image;
118     }
119 
120     BackupProtos.BackupImage toProto() {
121       BackupProtos.BackupImage.Builder builder = BackupProtos.BackupImage.newBuilder();
122       builder.setBackupId(backupId);
123       builder.setCompleteTs(completeTs);
124       builder.setStartTs(startTs);
125       builder.setRootDir(rootDir);
126       if (type == BackupType.FULL) {
127         builder.setBackupType(BackupProtos.BackupType.FULL);
128       } else{
129         builder.setBackupType(BackupProtos.BackupType.INCREMENTAL);
130       }
131 
132       for (TableName name: tableList) {
133         builder.addTableList(ProtobufUtil.toProtoTableName(name));
134       }
135 
136       if (ancestors != null){
137         for (BackupImage im: ancestors){
138           builder.addAncestors(im.toProto());
139         }
140       }
141 
142       return builder.build();
143     }
144 
145     public String getBackupId() {
146       return backupId;
147     }
148 
149     public void setBackupId(String backupId) {
150       this.backupId = backupId;
151     }
152 
153     public BackupType getType() {
154       return type;
155     }
156 
157     public void setType(BackupType type) {
158       this.type = type;
159     }
160 
161     public String getRootDir() {
162       return rootDir;
163     }
164 
165     public void setRootDir(String rootDir) {
166       this.rootDir = rootDir;
167     }
168 
169     public List<TableName> getTableNames() {
170       return tableList;
171     }
172 
173     public void setTableList(List<TableName> tableList) {
174       this.tableList = tableList;
175     }
176 
177     public long getStartTs() {
178       return startTs;
179     }
180 
181     public void setStartTs(long startTs) {
182       this.startTs = startTs;
183     }
184 
185     public long getCompleteTs() {
186       return completeTs;
187     }
188 
189     public void setCompleteTs(long completeTs) {
190       this.completeTs = completeTs;
191     }
192 
193     public ArrayList<BackupImage> getAncestors() {
194       if (this.ancestors == null) {
195         this.ancestors = new ArrayList<BackupImage>();
196       }
197       return this.ancestors;
198     }
199 
200     public void addAncestor(BackupImage backupImage) {
201       this.getAncestors().add(backupImage);
202     }
203 
204     public boolean hasAncestor(String token) {
205       for (BackupImage image : this.getAncestors()) {
206         if (image.getBackupId().equals(token)) {
207           return true;
208         }
209       }
210       return false;
211     }
212 
213     public boolean hasTable(TableName table) {
214       for (TableName t : tableList) {
215         if (t.equals(table)) {
216           return true;
217         }
218       }
219       return false;
220     }
221 
222     @Override
223     public int compareTo(BackupImage other) {
224       String thisBackupId = this.getBackupId();
225       String otherBackupId = other.getBackupId();
226       Long thisTS = new Long(thisBackupId.substring(thisBackupId.lastIndexOf("_") + 1));
227       Long otherTS = new Long(otherBackupId.substring(otherBackupId.lastIndexOf("_") + 1));
228       return thisTS.compareTo(otherTS);
229     }
230   }
231 
232   // manifest version
233   private String version = MANIFEST_VERSION;
234 
235   // hadoop hbase configuration
236   protected Configuration config = null;
237 
238   // backup root directory
239   private String rootDir = null;
240 
241   // backup image directory
242   private String tableBackupDir = null;
243 
244   // backup log directory if this is an incremental backup
245   private String logBackupDir = null;
246 
247   // backup token
248   private String backupId;
249 
250   // backup type, full or incremental
251   private BackupType type;
252 
253   // the table list for the backup
254   private ArrayList<TableName> tableList;
255 
256   // actual start timestamp of the backup process
257   private long startTs;
258 
259   // actual complete timestamp of the backup process
260   private long completeTs;
261 
262   // the region server timestamp for tables:
263   // <table, <rs, timestamp>>
264   private Map<TableName, HashMap<String, Long>> incrTimeRanges;
265 
266   // dependency of this backup, including all the dependent images to do PIT recovery
267   private Map<String, BackupImage> dependency;
268   
269   /**
270    * Construct manifest for a ongoing backup.
271    * @param backupCtx The ongoing backup context
272    */
273   public BackupManifest(BackupInfo backupCtx) {
274     this.backupId = backupCtx.getBackupId();
275     this.type = backupCtx.getType();
276     this.rootDir = backupCtx.getTargetRootDir();
277     if (this.type == BackupType.INCREMENTAL) {
278       this.logBackupDir = backupCtx.getHLogTargetDir();
279     }
280     this.startTs = backupCtx.getStartTs();
281     this.completeTs = backupCtx.getEndTs();
282     this.loadTableList(backupCtx.getTableNames());
283   }
284   
285   
286   /**
287    * Construct a table level manifest for a backup of the named table.
288    * @param backupCtx The ongoing backup context
289    */
290   public BackupManifest(BackupInfo backupCtx, TableName table) {
291     this.backupId = backupCtx.getBackupId();
292     this.type = backupCtx.getType();
293     this.rootDir = backupCtx.getTargetRootDir();
294     this.tableBackupDir = backupCtx.getBackupStatus(table).getTargetDir();
295     if (this.type == BackupType.INCREMENTAL) {
296       this.logBackupDir = backupCtx.getHLogTargetDir();
297     }
298     this.startTs = backupCtx.getStartTs();
299     this.completeTs = backupCtx.getEndTs();
300     List<TableName> tables = new ArrayList<TableName>();
301     tables.add(table);
302     this.loadTableList(tables);
303   }
304 
305   /**
306    * Construct manifest from a backup directory.
307    * @param conf configuration
308    * @param backupPath backup path
309    * @throws BackupException exception
310    */
311 
312   public BackupManifest(Configuration conf, Path backupPath) throws BackupException {
313     if (LOG.isDebugEnabled()) {
314       LOG.debug("Loading manifest from: " + backupPath.toString());
315     }
316     // The input backupDir may not exactly be the backup table dir.
317     // It could be the backup log dir where there is also a manifest file stored.
318     // This variable's purpose is to keep the correct and original location so
319     // that we can store/persist it.
320     this.tableBackupDir = backupPath.toString();
321     this.config = conf;
322     try {
323 
324       FileSystem fs = backupPath.getFileSystem(conf);
325       FileStatus[] subFiles = BackupClientUtil.listStatus(fs, backupPath, null);
326       if (subFiles == null) {
327         String errorMsg = backupPath.toString() + " does not exist";
328         LOG.error(errorMsg);
329         throw new IOException(errorMsg);
330       }
331       for (FileStatus subFile : subFiles) {
332         if (subFile.getPath().getName().equals(MANIFEST_FILE_NAME)) {
333 
334           // load and set manifest field from file content
335           FSDataInputStream in = fs.open(subFile.getPath());
336           long len = subFile.getLen();
337           byte[] pbBytes = new byte[(int) len];
338           in.readFully(pbBytes);
339           BackupProtos.BackupManifest proto = null;
340           try{
341             proto = parseFrom(pbBytes);
342           } catch(Exception e){
343             throw new BackupException(e);
344           }
345           this.version = proto.getVersion();
346           this.backupId = proto.getBackupId();
347           this.type = BackupType.valueOf(proto.getType().name());
348           // Here the parameter backupDir is where the manifest file is.
349           // There should always be a manifest file under:
350           // backupRootDir/namespace/table/backupId/.backup.manifest
351           this.rootDir = backupPath.getParent().getParent().getParent().toString();
352 
353           Path p = backupPath.getParent();
354           if (p.getName().equals(HConstants.HREGION_LOGDIR_NAME)) {
355             this.rootDir = p.getParent().toString();
356           } else {
357             this.rootDir = p.getParent().getParent().toString();
358           }
359 
360           loadTableList(proto);
361           this.startTs = proto.getStartTs();
362           this.completeTs = proto.getCompleteTs();
363           loadIncrementalTimestampMap(proto);
364           loadDependency(proto);
365           //TODO: merge will be implemented by future jira
366           LOG.debug("Loaded manifest instance from manifest file: "
367               + BackupClientUtil.getPath(subFile.getPath()));
368           return;
369         }
370       }
371       String errorMsg = "No manifest file found in: " + backupPath.toString();
372       throw new IOException(errorMsg);
373 
374     } catch (IOException e) {
375       throw new BackupException(e.getMessage());
376     }
377   }
378 
379   private void loadIncrementalTimestampMap(BackupProtos.BackupManifest proto) {
380     List<BackupProtos.TableServerTimestamp> list = proto.getTstMapList();
381     if(list == null || list.size() == 0) return;
382     this.incrTimeRanges = new HashMap<TableName, HashMap<String, Long>>();
383     for(BackupProtos.TableServerTimestamp tst: list){
384       TableName tn = ProtobufUtil.toTableName(tst.getTable());
385       HashMap<String, Long> map = this.incrTimeRanges.get(tn);
386       if(map == null){
387         map = new HashMap<String, Long>();
388         this.incrTimeRanges.put(tn, map);
389       }
390       List<BackupProtos.ServerTimestamp> listSt = tst.getServerTimestampList();
391       for(BackupProtos.ServerTimestamp stm: listSt) {
392         map.put(stm.getServer(), stm.getTimestamp());
393       }
394     }
395   }
396 
397   private void loadDependency(BackupProtos.BackupManifest proto) {
398     if(LOG.isDebugEnabled()) {
399       LOG.debug("load dependency for: "+proto.getBackupId());
400     }
401 
402     dependency = new HashMap<String, BackupImage>();
403     List<BackupProtos.BackupImage> list = proto.getDependentBackupImageList();
404     for (BackupProtos.BackupImage im : list) {
405       BackupImage bim = BackupImage.fromProto(im);
406       if(im.getBackupId() != null){
407         dependency.put(im.getBackupId(), bim);
408       } else{
409         LOG.warn("Load dependency for backup manifest: "+ backupId+ 
410           ". Null backup id in dependent image");
411       }
412     }
413   }
414 
415   private void loadTableList(BackupProtos.BackupManifest proto) {
416     this.tableList = new ArrayList<TableName>();
417     List<HBaseProtos.TableName> list = proto.getTableListList();
418     for (HBaseProtos.TableName name: list) {
419       this.tableList.add(ProtobufUtil.toTableName(name));
420     }
421   }
422 
423   public BackupType getType() {
424     return type;
425   }
426 
427   public void setType(BackupType type) {
428     this.type = type;
429   }
430 
431   /**
432    * Loads table list.
433    * @param tableList Table list
434    */
435   private void loadTableList(List<TableName> tableList) {
436 
437     this.tableList = this.getTableList();
438     if (this.tableList.size() > 0) {
439       this.tableList.clear();
440     }
441     for (int i = 0; i < tableList.size(); i++) {
442       this.tableList.add(tableList.get(i));
443     }
444 
445     LOG.debug(tableList.size() + " tables exist in table set.");
446   }
447 
448   /**
449    * Get the table set of this image.
450    * @return The table set list
451    */
452   public ArrayList<TableName> getTableList() {
453     if (this.tableList == null) {
454       this.tableList = new ArrayList<TableName>();
455     }
456     return this.tableList;
457   }
458 
459   /**
460    * Persist the manifest file.
461    * @throws IOException IOException when storing the manifest file.
462    */
463 
464   public void store(Configuration conf) throws BackupException {
465     byte[] data = toByteArray();
466 
467     // write the file, overwrite if already exist
468     Path manifestFilePath =
469         new Path(new Path((this.tableBackupDir != null ? this.tableBackupDir : this.logBackupDir))
470             ,MANIFEST_FILE_NAME);
471     try {
472       FSDataOutputStream out =
473           manifestFilePath.getFileSystem(conf).create(manifestFilePath, true);
474       out.write(data);
475       out.close();
476     } catch (IOException e) {      
477       throw new BackupException(e.getMessage());
478     }
479 
480     LOG.info("Manifest file stored to " + manifestFilePath);
481   }
482 
483   /**
484    * Protobuf serialization
485    * @return The filter serialized using pb
486    */
487   public byte[] toByteArray() {
488     BackupProtos.BackupManifest.Builder builder = BackupProtos.BackupManifest.newBuilder();
489     builder.setVersion(this.version);
490     builder.setBackupId(this.backupId);
491     builder.setType(BackupProtos.BackupType.valueOf(this.type.name()));
492     setTableList(builder);
493     builder.setStartTs(this.startTs);
494     builder.setCompleteTs(this.completeTs);
495     setIncrementalTimestampMap(builder);
496     setDependencyMap(builder);
497     return builder.build().toByteArray();
498   }
499 
500   private void setIncrementalTimestampMap(BackupProtos.BackupManifest.Builder builder) {
501     if (this.incrTimeRanges == null) {
502       return;
503     }
504     for (Entry<TableName, HashMap<String,Long>> entry: this.incrTimeRanges.entrySet()) {
505       TableName key = entry.getKey();
506       HashMap<String, Long> value = entry.getValue();
507       BackupProtos.TableServerTimestamp.Builder tstBuilder =
508           BackupProtos.TableServerTimestamp.newBuilder();
509       tstBuilder.setTable(ProtobufUtil.toProtoTableName(key));
510 
511       for (String s : value.keySet()) {
512         BackupProtos.ServerTimestamp.Builder stBuilder = BackupProtos.ServerTimestamp.newBuilder();
513         stBuilder.setServer(s);
514         stBuilder.setTimestamp(value.get(s));
515         tstBuilder.addServerTimestamp(stBuilder.build());
516       }
517       builder.addTstMap(tstBuilder.build());
518     }
519   }
520 
521   private void setDependencyMap(BackupProtos.BackupManifest.Builder builder) {
522     for (BackupImage image: getDependency().values()) {
523       builder.addDependentBackupImage(image.toProto());
524     }
525   }
526 
527   private void setTableList(BackupProtos.BackupManifest.Builder builder) {
528     for(TableName name: tableList){
529       builder.addTableList(ProtobufUtil.toProtoTableName(name));
530     }
531   }
532 
533   /**
534    * Parse protobuf from byte array
535    * @param pbBytes A pb serialized BackupManifest instance
536    * @return An instance of  made from <code>bytes</code>
537    * @throws DeserializationException
538    */
539   private static BackupProtos.BackupManifest parseFrom(final byte[] pbBytes)
540       throws DeserializationException {
541     BackupProtos.BackupManifest proto;
542     try {
543       proto = BackupProtos.BackupManifest.parseFrom(pbBytes);
544     } catch (InvalidProtocolBufferException e) {
545       throw new DeserializationException(e);
546     }
547     return proto;
548   }
549 
550   /**
551    * Get manifest file version
552    * @return version
553    */
554   public String getVersion() {
555     return version;
556   }
557 
558   /**
559    * Get this backup image.
560    * @return the backup image.
561    */
562   public BackupImage getBackupImage() {
563     return this.getDependency().get(this.backupId);
564   }
565 
566   /**
567    * Add dependent backup image for this backup.
568    * @param image The direct dependent backup image
569    */
570   public void addDependentImage(BackupImage image) {
571     this.getDependency().get(this.backupId).addAncestor(image);
572     this.setDependencyMap(this.getDependency(), image);
573   }
574 
575 
576 
577   /**
578    * Get all dependent backup images. The image of this backup is also contained.
579    * @return The dependent backup images map
580    */
581   public Map<String, BackupImage> getDependency() {
582     if (this.dependency == null) {
583       this.dependency = new HashMap<String, BackupImage>();
584       LOG.debug(this.rootDir + " " + this.backupId + " " + this.type);
585       this.dependency.put(this.backupId,
586         new BackupImage(this.backupId, this.type, this.rootDir, tableList, this.startTs,
587             this.completeTs));
588     }
589     return this.dependency;
590   }
591 
592   /**
593    * Set the incremental timestamp map directly.
594    * @param incrTimestampMap timestamp map
595    */
596   public void setIncrTimestampMap(HashMap<TableName, HashMap<String, Long>> incrTimestampMap) {
597     this.incrTimeRanges = incrTimestampMap;
598   }
599 
600 
601   public Map<TableName, HashMap<String, Long>> getIncrTimestampMap() {
602     if (this.incrTimeRanges == null) {
603       this.incrTimeRanges = new HashMap<TableName, HashMap<String, Long>>();
604     }
605     return this.incrTimeRanges;
606   }
607 
608 
609   /**
610    * Get the image list of this backup for restore in time order.
611    * @param reverse If true, then output in reverse order, otherwise in time order from old to new
612    * @return the backup image list for restore in time order
613    */
614   public ArrayList<BackupImage> getRestoreDependentList(boolean reverse) {
615     TreeMap<Long, BackupImage> restoreImages = new TreeMap<Long, BackupImage>();
616     for (BackupImage image : this.getDependency().values()) {
617       restoreImages.put(Long.valueOf(image.startTs), image);
618     }
619     return new ArrayList<BackupImage>(reverse ? (restoreImages.descendingMap().values())
620         : (restoreImages.values()));
621   }
622 
623   /**
624    * Get the dependent image list for a specific table of this backup in time order from old to new
625    * if want to restore to this backup image level.
626    * @param table table
627    * @return the backup image list for a table in time order
628    */
629   public ArrayList<BackupImage> getDependentListByTable(TableName table) {
630     ArrayList<BackupImage> tableImageList = new ArrayList<BackupImage>();
631     ArrayList<BackupImage> imageList = getRestoreDependentList(true);
632     for (BackupImage image : imageList) {
633       if (image.hasTable(table)) {
634         tableImageList.add(image);
635         if (image.getType() == BackupType.FULL) {
636           break;
637         }
638       }
639     }
640     Collections.reverse(tableImageList);
641     return tableImageList;
642   }
643 
644   /**
645    * Get the full dependent image list in the whole dependency scope for a specific table of this
646    * backup in time order from old to new.
647    * @param table table
648    * @return the full backup image list for a table in time order in the whole scope of the
649    *         dependency of this image
650    */
651   public ArrayList<BackupImage> getAllDependentListByTable(TableName table) {
652     ArrayList<BackupImage> tableImageList = new ArrayList<BackupImage>();
653     ArrayList<BackupImage> imageList = getRestoreDependentList(false);
654     for (BackupImage image : imageList) {
655       if (image.hasTable(table)) {
656         tableImageList.add(image);
657       }
658     }
659     return tableImageList;
660   }
661 
662 
663   /**
664    * Recursively set the dependency map of the backup images.
665    * @param map The dependency map
666    * @param image The backup image
667    */
668   private void setDependencyMap(Map<String, BackupImage> map, BackupImage image) {
669     if (image == null) {
670       return;
671     } else {
672       map.put(image.getBackupId(), image);
673       for (BackupImage img : image.getAncestors()) {
674         setDependencyMap(map, img);
675       }
676     }
677   }
678 
679   /**
680    * Check whether backup image1 could cover backup image2 or not.
681    * @param image1 backup image 1
682    * @param image2 backup image 2
683    * @return true if image1 can cover image2, otherwise false
684    */
685   public static boolean canCoverImage(BackupImage image1, BackupImage image2) {
686     // image1 can cover image2 only when the following conditions are satisfied:
687     // - image1 must not be an incremental image;
688     // - image1 must be taken after image2 has been taken;
689     // - table set of image1 must cover the table set of image2.
690     if (image1.getType() == BackupType.INCREMENTAL) {
691       return false;
692     }
693     if (image1.getStartTs() < image2.getStartTs()) {
694       return false;
695     }
696     List<TableName> image1TableList = image1.getTableNames();
697     List<TableName> image2TableList = image2.getTableNames();
698     boolean found = false;
699     for (int i = 0; i < image2TableList.size(); i++) {
700       found = false;
701       for (int j = 0; j < image1TableList.size(); j++) {
702         if (image2TableList.get(i).equals(image1TableList.get(j))) {
703           found = true;
704           break;
705         }
706       }
707       if (!found) {
708         return false;
709       }
710     }
711 
712     LOG.debug("Backup image " + image1.getBackupId() + " can cover " + image2.getBackupId());
713     return true;
714   }
715 
716   /**
717    * Check whether backup image set could cover a backup image or not.
718    * @param fullImages The backup image set
719    * @param image The target backup image
720    * @return true if fullImages can cover image, otherwise false
721    */
722   public static boolean canCoverImage(ArrayList<BackupImage> fullImages, BackupImage image) {
723     // fullImages can cover image only when the following conditions are satisfied:
724     // - each image of fullImages must not be an incremental image;
725     // - each image of fullImages must be taken after image has been taken;
726     // - sum table set of fullImages must cover the table set of image.
727     for (BackupImage image1 : fullImages) {
728       if (image1.getType() == BackupType.INCREMENTAL) {
729         return false;
730       }
731       if (image1.getStartTs() < image.getStartTs()) {
732         return false;
733       }
734     }
735 
736     ArrayList<String> image1TableList = new ArrayList<String>();
737     for (BackupImage image1 : fullImages) {
738       List<TableName> tableList = image1.getTableNames();
739       for (TableName table : tableList) {
740         image1TableList.add(table.getNameAsString());
741       }
742     }
743     ArrayList<String> image2TableList = new ArrayList<String>();
744     List<TableName> tableList = image.getTableNames();
745     for (TableName table : tableList) {
746       image2TableList.add(table.getNameAsString());
747     }
748 
749     for (int i = 0; i < image2TableList.size(); i++) {
750       if (image1TableList.contains(image2TableList.get(i)) == false) {
751         return false;
752       }
753     }
754 
755     LOG.debug("Full image set can cover image " + image.getBackupId());
756     return true;
757   }
758 }