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.regionserver; 019 020import java.io.FileNotFoundException; 021import java.io.IOException; 022import java.util.ArrayList; 023import java.util.Date; 024import java.util.List; 025import java.util.Map; 026import java.util.NavigableSet; 027import java.util.UUID; 028import java.util.concurrent.ConcurrentHashMap; 029import java.util.concurrent.atomic.AtomicLong; 030 031import org.apache.hadoop.conf.Configuration; 032import org.apache.hadoop.fs.FileSystem; 033import org.apache.hadoop.fs.Path; 034import org.apache.hadoop.hbase.ArrayBackedTag; 035import org.apache.hadoop.hbase.Cell; 036import org.apache.hadoop.hbase.CellBuilderType; 037import org.apache.hadoop.hbase.CellComparator; 038import org.apache.hadoop.hbase.DoNotRetryIOException; 039import org.apache.hadoop.hbase.ExtendedCellBuilderFactory; 040import org.apache.hadoop.hbase.HConstants; 041import org.apache.hadoop.hbase.TableName; 042import org.apache.hadoop.hbase.Tag; 043import org.apache.hadoop.hbase.TagType; 044import org.apache.hadoop.hbase.TagUtil; 045import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; 046import org.apache.hadoop.hbase.client.Scan; 047import org.apache.hadoop.hbase.filter.Filter; 048import org.apache.hadoop.hbase.filter.FilterList; 049import org.apache.hadoop.hbase.io.compress.Compression; 050import org.apache.hadoop.hbase.io.hfile.CorruptHFileException; 051import org.apache.hadoop.hbase.mob.MobConstants; 052import org.apache.hadoop.hbase.mob.MobFile; 053import org.apache.hadoop.hbase.mob.MobFileCache; 054import org.apache.hadoop.hbase.mob.MobFileName; 055import org.apache.hadoop.hbase.mob.MobStoreEngine; 056import org.apache.hadoop.hbase.mob.MobUtils; 057import org.apache.hadoop.hbase.util.Bytes; 058import org.apache.hadoop.hbase.util.HFileArchiveUtil; 059import org.apache.hadoop.hbase.util.IdLock; 060import org.apache.yetus.audience.InterfaceAudience; 061import org.slf4j.Logger; 062import org.slf4j.LoggerFactory; 063 064/** 065 * The store implementation to save MOBs (medium objects), it extends the HStore. 066 * When a descriptor of a column family has the value "IS_MOB", it means this column family 067 * is a mob one. When a HRegion instantiate a store for this column family, the HMobStore is 068 * created. 069 * HMobStore is almost the same with the HStore except using different types of scanners. 070 * In the method of getScanner, the MobStoreScanner and MobReversedStoreScanner are returned. 071 * In these scanners, a additional seeks in the mob files should be performed after the seek 072 * to HBase is done. 073 * The store implements how we save MOBs by extending HStore. When a descriptor 074 * of a column family has the value "IS_MOB", it means this column family is a mob one. When a 075 * HRegion instantiate a store for this column family, the HMobStore is created. HMobStore is 076 * almost the same with the HStore except using different types of scanners. In the method of 077 * getScanner, the MobStoreScanner and MobReversedStoreScanner are returned. In these scanners, a 078 * additional seeks in the mob files should be performed after the seek in HBase is done. 079 */ 080@InterfaceAudience.Private 081public class HMobStore extends HStore { 082 private static final Logger LOG = LoggerFactory.getLogger(HMobStore.class); 083 private MobFileCache mobFileCache; 084 private Path homePath; 085 private Path mobFamilyPath; 086 private AtomicLong cellsCountCompactedToMob = new AtomicLong(); 087 private AtomicLong cellsCountCompactedFromMob = new AtomicLong(); 088 private AtomicLong cellsSizeCompactedToMob = new AtomicLong(); 089 private AtomicLong cellsSizeCompactedFromMob = new AtomicLong(); 090 private AtomicLong mobFlushCount = new AtomicLong(); 091 private AtomicLong mobFlushedCellsCount = new AtomicLong(); 092 private AtomicLong mobFlushedCellsSize = new AtomicLong(); 093 private AtomicLong mobScanCellsCount = new AtomicLong(); 094 private AtomicLong mobScanCellsSize = new AtomicLong(); 095 private ColumnFamilyDescriptor family; 096 private Map<String, List<Path>> map = new ConcurrentHashMap<>(); 097 private final IdLock keyLock = new IdLock(); 098 // When we add a MOB reference cell to the HFile, we will add 2 tags along with it 099 // 1. A ref tag with type TagType.MOB_REFERENCE_TAG_TYPE. This just denote this this cell is not 100 // original one but a ref to another MOB Cell. 101 // 2. Table name tag. It's very useful in cloning the snapshot. When reading from the cloning 102 // table, we need to find the original mob files by this table name. For details please see 103 // cloning snapshot for mob files. 104 private final byte[] refCellTags; 105 106 public HMobStore(final HRegion region, final ColumnFamilyDescriptor family, 107 final Configuration confParam, boolean warmup) throws IOException { 108 super(region, family, confParam, warmup); 109 this.family = family; 110 this.mobFileCache = region.getMobFileCache(); 111 this.homePath = MobUtils.getMobHome(conf); 112 this.mobFamilyPath = MobUtils.getMobFamilyPath(conf, this.getTableName(), 113 family.getNameAsString()); 114 List<Path> locations = new ArrayList<>(2); 115 locations.add(mobFamilyPath); 116 TableName tn = region.getTableDescriptor().getTableName(); 117 locations.add(HFileArchiveUtil.getStoreArchivePath(conf, tn, MobUtils.getMobRegionInfo(tn) 118 .getEncodedName(), family.getNameAsString())); 119 map.put(Bytes.toString(tn.getName()), locations); 120 List<Tag> tags = new ArrayList<>(2); 121 tags.add(MobConstants.MOB_REF_TAG); 122 Tag tableNameTag = new ArrayBackedTag(TagType.MOB_TABLE_NAME_TAG_TYPE, 123 getTableName().getName()); 124 tags.add(tableNameTag); 125 this.refCellTags = TagUtil.fromList(tags); 126 } 127 128 /** 129 * Gets current config. 130 */ 131 public Configuration getConfiguration() { 132 return this.conf; 133 } 134 135 /** 136 * Gets the MobStoreScanner or MobReversedStoreScanner. In these scanners, a additional seeks in 137 * the mob files should be performed after the seek in HBase is done. 138 */ 139 @Override 140 protected KeyValueScanner createScanner(Scan scan, ScanInfo scanInfo, 141 NavigableSet<byte[]> targetCols, long readPt) throws IOException { 142 if (MobUtils.isRefOnlyScan(scan)) { 143 Filter refOnlyFilter = new MobReferenceOnlyFilter(); 144 Filter filter = scan.getFilter(); 145 if (filter != null) { 146 scan.setFilter(new FilterList(filter, refOnlyFilter)); 147 } else { 148 scan.setFilter(refOnlyFilter); 149 } 150 } 151 return scan.isReversed() ? new ReversedMobStoreScanner(this, scanInfo, scan, targetCols, readPt) 152 : new MobStoreScanner(this, scanInfo, scan, targetCols, readPt); 153 } 154 155 /** 156 * Creates the mob store engine. 157 */ 158 @Override 159 protected StoreEngine<?, ?, ?, ?> createStoreEngine(HStore store, Configuration conf, 160 CellComparator cellComparator) throws IOException { 161 MobStoreEngine engine = new MobStoreEngine(); 162 engine.createComponents(conf, store, cellComparator); 163 return engine; 164 } 165 166 /** 167 * Gets the temp directory. 168 * @return The temp directory. 169 */ 170 private Path getTempDir() { 171 return new Path(homePath, MobConstants.TEMP_DIR_NAME); 172 } 173 174 /** 175 * Creates the writer for the mob file in temp directory. 176 * @param date The latest date of written cells. 177 * @param maxKeyCount The key count. 178 * @param compression The compression algorithm. 179 * @param startKey The start key. 180 * @param isCompaction If the writer is used in compaction. 181 * @return The writer for the mob file. 182 * @throws IOException 183 */ 184 public StoreFileWriter createWriterInTmp(Date date, long maxKeyCount, 185 Compression.Algorithm compression, byte[] startKey, 186 boolean isCompaction) throws IOException { 187 if (startKey == null) { 188 startKey = HConstants.EMPTY_START_ROW; 189 } 190 Path path = getTempDir(); 191 return createWriterInTmp(MobUtils.formatDate(date), path, maxKeyCount, compression, startKey, 192 isCompaction); 193 } 194 195 /** 196 * Creates the writer for the del file in temp directory. 197 * The del file keeps tracking the delete markers. Its name has a suffix _del, 198 * the format is [0-9a-f]+(_del)?. 199 * @param date The latest date of written cells. 200 * @param maxKeyCount The key count. 201 * @param compression The compression algorithm. 202 * @param startKey The start key. 203 * @return The writer for the del file. 204 * @throws IOException 205 */ 206 public StoreFileWriter createDelFileWriterInTmp(Date date, long maxKeyCount, 207 Compression.Algorithm compression, byte[] startKey) throws IOException { 208 if (startKey == null) { 209 startKey = HConstants.EMPTY_START_ROW; 210 } 211 Path path = getTempDir(); 212 String suffix = UUID 213 .randomUUID().toString().replaceAll("-", "") + "_del"; 214 MobFileName mobFileName = MobFileName.create(startKey, MobUtils.formatDate(date), suffix); 215 return createWriterInTmp(mobFileName, path, maxKeyCount, compression, true); 216 } 217 218 /** 219 * Creates the writer for the mob file in temp directory. 220 * @param date The date string, its format is yyyymmmdd. 221 * @param basePath The basic path for a temp directory. 222 * @param maxKeyCount The key count. 223 * @param compression The compression algorithm. 224 * @param startKey The start key. 225 * @param isCompaction If the writer is used in compaction. 226 * @return The writer for the mob file. 227 * @throws IOException 228 */ 229 public StoreFileWriter createWriterInTmp(String date, Path basePath, long maxKeyCount, 230 Compression.Algorithm compression, byte[] startKey, 231 boolean isCompaction) throws IOException { 232 MobFileName mobFileName = MobFileName.create(startKey, date, UUID.randomUUID() 233 .toString().replaceAll("-", "")); 234 return createWriterInTmp(mobFileName, basePath, maxKeyCount, compression, isCompaction); 235 } 236 237 /** 238 * Creates the writer for the mob file in temp directory. 239 * @param mobFileName The mob file name. 240 * @param basePath The basic path for a temp directory. 241 * @param maxKeyCount The key count. 242 * @param compression The compression algorithm. 243 * @param isCompaction If the writer is used in compaction. 244 * @return The writer for the mob file. 245 * @throws IOException 246 */ 247 public StoreFileWriter createWriterInTmp(MobFileName mobFileName, Path basePath, 248 long maxKeyCount, Compression.Algorithm compression, 249 boolean isCompaction) throws IOException { 250 return MobUtils.createWriter(conf, region.getFilesystem(), family, 251 new Path(basePath, mobFileName.getFileName()), maxKeyCount, compression, cacheConf, 252 cryptoContext, checksumType, bytesPerChecksum, blocksize, BloomType.NONE, isCompaction); 253 } 254 255 /** 256 * Commits the mob file. 257 * @param sourceFile The source file. 258 * @param targetPath The directory path where the source file is renamed to. 259 * @throws IOException 260 */ 261 public void commitFile(final Path sourceFile, Path targetPath) throws IOException { 262 if (sourceFile == null) { 263 return; 264 } 265 Path dstPath = new Path(targetPath, sourceFile.getName()); 266 validateMobFile(sourceFile); 267 String msg = "Renaming flushed file from " + sourceFile + " to " + dstPath; 268 LOG.info(msg); 269 Path parent = dstPath.getParent(); 270 if (!region.getFilesystem().exists(parent)) { 271 region.getFilesystem().mkdirs(parent); 272 } 273 if (!region.getFilesystem().rename(sourceFile, dstPath)) { 274 throw new IOException("Failed rename of " + sourceFile + " to " + dstPath); 275 } 276 } 277 278 /** 279 * Validates a mob file by opening and closing it. 280 * 281 * @param path the path to the mob file 282 */ 283 private void validateMobFile(Path path) throws IOException { 284 HStoreFile storeFile = null; 285 try { 286 storeFile = new HStoreFile(region.getFilesystem(), path, conf, this.cacheConf, 287 BloomType.NONE, isPrimaryReplicaStore()); 288 storeFile.initReader(); 289 } catch (IOException e) { 290 LOG.error("Fail to open mob file[" + path + "], keep it in temp directory.", e); 291 throw e; 292 } finally { 293 if (storeFile != null) { 294 storeFile.closeStoreFile(false); 295 } 296 } 297 } 298 299 /** 300 * Reads the cell from the mob file, and the read point does not count. 301 * This is used for DefaultMobStoreCompactor where we can read empty value for the missing cell. 302 * @param reference The cell found in the HBase, its value is a path to a mob file. 303 * @param cacheBlocks Whether the scanner should cache blocks. 304 * @return The cell found in the mob file. 305 * @throws IOException 306 */ 307 public Cell resolve(Cell reference, boolean cacheBlocks) throws IOException { 308 return resolve(reference, cacheBlocks, -1, true); 309 } 310 311 /** 312 * Reads the cell from the mob file. 313 * @param reference The cell found in the HBase, its value is a path to a mob file. 314 * @param cacheBlocks Whether the scanner should cache blocks. 315 * @param readPt the read point. 316 * @param readEmptyValueOnMobCellMiss Whether return null value when the mob file is 317 * missing or corrupt. 318 * @return The cell found in the mob file. 319 * @throws IOException 320 */ 321 public Cell resolve(Cell reference, boolean cacheBlocks, long readPt, 322 boolean readEmptyValueOnMobCellMiss) throws IOException { 323 Cell result = null; 324 if (MobUtils.hasValidMobRefCellValue(reference)) { 325 String fileName = MobUtils.getMobFileName(reference); 326 Tag tableNameTag = MobUtils.getTableNameTag(reference); 327 if (tableNameTag != null) { 328 String tableNameString = Tag.getValueAsString(tableNameTag); 329 List<Path> locations = map.get(tableNameString); 330 if (locations == null) { 331 IdLock.Entry lockEntry = keyLock.getLockEntry(tableNameString.hashCode()); 332 try { 333 locations = map.get(tableNameString); 334 if (locations == null) { 335 locations = new ArrayList<>(2); 336 TableName tn = TableName.valueOf(tableNameString); 337 locations.add(MobUtils.getMobFamilyPath(conf, tn, family.getNameAsString())); 338 locations.add(HFileArchiveUtil.getStoreArchivePath(conf, tn, MobUtils 339 .getMobRegionInfo(tn).getEncodedName(), family.getNameAsString())); 340 map.put(tableNameString, locations); 341 } 342 } finally { 343 keyLock.releaseLockEntry(lockEntry); 344 } 345 } 346 result = readCell(locations, fileName, reference, cacheBlocks, readPt, 347 readEmptyValueOnMobCellMiss); 348 } 349 } 350 if (result == null) { 351 LOG.warn("The Cell result is null, assemble a new Cell with the same row,family," 352 + "qualifier,timestamp,type and tags but with an empty value to return."); 353 result = ExtendedCellBuilderFactory.create(CellBuilderType.DEEP_COPY) 354 .setRow(reference.getRowArray(), reference.getRowOffset(), reference.getRowLength()) 355 .setFamily(reference.getFamilyArray(), reference.getFamilyOffset(), 356 reference.getFamilyLength()) 357 .setQualifier(reference.getQualifierArray(), 358 reference.getQualifierOffset(), reference.getQualifierLength()) 359 .setTimestamp(reference.getTimestamp()) 360 .setType(reference.getTypeByte()) 361 .setValue(HConstants.EMPTY_BYTE_ARRAY) 362 .setTags(reference.getTagsArray(), reference.getTagsOffset(), 363 reference.getTagsLength()) 364 .build(); 365 } 366 return result; 367 } 368 369 /** 370 * Reads the cell from a mob file. 371 * The mob file might be located in different directories. 372 * 1. The working directory. 373 * 2. The archive directory. 374 * Reads the cell from the files located in both of the above directories. 375 * @param locations The possible locations where the mob files are saved. 376 * @param fileName The file to be read. 377 * @param search The cell to be searched. 378 * @param cacheMobBlocks Whether the scanner should cache blocks. 379 * @param readPt the read point. 380 * @param readEmptyValueOnMobCellMiss Whether return null value when the mob file is 381 * missing or corrupt. 382 * @return The found cell. Null if there's no such a cell. 383 * @throws IOException 384 */ 385 private Cell readCell(List<Path> locations, String fileName, Cell search, boolean cacheMobBlocks, 386 long readPt, boolean readEmptyValueOnMobCellMiss) throws IOException { 387 FileSystem fs = getFileSystem(); 388 Throwable throwable = null; 389 for (Path location : locations) { 390 MobFile file = null; 391 Path path = new Path(location, fileName); 392 try { 393 file = mobFileCache.openFile(fs, path, cacheConf); 394 return readPt != -1 ? file.readCell(search, cacheMobBlocks, readPt) : file.readCell(search, 395 cacheMobBlocks); 396 } catch (IOException e) { 397 mobFileCache.evictFile(fileName); 398 throwable = e; 399 if ((e instanceof FileNotFoundException) || 400 (e.getCause() instanceof FileNotFoundException)) { 401 LOG.debug("Fail to read the cell, the mob file " + path + " doesn't exist", e); 402 } else if (e instanceof CorruptHFileException) { 403 LOG.error("The mob file " + path + " is corrupt", e); 404 break; 405 } else { 406 throw e; 407 } 408 } catch (NullPointerException e) { // HDFS 1.x - DFSInputStream.getBlockAt() 409 mobFileCache.evictFile(fileName); 410 LOG.debug("Fail to read the cell", e); 411 throwable = e; 412 } catch (AssertionError e) { // assert in HDFS 1.x - DFSInputStream.getBlockAt() 413 mobFileCache.evictFile(fileName); 414 LOG.debug("Fail to read the cell", e); 415 throwable = e; 416 } finally { 417 if (file != null) { 418 mobFileCache.closeFile(file); 419 } 420 } 421 } 422 LOG.error("The mob file " + fileName + " could not be found in the locations " + locations 423 + " or it is corrupt"); 424 if (readEmptyValueOnMobCellMiss) { 425 return null; 426 } else if ((throwable instanceof FileNotFoundException) 427 || (throwable.getCause() instanceof FileNotFoundException)) { 428 // The region is re-opened when FileNotFoundException is thrown. 429 // This is not necessary when MOB files cannot be found, because the store files 430 // in a region only contain the references to MOB files and a re-open on a region 431 // doesn't help fix the lost MOB files. 432 throw new DoNotRetryIOException(throwable); 433 } else if (throwable instanceof IOException) { 434 throw (IOException) throwable; 435 } else { 436 throw new IOException(throwable); 437 } 438 } 439 440 /** 441 * Gets the mob file path. 442 * @return The mob file path. 443 */ 444 public Path getPath() { 445 return mobFamilyPath; 446 } 447 448 public void updateCellsCountCompactedToMob(long count) { 449 cellsCountCompactedToMob.addAndGet(count); 450 } 451 452 public long getCellsCountCompactedToMob() { 453 return cellsCountCompactedToMob.get(); 454 } 455 456 public void updateCellsCountCompactedFromMob(long count) { 457 cellsCountCompactedFromMob.addAndGet(count); 458 } 459 460 public long getCellsCountCompactedFromMob() { 461 return cellsCountCompactedFromMob.get(); 462 } 463 464 public void updateCellsSizeCompactedToMob(long size) { 465 cellsSizeCompactedToMob.addAndGet(size); 466 } 467 468 public long getCellsSizeCompactedToMob() { 469 return cellsSizeCompactedToMob.get(); 470 } 471 472 public void updateCellsSizeCompactedFromMob(long size) { 473 cellsSizeCompactedFromMob.addAndGet(size); 474 } 475 476 public long getCellsSizeCompactedFromMob() { 477 return cellsSizeCompactedFromMob.get(); 478 } 479 480 public void updateMobFlushCount() { 481 mobFlushCount.incrementAndGet(); 482 } 483 484 public long getMobFlushCount() { 485 return mobFlushCount.get(); 486 } 487 488 public void updateMobFlushedCellsCount(long count) { 489 mobFlushedCellsCount.addAndGet(count); 490 } 491 492 public long getMobFlushedCellsCount() { 493 return mobFlushedCellsCount.get(); 494 } 495 496 public void updateMobFlushedCellsSize(long size) { 497 mobFlushedCellsSize.addAndGet(size); 498 } 499 500 public long getMobFlushedCellsSize() { 501 return mobFlushedCellsSize.get(); 502 } 503 504 public void updateMobScanCellsCount(long count) { 505 mobScanCellsCount.addAndGet(count); 506 } 507 508 public long getMobScanCellsCount() { 509 return mobScanCellsCount.get(); 510 } 511 512 public void updateMobScanCellsSize(long size) { 513 mobScanCellsSize.addAndGet(size); 514 } 515 516 public long getMobScanCellsSize() { 517 return mobScanCellsSize.get(); 518 } 519 520 public byte[] getRefCellTags() { 521 return this.refCellTags; 522 } 523}