HBASE-24371 Add more details when print CompactionConfiguration info (#1711)

Signed-off-by: Anoop Sam John <anoopsamjohn@apache.org>
This commit is contained in:
binlijin 2020-05-27 19:41:43 +08:00 committed by GitHub
parent 476cb16232
commit aacb69d098
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 17 additions and 2 deletions

View File

@ -18,6 +18,7 @@
*/
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
@ -53,4 +54,8 @@ public interface StoreConfigInformation {
* The number of files required before flushes for this store will be blocked.
*/
long getBlockingFileCount();
RegionInfo getRegionInfo();
String getColumnFamilyName();
}

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.regionserver.compactions;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.util.StringUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
@ -156,7 +157,8 @@ public class CompactionConfiguration {
+ " major period %d, major jitter %f, min locality to compact %f;"
+ " tiered compaction: max_age %d, incoming window min %d,"
+ " compaction policy for tiered window %s, single output for minor %b,"
+ " compaction window factory %s",
+ " compaction window factory %s,"
+ " region %s columnFamilyName %s",
StringUtils.byteDesc(minCompactSize),
StringUtils.byteDesc(maxCompactSize),
StringUtils.byteDesc(offPeakMaxCompactSize),
@ -172,7 +174,9 @@ public class CompactionConfiguration {
dateTieredIncomingWindowMin,
compactionPolicyForDateTieredWindow,
dateTieredSingleOutputForMinorCompaction,
dateTieredCompactionWindowFactory
dateTieredCompactionWindowFactory,
RegionInfo.prettyPrint(storeConfigInfo.getRegionInfo().getEncodedName()),
storeConfigInfo.getColumnFamilyName()
);
}

View File

@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CellComparatorImpl;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@ -65,6 +66,7 @@ public class TestDefaultStoreEngine {
conf.set(DefaultStoreEngine.DEFAULT_STORE_FLUSHER_CLASS_KEY,
DummyStoreFlusher.class.getName());
HStore mockStore = Mockito.mock(HStore.class);
Mockito.when(mockStore.getRegionInfo()).thenReturn(HRegionInfo.FIRST_META_REGIONINFO);
StoreEngine<?, ?, ?, ?> se = StoreEngine.create(mockStore, conf, CellComparatorImpl.COMPARATOR);
Assert.assertTrue(se instanceof DefaultStoreEngine);
Assert.assertTrue(se.getCompactionPolicy() instanceof DummyCompactionPolicy);

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CellComparatorImpl;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl;
import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy;
@ -118,6 +119,7 @@ public class TestStripeStoreEngine {
private static TestStoreEngine createEngine(Configuration conf) throws Exception {
HStore store = mock(HStore.class);
when(store.getRegionInfo()).thenReturn(HRegionInfo.FIRST_META_REGIONINFO);
CellComparatorImpl kvComparator = mock(CellComparatorImpl.class);
return (TestStoreEngine)StoreEngine.create(store, conf, kvComparator);
}

View File

@ -163,6 +163,7 @@ public class TestStripeCompactionPolicy {
conf.setInt(StripeStoreConfig.MAX_FILES_KEY, 4);
conf.setLong(StripeStoreConfig.SIZE_TO_SPLIT_KEY, 1000); // make sure the are no splits
StoreConfigInformation sci = mock(StoreConfigInformation.class);
when(sci.getRegionInfo()).thenReturn(HRegionInfo.FIRST_META_REGIONINFO);
StripeStoreConfig ssc = new StripeStoreConfig(conf, sci);
StripeCompactionPolicy policy = new StripeCompactionPolicy(conf, sci, ssc) {
@Override
@ -479,6 +480,7 @@ public class TestStripeCompactionPolicy {
conf.setInt(StripeStoreConfig.INITIAL_STRIPE_COUNT_KEY, initialCount);
StoreConfigInformation sci = mock(StoreConfigInformation.class);
when(sci.getStoreFileTtl()).thenReturn(hasTtl ? defaultTtl : Long.MAX_VALUE);
when(sci.getRegionInfo()).thenReturn(HRegionInfo.FIRST_META_REGIONINFO);
StripeStoreConfig ssc = new StripeStoreConfig(conf, sci);
return new StripeCompactionPolicy(conf, sci, ssc);
}