View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.mob.compactions;
20  
21  import java.io.FileNotFoundException;
22  import java.io.IOException;
23  import java.util.ArrayList;
24  import java.util.Collection;
25  import java.util.Collections;
26  import java.util.Date;
27  import java.util.HashMap;
28  import java.util.List;
29  import java.util.Map;
30  import java.util.Map.Entry;
31  import java.util.concurrent.Callable;
32  import java.util.concurrent.ExecutorService;
33  import java.util.concurrent.Future;
34  
35  import org.apache.commons.logging.Log;
36  import org.apache.commons.logging.LogFactory;
37  import org.apache.hadoop.hbase.classification.InterfaceAudience;
38  import org.apache.hadoop.conf.Configuration;
39  import org.apache.hadoop.fs.FileStatus;
40  import org.apache.hadoop.fs.FileSystem;
41  import org.apache.hadoop.fs.Path;
42  import org.apache.hadoop.hbase.Cell;
43  import org.apache.hadoop.hbase.HColumnDescriptor;
44  import org.apache.hadoop.hbase.HConstants;
45  import org.apache.hadoop.hbase.KeyValue;
46  import org.apache.hadoop.hbase.TableName;
47  import org.apache.hadoop.hbase.Tag;
48  import org.apache.hadoop.hbase.TagType;
49  import org.apache.hadoop.hbase.client.Connection;
50  import org.apache.hadoop.hbase.client.ConnectionFactory;
51  import org.apache.hadoop.hbase.client.HTable;
52  import org.apache.hadoop.hbase.client.Scan;
53  import org.apache.hadoop.hbase.client.Table;
54  import org.apache.hadoop.hbase.io.HFileLink;
55  import org.apache.hadoop.hbase.io.crypto.Encryption;
56  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
57  import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
58  import org.apache.hadoop.hbase.mob.MobConstants;
59  import org.apache.hadoop.hbase.mob.MobFileName;
60  import org.apache.hadoop.hbase.mob.MobUtils;
61  import org.apache.hadoop.hbase.mob.compactions.MobCompactionRequest.CompactionType;
62  import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartition;
63  import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId;
64  import org.apache.hadoop.hbase.regionserver.BloomType;
65  import org.apache.hadoop.hbase.regionserver.HStore;
66  import org.apache.hadoop.hbase.regionserver.ScanInfo;
67  import org.apache.hadoop.hbase.regionserver.ScanType;
68  import org.apache.hadoop.hbase.regionserver.ScannerContext;
69  import org.apache.hadoop.hbase.regionserver.StoreFile;
70  import org.apache.hadoop.hbase.regionserver.StoreFile.Writer;
71  import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
72  import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
73  import org.apache.hadoop.hbase.regionserver.StoreScanner;
74  import org.apache.hadoop.hbase.util.Bytes;
75  import org.apache.hadoop.hbase.util.Pair;
76  
77  /**
78   * An implementation of {@link MobCompactor} that compacts the mob files in partitions.
79   */
80  @InterfaceAudience.Private
81  public class PartitionedMobCompactor extends MobCompactor {
82  
83    private static final Log LOG = LogFactory.getLog(PartitionedMobCompactor.class);
84    protected long mergeableSize;
85    protected int delFileMaxCount;
86    /** The number of files compacted in a batch */
87    protected int compactionBatchSize;
88    protected int compactionKVMax;
89  
90    private Path tempPath;
91    private Path bulkloadPath;
92    private CacheConfig compactionCacheConfig;
93    private Tag tableNameTag;
94    private Encryption.Context cryptoContext = Encryption.Context.NONE;
95  
96    public PartitionedMobCompactor(Configuration conf, FileSystem fs, TableName tableName,
97      HColumnDescriptor column, ExecutorService pool) throws IOException {
98      super(conf, fs, tableName, column, pool);
99      mergeableSize = conf.getLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD,
100       MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD);
101     delFileMaxCount = conf.getInt(MobConstants.MOB_DELFILE_MAX_COUNT,
102       MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT);
103     // default is 100
104     compactionBatchSize = conf.getInt(MobConstants.MOB_COMPACTION_BATCH_SIZE,
105       MobConstants.DEFAULT_MOB_COMPACTION_BATCH_SIZE);
106     tempPath = new Path(MobUtils.getMobHome(conf), MobConstants.TEMP_DIR_NAME);
107     bulkloadPath = new Path(tempPath, new Path(MobConstants.BULKLOAD_DIR_NAME, new Path(
108       tableName.getNamespaceAsString(), tableName.getQualifierAsString())));
109     compactionKVMax = this.conf.getInt(HConstants.COMPACTION_KV_MAX,
110       HConstants.COMPACTION_KV_MAX_DEFAULT);
111     Configuration copyOfConf = new Configuration(conf);
112     copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f);
113     compactionCacheConfig = new CacheConfig(copyOfConf);
114     tableNameTag = new Tag(TagType.MOB_TABLE_NAME_TAG_TYPE, tableName.getName());
115     cryptoContext = MobUtils.createEncryptionContext(copyOfConf, column);
116   }
117 
118   @Override
119   public List<Path> compact(List<FileStatus> files, boolean allFiles) throws IOException {
120     if (files == null || files.isEmpty()) {
121       LOG.info("No candidate mob files");
122       return null;
123     }
124     LOG.info("is allFiles: " + allFiles);
125     // find the files to compact.
126     PartitionedMobCompactionRequest request = select(files, allFiles);
127     // compact the files.
128     return performCompaction(request);
129   }
130 
131   /**
132    * Selects the compacted mob/del files.
133    * Iterates the candidates to find out all the del files and small mob files.
134    * @param candidates All the candidates.
135    * @param allFiles Whether add all mob files into the compaction.
136    * @return A compaction request.
137    * @throws IOException
138    */
139   protected PartitionedMobCompactionRequest select(List<FileStatus> candidates,
140     boolean allFiles) throws IOException {
141     Collection<FileStatus> allDelFiles = new ArrayList<FileStatus>();
142     Map<CompactionPartitionId, CompactionPartition> filesToCompact =
143       new HashMap<CompactionPartitionId, CompactionPartition>();
144     int selectedFileCount = 0;
145     int irrelevantFileCount = 0;
146     for (FileStatus file : candidates) {
147       if (!file.isFile()) {
148         irrelevantFileCount++;
149         continue;
150       }
151       // group the del files and small files.
152       FileStatus linkedFile = file;
153       if (HFileLink.isHFileLink(file.getPath())) {
154         HFileLink link = HFileLink.buildFromHFileLinkPattern(conf, file.getPath());
155         linkedFile = getLinkedFileStatus(link);
156         if (linkedFile == null) {
157           // If the linked file cannot be found, regard it as an irrelevantFileCount file
158           irrelevantFileCount++;
159           continue;
160         }
161       }
162       if (StoreFileInfo.isDelFile(linkedFile.getPath())) {
163         allDelFiles.add(file);
164       } else if (allFiles || linkedFile.getLen() < mergeableSize) {
165         // add all files if allFiles is true,
166         // otherwise add the small files to the merge pool
167         MobFileName fileName = MobFileName.create(linkedFile.getPath().getName());
168         CompactionPartitionId id = new CompactionPartitionId(fileName.getStartKey(),
169           fileName.getDate());
170         CompactionPartition compactionPartition = filesToCompact.get(id);
171         if (compactionPartition == null) {
172           compactionPartition = new CompactionPartition(id);
173           compactionPartition.addFile(file);
174           filesToCompact.put(id, compactionPartition);
175         } else {
176           compactionPartition.addFile(file);
177         }
178         selectedFileCount++;
179       }
180     }
181     PartitionedMobCompactionRequest request = new PartitionedMobCompactionRequest(
182       filesToCompact.values(), allDelFiles);
183     if (candidates.size() == (allDelFiles.size() + selectedFileCount + irrelevantFileCount)) {
184       // all the files are selected
185       request.setCompactionType(CompactionType.ALL_FILES);
186     }
187     LOG.info("The compaction type is " + request.getCompactionType() + ", the request has "
188       + allDelFiles.size() + " del files, " + selectedFileCount + " selected files, and "
189       + irrelevantFileCount + " irrelevant files");
190     return request;
191   }
192 
193   /**
194    * Performs the compaction on the selected files.
195    * <ol>
196    * <li>Compacts the del files.</li>
197    * <li>Compacts the selected small mob files and all the del files.</li>
198    * <li>If all the candidates are selected, delete the del files.</li>
199    * </ol>
200    * @param request The compaction request.
201    * @return The paths of new mob files generated in the compaction.
202    * @throws IOException
203    */
204   protected List<Path> performCompaction(PartitionedMobCompactionRequest request)
205     throws IOException {
206     // merge the del files
207     List<Path> delFilePaths = new ArrayList<Path>();
208     for (FileStatus delFile : request.delFiles) {
209       delFilePaths.add(delFile.getPath());
210     }
211     List<Path> newDelPaths = compactDelFiles(request, delFilePaths);
212     List<StoreFile> newDelFiles = new ArrayList<StoreFile>();
213     List<Path> paths = null;
214     try {
215       for (Path newDelPath : newDelPaths) {
216         StoreFile sf = new StoreFile(fs, newDelPath, conf, compactionCacheConfig, BloomType.NONE);
217         // pre-create reader of a del file to avoid race condition when opening the reader in each
218         // partition.
219         sf.createReader();
220         newDelFiles.add(sf);
221       }
222       LOG.info("After merging, there are " + newDelFiles.size() + " del files");
223       // compact the mob files by partitions.
224       paths = compactMobFiles(request, newDelFiles);
225       LOG.info("After compaction, there are " + paths.size() + " mob files");
226     } finally {
227       closeStoreFileReaders(newDelFiles);
228     }
229     // archive the del files if all the mob files are selected.
230     if (request.type == CompactionType.ALL_FILES && !newDelPaths.isEmpty()) {
231       LOG.info("After a mob compaction with all files selected, archiving the del files "
232         + newDelPaths);
233       try {
234         MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), newDelFiles);
235       } catch (IOException e) {
236         LOG.error("Failed to archive the del files " + newDelPaths, e);
237       }
238     }
239     return paths;
240   }
241 
242   /**
243    * Compacts the selected small mob files and all the del files.
244    * @param request The compaction request.
245    * @param delFiles The del files.
246    * @return The paths of new mob files after compactions.
247    * @throws IOException
248    */
249   protected List<Path> compactMobFiles(final PartitionedMobCompactionRequest request,
250     final List<StoreFile> delFiles) throws IOException {
251     Collection<CompactionPartition> partitions = request.compactionPartitions;
252     if (partitions == null || partitions.isEmpty()) {
253       LOG.info("No partitions of mob files");
254       return Collections.emptyList();
255     }
256     List<Path> paths = new ArrayList<Path>();
257     Connection c = ConnectionFactory.createConnection(conf);
258     final Table table = c.getTable(tableName);
259     try {
260       Map<CompactionPartitionId, Future<List<Path>>> results =
261         new HashMap<CompactionPartitionId, Future<List<Path>>>();
262       // compact the mob files by partitions in parallel.
263       for (final CompactionPartition partition : partitions) {
264         results.put(partition.getPartitionId(), pool.submit(new Callable<List<Path>>() {
265           @Override
266           public List<Path> call() throws Exception {
267             LOG.info("Compacting mob files for partition " + partition.getPartitionId());
268             return compactMobFilePartition(request, partition, delFiles, table);
269           }
270         }));
271       }
272       // compact the partitions in parallel.
273       List<CompactionPartitionId> failedPartitions = new ArrayList<CompactionPartitionId>();
274       for (Entry<CompactionPartitionId, Future<List<Path>>> result : results.entrySet()) {
275         try {
276           paths.addAll(result.getValue().get());
277         } catch (Exception e) {
278           // just log the error
279           LOG.error("Failed to compact the partition " + result.getKey(), e);
280           failedPartitions.add(result.getKey());
281         }
282       }
283       if (!failedPartitions.isEmpty()) {
284         // if any partition fails in the compaction, directly throw an exception.
285         throw new IOException("Failed to compact the partitions " + failedPartitions);
286       }
287     } finally {
288       try {
289         table.close();
290       } catch (IOException e) {
291         LOG.error("Failed to close the HTable", e);
292       }
293     }
294     return paths;
295   }
296 
297   /**
298    * Compacts a partition of selected small mob files and all the del files.
299    * @param request The compaction request.
300    * @param partition A compaction partition.
301    * @param delFiles The del files.
302    * @param table The current table.
303    * @return The paths of new mob files after compactions.
304    * @throws IOException
305    */
306   private List<Path> compactMobFilePartition(PartitionedMobCompactionRequest request,
307     CompactionPartition partition, List<StoreFile> delFiles, Table table) throws IOException {
308     List<Path> newFiles = new ArrayList<Path>();
309     List<FileStatus> files = partition.listFiles();
310     int offset = 0;
311     Path bulkloadPathOfPartition = new Path(bulkloadPath, partition.getPartitionId().toString());
312     Path bulkloadColumnPath = new Path(bulkloadPathOfPartition, column.getNameAsString());
313     while (offset < files.size()) {
314       int batch = compactionBatchSize;
315       if (files.size() - offset < compactionBatchSize) {
316         batch = files.size() - offset;
317       }
318       if (batch == 1 && delFiles.isEmpty()) {
319         // only one file left and no del files, do not compact it,
320         // and directly add it to the new files.
321         newFiles.add(files.get(offset).getPath());
322         offset++;
323         continue;
324       }
325       // clean the bulkload directory to avoid loading old files.
326       fs.delete(bulkloadPathOfPartition, true);
327       // add the selected mob files and del files into filesToCompact
328       List<StoreFile> filesToCompact = new ArrayList<StoreFile>();
329       for (int i = offset; i < batch + offset; i++) {
330         StoreFile sf = new StoreFile(fs, files.get(i).getPath(), conf, compactionCacheConfig,
331           BloomType.NONE);
332         filesToCompact.add(sf);
333       }
334       filesToCompact.addAll(delFiles);
335       // compact the mob files in a batch.
336       compactMobFilesInBatch(request, partition, table, filesToCompact, batch,
337         bulkloadPathOfPartition, bulkloadColumnPath, newFiles);
338       // move to the next batch.
339       offset += batch;
340     }
341     LOG.info("Compaction is finished. The number of mob files is changed from " + files.size()
342       + " to " + newFiles.size());
343     return newFiles;
344   }
345 
346   /**
347    * Closes the readers of store files.
348    * @param storeFiles The store files to be closed.
349    */
350   private void closeStoreFileReaders(List<StoreFile> storeFiles) {
351     for (StoreFile storeFile : storeFiles) {
352       try {
353         storeFile.closeReader(true);
354       } catch (IOException e) {
355         LOG.warn("Failed to close the reader on store file " + storeFile.getPath(), e);
356       }
357     }
358   }
359 
360   /**
361    * Compacts a partition of selected small mob files and all the del files in a batch.
362    * @param request The compaction request.
363    * @param partition A compaction partition.
364    * @param table The current table.
365    * @param filesToCompact The files to be compacted.
366    * @param batch The number of mob files to be compacted in a batch.
367    * @param bulkloadPathOfPartition The directory where the bulkload column of the current
368    *        partition is saved.
369    * @param bulkloadColumnPath The directory where the bulkload files of current partition
370    *        are saved.
371    * @param newFiles The paths of new mob files after compactions.
372    * @throws IOException
373    */
374   private void compactMobFilesInBatch(PartitionedMobCompactionRequest request,
375     CompactionPartition partition, Table table, List<StoreFile> filesToCompact, int batch,
376     Path bulkloadPathOfPartition, Path bulkloadColumnPath, List<Path> newFiles)
377     throws IOException {
378     // open scanner to the selected mob files and del files.
379     StoreScanner scanner = createScanner(filesToCompact, ScanType.COMPACT_DROP_DELETES);
380     // the mob files to be compacted, not include the del files.
381     List<StoreFile> mobFilesToCompact = filesToCompact.subList(0, batch);
382     // Pair(maxSeqId, cellsCount)
383     Pair<Long, Long> fileInfo = getFileInfo(mobFilesToCompact);
384     // open writers for the mob files and new ref store files.
385     Writer writer = null;
386     Writer refFileWriter = null;
387     Path filePath = null;
388     Path refFilePath = null;
389     long mobCells = 0;
390     try {
391       writer = MobUtils.createWriter(conf, fs, column, partition.getPartitionId().getDate(),
392         tempPath, Long.MAX_VALUE, column.getCompactionCompression(), partition.getPartitionId()
393           .getStartKey(), compactionCacheConfig, cryptoContext);
394       filePath = writer.getPath();
395       byte[] fileName = Bytes.toBytes(filePath.getName());
396       // create a temp file and open a writer for it in the bulkloadPath
397       refFileWriter = MobUtils.createRefFileWriter(conf, fs, column, bulkloadColumnPath, fileInfo
398         .getSecond().longValue(), compactionCacheConfig, cryptoContext);
399       refFilePath = refFileWriter.getPath();
400       List<Cell> cells = new ArrayList<Cell>();
401       boolean hasMore = false;
402       ScannerContext scannerContext =
403               ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build();
404       do {
405         hasMore = scanner.next(cells, scannerContext);
406         for (Cell cell : cells) {
407           // write the mob cell to the mob file.
408           writer.append(cell);
409           // write the new reference cell to the store file.
410           KeyValue reference = MobUtils.createMobRefKeyValue(cell, fileName, tableNameTag);
411           refFileWriter.append(reference);
412           mobCells++;
413         }
414         cells.clear();
415       } while (hasMore);
416     } finally {
417       // close the scanner.
418       scanner.close();
419       // append metadata to the mob file, and close the mob file writer.
420       closeMobFileWriter(writer, fileInfo.getFirst(), mobCells);
421       // append metadata and bulkload info to the ref mob file, and close the writer.
422       closeRefFileWriter(refFileWriter, fileInfo.getFirst(), request.selectionTime);
423     }
424     if (mobCells > 0) {
425       // commit mob file
426       MobUtils.commitFile(conf, fs, filePath, mobFamilyDir, compactionCacheConfig);
427       // bulkload the ref file
428       bulkloadRefFile(table, bulkloadPathOfPartition, filePath.getName());
429       newFiles.add(new Path(mobFamilyDir, filePath.getName()));
430     } else {
431       // remove the new files
432       // the mob file is empty, delete it instead of committing.
433       deletePath(filePath);
434       // the ref file is empty, delete it instead of committing.
435       deletePath(refFilePath);
436     }
437     // archive the old mob files, do not archive the del files.
438     try {
439       closeStoreFileReaders(mobFilesToCompact);
440       MobUtils
441         .removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), mobFilesToCompact);
442     } catch (IOException e) {
443       LOG.error("Failed to archive the files " + mobFilesToCompact, e);
444     }
445   }
446 
447   /**
448    * Compacts the del files in batches which avoids opening too many files.
449    * @param request The compaction request.
450    * @param delFilePaths
451    * @return The paths of new del files after merging or the original files if no merging
452    *         is necessary.
453    * @throws IOException
454    */
455   protected List<Path> compactDelFiles(PartitionedMobCompactionRequest request,
456     List<Path> delFilePaths) throws IOException {
457     if (delFilePaths.size() <= delFileMaxCount) {
458       return delFilePaths;
459     }
460     // when there are more del files than the number that is allowed, merge it firstly.
461     int offset = 0;
462     List<Path> paths = new ArrayList<Path>();
463     while (offset < delFilePaths.size()) {
464       // get the batch
465       int batch = compactionBatchSize;
466       if (delFilePaths.size() - offset < compactionBatchSize) {
467         batch = delFilePaths.size() - offset;
468       }
469       List<StoreFile> batchedDelFiles = new ArrayList<StoreFile>();
470       if (batch == 1) {
471         // only one file left, do not compact it, directly add it to the new files.
472         paths.add(delFilePaths.get(offset));
473         offset++;
474         continue;
475       }
476       for (int i = offset; i < batch + offset; i++) {
477         batchedDelFiles.add(new StoreFile(fs, delFilePaths.get(i), conf, compactionCacheConfig,
478           BloomType.NONE));
479       }
480       // compact the del files in a batch.
481       paths.add(compactDelFilesInBatch(request, batchedDelFiles));
482       // move to the next batch.
483       offset += batch;
484     }
485     return compactDelFiles(request, paths);
486   }
487 
488   /**
489    * Compacts the del file in a batch.
490    * @param request The compaction request.
491    * @param delFiles The del files.
492    * @return The path of new del file after merging.
493    * @throws IOException
494    */
495   private Path compactDelFilesInBatch(PartitionedMobCompactionRequest request,
496     List<StoreFile> delFiles) throws IOException {
497     // create a scanner for the del files.
498     StoreScanner scanner = createScanner(delFiles, ScanType.COMPACT_RETAIN_DELETES);
499     Writer writer = null;
500     Path filePath = null;
501     try {
502       writer = MobUtils.createDelFileWriter(conf, fs, column,
503         MobUtils.formatDate(new Date(request.selectionTime)), tempPath, Long.MAX_VALUE,
504         column.getCompactionCompression(), HConstants.EMPTY_START_ROW, compactionCacheConfig,
505         cryptoContext);
506       filePath = writer.getPath();
507       List<Cell> cells = new ArrayList<Cell>();
508       boolean hasMore = false;
509       ScannerContext scannerContext =
510               ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build();
511       do {
512         hasMore = scanner.next(cells, scannerContext);
513         for (Cell cell : cells) {
514           writer.append(cell);
515         }
516         cells.clear();
517       } while (hasMore);
518     } finally {
519       scanner.close();
520       if (writer != null) {
521         try {
522           writer.close();
523         } catch (IOException e) {
524           LOG.error("Failed to close the writer of the file " + filePath, e);
525         }
526       }
527     }
528     // commit the new del file
529     Path path = MobUtils.commitFile(conf, fs, filePath, mobFamilyDir, compactionCacheConfig);
530     // archive the old del files
531     try {
532       MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), delFiles);
533     } catch (IOException e) {
534       LOG.error("Failed to archive the old del files " + delFiles, e);
535     }
536     return path;
537   }
538 
539   /**
540    * Creates a store scanner.
541    * @param filesToCompact The files to be compacted.
542    * @param scanType The scan type.
543    * @return The store scanner.
544    * @throws IOException
545    */
546   private StoreScanner createScanner(List<StoreFile> filesToCompact, ScanType scanType)
547     throws IOException {
548     List scanners = StoreFileScanner.getScannersForStoreFiles(filesToCompact, false, true, false,
549       null, HConstants.LATEST_TIMESTAMP);
550     Scan scan = new Scan();
551     scan.setMaxVersions(column.getMaxVersions());
552     long ttl = HStore.determineTTLFromFamily(column);
553     ScanInfo scanInfo = new ScanInfo(column, ttl, 0, KeyValue.COMPARATOR);
554     StoreScanner scanner = new StoreScanner(scan, scanInfo, scanType, null, scanners, 0L,
555       HConstants.LATEST_TIMESTAMP);
556     return scanner;
557   }
558 
559   /**
560    * Bulkloads the current file.
561    * @param table The current table.
562    * @param bulkloadDirectory The path of bulkload directory.
563    * @param fileName The current file name.
564    * @throws IOException
565    */
566   private void bulkloadRefFile(Table table, Path bulkloadDirectory, String fileName)
567     throws IOException {
568     // bulkload the ref file
569     try {
570       LoadIncrementalHFiles bulkload = new LoadIncrementalHFiles(conf);
571       bulkload.doBulkLoad(bulkloadDirectory, (HTable)table);
572     } catch (Exception e) {
573       // delete the committed mob file
574       deletePath(new Path(mobFamilyDir, fileName));
575       throw new IOException(e);
576     } finally {
577       // delete the bulkload files in bulkloadPath
578       deletePath(bulkloadDirectory);
579     }
580   }
581 
582   /**
583    * Closes the mob file writer.
584    * @param writer The mob file writer.
585    * @param maxSeqId Maximum sequence id.
586    * @param mobCellsCount The number of mob cells.
587    * @throws IOException
588    */
589   private void closeMobFileWriter(Writer writer, long maxSeqId, long mobCellsCount)
590     throws IOException {
591     if (writer != null) {
592       writer.appendMetadata(maxSeqId, false, mobCellsCount);
593       try {
594         writer.close();
595       } catch (IOException e) {
596         LOG.error("Failed to close the writer of the file " + writer.getPath(), e);
597       }
598     }
599   }
600 
601   /**
602    * Closes the ref file writer.
603    * @param writer The ref file writer.
604    * @param maxSeqId Maximum sequence id.
605    * @param bulkloadTime The timestamp at which the bulk load file is created.
606    * @throws IOException
607    */
608   private void closeRefFileWriter(Writer writer, long maxSeqId, long bulkloadTime)
609     throws IOException {
610     if (writer != null) {
611       writer.appendMetadata(maxSeqId, false);
612       writer.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY, Bytes.toBytes(bulkloadTime));
613       writer.appendFileInfo(StoreFile.SKIP_RESET_SEQ_ID, Bytes.toBytes(true));
614       try {
615         writer.close();
616       } catch (IOException e) {
617         LOG.error("Failed to close the writer of the ref file " + writer.getPath(), e);
618       }
619     }
620   }
621 
622   /**
623    * Gets the max seqId and number of cells of the store files.
624    * @param storeFiles The store files.
625    * @return The pair of the max seqId and number of cells of the store files.
626    * @throws IOException
627    */
628   private Pair<Long, Long> getFileInfo(List<StoreFile> storeFiles) throws IOException {
629     long maxSeqId = 0;
630     long maxKeyCount = 0;
631     for (StoreFile sf : storeFiles) {
632       // the readers will be closed later after the merge.
633       maxSeqId = Math.max(maxSeqId, sf.getMaxSequenceId());
634       byte[] count = sf.createReader().loadFileInfo().get(StoreFile.MOB_CELLS_COUNT);
635       if (count != null) {
636         maxKeyCount += Bytes.toLong(count);
637       }
638     }
639     return new Pair<Long, Long>(Long.valueOf(maxSeqId), Long.valueOf(maxKeyCount));
640   }
641 
642   /**
643    * Deletes a file.
644    * @param path The path of the file to be deleted.
645    */
646   private void deletePath(Path path) {
647     try {
648       if (path != null) {
649         fs.delete(path, true);
650       }
651     } catch (IOException e) {
652       LOG.error("Failed to delete the file " + path, e);
653     }
654   }
655 
656   private FileStatus getLinkedFileStatus(HFileLink link) throws IOException {
657     Path[] locations = link.getLocations();
658     for (Path location : locations) {
659       FileStatus file = getFileStatus(location);
660       if (file != null) {
661         return file;
662       }
663     }
664     return null;
665   }
666 
667   private FileStatus getFileStatus(Path path) throws IOException {
668     try {
669       if (path != null) {
670         FileStatus file = fs.getFileStatus(path);
671         return file;
672       }
673     } catch (FileNotFoundException e) {
674       LOG.warn("The file " + path + " can not be found", e);
675     }
676     return null;
677   }
678 }