HBASE-21263 Mention compression algorithm along with other storefile details
Signed-off-by: Andrew Purtell <apurtell@apache.org> Amending-Author: Andrew Purtell <apurtell@apache.org>
This commit is contained in:
parent
85c3ec3fb4
commit
dd836aae12
|
@ -321,10 +321,11 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
|
|||
confPrintThreshold = 10;
|
||||
}
|
||||
this.parallelPutCountPrintThreshold = confPrintThreshold;
|
||||
LOG.info("Store={}, memstore type={}, storagePolicy={}, verifyBulkLoads={}, " +
|
||||
"parallelPutCountPrintThreshold={}", getColumnFamilyName(),
|
||||
this.memstore.getClass().getSimpleName(), policyName,
|
||||
this.verifyBulkLoads, this.parallelPutCountPrintThreshold);
|
||||
LOG.info("Store={}, memstore type={}, storagePolicy={}, verifyBulkLoads={}, "
|
||||
+ "parallelPutCountPrintThreshold={}, encoding={}, compression={}",
|
||||
getColumnFamilyName(), memstore.getClass().getSimpleName(), policyName, verifyBulkLoads,
|
||||
parallelPutCountPrintThreshold, family.getDataBlockEncoding(),
|
||||
family.getCompressionType());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -199,12 +199,14 @@ public abstract class Compactor<T extends CellSink> {
|
|||
}
|
||||
tmp = fileInfo.get(TIMERANGE_KEY);
|
||||
fd.latestPutTs = tmp == null ? HConstants.LATEST_TIMESTAMP: TimeRangeTracker.parseFrom(tmp).getMax();
|
||||
LOG.debug("Compacting {}, keycount={}, bloomtype={}, size={}, encoding={}, seqNum={}{}",
|
||||
LOG.debug("Compacting {}, keycount={}, bloomtype={}, size={}, "
|
||||
+ "encoding={}, compression={}, seqNum={}{}",
|
||||
(file.getPath() == null? null: file.getPath().getName()),
|
||||
keyCount,
|
||||
r.getBloomFilterType().toString(),
|
||||
TraditionalBinaryPrefix.long2String(r.length(), "", 1),
|
||||
r.getHFileReader().getDataBlockEncoding(),
|
||||
compactionCompression,
|
||||
seqNum,
|
||||
(allFiles? ", earliestPutTs=" + earliestPutTs: ""));
|
||||
}
|
||||
|
|
|
@ -299,11 +299,9 @@ public class TestChecksum {
|
|||
long expectedChunks = ChecksumUtil.numChunks(
|
||||
dataSize + HConstants.HFILEBLOCK_HEADER_SIZE,
|
||||
bytesPerChecksum);
|
||||
LOG.info("testChecksumChunks: pread=" + pread +
|
||||
", bytesPerChecksum=" + bytesPerChecksum +
|
||||
", fileSize=" + totalSize +
|
||||
", dataSize=" + dataSize +
|
||||
", expectedChunks=" + expectedChunks);
|
||||
LOG.info("testChecksumChunks: pread={}, bytesPerChecksum={}, fileSize={}, "
|
||||
+ "dataSize={}, expectedChunks={}, compression={}", pread, bytesPerChecksum,
|
||||
totalSize, dataSize, expectedChunks, algo.toString());
|
||||
|
||||
// Verify hbase checksums.
|
||||
assertEquals(true, hfs.useHBaseChecksum());
|
||||
|
|
|
@ -376,6 +376,8 @@ public class TestHFileBlock {
|
|||
for (Compression.Algorithm algo : COMPRESSION_ALGORITHMS) {
|
||||
for (boolean pread : new boolean[] { false, true }) {
|
||||
for (DataBlockEncoding encoding : DataBlockEncoding.values()) {
|
||||
LOG.info("testDataBlockEncoding: Compression algorithm={}, pread={}, dataBlockEncoder={}",
|
||||
algo.toString(), pread, encoding);
|
||||
Path path = new Path(TEST_UTIL.getDataTestDir(), "blocks_v2_"
|
||||
+ algo + "_" + encoding.toString());
|
||||
FSDataOutputStream os = fs.create(path);
|
||||
|
@ -534,9 +536,8 @@ public class TestHFileBlock {
|
|||
for (boolean pread : BOOLEAN_VALUES) {
|
||||
for (boolean cacheOnWrite : BOOLEAN_VALUES) {
|
||||
Random rand = defaultRandom();
|
||||
LOG.info("testPreviousOffset:Compression algorithm: " + algo +
|
||||
", pread=" + pread +
|
||||
", cacheOnWrite=" + cacheOnWrite);
|
||||
LOG.info("testPreviousOffset: Compression algorithm={}, pread={}, cacheOnWrite={}",
|
||||
algo.toString(), pread, cacheOnWrite);
|
||||
Path path = new Path(TEST_UTIL.getDataTestDir(), "prev_offset");
|
||||
List<Long> expectedOffsets = new ArrayList<>();
|
||||
List<Long> expectedPrevOffsets = new ArrayList<>();
|
||||
|
|
|
@ -202,7 +202,7 @@ public class TestHFileBlockIndex {
|
|||
|
||||
private void readIndex(boolean useTags) throws IOException {
|
||||
long fileSize = fs.getFileStatus(path).getLen();
|
||||
LOG.info("Size of " + path + ": " + fileSize);
|
||||
LOG.info("Size of {}: {} compression={}", path, fileSize, compr.toString());
|
||||
|
||||
FSDataInputStream istream = fs.open(path);
|
||||
HFileContext meta = new HFileContextBuilder()
|
||||
|
|
|
@ -209,7 +209,7 @@ public class CreateRandomStoreFile {
|
|||
|
||||
Path storeFilePath = sfw.getPath();
|
||||
long fileSize = fs.getFileStatus(storeFilePath).getLen();
|
||||
LOG.info("Created " + storeFilePath + ", " + fileSize + " bytes");
|
||||
LOG.info("Created {}, {} bytes, compression={}", storeFilePath, fileSize, compr.toString());
|
||||
|
||||
return true;
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue