@InterfaceAudience.Private public class PartitionedMobCompactor extends MobCompactor
MobCompactor
that compacts the mob files in partitions.Modifier and Type | Class and Description |
---|---|
(package private) static class |
PartitionedMobCompactor.DelPartitionComparator |
Modifier and Type | Field and Description |
---|---|
private org.apache.hadoop.fs.Path |
bulkloadPath |
protected int |
compactionBatchSize
The number of files compacted in a batch
|
private CacheConfig |
compactionCacheConfig |
protected int |
compactionKVMax |
private Encryption.Context |
cryptoContext |
protected int |
delFileMaxCount |
private static org.slf4j.Logger |
LOG |
protected long |
mergeableSize |
private byte[] |
refCellTags |
private org.apache.hadoop.fs.Path |
tempPath |
column, conf, fs, mobFamilyDir, mobTableDir, pool, tableName
Constructor and Description |
---|
PartitionedMobCompactor(org.apache.hadoop.conf.Configuration conf,
org.apache.hadoop.fs.FileSystem fs,
TableName tableName,
ColumnFamilyDescriptor column,
ExecutorService pool) |
Modifier and Type | Method and Description |
---|---|
private void |
bulkloadRefFile(Connection connection,
Table table,
org.apache.hadoop.fs.Path bulkloadDirectory,
String fileName)
Bulkloads the current file.
|
private void |
closeMobFileWriter(StoreFileWriter writer,
long maxSeqId,
long mobCellsCount)
Closes the mob file writer.
|
private void |
closeRefFileWriter(StoreFileWriter writer,
long maxSeqId,
long bulkloadTime)
Closes the ref file writer.
|
private void |
closeStoreFileReaders(List<HStoreFile> storeFiles)
Closes the readers of store files.
|
List<org.apache.hadoop.fs.Path> |
compact(List<org.apache.hadoop.fs.FileStatus> files,
boolean allFiles)
Compacts the candidate mob files.
|
protected List<org.apache.hadoop.fs.Path> |
compactDelFiles(PartitionedMobCompactionRequest request,
List<org.apache.hadoop.fs.Path> delFilePaths)
Compacts the del files in batches which avoids opening too many files.
|
private org.apache.hadoop.fs.Path |
compactDelFilesInBatch(PartitionedMobCompactionRequest request,
List<HStoreFile> delFiles)
Compacts the del file in a batch.
|
private List<org.apache.hadoop.fs.Path> |
compactMobFilePartition(PartitionedMobCompactionRequest request,
PartitionedMobCompactionRequest.CompactionPartition partition,
List<HStoreFile> delFiles,
Connection connection,
Table table)
Compacts a partition of selected small mob files and all the del files.
|
protected List<org.apache.hadoop.fs.Path> |
compactMobFiles(PartitionedMobCompactionRequest request)
Compacts the selected small mob files and all the del files.
|
private void |
compactMobFilesInBatch(PartitionedMobCompactionRequest request,
PartitionedMobCompactionRequest.CompactionPartition partition,
Connection connection,
Table table,
List<HStoreFile> filesToCompact,
int batch,
org.apache.hadoop.fs.Path bulkloadPathOfPartition,
org.apache.hadoop.fs.Path bulkloadColumnPath,
List<org.apache.hadoop.fs.Path> newFiles)
Compacts a partition of selected small mob files and all the del files in a batch.
|
private StoreScanner |
createScanner(List<HStoreFile> filesToCompact,
ScanType scanType)
Creates a store scanner.
|
private void |
deletePath(org.apache.hadoop.fs.Path path)
Deletes a file.
|
private Pair<Long,Long> |
getFileInfo(List<HStoreFile> storeFiles)
Gets the max seqId and number of cells of the store files.
|
private org.apache.hadoop.fs.FileStatus |
getLinkedFileStatus(HFileLink link) |
(package private) List<HStoreFile> |
getListOfDelFilesForPartition(PartitionedMobCompactionRequest.CompactionPartition partition,
List<PartitionedMobCompactionRequest.CompactionDelPartition> delPartitions) |
protected List<org.apache.hadoop.fs.Path> |
performCompaction(PartitionedMobCompactionRequest request)
Performs the compaction on the selected files.
|
protected PartitionedMobCompactionRequest |
select(List<org.apache.hadoop.fs.FileStatus> candidates,
boolean allFiles)
Selects the compacted mob/del files.
|
compact, compact
private static final org.slf4j.Logger LOG
protected long mergeableSize
protected int delFileMaxCount
protected int compactionBatchSize
protected int compactionKVMax
private final org.apache.hadoop.fs.Path tempPath
private final org.apache.hadoop.fs.Path bulkloadPath
private final CacheConfig compactionCacheConfig
private final byte[] refCellTags
private Encryption.Context cryptoContext
public PartitionedMobCompactor(org.apache.hadoop.conf.Configuration conf, org.apache.hadoop.fs.FileSystem fs, TableName tableName, ColumnFamilyDescriptor column, ExecutorService pool) throws IOException
IOException
public List<org.apache.hadoop.fs.Path> compact(List<org.apache.hadoop.fs.FileStatus> files, boolean allFiles) throws IOException
MobCompactor
compact
in class MobCompactor
files
- The candidate mob files.allFiles
- Whether add all mob files into the compaction.IOException
protected PartitionedMobCompactionRequest select(List<org.apache.hadoop.fs.FileStatus> candidates, boolean allFiles) throws IOException
candidates
- All the candidates.allFiles
- Whether add all mob files into the compaction.IOException
- if IO failure is encounteredprotected List<org.apache.hadoop.fs.Path> performCompaction(PartitionedMobCompactionRequest request) throws IOException
request
- The compaction request.IOException
- if IO failure is encounteredList<HStoreFile> getListOfDelFilesForPartition(PartitionedMobCompactionRequest.CompactionPartition partition, List<PartitionedMobCompactionRequest.CompactionDelPartition> delPartitions)
protected List<org.apache.hadoop.fs.Path> compactMobFiles(PartitionedMobCompactionRequest request) throws IOException
request
- The compaction request.IOException
- if IO failure is encounteredprivate List<org.apache.hadoop.fs.Path> compactMobFilePartition(PartitionedMobCompactionRequest request, PartitionedMobCompactionRequest.CompactionPartition partition, List<HStoreFile> delFiles, Connection connection, Table table) throws IOException
request
- The compaction request.partition
- A compaction partition.delFiles
- The del files.connection
- The connection to use.table
- The current table.IOException
- if IO failure is encounteredprivate void closeStoreFileReaders(List<HStoreFile> storeFiles)
storeFiles
- The store files to be closed.private void compactMobFilesInBatch(PartitionedMobCompactionRequest request, PartitionedMobCompactionRequest.CompactionPartition partition, Connection connection, Table table, List<HStoreFile> filesToCompact, int batch, org.apache.hadoop.fs.Path bulkloadPathOfPartition, org.apache.hadoop.fs.Path bulkloadColumnPath, List<org.apache.hadoop.fs.Path> newFiles) throws IOException
request
- The compaction request.partition
- A compaction partition.connection
- To use for transporttable
- The current table.filesToCompact
- The files to be compacted.batch
- The number of mob files to be compacted in a batch.bulkloadPathOfPartition
- The directory where the bulkload column of the current
partition is saved.bulkloadColumnPath
- The directory where the bulkload files of current partition
are saved.newFiles
- The paths of new mob files after compactions.IOException
- if IO failure is encounteredprotected List<org.apache.hadoop.fs.Path> compactDelFiles(PartitionedMobCompactionRequest request, List<org.apache.hadoop.fs.Path> delFilePaths) throws IOException
request
- The compaction request.delFilePaths
- Del file paths to compactIOException
- if IO failure is encounteredprivate org.apache.hadoop.fs.Path compactDelFilesInBatch(PartitionedMobCompactionRequest request, List<HStoreFile> delFiles) throws IOException
request
- The compaction request.delFiles
- The del files.IOException
- if IO failure is encounteredprivate StoreScanner createScanner(List<HStoreFile> filesToCompact, ScanType scanType) throws IOException
filesToCompact
- The files to be compacted.scanType
- The scan type.IOException
- if IO failure is encounteredprivate void bulkloadRefFile(Connection connection, Table table, org.apache.hadoop.fs.Path bulkloadDirectory, String fileName) throws IOException
connection
- to use to get admin/RegionLocatortable
- The current table.bulkloadDirectory
- The path of bulkload directory.fileName
- The current file name.IOException
- if IO failure is encounteredprivate void closeMobFileWriter(StoreFileWriter writer, long maxSeqId, long mobCellsCount) throws IOException
writer
- The mob file writer.maxSeqId
- Maximum sequence id.mobCellsCount
- The number of mob cells.IOException
- if IO failure is encounteredprivate void closeRefFileWriter(StoreFileWriter writer, long maxSeqId, long bulkloadTime) throws IOException
writer
- The ref file writer.maxSeqId
- Maximum sequence id.bulkloadTime
- The timestamp at which the bulk load file is created.IOException
- if IO failure is encounteredprivate Pair<Long,Long> getFileInfo(List<HStoreFile> storeFiles) throws IOException
storeFiles
- The store files.IOException
- if IO failure is encounteredprivate void deletePath(org.apache.hadoop.fs.Path path)
path
- The path of the file to be deleted.private org.apache.hadoop.fs.FileStatus getLinkedFileStatus(HFileLink link) throws IOException
IOException
Copyright © 2007–2020 The Apache Software Foundation. All rights reserved.