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  package org.apache.hadoop.hbase.regionserver;
19  
20  import java.io.FileNotFoundException;
21  import java.io.IOException;
22  import java.util.ArrayList;
23  import java.util.Date;
24  import java.util.List;
25  import java.util.Map;
26  import java.util.NavigableSet;
27  import java.util.UUID;
28  import java.util.concurrent.ConcurrentHashMap;
29  
30  import org.apache.commons.logging.Log;
31  import org.apache.commons.logging.LogFactory;
32  import org.apache.hadoop.hbase.classification.InterfaceAudience;
33  import org.apache.hadoop.conf.Configuration;
34  import org.apache.hadoop.fs.FileSystem;
35  import org.apache.hadoop.fs.Path;
36  import org.apache.hadoop.hbase.Cell;
37  import org.apache.hadoop.hbase.CellComparator;
38  import org.apache.hadoop.hbase.HColumnDescriptor;
39  import org.apache.hadoop.hbase.HConstants;
40  import org.apache.hadoop.hbase.KeyValue;
41  import org.apache.hadoop.hbase.KeyValue.KVComparator;
42  import org.apache.hadoop.hbase.KeyValue.Type;
43  import org.apache.hadoop.hbase.TableName;
44  import org.apache.hadoop.hbase.Tag;
45  import org.apache.hadoop.hbase.client.Scan;
46  import org.apache.hadoop.hbase.filter.Filter;
47  import org.apache.hadoop.hbase.filter.FilterList;
48  import org.apache.hadoop.hbase.io.compress.Compression;
49  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
50  import org.apache.hadoop.hbase.io.hfile.CorruptHFileException;
51  import org.apache.hadoop.hbase.io.hfile.HFileContext;
52  import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
53  import org.apache.hadoop.hbase.master.TableLockManager;
54  import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
55  import org.apache.hadoop.hbase.mob.MobCacheConfig;
56  import org.apache.hadoop.hbase.mob.MobConstants;
57  import org.apache.hadoop.hbase.mob.MobFile;
58  import org.apache.hadoop.hbase.mob.MobFileName;
59  import org.apache.hadoop.hbase.mob.MobStoreEngine;
60  import org.apache.hadoop.hbase.mob.MobUtils;
61  import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
62  import org.apache.hadoop.hbase.regionserver.compactions.CompactionThroughputController;
63  import org.apache.hadoop.hbase.util.Bytes;
64  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
65  import org.apache.hadoop.hbase.util.HFileArchiveUtil;
66  import org.apache.hadoop.hbase.util.IdLock;
67  
68  /**
69   * The store implementation to save MOBs (medium objects), it extends the HStore.
70   * When a descriptor of a column family has the value "IS_MOB", it means this column family
71   * is a mob one. When a HRegion instantiate a store for this column family, the HMobStore is
72   * created.
73   * HMobStore is almost the same with the HStore except using different types of scanners.
74   * In the method of getScanner, the MobStoreScanner and MobReversedStoreScanner are returned.
75   * In these scanners, a additional seeks in the mob files should be performed after the seek
76   * to HBase is done.
77   * The store implements how we save MOBs by extending HStore. When a descriptor
78   * of a column family has the value "IS_MOB", it means this column family is a mob one. When a
79   * HRegion instantiate a store for this column family, the HMobStore is created. HMobStore is
80   * almost the same with the HStore except using different types of scanners. In the method of
81   * getScanner, the MobStoreScanner and MobReversedStoreScanner are returned. In these scanners, a
82   * additional seeks in the mob files should be performed after the seek in HBase is done.
83   */
84  @InterfaceAudience.Private
85  public class HMobStore extends HStore {
86    private static final Log LOG = LogFactory.getLog(HMobStore.class);
87    private MobCacheConfig mobCacheConfig;
88    private Path homePath;
89    private Path mobFamilyPath;
90    private volatile long cellsCountCompactedToMob = 0;
91    private volatile long cellsCountCompactedFromMob = 0;
92    private volatile long cellsSizeCompactedToMob = 0;
93    private volatile long cellsSizeCompactedFromMob = 0;
94    private volatile long mobFlushCount = 0;
95    private volatile long mobFlushedCellsCount = 0;
96    private volatile long mobFlushedCellsSize = 0;
97    private volatile long mobScanCellsCount = 0;
98    private volatile long mobScanCellsSize = 0;
99    private HColumnDescriptor family;
100   private TableLockManager tableLockManager;
101   private TableName tableLockName;
102   private Map<String, List<Path>> map = new ConcurrentHashMap<String, List<Path>>();
103   private final IdLock keyLock = new IdLock();
104 
105   public HMobStore(final HRegion region, final HColumnDescriptor family,
106       final Configuration confParam) throws IOException {
107     super(region, family, confParam);
108     this.family = family;
109     this.mobCacheConfig = (MobCacheConfig) cacheConf;
110     this.homePath = MobUtils.getMobHome(conf);
111     this.mobFamilyPath = MobUtils.getMobFamilyPath(conf, this.getTableName(),
112         family.getNameAsString());
113     List<Path> locations = new ArrayList<Path>(2);
114     locations.add(mobFamilyPath);
115     TableName tn = region.getTableDesc().getTableName();
116     locations.add(HFileArchiveUtil.getStoreArchivePath(conf, tn, MobUtils.getMobRegionInfo(tn)
117         .getEncodedName(), family.getNameAsString()));
118     map.put(Bytes.toString(tn.getName()), locations);
119     if (region.getRegionServerServices() != null) {
120       tableLockManager = region.getRegionServerServices().getTableLockManager();
121       tableLockName = MobUtils.getTableLockName(getTableName());
122     }
123   }
124 
125   /**
126    * Creates the mob cache config.
127    */
128   @Override
129   protected void createCacheConf(HColumnDescriptor family) {
130     cacheConf = new MobCacheConfig(conf, family);
131   }
132 
133   /**
134    * Gets current config.
135    */
136   public Configuration getConfiguration() {
137     return this.conf;
138   }
139 
140   /**
141    * Gets the MobStoreScanner or MobReversedStoreScanner. In these scanners, a additional seeks in
142    * the mob files should be performed after the seek in HBase is done.
143    */
144   @Override
145   protected KeyValueScanner createScanner(Scan scan, final NavigableSet<byte[]> targetCols,
146       long readPt, KeyValueScanner scanner) throws IOException {
147     if (scanner == null) {
148       if (MobUtils.isRefOnlyScan(scan)) {
149         Filter refOnlyFilter = new MobReferenceOnlyFilter();
150         Filter filter = scan.getFilter();
151         if (filter != null) {
152           scan.setFilter(new FilterList(filter, refOnlyFilter));
153         } else {
154           scan.setFilter(refOnlyFilter);
155         }
156       }
157       scanner = scan.isReversed() ? new ReversedMobStoreScanner(this, getScanInfo(), scan,
158           targetCols, readPt) : new MobStoreScanner(this, getScanInfo(), scan, targetCols, readPt);
159     }
160     return scanner;
161   }
162 
163   /**
164    * Creates the mob store engine.
165    */
166   @Override
167   protected StoreEngine<?, ?, ?, ?> createStoreEngine(Store store, Configuration conf,
168       KVComparator cellComparator) throws IOException {
169     MobStoreEngine engine = new MobStoreEngine();
170     engine.createComponents(conf, store, cellComparator);
171     return engine;
172   }
173 
174   /**
175    * Gets the temp directory.
176    * @return The temp directory.
177    */
178   private Path getTempDir() {
179     return new Path(homePath, MobConstants.TEMP_DIR_NAME);
180   }
181 
182   /**
183    * Creates the writer for the mob file in temp directory.
184    * @param date The latest date of written cells.
185    * @param maxKeyCount The key count.
186    * @param compression The compression algorithm.
187    * @param startKey The start key.
188    * @return The writer for the mob file.
189    * @throws IOException
190    */
191   public StoreFile.Writer createWriterInTmp(Date date, long maxKeyCount,
192       Compression.Algorithm compression, byte[] startKey) throws IOException {
193     if (startKey == null) {
194       startKey = HConstants.EMPTY_START_ROW;
195     }
196     Path path = getTempDir();
197     return createWriterInTmp(MobUtils.formatDate(date), path, maxKeyCount, compression, startKey);
198   }
199 
200   /**
201    * Creates the writer for the del file in temp directory.
202    * The del file keeps tracking the delete markers. Its name has a suffix _del,
203    * the format is [0-9a-f]+(_del)?.
204    * @param date The latest date of written cells.
205    * @param maxKeyCount The key count.
206    * @param compression The compression algorithm.
207    * @param startKey The start key.
208    * @return The writer for the del file.
209    * @throws IOException
210    */
211   public StoreFile.Writer createDelFileWriterInTmp(Date date, long maxKeyCount,
212       Compression.Algorithm compression, byte[] startKey) throws IOException {
213     if (startKey == null) {
214       startKey = HConstants.EMPTY_START_ROW;
215     }
216     Path path = getTempDir();
217     String suffix = UUID
218         .randomUUID().toString().replaceAll("-", "") + "_del";
219     MobFileName mobFileName = MobFileName.create(startKey, MobUtils.formatDate(date), suffix);
220     return createWriterInTmp(mobFileName, path, maxKeyCount, compression);
221   }
222 
223   /**
224    * Creates the writer for the mob file in temp directory.
225    * @param date The date string, its format is yyyymmmdd.
226    * @param basePath The basic path for a temp directory.
227    * @param maxKeyCount The key count.
228    * @param compression The compression algorithm.
229    * @param startKey The start key.
230    * @return The writer for the mob file.
231    * @throws IOException
232    */
233   public StoreFile.Writer createWriterInTmp(String date, Path basePath, long maxKeyCount,
234       Compression.Algorithm compression, byte[] startKey) throws IOException {
235     MobFileName mobFileName = MobFileName.create(startKey, date, UUID.randomUUID()
236         .toString().replaceAll("-", ""));
237     return createWriterInTmp(mobFileName, basePath, maxKeyCount, compression);
238   }
239 
240   /**
241    * Creates the writer for the mob file in temp directory.
242    * @param mobFileName The mob file name.
243    * @param basePath The basic path for a temp directory.
244    * @param maxKeyCount The key count.
245    * @param compression The compression algorithm.
246    * @return The writer for the mob file.
247    * @throws IOException
248    */
249   public StoreFile.Writer createWriterInTmp(MobFileName mobFileName, Path basePath, long maxKeyCount,
250       Compression.Algorithm compression) throws IOException {
251     final CacheConfig writerCacheConf = mobCacheConfig;
252     HFileContext hFileContext = new HFileContextBuilder().withCompression(compression)
253         .withIncludesMvcc(true).withIncludesTags(true)
254         .withCompressTags(family.isCompressTags())
255         .withChecksumType(checksumType)
256         .withBytesPerCheckSum(bytesPerChecksum)
257         .withBlockSize(blocksize)
258         .withHBaseCheckSum(true).withDataBlockEncoding(getFamily().getDataBlockEncoding())
259         .withEncryptionContext(cryptoContext)
260         .withCreateTime(EnvironmentEdgeManager.currentTime()).build();
261 
262     StoreFile.Writer w = new StoreFile.WriterBuilder(conf, writerCacheConf, region.getFilesystem())
263         .withFilePath(new Path(basePath, mobFileName.getFileName()))
264         .withComparator(KeyValue.COMPARATOR).withBloomType(BloomType.NONE)
265         .withMaxKeyCount(maxKeyCount).withFileContext(hFileContext).build();
266     return w;
267   }
268 
269   /**
270    * Commits the mob file.
271    * @param sourceFile The source file.
272    * @param targetPath The directory path where the source file is renamed to.
273    * @throws IOException
274    */
275   public void commitFile(final Path sourceFile, Path targetPath) throws IOException {
276     if (sourceFile == null) {
277       return;
278     }
279     Path dstPath = new Path(targetPath, sourceFile.getName());
280     validateMobFile(sourceFile);
281     String msg = "Renaming flushed file from " + sourceFile + " to " + dstPath;
282     LOG.info(msg);
283     Path parent = dstPath.getParent();
284     if (!region.getFilesystem().exists(parent)) {
285       region.getFilesystem().mkdirs(parent);
286     }
287     if (!region.getFilesystem().rename(sourceFile, dstPath)) {
288       throw new IOException("Failed rename of " + sourceFile + " to " + dstPath);
289     }
290   }
291 
292   /**
293    * Validates a mob file by opening and closing it.
294    *
295    * @param path the path to the mob file
296    */
297   private void validateMobFile(Path path) throws IOException {
298     StoreFile storeFile = null;
299     try {
300       storeFile =
301           new StoreFile(region.getFilesystem(), path, conf, this.mobCacheConfig, BloomType.NONE);
302       storeFile.createReader();
303     } catch (IOException e) {
304       LOG.error("Fail to open mob file[" + path + "], keep it in temp directory.", e);
305       throw e;
306     } finally {
307       if (storeFile != null) {
308         storeFile.closeReader(false);
309       }
310     }
311   }
312 
313   /**
314    * Reads the cell from the mob file, and the read point does not count.
315    * This is used for DefaultMobStoreCompactor where we can read empty value for the missing cell.
316    * @param reference The cell found in the HBase, its value is a path to a mob file.
317    * @param cacheBlocks Whether the scanner should cache blocks.
318    * @return The cell found in the mob file.
319    * @throws IOException
320    */
321   public Cell resolve(Cell reference, boolean cacheBlocks) throws IOException {
322     return resolve(reference, cacheBlocks, -1, true);
323   }
324 
325   /**
326    * Reads the cell from the mob file.
327    * @param reference The cell found in the HBase, its value is a path to a mob file.
328    * @param cacheBlocks Whether the scanner should cache blocks.
329    * @param readPt the read point.
330    * @param readEmptyValueOnMobCellMiss Whether return null value when the mob file is
331    *        missing or corrupt.
332    * @return The cell found in the mob file.
333    * @throws IOException
334    */
335   public Cell resolve(Cell reference, boolean cacheBlocks, long readPt,
336     boolean readEmptyValueOnMobCellMiss) throws IOException {
337     Cell result = null;
338     if (MobUtils.hasValidMobRefCellValue(reference)) {
339       String fileName = MobUtils.getMobFileName(reference);
340       Tag tableNameTag = MobUtils.getTableNameTag(reference);
341       if (tableNameTag != null) {
342         byte[] tableName = tableNameTag.getValue();
343         String tableNameString = Bytes.toString(tableName);
344         List<Path> locations = map.get(tableNameString);
345         if (locations == null) {
346           IdLock.Entry lockEntry = keyLock.getLockEntry(tableNameString.hashCode());
347           try {
348             locations = map.get(tableNameString);
349             if (locations == null) {
350               locations = new ArrayList<Path>(2);
351               TableName tn = TableName.valueOf(tableName);
352               locations.add(MobUtils.getMobFamilyPath(conf, tn, family.getNameAsString()));
353               locations.add(HFileArchiveUtil.getStoreArchivePath(conf, tn, MobUtils
354                   .getMobRegionInfo(tn).getEncodedName(), family.getNameAsString()));
355               map.put(tableNameString, locations);
356             }
357           } finally {
358             keyLock.releaseLockEntry(lockEntry);
359           }
360         }
361         result = readCell(locations, fileName, reference, cacheBlocks, readPt,
362           readEmptyValueOnMobCellMiss);
363       }
364     }
365     if (result == null) {
366       LOG.warn("The KeyValue result is null, assemble a new KeyValue with the same row,family,"
367           + "qualifier,timestamp,type and tags but with an empty value to return.");
368       result = new KeyValue(reference.getRowArray(), reference.getRowOffset(),
369           reference.getRowLength(), reference.getFamilyArray(), reference.getFamilyOffset(),
370           reference.getFamilyLength(), reference.getQualifierArray(),
371           reference.getQualifierOffset(), reference.getQualifierLength(), reference.getTimestamp(),
372           Type.codeToType(reference.getTypeByte()), HConstants.EMPTY_BYTE_ARRAY,
373           0, 0, reference.getTagsArray(), reference.getTagsOffset(),
374           reference.getTagsLength());
375     }
376     return result;
377   }
378 
379   /**
380    * Reads the cell from a mob file.
381    * The mob file might be located in different directories.
382    * 1. The working directory.
383    * 2. The archive directory.
384    * Reads the cell from the files located in both of the above directories.
385    * @param locations The possible locations where the mob files are saved.
386    * @param fileName The file to be read.
387    * @param search The cell to be searched.
388    * @param cacheMobBlocks Whether the scanner should cache blocks.
389    * @param readPt the read point.
390    * @param readEmptyValueOnMobCellMiss Whether return null value when the mob file is
391    *        missing or corrupt.
392    * @return The found cell. Null if there's no such a cell.
393    * @throws IOException
394    */
395   private Cell readCell(List<Path> locations, String fileName, Cell search, boolean cacheMobBlocks,
396     long readPt, boolean readEmptyValueOnMobCellMiss) throws IOException {
397     FileSystem fs = getFileSystem();
398     Throwable throwable = null;
399     for (Path location : locations) {
400       MobFile file = null;
401       Path path = new Path(location, fileName);
402       try {
403         file = mobCacheConfig.getMobFileCache().openFile(fs, path, mobCacheConfig);
404         return readPt != -1 ? file.readCell(search, cacheMobBlocks, readPt) : file.readCell(search,
405           cacheMobBlocks);
406       } catch (IOException e) {
407         mobCacheConfig.getMobFileCache().evictFile(fileName);
408         throwable = e;
409         if ((e instanceof FileNotFoundException) ||
410             (e.getCause() instanceof FileNotFoundException)) {
411           LOG.warn("Fail to read the cell, the mob file " + path + " doesn't exist", e);
412         } else if (e instanceof CorruptHFileException) {
413           LOG.error("The mob file " + path + " is corrupt", e);
414           break;
415         } else {
416           throw e;
417         }
418       } catch (NullPointerException e) { // HDFS 1.x - DFSInputStream.getBlockAt()
419         mobCacheConfig.getMobFileCache().evictFile(fileName);
420         LOG.warn("Fail to read the cell", e);
421         throwable = e;
422       } catch (AssertionError e) { // assert in HDFS 1.x - DFSInputStream.getBlockAt()
423         mobCacheConfig.getMobFileCache().evictFile(fileName);
424         LOG.warn("Fail to read the cell", e);
425         throwable = e;
426       } finally {
427         if (file != null) {
428           mobCacheConfig.getMobFileCache().closeFile(file);
429         }
430       }
431     }
432     LOG.error("The mob file " + fileName + " could not be found in the locations " + locations
433       + " or it is corrupt");
434     if (readEmptyValueOnMobCellMiss) {
435       return null;
436     } else if (throwable instanceof IOException) {
437       throw (IOException) throwable;
438     } else {
439       throw new IOException(throwable);
440     }
441   }
442 
443   /**
444    * Gets the mob file path.
445    * @return The mob file path.
446    */
447   public Path getPath() {
448     return mobFamilyPath;
449   }
450 
451   /**
452    * The compaction in the store of mob.
453    * The cells in this store contains the path of the mob files. There might be race
454    * condition between the major compaction and the sweeping in mob files.
455    * In order to avoid this, we need mutually exclude the running of the major compaction and
456    * sweeping in mob files.
457    * The minor compaction is not affected.
458    * The major compaction is marked as retainDeleteMarkers when a sweeping is in progress.
459    */
460   @Override
461   public List<StoreFile> compact(CompactionContext compaction,
462       CompactionThroughputController throughputController) throws IOException {
463     // If it's major compaction, try to find whether there's a sweeper is running
464     // If yes, mark the major compaction as retainDeleteMarkers
465     if (compaction.getRequest().isAllFiles()) {
466       // Use the Zookeeper to coordinate.
467       // 1. Acquire a operation lock.
468       //   1.1. If no, mark the major compaction as retainDeleteMarkers and continue the compaction.
469       //   1.2. If the lock is obtained, search the node of sweeping.
470       //      1.2.1. If the node is there, the sweeping is in progress, mark the major
471       //             compaction as retainDeleteMarkers and continue the compaction.
472       //      1.2.2. If the node is not there, add a child to the major compaction node, and
473       //             run the compaction directly.
474       TableLock lock = null;
475       if (tableLockManager != null) {
476         lock = tableLockManager.readLock(tableLockName, "Major compaction in HMobStore");
477       }
478       boolean tableLocked = false;
479       String tableName = getTableName().getNameAsString();
480       if (lock != null) {
481         try {
482           LOG.info("Start to acquire a read lock for the table[" + tableName
483               + "], ready to perform the major compaction");
484           lock.acquire();
485           tableLocked = true;
486         } catch (Exception e) {
487           LOG.error("Fail to lock the table " + tableName, e);
488         }
489       } else {
490         // If the tableLockManager is null, mark the tableLocked as true.
491         tableLocked = true;
492       }
493       try {
494         if (!tableLocked) {
495           LOG.warn("Cannot obtain the table lock, maybe a sweep tool is running on this table["
496               + tableName + "], forcing the delete markers to be retained");
497           compaction.getRequest().forceRetainDeleteMarkers();
498         }
499         return super.compact(compaction, throughputController);
500       } finally {
501         if (tableLocked && lock != null) {
502           try {
503             lock.release();
504           } catch (IOException e) {
505             LOG.error("Fail to release the table lock " + tableName, e);
506           }
507         }
508       }
509     } else {
510       // If it's not a major compaction, continue the compaction.
511       return super.compact(compaction, throughputController);
512     }
513   }
514 
515   public void updateCellsCountCompactedToMob(long count) {
516     cellsCountCompactedToMob += count;
517   }
518 
519   public long getCellsCountCompactedToMob() {
520     return cellsCountCompactedToMob;
521   }
522 
523   public void updateCellsCountCompactedFromMob(long count) {
524     cellsCountCompactedFromMob += count;
525   }
526 
527   public long getCellsCountCompactedFromMob() {
528     return cellsCountCompactedFromMob;
529   }
530 
531   public void updateCellsSizeCompactedToMob(long size) {
532     cellsSizeCompactedToMob += size;
533   }
534 
535   public long getCellsSizeCompactedToMob() {
536     return cellsSizeCompactedToMob;
537   }
538 
539   public void updateCellsSizeCompactedFromMob(long size) {
540     cellsSizeCompactedFromMob += size;
541   }
542 
543   public long getCellsSizeCompactedFromMob() {
544     return cellsSizeCompactedFromMob;
545   }
546 
547   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "VO_VOLATILE_INCREMENT")
548   public void updateMobFlushCount() {
549     mobFlushCount++;
550   }
551 
552   public long getMobFlushCount() {
553     return mobFlushCount;
554   }
555 
556   public void updateMobFlushedCellsCount(long count) {
557     mobFlushedCellsCount += count;
558   }
559 
560   public long getMobFlushedCellsCount() {
561     return mobFlushedCellsCount;
562   }
563 
564   public void updateMobFlushedCellsSize(long size) {
565     mobFlushedCellsSize += size;
566   }
567 
568   public long getMobFlushedCellsSize() {
569     return mobFlushedCellsSize;
570   }
571 
572   public void updateMobScanCellsCount(long count) {
573     mobScanCellsCount += count;
574   }
575 
576   public long getMobScanCellsCount() {
577     return mobScanCellsCount;
578   }
579 
580   public void updateMobScanCellsSize(long size) {
581     mobScanCellsSize += size;
582   }
583 
584   public long getMobScanCellsSize() {
585     return mobScanCellsSize;
586   }
587 }