Modifier and Type | Field and Description |
---|---|
private static ThreadLocal<SimpleDateFormat> |
LOCAL_FORMAT |
private static org.slf4j.Logger |
LOG |
private static long |
MONTHLY_THRESHOLD_MULTIPLIER |
private static byte[] |
REF_DELETE_MARKER_TAG_BYTES |
private static long |
WEEKLY_THRESHOLD_MULTIPLIER |
Modifier | Constructor and Description |
---|---|
private |
MobUtils()
Private constructor to keep this class from being instantiated.
|
Modifier and Type | Method and Description |
---|---|
static void |
cleanExpiredMobFiles(org.apache.hadoop.fs.FileSystem fs,
org.apache.hadoop.conf.Configuration conf,
TableName tableName,
ColumnFamilyDescriptor columnDescriptor,
CacheConfig cacheConfig,
long current)
Cleans the expired mob files.
|
static org.apache.hadoop.fs.Path |
commitFile(org.apache.hadoop.conf.Configuration conf,
org.apache.hadoop.fs.FileSystem fs,
org.apache.hadoop.fs.Path sourceFile,
org.apache.hadoop.fs.Path targetPath,
CacheConfig cacheConfig)
Commits the mob file.
|
static StoreFileWriter |
createDelFileWriter(org.apache.hadoop.conf.Configuration conf,
org.apache.hadoop.fs.FileSystem fs,
ColumnFamilyDescriptor family,
String date,
org.apache.hadoop.fs.Path basePath,
long maxKeyCount,
Compression.Algorithm compression,
byte[] startKey,
CacheConfig cacheConfig,
Encryption.Context cryptoContext)
Creates a writer for the del file in temp directory.
|
static ExecutorService |
createMobCompactorThreadPool(org.apache.hadoop.conf.Configuration conf)
Creates a thread pool.
|
static Cell |
createMobRefCell(Cell cell,
byte[] fileName,
byte[] refCellTags) |
static Cell |
createMobRefCell(Cell cell,
byte[] fileName,
Tag tableNameTag)
Creates a mob reference KeyValue.
|
static Cell |
createMobRefDeleteMarker(Cell cell)
Creates a mob ref delete marker.
|
static StoreFileWriter |
createRefFileWriter(org.apache.hadoop.conf.Configuration conf,
org.apache.hadoop.fs.FileSystem fs,
ColumnFamilyDescriptor family,
org.apache.hadoop.fs.Path basePath,
long maxKeyCount,
CacheConfig cacheConfig,
Encryption.Context cryptoContext,
boolean isCompaction)
Creates a writer for the ref file in temp directory.
|
static StoreFileWriter |
createWriter(org.apache.hadoop.conf.Configuration conf,
org.apache.hadoop.fs.FileSystem fs,
ColumnFamilyDescriptor family,
MobFileName mobFileName,
org.apache.hadoop.fs.Path basePath,
long maxKeyCount,
Compression.Algorithm compression,
CacheConfig cacheConfig,
Encryption.Context cryptoContext,
boolean isCompaction)
Creates a writer for the mob file in temp directory.
|
static StoreFileWriter |
createWriter(org.apache.hadoop.conf.Configuration conf,
org.apache.hadoop.fs.FileSystem fs,
ColumnFamilyDescriptor family,
org.apache.hadoop.fs.Path path,
long maxKeyCount,
Compression.Algorithm compression,
CacheConfig cacheConfig,
Encryption.Context cryptoContext,
ChecksumType checksumType,
int bytesPerChecksum,
int blocksize,
BloomType bloomType,
boolean isCompaction)
Creates a writer for the mob file in temp directory.
|
static StoreFileWriter |
createWriter(org.apache.hadoop.conf.Configuration conf,
org.apache.hadoop.fs.FileSystem fs,
ColumnFamilyDescriptor family,
String date,
org.apache.hadoop.fs.Path basePath,
long maxKeyCount,
Compression.Algorithm compression,
byte[] startKey,
CacheConfig cacheConfig,
Encryption.Context cryptoContext,
boolean isCompaction)
Creates a writer for the mob file in temp directory.
|
static StoreFileWriter |
createWriter(org.apache.hadoop.conf.Configuration conf,
org.apache.hadoop.fs.FileSystem fs,
ColumnFamilyDescriptor family,
String date,
org.apache.hadoop.fs.Path basePath,
long maxKeyCount,
Compression.Algorithm compression,
String startKey,
CacheConfig cacheConfig,
Encryption.Context cryptoContext,
boolean isCompaction)
Creates a writer for the mob file in temp directory.
|
static void |
doMobCompaction(org.apache.hadoop.conf.Configuration conf,
org.apache.hadoop.fs.FileSystem fs,
TableName tableName,
ColumnFamilyDescriptor hcd,
ExecutorService pool,
boolean allFiles,
LockManager.MasterLock lock)
Performs the mob compaction.
|
static boolean |
fillPartitionId(PartitionedMobCompactionRequest.CompactionPartitionId id,
Date firstDayOfCurrentMonth,
Date firstDayOfCurrentWeek,
String dateStr,
MobCompactPartitionPolicy policy,
Calendar calendar,
long threshold)
fill out partition id based on compaction policy and date, threshold...
|
static String |
formatDate(Date date)
Formats a date to a string.
|
static org.apache.hadoop.fs.Path |
getCompactionWorkingPath(org.apache.hadoop.fs.Path root,
String jobName)
Gets the working directory of the mob compaction.
|
static Date |
getFirstDayOfMonth(Calendar calendar,
Date date)
Get the first day of the input date's month
|
static Date |
getFirstDayOfWeek(Calendar calendar,
Date date)
Get the first day of the input date's week
|
static org.apache.hadoop.fs.Path |
getMobFamilyPath(org.apache.hadoop.conf.Configuration conf,
TableName tableName,
String familyName)
Gets the family dir of the mob files.
|
static org.apache.hadoop.fs.Path |
getMobFamilyPath(org.apache.hadoop.fs.Path regionPath,
String familyName)
Gets the family dir of the mob files.
|
static String |
getMobFileName(Cell cell)
Gets the mob file name from the mob ref cell.
|
static org.apache.hadoop.fs.Path |
getMobHome(org.apache.hadoop.conf.Configuration conf)
Gets the root dir of the mob files.
|
static org.apache.hadoop.fs.Path |
getMobHome(org.apache.hadoop.fs.Path rootDir)
Gets the root dir of the mob files under the qualified HBase root dir.
|
static RegionInfo |
getMobRegionInfo(TableName tableName)
Gets the RegionInfo of the mob files.
|
static org.apache.hadoop.fs.Path |
getMobRegionPath(org.apache.hadoop.conf.Configuration conf,
TableName tableName)
Gets the region dir of the mob files.
|
static org.apache.hadoop.fs.Path |
getMobRegionPath(org.apache.hadoop.fs.Path rootDir,
TableName tableName)
Gets the region dir of the mob files under the specified root dir.
|
static org.apache.hadoop.fs.Path |
getMobTableDir(org.apache.hadoop.fs.Path rootDir,
TableName tableName)
Gets the table dir of the mob files under the qualified HBase root dir.
|
static int |
getMobValueLength(Cell cell)
Gets the mob value length from the mob ref cell.
|
static org.apache.hadoop.fs.Path |
getQualifiedMobRootDir(org.apache.hadoop.conf.Configuration conf)
Gets the qualified root dir of the mob files.
|
static TableName |
getTableLockName(TableName tn)
Gets the table name used in the table lock.
|
static Tag |
getTableNameTag(Cell cell)
Gets the table name tag.
|
static boolean |
hasMobColumns(TableDescriptor htd)
Checks whether this table has mob-enabled columns.
|
static boolean |
hasMobReferenceTag(List<Tag> tags)
Whether the tag list has a mob reference tag.
|
static boolean |
hasValidMobRefCellValue(Cell cell)
Indicates whether the current mob ref cell has a valid value.
|
static boolean |
isCacheMobBlocks(Scan scan)
Indicates whether the scan contains the information of caching blocks.
|
static boolean |
isMobFileExpired(ColumnFamilyDescriptor column,
long current,
String fileDate)
Checks if the mob file is expired.
|
static boolean |
isMobReferenceCell(Cell cell)
Whether the current cell is a mob reference cell.
|
static boolean |
isMobRegionInfo(RegionInfo regionInfo)
Gets whether the current RegionInfo is a mob one.
|
static boolean |
isMobRegionName(TableName tableName,
byte[] regionName)
Gets whether the current region name follows the pattern of a mob region name.
|
static boolean |
isRawMobScan(Scan scan)
Indicates whether it's a raw scan.
|
static boolean |
isReadEmptyValueOnMobCellMiss(Scan scan)
Indicates whether return null value when the mob file is missing or corrupt.
|
static boolean |
isRefOnlyScan(Scan scan)
Indicates whether it's a reference only scan.
|
static Date |
parseDate(String dateString)
Parses the string to a date.
|
static void |
removeMobFiles(org.apache.hadoop.conf.Configuration conf,
org.apache.hadoop.fs.FileSystem fs,
TableName tableName,
org.apache.hadoop.fs.Path tableDir,
byte[] family,
Collection<HStoreFile> storeFiles)
Archives the mob files.
|
static void |
setCacheMobBlocks(Scan scan,
boolean cacheBlocks)
Sets the attribute of caching blocks in the scan.
|
private static void |
validateMobFile(org.apache.hadoop.conf.Configuration conf,
org.apache.hadoop.fs.FileSystem fs,
org.apache.hadoop.fs.Path path,
CacheConfig cacheConfig,
boolean primaryReplica)
Validates a mob file by opening and closing it.
|
private static final org.slf4j.Logger LOG
private static final long WEEKLY_THRESHOLD_MULTIPLIER
private static final long MONTHLY_THRESHOLD_MULTIPLIER
private static final ThreadLocal<SimpleDateFormat> LOCAL_FORMAT
private static final byte[] REF_DELETE_MARKER_TAG_BYTES
private MobUtils()
public static String formatDate(Date date)
date
- The date.public static Date parseDate(String dateString) throws ParseException
dateString
- The string format of a date, it's yyyymmdd.ParseException
public static Date getFirstDayOfMonth(Calendar calendar, Date date)
calendar
- Calendar objectdate
- The date to find out its first day of that monthpublic static Date getFirstDayOfWeek(Calendar calendar, Date date)
calendar
- Calendar objectdate
- The date to find out its first day of that weekpublic static boolean isMobReferenceCell(Cell cell)
cell
- The current cell.public static Tag getTableNameTag(Cell cell)
cell
- The current cell.public static boolean hasMobReferenceTag(List<Tag> tags)
tags
- The tag list.public static boolean isRawMobScan(Scan scan)
scan
- The current scan.public static boolean isRefOnlyScan(Scan scan)
scan
- The current scan.public static boolean isCacheMobBlocks(Scan scan)
scan
- The current scan.public static void setCacheMobBlocks(Scan scan, boolean cacheBlocks)
scan
- The current scan.cacheBlocks
- True, set the attribute of caching blocks into the scan, the scanner with this scan
caches blocks.
False, the scanner doesn't cache blocks for this scan.public static void cleanExpiredMobFiles(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.conf.Configuration conf, TableName tableName, ColumnFamilyDescriptor columnDescriptor, CacheConfig cacheConfig, long current) throws IOException
fs
- The current file system.conf
- The current configuration.tableName
- The current table name.columnDescriptor
- The descriptor of the current column family.cacheConfig
- The cacheConfig that disables the block cache.current
- The current time.IOException
public static org.apache.hadoop.fs.Path getMobHome(org.apache.hadoop.conf.Configuration conf)
conf
- The current configuration.public static org.apache.hadoop.fs.Path getMobHome(org.apache.hadoop.fs.Path rootDir)
rootDir
- The qualified path of HBase root directory.public static org.apache.hadoop.fs.Path getQualifiedMobRootDir(org.apache.hadoop.conf.Configuration conf) throws IOException
conf
- The current configuration.IOException
public static org.apache.hadoop.fs.Path getMobTableDir(org.apache.hadoop.fs.Path rootDir, TableName tableName)
rootDir
- The qualified path of HBase root directory.tableName
- The name of table.public static org.apache.hadoop.fs.Path getMobRegionPath(org.apache.hadoop.conf.Configuration conf, TableName tableName)
conf
- The current configuration.tableName
- The current table name.public static org.apache.hadoop.fs.Path getMobRegionPath(org.apache.hadoop.fs.Path rootDir, TableName tableName)
rootDir
- The qualified path of HBase root directory.tableName
- The current table name.public static org.apache.hadoop.fs.Path getMobFamilyPath(org.apache.hadoop.conf.Configuration conf, TableName tableName, String familyName)
conf
- The current configuration.tableName
- The current table name.familyName
- The current family name.public static org.apache.hadoop.fs.Path getMobFamilyPath(org.apache.hadoop.fs.Path regionPath, String familyName)
regionPath
- The path of mob region which is a dummy one.familyName
- The current family name.public static RegionInfo getMobRegionInfo(TableName tableName)
tableName
- public static boolean isMobRegionInfo(RegionInfo regionInfo)
regionInfo
- The current RegionInfo.public static boolean isMobRegionName(TableName tableName, byte[] regionName)
tableName
- The current table name.regionName
- The current region name.public static org.apache.hadoop.fs.Path getCompactionWorkingPath(org.apache.hadoop.fs.Path root, String jobName)
root
- The root directory of the mob compaction.jobName
- The current job name.public static void removeMobFiles(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, TableName tableName, org.apache.hadoop.fs.Path tableDir, byte[] family, Collection<HStoreFile> storeFiles) throws IOException
conf
- The current configuration.fs
- The current file system.tableName
- The table name.tableDir
- The table directory.family
- The name of the column family.storeFiles
- The files to be deleted.IOException
public static Cell createMobRefCell(Cell cell, byte[] fileName, Tag tableNameTag)
cell
- The original Cell.fileName
- The mob file name where the mob reference KeyValue is written.tableNameTag
- The tag of the current table name. It's very important in
cloning the snapshot.public static Cell createMobRefCell(Cell cell, byte[] fileName, byte[] refCellTags)
public static StoreFileWriter createWriter(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, ColumnFamilyDescriptor family, String date, org.apache.hadoop.fs.Path basePath, long maxKeyCount, Compression.Algorithm compression, String startKey, CacheConfig cacheConfig, Encryption.Context cryptoContext, boolean isCompaction) throws IOException
conf
- The current configuration.fs
- The current file system.family
- The descriptor of the current column family.date
- The date string, its format is yyyymmmdd.basePath
- The basic path for a temp directory.maxKeyCount
- The key count.compression
- The compression algorithm.startKey
- The hex string of the start key.cacheConfig
- The current cache config.cryptoContext
- The encryption context.isCompaction
- If the writer is used in compaction.IOException
public static StoreFileWriter createRefFileWriter(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, ColumnFamilyDescriptor family, org.apache.hadoop.fs.Path basePath, long maxKeyCount, CacheConfig cacheConfig, Encryption.Context cryptoContext, boolean isCompaction) throws IOException
conf
- The current configuration.fs
- The current file system.family
- The descriptor of the current column family.basePath
- The basic path for a temp directory.maxKeyCount
- The key count.cacheConfig
- The current cache config.cryptoContext
- The encryption context.isCompaction
- If the writer is used in compaction.IOException
public static StoreFileWriter createWriter(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, ColumnFamilyDescriptor family, String date, org.apache.hadoop.fs.Path basePath, long maxKeyCount, Compression.Algorithm compression, byte[] startKey, CacheConfig cacheConfig, Encryption.Context cryptoContext, boolean isCompaction) throws IOException
conf
- The current configuration.fs
- The current file system.family
- The descriptor of the current column family.date
- The date string, its format is yyyymmmdd.basePath
- The basic path for a temp directory.maxKeyCount
- The key count.compression
- The compression algorithm.startKey
- The start key.cacheConfig
- The current cache config.cryptoContext
- The encryption context.isCompaction
- If the writer is used in compaction.IOException
public static StoreFileWriter createDelFileWriter(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, ColumnFamilyDescriptor family, String date, org.apache.hadoop.fs.Path basePath, long maxKeyCount, Compression.Algorithm compression, byte[] startKey, CacheConfig cacheConfig, Encryption.Context cryptoContext) throws IOException
conf
- The current configuration.fs
- The current file system.family
- The descriptor of the current column family.date
- The date string, its format is yyyymmmdd.basePath
- The basic path for a temp directory.maxKeyCount
- The key count.compression
- The compression algorithm.startKey
- The start key.cacheConfig
- The current cache config.cryptoContext
- The encryption context.IOException
public static StoreFileWriter createWriter(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, ColumnFamilyDescriptor family, MobFileName mobFileName, org.apache.hadoop.fs.Path basePath, long maxKeyCount, Compression.Algorithm compression, CacheConfig cacheConfig, Encryption.Context cryptoContext, boolean isCompaction) throws IOException
conf
- The current configuration.fs
- The current file system.family
- The descriptor of the current column family.mobFileName
- The mob file name.basePath
- The basic path for a temp directory.maxKeyCount
- The key count.compression
- The compression algorithm.cacheConfig
- The current cache config.cryptoContext
- The encryption context.isCompaction
- If the writer is used in compaction.IOException
public static StoreFileWriter createWriter(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, ColumnFamilyDescriptor family, org.apache.hadoop.fs.Path path, long maxKeyCount, Compression.Algorithm compression, CacheConfig cacheConfig, Encryption.Context cryptoContext, ChecksumType checksumType, int bytesPerChecksum, int blocksize, BloomType bloomType, boolean isCompaction) throws IOException
conf
- The current configuration.fs
- The current file system.family
- The descriptor of the current column family.path
- The path for a temp directory.maxKeyCount
- The key count.compression
- The compression algorithm.cacheConfig
- The current cache config.cryptoContext
- The encryption context.checksumType
- The checksum type.bytesPerChecksum
- The bytes per checksum.blocksize
- The HFile block size.bloomType
- The bloom filter type.isCompaction
- If the writer is used in compaction.IOException
public static org.apache.hadoop.fs.Path commitFile(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path sourceFile, org.apache.hadoop.fs.Path targetPath, CacheConfig cacheConfig) throws IOException
conf
- The current configuration.fs
- The current file system.sourceFile
- The path where the mob file is saved.targetPath
- The directory path where the source file is renamed to.cacheConfig
- The current cache config.IOException
private static void validateMobFile(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path path, CacheConfig cacheConfig, boolean primaryReplica) throws IOException
conf
- The current configuration.fs
- The current file system.path
- The path where the mob file is saved.cacheConfig
- The current cache config.IOException
public static boolean hasValidMobRefCellValue(Cell cell)
cell
- The mob ref cell.public static int getMobValueLength(Cell cell)
cell
- The mob ref cell.public static String getMobFileName(Cell cell)
cell
- The mob ref cell.public static TableName getTableLockName(TableName tn)
tn
- The table name.public static void doMobCompaction(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, TableName tableName, ColumnFamilyDescriptor hcd, ExecutorService pool, boolean allFiles, LockManager.MasterLock lock) throws IOException
conf
- the Configurationfs
- the file systemtableName
- the table the compacthcd
- the column descriptorpool
- the thread poolallFiles
- Whether add all mob files into the compaction.IOException
public static ExecutorService createMobCompactorThreadPool(org.apache.hadoop.conf.Configuration conf)
conf
- the Configurationpublic static boolean hasMobColumns(TableDescriptor htd)
htd
- The current table descriptor.public static boolean isReadEmptyValueOnMobCellMiss(Scan scan)
scan
- The current scan.public static Cell createMobRefDeleteMarker(Cell cell)
cell
- The current delete marker.public static boolean isMobFileExpired(ColumnFamilyDescriptor column, long current, String fileDate)
column
- The descriptor of the current column family.current
- The current time.fileDate
- The date string parsed from the mob file name.public static boolean fillPartitionId(PartitionedMobCompactionRequest.CompactionPartitionId id, Date firstDayOfCurrentMonth, Date firstDayOfCurrentWeek, String dateStr, MobCompactPartitionPolicy policy, Calendar calendar, long threshold)
id
- Partition id to be filled outfirstDayOfCurrentMonth
- The first day in the current monthfirstDayOfCurrentWeek
- The first day in the current weekdateStr
- Date string from the mob filepolicy
- Mob compaction policycalendar
- Calendar objectthreshold
- Mob compaciton threshold configuredCopyright © 2007–2019 The Apache Software Foundation. All rights reserved.