HBASE-22749 Distributed MOB compactions
- MOB compaction is now handled in-line with per-region compaction on region servers - regions with mob data store per-hfile metadata about which mob hfiles are referenced - admin requested major compaction will also rewrite MOB files; periodic RS initiated major compaction will not - periodically a chore in the master will initiate a major compaction that will rewrite MOB values to ensure it happens. controlled by 'hbase.mob.compaction.chore.period'. default is weekly - control how many RS the chore requests major compaction on in parallel with 'hbase.mob.major.compaction.region.batch.size'. default is as parallel as possible. - periodic chore in master will scan backing hfiles from regions to get the set of referenced mob hfiles and archive those that are no longer referenced. control period with 'hbase.master.mob.cleaner.period' - Optionally, RS that are compacting mob files can limit write amplification by not rewriting values from mob hfiles over a certain size limit. opt-in by setting 'hbase.mob.compaction.type' to 'optimized'. control threshold by 'hbase.mob.compactions.max.file.size'. default is 1GiB - Should smoothly integrate with existing MOB users via rolling upgrade. will delay old MOB file cleanup until per-region compaction has managed to compact each region at least once so that used mob hfile metadata can be gathered.
This commit is contained in:
parent
53afc1dc10
commit
b8194b4902
|
@ -55,4 +55,5 @@
|
||||||
<suppress checks="HideUtilityClassConstructor" files="org.apache.hadoop.hbase.util.ByteRangeUtils"/>
|
<suppress checks="HideUtilityClassConstructor" files="org.apache.hadoop.hbase.util.ByteRangeUtils"/>
|
||||||
<!-- Will not be final, because it is InterfaceAudience.Public -->
|
<!-- Will not be final, because it is InterfaceAudience.Public -->
|
||||||
<suppress checks="FinalClass" files="org.apache.hadoop.hbase.net.Address"/>
|
<suppress checks="FinalClass" files="org.apache.hadoop.hbase.net.Address"/>
|
||||||
|
<suppress checks="MethodLength" files="org.apache.hadoop.hbase.mob.FaultyMobStoreCompactor.java"/>
|
||||||
</suppressions>
|
</suppressions>
|
||||||
|
|
|
@ -1814,10 +1814,10 @@ possible configurations would overwhelm and obscure the important.
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>hbase.master.mob.ttl.cleaner.period</name>
|
<name>hbase.master.mob.cleaner.period</name>
|
||||||
<value>86400</value>
|
<value>86400</value>
|
||||||
<description>
|
<description>
|
||||||
The period that ExpiredMobFileCleanerChore runs. The unit is second.
|
The period that MobFileCleanerChore runs. The unit is second.
|
||||||
The default value is one day. The MOB file name uses only the date part of
|
The default value is one day. The MOB file name uses only the date part of
|
||||||
the file creation time in it. We use this time for deciding TTL expiry of
|
the file creation time in it. We use this time for deciding TTL expiry of
|
||||||
the files. So the removal of TTL expired files might be delayed. The max
|
the files. So the removal of TTL expired files might be delayed. The max
|
||||||
|
@ -1825,32 +1825,13 @@ possible configurations would overwhelm and obscure the important.
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>hbase.mob.compaction.mergeable.threshold</name>
|
<name>hbase.mob.major.compaction.region.batch.size</name>
|
||||||
<value>1342177280</value>
|
<value>0</value>
|
||||||
<description>
|
<description>
|
||||||
If the size of a mob file is less than this value, it's regarded as a small
|
The max number of a MOB table regions that is allowed in a batch of the mob compaction. By
|
||||||
file and needs to be merged in mob compaction. The default value is 1280MB.
|
setting this number to a custom value, users can control the overall effect of a major
|
||||||
</description>
|
compaction of a large MOB-enabled table.
|
||||||
</property>
|
Default is 0 - means no limit - all regions of a MOB table will be compacted at once
|
||||||
<property>
|
|
||||||
<name>hbase.mob.delfile.max.count</name>
|
|
||||||
<value>3</value>
|
|
||||||
<description>
|
|
||||||
The max number of del files that is allowed in the mob compaction.
|
|
||||||
In the mob compaction, when the number of existing del files is larger than
|
|
||||||
this value, they are merged until number of del files is not larger this value.
|
|
||||||
The default value is 3.
|
|
||||||
</description>
|
|
||||||
</property>
|
|
||||||
<property>
|
|
||||||
<name>hbase.mob.compaction.batch.size</name>
|
|
||||||
<value>100</value>
|
|
||||||
<description>
|
|
||||||
The max number of the mob files that is allowed in a batch of the mob compaction.
|
|
||||||
The mob compaction merges the small mob files to bigger ones. If the number of the
|
|
||||||
small files is very large, it could lead to a "too many opened file handlers" in the merge.
|
|
||||||
And the merge has to be split into batches. This value limits the number of mob files
|
|
||||||
that are selected in a batch of the mob compaction. The default value is 100.
|
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
|
@ -1861,20 +1842,6 @@ possible configurations would overwhelm and obscure the important.
|
||||||
The default value is one week.
|
The default value is one week.
|
||||||
</description>
|
</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
|
||||||
<name>hbase.mob.compactor.class</name>
|
|
||||||
<value>org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactor</value>
|
|
||||||
<description>
|
|
||||||
Implementation of mob compactor, the default one is PartitionedMobCompactor.
|
|
||||||
</description>
|
|
||||||
</property>
|
|
||||||
<property>
|
|
||||||
<name>hbase.mob.compaction.threads.max</name>
|
|
||||||
<value>1</value>
|
|
||||||
<description>
|
|
||||||
The max number of threads used in MobCompactor.
|
|
||||||
</description>
|
|
||||||
</property>
|
|
||||||
<property>
|
<property>
|
||||||
<name>hbase.snapshot.master.timeout.millis</name>
|
<name>hbase.snapshot.master.timeout.millis</name>
|
||||||
<value>300000</value>
|
<value>300000</value>
|
||||||
|
|
|
@ -0,0 +1,419 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.hbase;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.hbase.client.Admin;
|
||||||
|
import org.apache.hadoop.hbase.client.Put;
|
||||||
|
import org.apache.hadoop.hbase.client.Result;
|
||||||
|
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||||
|
import org.apache.hadoop.hbase.client.Table;
|
||||||
|
import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner;
|
||||||
|
import org.apache.hadoop.hbase.mob.FaultyMobStoreCompactor;
|
||||||
|
import org.apache.hadoop.hbase.mob.MobConstants;
|
||||||
|
import org.apache.hadoop.hbase.mob.MobFileCleanerChore;
|
||||||
|
import org.apache.hadoop.hbase.mob.MobStoreEngine;
|
||||||
|
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hbase.testclassification.IntegrationTests;
|
||||||
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.hadoop.util.ToolRunner;
|
||||||
|
|
||||||
|
import org.junit.After;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects;
|
||||||
|
import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An integration test to detect regressions in HBASE-22749. Test creates
|
||||||
|
* MOB-enabled table, and runs in parallel, the following tasks: loads data,
|
||||||
|
* runs MOB compactions, runs MOB cleaning chore. The failure injections into MOB
|
||||||
|
* compaction cycle is implemented via specific sub-class of DefaultMobStoreCompactor -
|
||||||
|
* FaultyMobStoreCompactor. The probability of failure is controlled by command-line
|
||||||
|
* argument 'failprob'.
|
||||||
|
* @see <a href="https://issues.apache.org/jira/browse/HBASE-22749">HBASE-22749</a>
|
||||||
|
* <p>
|
||||||
|
* Sample usage:
|
||||||
|
* <pre>
|
||||||
|
* hbase org.apache.hadoop.hbase.IntegrationTestMobCompaction -Dservers=10 -Drows=1000000
|
||||||
|
* -Dfailprob=0.2
|
||||||
|
* </pre>
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("deprecation")
|
||||||
|
|
||||||
|
@Category(IntegrationTests.class)
|
||||||
|
public class IntegrationTestMobCompaction extends IntegrationTestBase {
|
||||||
|
protected static final Logger LOG = LoggerFactory.getLogger(IntegrationTestMobCompaction.class);
|
||||||
|
|
||||||
|
protected static final String REGIONSERVER_COUNT_KEY = "servers";
|
||||||
|
protected static final String ROWS_COUNT_KEY = "rows";
|
||||||
|
protected static final String FAILURE_PROB_KEY = "failprob";
|
||||||
|
|
||||||
|
protected static final int DEFAULT_REGIONSERVER_COUNT = 3;
|
||||||
|
protected static final int DEFAULT_ROWS_COUNT = 5000000;
|
||||||
|
protected static final double DEFAULT_FAILURE_PROB = 0.1;
|
||||||
|
|
||||||
|
protected static int regionServerCount = DEFAULT_REGIONSERVER_COUNT;
|
||||||
|
protected static long rowsToLoad = DEFAULT_ROWS_COUNT;
|
||||||
|
protected static double failureProb = DEFAULT_FAILURE_PROB;
|
||||||
|
|
||||||
|
protected static String famStr = "f1";
|
||||||
|
protected static byte[] fam = Bytes.toBytes(famStr);
|
||||||
|
protected static byte[] qualifier = Bytes.toBytes("q1");
|
||||||
|
protected static long mobLen = 10;
|
||||||
|
protected static byte[] mobVal = Bytes
|
||||||
|
.toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789");
|
||||||
|
|
||||||
|
private static Configuration conf;
|
||||||
|
private static HTableDescriptor hdt;
|
||||||
|
private static HColumnDescriptor hcd;
|
||||||
|
private static Admin admin;
|
||||||
|
private static Table table = null;
|
||||||
|
private static MobFileCleanerChore chore;
|
||||||
|
|
||||||
|
private static volatile boolean run = true;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Before
|
||||||
|
public void setUp() throws Exception {
|
||||||
|
util = getTestingUtil(getConf());
|
||||||
|
conf = util.getConfiguration();
|
||||||
|
// Initialize with test-specific configuration values
|
||||||
|
initConf(conf);
|
||||||
|
regionServerCount =
|
||||||
|
conf.getInt(REGIONSERVER_COUNT_KEY, DEFAULT_REGIONSERVER_COUNT);
|
||||||
|
LOG.info("Initializing cluster with {} region servers.", regionServerCount);
|
||||||
|
util.initializeCluster(regionServerCount);
|
||||||
|
admin = util.getAdmin();
|
||||||
|
|
||||||
|
createTestTable();
|
||||||
|
|
||||||
|
LOG.info("Cluster initialized and ready");
|
||||||
|
}
|
||||||
|
|
||||||
|
private void createTestTable() throws IOException {
|
||||||
|
// Create test table
|
||||||
|
hdt = util.createTableDescriptor("testMobCompactTable");
|
||||||
|
hcd = new HColumnDescriptor(fam);
|
||||||
|
hcd.setMobEnabled(true);
|
||||||
|
hcd.setMobThreshold(mobLen);
|
||||||
|
hcd.setMaxVersions(1);
|
||||||
|
hdt.addFamily(hcd);
|
||||||
|
table = util.createTable(hdt, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void tearDown() throws IOException {
|
||||||
|
LOG.info("Cleaning up after test.");
|
||||||
|
if(util.isDistributedCluster()) {
|
||||||
|
deleteTablesIfAny();
|
||||||
|
// TODO
|
||||||
|
}
|
||||||
|
LOG.info("Restoring cluster.");
|
||||||
|
util.restoreCluster();
|
||||||
|
LOG.info("Cluster restored.");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setUpMonkey() throws Exception {
|
||||||
|
// Sorry, no Monkey
|
||||||
|
String msg = "Chaos monkey is not supported";
|
||||||
|
LOG.warn(msg);
|
||||||
|
throw new IOException(msg);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void deleteTablesIfAny() throws IOException {
|
||||||
|
if (table != null) {
|
||||||
|
util.deleteTableIfAny(table.getName());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void setUpCluster() throws Exception {
|
||||||
|
util = getTestingUtil(getConf());
|
||||||
|
LOG.debug("Initializing/checking cluster has {} servers",regionServerCount);
|
||||||
|
util.initializeCluster(regionServerCount);
|
||||||
|
LOG.debug("Done initializing/checking cluster");
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* @return status of CLI execution
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public int runTestFromCommandLine() throws Exception {
|
||||||
|
testMobCompaction();
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public TableName getTablename() {
|
||||||
|
// That is only valid when Monkey is CALM (no monkey)
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Set<String> getColumnFamilies() {
|
||||||
|
// That is only valid when Monkey is CALM (no monkey)
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void addOptions() {
|
||||||
|
addOptWithArg(REGIONSERVER_COUNT_KEY,
|
||||||
|
"Total number of region servers. Default: '" + DEFAULT_REGIONSERVER_COUNT + "'");
|
||||||
|
addOptWithArg(ROWS_COUNT_KEY,
|
||||||
|
"Total number of data rows to load. Default: '" + DEFAULT_ROWS_COUNT + "'");
|
||||||
|
addOptWithArg(FAILURE_PROB_KEY,
|
||||||
|
"Probability of a failure of a region MOB compaction request. Default: '"
|
||||||
|
+ DEFAULT_FAILURE_PROB + "'");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void processOptions(CommandLine cmd) {
|
||||||
|
super.processOptions(cmd);
|
||||||
|
|
||||||
|
regionServerCount =
|
||||||
|
Integer.parseInt(cmd.getOptionValue(REGIONSERVER_COUNT_KEY,
|
||||||
|
Integer.toString(DEFAULT_REGIONSERVER_COUNT)));
|
||||||
|
rowsToLoad =
|
||||||
|
Long.parseLong(cmd.getOptionValue(ROWS_COUNT_KEY,
|
||||||
|
Long.toString(DEFAULT_ROWS_COUNT)));
|
||||||
|
failureProb = Double.parseDouble(cmd.getOptionValue(FAILURE_PROB_KEY,
|
||||||
|
Double.toString(DEFAULT_FAILURE_PROB)));
|
||||||
|
|
||||||
|
LOG.info(MoreObjects.toStringHelper("Parsed Options")
|
||||||
|
.add(REGIONSERVER_COUNT_KEY, regionServerCount)
|
||||||
|
.add(ROWS_COUNT_KEY, rowsToLoad)
|
||||||
|
.add(FAILURE_PROB_KEY, failureProb)
|
||||||
|
.toString());
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void initConf(Configuration conf) {
|
||||||
|
|
||||||
|
conf.setInt("hfile.format.version", 3);
|
||||||
|
conf.setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0);
|
||||||
|
conf.setInt("hbase.client.retries.number", 100);
|
||||||
|
conf.setInt("hbase.hregion.max.filesize", 200000000);
|
||||||
|
conf.setInt("hbase.hregion.memstore.flush.size", 800000);
|
||||||
|
conf.setInt("hbase.hstore.blockingStoreFiles", 150);
|
||||||
|
conf.setInt("hbase.hstore.compaction.throughput.lower.bound", 52428800);
|
||||||
|
conf.setInt("hbase.hstore.compaction.throughput.higher.bound", 2 * 52428800);
|
||||||
|
conf.setDouble("hbase.mob.compaction.fault.probability", failureProb);
|
||||||
|
conf.set(MobStoreEngine.MOB_COMPACTOR_CLASS_KEY,
|
||||||
|
FaultyMobStoreCompactor.class.getName());
|
||||||
|
conf.setBoolean("hbase.table.sanity.checks", false);
|
||||||
|
conf.setLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, 20000);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
class MajorCompaction implements Runnable {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
while (run) {
|
||||||
|
try {
|
||||||
|
admin.majorCompact(hdt.getTableName(), fam);
|
||||||
|
Thread.sleep(120000);
|
||||||
|
} catch (Exception e) {
|
||||||
|
LOG.error("MOB Stress Test FAILED", e);
|
||||||
|
System.exit(-1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
class CleanMobAndArchive implements Runnable {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
while (run) {
|
||||||
|
try {
|
||||||
|
LOG.info("MOB cleanup chore started ...");
|
||||||
|
if (chore == null) {
|
||||||
|
chore = new MobFileCleanerChore();
|
||||||
|
}
|
||||||
|
chore.cleanupObsoleteMobFiles(conf, table.getName());
|
||||||
|
LOG.info("MOB cleanup chore finished");
|
||||||
|
|
||||||
|
Thread.sleep(130000);
|
||||||
|
} catch (Exception e) {
|
||||||
|
e.printStackTrace();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
class WriteData implements Runnable {
|
||||||
|
|
||||||
|
private long rows = -1;
|
||||||
|
|
||||||
|
public WriteData(long rows) {
|
||||||
|
this.rows = rows;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
try {
|
||||||
|
|
||||||
|
// BufferedMutator bm = admin.getConnection().getBufferedMutator(table.getName());
|
||||||
|
// Put Operation
|
||||||
|
for (int i = 0; i < rows; i++) {
|
||||||
|
Put p = new Put(Bytes.toBytes(i));
|
||||||
|
p.addColumn(fam, qualifier, mobVal);
|
||||||
|
table.put(p);
|
||||||
|
|
||||||
|
// bm.mutate(p);
|
||||||
|
if (i % 10000 == 0) {
|
||||||
|
LOG.info("LOADED=" + i);
|
||||||
|
try {
|
||||||
|
Thread.sleep(500);
|
||||||
|
} catch (InterruptedException ee) {
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (i % 100000 == 0) {
|
||||||
|
printStats(i);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// bm.flush();
|
||||||
|
admin.flush(table.getName());
|
||||||
|
run = false;
|
||||||
|
} catch (Exception e) {
|
||||||
|
LOG.error("MOB Stress Test FAILED", e);
|
||||||
|
System.exit(-1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMobCompaction() throws InterruptedException, IOException {
|
||||||
|
|
||||||
|
try {
|
||||||
|
|
||||||
|
Thread writeData = new Thread(new WriteData(rowsToLoad));
|
||||||
|
writeData.start();
|
||||||
|
|
||||||
|
Thread majorcompact = new Thread(new MajorCompaction());
|
||||||
|
majorcompact.start();
|
||||||
|
|
||||||
|
Thread cleaner = new Thread(new CleanMobAndArchive());
|
||||||
|
cleaner.start();
|
||||||
|
|
||||||
|
while (run) {
|
||||||
|
Thread.sleep(1000);
|
||||||
|
}
|
||||||
|
|
||||||
|
getNumberOfMobFiles(conf, table.getName(), new String(fam));
|
||||||
|
LOG.info("Waiting for write thread to finish ...");
|
||||||
|
writeData.join();
|
||||||
|
// Cleanup again
|
||||||
|
chore.cleanupObsoleteMobFiles(conf, table.getName());
|
||||||
|
|
||||||
|
if (util != null) {
|
||||||
|
LOG.info("Archive cleaner started ...");
|
||||||
|
// Call archive cleaner again
|
||||||
|
util.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting();
|
||||||
|
LOG.info("Archive cleaner finished");
|
||||||
|
}
|
||||||
|
|
||||||
|
scanTable();
|
||||||
|
|
||||||
|
} finally {
|
||||||
|
|
||||||
|
admin.disableTable(hdt.getTableName());
|
||||||
|
admin.deleteTable(hdt.getTableName());
|
||||||
|
}
|
||||||
|
LOG.info("MOB Stress Test finished OK");
|
||||||
|
printStats(rowsToLoad);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
private long getNumberOfMobFiles(Configuration conf, TableName tableName, String family)
|
||||||
|
throws IOException {
|
||||||
|
FileSystem fs = FileSystem.get(conf);
|
||||||
|
Path dir = MobUtils.getMobFamilyPath(conf, tableName, family);
|
||||||
|
FileStatus[] stat = fs.listStatus(dir);
|
||||||
|
for (FileStatus st : stat) {
|
||||||
|
LOG.debug("MOB Directory content: {}", st.getPath());
|
||||||
|
}
|
||||||
|
LOG.debug("MOB Directory content total files: {}", stat.length);
|
||||||
|
|
||||||
|
return stat.length;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void printStats(long loaded) {
|
||||||
|
LOG.info("MOB Stress Test: loaded=" + loaded + " compactions="
|
||||||
|
+ FaultyMobStoreCompactor.totalCompactions.get() + " major="
|
||||||
|
+ FaultyMobStoreCompactor.totalMajorCompactions.get() + " mob="
|
||||||
|
+ FaultyMobStoreCompactor.mobCounter.get() + " injected failures="
|
||||||
|
+ FaultyMobStoreCompactor.totalFailures.get());
|
||||||
|
}
|
||||||
|
|
||||||
|
private void scanTable() {
|
||||||
|
try {
|
||||||
|
|
||||||
|
Result result;
|
||||||
|
ResultScanner scanner = table.getScanner(fam);
|
||||||
|
int counter = 0;
|
||||||
|
while ((result = scanner.next()) != null) {
|
||||||
|
assertTrue(Arrays.equals(result.getValue(fam, qualifier), mobVal));
|
||||||
|
if (counter % 10000 == 0) {
|
||||||
|
LOG.info("GET=" + counter);
|
||||||
|
}
|
||||||
|
counter++;
|
||||||
|
}
|
||||||
|
assertEquals(rowsToLoad, counter);
|
||||||
|
} catch (Exception e) {
|
||||||
|
e.printStackTrace();
|
||||||
|
LOG.error("MOB Stress Test FAILED");
|
||||||
|
if (util != null) {
|
||||||
|
assertTrue(false);
|
||||||
|
} else {
|
||||||
|
System.exit(-1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void main(String[] args) throws Exception {
|
||||||
|
Configuration conf = HBaseConfiguration.create();
|
||||||
|
initConf(conf);
|
||||||
|
IntegrationTestingUtility.setUseDistributedCluster(conf);
|
||||||
|
int status = ToolRunner.run(conf, new IntegrationTestMobCompaction(), args);
|
||||||
|
System.exit(status);
|
||||||
|
}
|
||||||
|
}
|
|
@ -217,6 +217,10 @@ public class HFileWriterImpl implements HFile.Writer {
|
||||||
HFile.updateWriteLatency(System.currentTimeMillis() - startTime);
|
HFile.updateWriteLatency(System.currentTimeMillis() - startTime);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public long getPos() throws IOException {
|
||||||
|
return outputStream.getPos();
|
||||||
|
|
||||||
|
}
|
||||||
/**
|
/**
|
||||||
* Checks that the given Cell's key does not violate the key order.
|
* Checks that the given Cell's key does not violate the key order.
|
||||||
*
|
*
|
||||||
|
|
|
@ -1,87 +0,0 @@
|
||||||
/**
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package org.apache.hadoop.hbase.master;
|
|
||||||
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.concurrent.TimeUnit;
|
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.ScheduledChore;
|
|
||||||
import org.apache.hadoop.hbase.TableDescriptors;
|
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
|
||||||
import org.slf4j.Logger;
|
|
||||||
import org.slf4j.LoggerFactory;
|
|
||||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
|
||||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
|
||||||
import org.apache.hadoop.hbase.master.locking.LockManager;
|
|
||||||
import org.apache.hadoop.hbase.mob.ExpiredMobFileCleaner;
|
|
||||||
import org.apache.hadoop.hbase.mob.MobConstants;
|
|
||||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
|
||||||
import org.apache.hadoop.hbase.procedure2.LockType;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The Class ExpiredMobFileCleanerChore for running cleaner regularly to remove the expired
|
|
||||||
* mob files.
|
|
||||||
*/
|
|
||||||
@InterfaceAudience.Private
|
|
||||||
public class ExpiredMobFileCleanerChore extends ScheduledChore {
|
|
||||||
|
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(ExpiredMobFileCleanerChore.class);
|
|
||||||
private final HMaster master;
|
|
||||||
private ExpiredMobFileCleaner cleaner;
|
|
||||||
|
|
||||||
public ExpiredMobFileCleanerChore(HMaster master) {
|
|
||||||
super(master.getServerName() + "-ExpiredMobFileCleanerChore", master, master.getConfiguration()
|
|
||||||
.getInt(MobConstants.MOB_CLEANER_PERIOD, MobConstants.DEFAULT_MOB_CLEANER_PERIOD), master
|
|
||||||
.getConfiguration().getInt(MobConstants.MOB_CLEANER_PERIOD,
|
|
||||||
MobConstants.DEFAULT_MOB_CLEANER_PERIOD), TimeUnit.SECONDS);
|
|
||||||
this.master = master;
|
|
||||||
cleaner = new ExpiredMobFileCleaner();
|
|
||||||
cleaner.setConf(master.getConfiguration());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION",
|
|
||||||
justification="Intentional")
|
|
||||||
protected void chore() {
|
|
||||||
try {
|
|
||||||
TableDescriptors htds = master.getTableDescriptors();
|
|
||||||
Map<String, TableDescriptor> map = htds.getAll();
|
|
||||||
for (TableDescriptor htd : map.values()) {
|
|
||||||
for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
|
|
||||||
if (hcd.isMobEnabled() && hcd.getMinVersions() == 0) {
|
|
||||||
// clean only for mob-enabled column.
|
|
||||||
// obtain a read table lock before cleaning, synchronize with MobFileCompactionChore.
|
|
||||||
final LockManager.MasterLock lock = master.getLockManager().createMasterLock(
|
|
||||||
MobUtils.getTableLockName(htd.getTableName()), LockType.SHARED,
|
|
||||||
this.getClass().getSimpleName() + ": Cleaning expired mob files");
|
|
||||||
try {
|
|
||||||
lock.acquire();
|
|
||||||
cleaner.cleanExpiredMobFiles(htd.getTableName().getNameAsString(), hcd);
|
|
||||||
} finally {
|
|
||||||
lock.release();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} catch (Exception e) {
|
|
||||||
LOG.error("Fail to clean the expired mob files", e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
|
@ -147,7 +147,8 @@ import org.apache.hadoop.hbase.master.replication.UpdatePeerConfigProcedure;
|
||||||
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
|
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
|
||||||
import org.apache.hadoop.hbase.master.zksyncer.MasterAddressSyncer;
|
import org.apache.hadoop.hbase.master.zksyncer.MasterAddressSyncer;
|
||||||
import org.apache.hadoop.hbase.master.zksyncer.MetaLocationSyncer;
|
import org.apache.hadoop.hbase.master.zksyncer.MetaLocationSyncer;
|
||||||
import org.apache.hadoop.hbase.mob.MobConstants;
|
import org.apache.hadoop.hbase.mob.MobFileCleanerChore;
|
||||||
|
import org.apache.hadoop.hbase.mob.MobFileCompactionChore;
|
||||||
import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
|
import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
|
||||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||||
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
|
||||||
|
@ -396,9 +397,8 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
private LogCleaner logCleaner;
|
private LogCleaner logCleaner;
|
||||||
private HFileCleaner hfileCleaner;
|
private HFileCleaner hfileCleaner;
|
||||||
private ReplicationBarrierCleaner replicationBarrierCleaner;
|
private ReplicationBarrierCleaner replicationBarrierCleaner;
|
||||||
private ExpiredMobFileCleanerChore expiredMobFileCleanerChore;
|
private MobFileCleanerChore mobFileCleanerChore;
|
||||||
private MobCompactionChore mobCompactChore;
|
private MobFileCompactionChore mobFileCompactionChore;
|
||||||
private MasterMobCompactionThread mobCompactThread;
|
|
||||||
// used to synchronize the mobCompactionStates
|
// used to synchronize the mobCompactionStates
|
||||||
private final IdLock mobCompactionLock = new IdLock();
|
private final IdLock mobCompactionLock = new IdLock();
|
||||||
// save the information of mob compactions in tables.
|
// save the information of mob compactions in tables.
|
||||||
|
@ -1265,14 +1265,10 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
}
|
}
|
||||||
|
|
||||||
private void initMobCleaner() {
|
private void initMobCleaner() {
|
||||||
this.expiredMobFileCleanerChore = new ExpiredMobFileCleanerChore(this);
|
this.mobFileCleanerChore = new MobFileCleanerChore(this);
|
||||||
getChoreService().scheduleChore(expiredMobFileCleanerChore);
|
getChoreService().scheduleChore(mobFileCleanerChore);
|
||||||
|
this.mobFileCompactionChore = new MobFileCompactionChore(this);
|
||||||
int mobCompactionPeriod = conf.getInt(MobConstants.MOB_COMPACTION_CHORE_PERIOD,
|
getChoreService().scheduleChore(mobFileCompactionChore);
|
||||||
MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD);
|
|
||||||
this.mobCompactChore = new MobCompactionChore(this, mobCompactionPeriod);
|
|
||||||
getChoreService().scheduleChore(mobCompactChore);
|
|
||||||
this.mobCompactThread = new MasterMobCompactionThread(this);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -1475,9 +1471,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
stopChores();
|
stopChores();
|
||||||
if (this.mobCompactThread != null) {
|
|
||||||
this.mobCompactThread.close();
|
|
||||||
}
|
|
||||||
super.stopServiceThreads();
|
super.stopServiceThreads();
|
||||||
if (cleanerPool != null) {
|
if (cleanerPool != null) {
|
||||||
cleanerPool.shutdownNow();
|
cleanerPool.shutdownNow();
|
||||||
|
@ -1600,8 +1594,8 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
private void stopChores() {
|
private void stopChores() {
|
||||||
ChoreService choreService = getChoreService();
|
ChoreService choreService = getChoreService();
|
||||||
if (choreService != null) {
|
if (choreService != null) {
|
||||||
choreService.cancelChore(this.expiredMobFileCleanerChore);
|
choreService.cancelChore(this.mobFileCleanerChore);
|
||||||
choreService.cancelChore(this.mobCompactChore);
|
choreService.cancelChore(this.mobFileCompactionChore);
|
||||||
choreService.cancelChore(this.balancerChore);
|
choreService.cancelChore(this.balancerChore);
|
||||||
choreService.cancelChore(this.normalizerChore);
|
choreService.cancelChore(this.normalizerChore);
|
||||||
choreService.cancelChore(this.clusterStatusChore);
|
choreService.cancelChore(this.clusterStatusChore);
|
||||||
|
@ -3444,16 +3438,6 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Requests mob compaction.
|
|
||||||
* @param tableName The table the compact.
|
|
||||||
* @param columns The compacted columns.
|
|
||||||
* @param allFiles Whether add all mob files into the compaction.
|
|
||||||
*/
|
|
||||||
public void requestMobCompaction(TableName tableName,
|
|
||||||
List<ColumnFamilyDescriptor> columns, boolean allFiles) throws IOException {
|
|
||||||
mobCompactThread.requestMobCompaction(conf, getFileSystem(), tableName, columns, allFiles);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Queries the state of the {@link LoadBalancerTracker}. If the balancer is not initialized,
|
* Queries the state of the {@link LoadBalancerTracker}. If the balancer is not initialized,
|
||||||
|
|
|
@ -1,181 +0,0 @@
|
||||||
/**
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
package org.apache.hadoop.hbase.master;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.concurrent.ExecutorService;
|
|
||||||
import java.util.concurrent.RejectedExecutionException;
|
|
||||||
import java.util.concurrent.SynchronousQueue;
|
|
||||||
import java.util.concurrent.ThreadPoolExecutor;
|
|
||||||
import java.util.concurrent.TimeUnit;
|
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.hadoop.hbase.TableName;
|
|
||||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
|
||||||
import org.apache.hadoop.hbase.master.locking.LockManager;
|
|
||||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
|
||||||
import org.apache.hadoop.hbase.procedure2.LockType;
|
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
|
||||||
import org.slf4j.Logger;
|
|
||||||
import org.slf4j.LoggerFactory;
|
|
||||||
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The mob compaction thread used in {@link MasterRpcServices}
|
|
||||||
*/
|
|
||||||
@InterfaceAudience.Private
|
|
||||||
public class MasterMobCompactionThread {
|
|
||||||
static final Logger LOG = LoggerFactory.getLogger(MasterMobCompactionThread.class);
|
|
||||||
private final HMaster master;
|
|
||||||
private final Configuration conf;
|
|
||||||
private final ExecutorService mobCompactorPool;
|
|
||||||
private final ExecutorService masterMobPool;
|
|
||||||
|
|
||||||
public MasterMobCompactionThread(HMaster master) {
|
|
||||||
this.master = master;
|
|
||||||
this.conf = master.getConfiguration();
|
|
||||||
final String n = Thread.currentThread().getName();
|
|
||||||
// this pool is used to run the mob compaction
|
|
||||||
this.masterMobPool = new ThreadPoolExecutor(1, 2, 60,
|
|
||||||
TimeUnit.SECONDS, new SynchronousQueue<>(),
|
|
||||||
new ThreadFactoryBuilder().setDaemon(true)
|
|
||||||
.setNameFormat(n + "-MasterMobCompaction-" + EnvironmentEdgeManager.currentTime())
|
|
||||||
.build());
|
|
||||||
((ThreadPoolExecutor) this.masterMobPool).allowCoreThreadTimeOut(true);
|
|
||||||
// this pool is used in the mob compaction to compact the mob files by partitions
|
|
||||||
// in parallel
|
|
||||||
this.mobCompactorPool = MobUtils
|
|
||||||
.createMobCompactorThreadPool(master.getConfiguration());
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Requests mob compaction
|
|
||||||
* @param conf The Configuration
|
|
||||||
* @param fs The file system
|
|
||||||
* @param tableName The table the compact
|
|
||||||
* @param columns The column descriptors
|
|
||||||
* @param allFiles Whether add all mob files into the compaction.
|
|
||||||
*/
|
|
||||||
public void requestMobCompaction(Configuration conf, FileSystem fs, TableName tableName,
|
|
||||||
List<ColumnFamilyDescriptor> columns, boolean allFiles) throws IOException {
|
|
||||||
master.reportMobCompactionStart(tableName);
|
|
||||||
try {
|
|
||||||
masterMobPool.execute(new CompactionRunner(fs, tableName, columns,
|
|
||||||
allFiles, mobCompactorPool));
|
|
||||||
} catch (RejectedExecutionException e) {
|
|
||||||
// in case the request is rejected by the pool
|
|
||||||
try {
|
|
||||||
master.reportMobCompactionEnd(tableName);
|
|
||||||
} catch (IOException e1) {
|
|
||||||
LOG.error("Failed to mark end of mob compaction", e1);
|
|
||||||
}
|
|
||||||
throw e;
|
|
||||||
}
|
|
||||||
if (LOG.isDebugEnabled()) {
|
|
||||||
LOG.debug("The mob compaction is requested for the columns " + columns
|
|
||||||
+ " of the table " + tableName.getNameAsString());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private class CompactionRunner implements Runnable {
|
|
||||||
private FileSystem fs;
|
|
||||||
private TableName tableName;
|
|
||||||
private List<ColumnFamilyDescriptor> hcds;
|
|
||||||
private boolean allFiles;
|
|
||||||
private ExecutorService pool;
|
|
||||||
|
|
||||||
public CompactionRunner(FileSystem fs, TableName tableName, List<ColumnFamilyDescriptor> hcds,
|
|
||||||
boolean allFiles, ExecutorService pool) {
|
|
||||||
super();
|
|
||||||
this.fs = fs;
|
|
||||||
this.tableName = tableName;
|
|
||||||
this.hcds = hcds;
|
|
||||||
this.allFiles = allFiles;
|
|
||||||
this.pool = pool;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void run() {
|
|
||||||
// These locks are on dummy table names, and only used for compaction/mob file cleaning.
|
|
||||||
final LockManager.MasterLock lock = master.getLockManager().createMasterLock(
|
|
||||||
MobUtils.getTableLockName(tableName), LockType.EXCLUSIVE,
|
|
||||||
this.getClass().getName() + ": mob compaction");
|
|
||||||
try {
|
|
||||||
for (ColumnFamilyDescriptor hcd : hcds) {
|
|
||||||
MobUtils.doMobCompaction(conf, fs, tableName, hcd, pool, allFiles, lock);
|
|
||||||
}
|
|
||||||
} catch (IOException e) {
|
|
||||||
LOG.error("Failed to perform the mob compaction", e);
|
|
||||||
} finally {
|
|
||||||
try {
|
|
||||||
master.reportMobCompactionEnd(tableName);
|
|
||||||
} catch (IOException e) {
|
|
||||||
LOG.error("Failed to mark end of mob compaction", e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Only interrupt once it's done with a run through the work loop.
|
|
||||||
*/
|
|
||||||
private void interruptIfNecessary() {
|
|
||||||
mobCompactorPool.shutdown();
|
|
||||||
masterMobPool.shutdown();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Wait for all the threads finish.
|
|
||||||
*/
|
|
||||||
private void join() {
|
|
||||||
waitFor(mobCompactorPool, "Mob Compaction Thread");
|
|
||||||
waitFor(masterMobPool, "Region Server Mob Compaction Thread");
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Closes the MasterMobCompactionThread.
|
|
||||||
*/
|
|
||||||
public void close() {
|
|
||||||
interruptIfNecessary();
|
|
||||||
join();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Wait for thread finish.
|
|
||||||
* @param t the thread to wait
|
|
||||||
* @param name the thread name.
|
|
||||||
*/
|
|
||||||
private void waitFor(ExecutorService t, String name) {
|
|
||||||
boolean done = false;
|
|
||||||
while (!done) {
|
|
||||||
try {
|
|
||||||
done = t.awaitTermination(60, TimeUnit.SECONDS);
|
|
||||||
LOG.info("Waiting for " + name + " to finish...");
|
|
||||||
if (!done) {
|
|
||||||
t.shutdownNow();
|
|
||||||
}
|
|
||||||
} catch (InterruptedException ie) {
|
|
||||||
LOG.warn("Interrupted waiting for " + name + " to finish...");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -32,6 +32,7 @@ import java.util.Map;
|
||||||
import java.util.Map.Entry;
|
import java.util.Map.Entry;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hbase.ClusterMetricsBuilder;
|
import org.apache.hadoop.hbase.ClusterMetricsBuilder;
|
||||||
|
@ -45,7 +46,6 @@ import org.apache.hadoop.hbase.ServerMetricsBuilder;
|
||||||
import org.apache.hadoop.hbase.ServerName;
|
import org.apache.hadoop.hbase.ServerName;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.UnknownRegionException;
|
import org.apache.hadoop.hbase.UnknownRegionException;
|
||||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
|
||||||
import org.apache.hadoop.hbase.client.MasterSwitchType;
|
import org.apache.hadoop.hbase.client.MasterSwitchType;
|
||||||
import org.apache.hadoop.hbase.client.Put;
|
import org.apache.hadoop.hbase.client.Put;
|
||||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||||
|
@ -104,6 +104,7 @@ import org.apache.hadoop.hbase.security.access.PermissionStorage;
|
||||||
import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil;
|
import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil;
|
||||||
import org.apache.hadoop.hbase.security.access.UserPermission;
|
import org.apache.hadoop.hbase.security.access.UserPermission;
|
||||||
import org.apache.hadoop.hbase.security.visibility.VisibilityController;
|
import org.apache.hadoop.hbase.security.visibility.VisibilityController;
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
|
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
|
||||||
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
@ -112,6 +113,8 @@ import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
|
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
|
||||||
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
||||||
|
|
||||||
|
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
import org.apache.zookeeper.KeeperException;
|
import org.apache.zookeeper.KeeperException;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
|
@ -121,6 +124,7 @@ import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||||
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
||||||
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
|
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
|
||||||
|
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
|
import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos;
|
||||||
|
@ -346,6 +350,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Updat
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
|
||||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
|
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Implements the master RPC services.
|
* Implements the master RPC services.
|
||||||
*/
|
*/
|
||||||
|
@ -1751,10 +1757,15 @@ public class MasterRpcServices extends RSRpcServices
|
||||||
master.checkInitialized();
|
master.checkInitialized();
|
||||||
byte[] regionName = request.getRegion().getValue().toByteArray();
|
byte[] regionName = request.getRegion().getValue().toByteArray();
|
||||||
TableName tableName = RegionInfo.getTable(regionName);
|
TableName tableName = RegionInfo.getTable(regionName);
|
||||||
|
// TODO: support CompactType.MOB
|
||||||
// if the region is a mob region, do the mob file compaction.
|
// if the region is a mob region, do the mob file compaction.
|
||||||
if (MobUtils.isMobRegionName(tableName, regionName)) {
|
if (MobUtils.isMobRegionName(tableName, regionName)) {
|
||||||
checkHFileFormatVersionForMob();
|
checkHFileFormatVersionForMob();
|
||||||
return compactMob(request, tableName);
|
//TODO: support CompactType.MOB
|
||||||
|
// HBASE-23571
|
||||||
|
LOG.warn("CompactType.MOB is not supported yet, will run regular compaction."+
|
||||||
|
" Refer to HBASE-23571.");
|
||||||
|
return super.compactRegion(controller, request);
|
||||||
} else {
|
} else {
|
||||||
return super.compactRegion(controller, request);
|
return super.compactRegion(controller, request);
|
||||||
}
|
}
|
||||||
|
@ -1817,57 +1828,6 @@ public class MasterRpcServices extends RSRpcServices
|
||||||
return builder.build();
|
return builder.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Compacts the mob files in the current table.
|
|
||||||
* @param request the request.
|
|
||||||
* @param tableName the current table name.
|
|
||||||
* @return The response of the mob file compaction.
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
private CompactRegionResponse compactMob(final CompactRegionRequest request,
|
|
||||||
TableName tableName) throws IOException {
|
|
||||||
if (!master.getTableStateManager().isTableState(tableName, TableState.State.ENABLED)) {
|
|
||||||
throw new DoNotRetryIOException("Table " + tableName + " is not enabled");
|
|
||||||
}
|
|
||||||
boolean allFiles = false;
|
|
||||||
List<ColumnFamilyDescriptor> compactedColumns = new ArrayList<>();
|
|
||||||
ColumnFamilyDescriptor[] hcds = master.getTableDescriptors().get(tableName).getColumnFamilies();
|
|
||||||
byte[] family = null;
|
|
||||||
if (request.hasFamily()) {
|
|
||||||
family = request.getFamily().toByteArray();
|
|
||||||
for (ColumnFamilyDescriptor hcd : hcds) {
|
|
||||||
if (Bytes.equals(family, hcd.getName())) {
|
|
||||||
if (!hcd.isMobEnabled()) {
|
|
||||||
LOG.error("Column family " + hcd.getNameAsString() + " is not a mob column family");
|
|
||||||
throw new DoNotRetryIOException("Column family " + hcd.getNameAsString()
|
|
||||||
+ " is not a mob column family");
|
|
||||||
}
|
|
||||||
compactedColumns.add(hcd);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
for (ColumnFamilyDescriptor hcd : hcds) {
|
|
||||||
if (hcd.isMobEnabled()) {
|
|
||||||
compactedColumns.add(hcd);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
if (compactedColumns.isEmpty()) {
|
|
||||||
LOG.error("No mob column families are assigned in the mob compaction");
|
|
||||||
throw new DoNotRetryIOException(
|
|
||||||
"No mob column families are assigned in the mob compaction");
|
|
||||||
}
|
|
||||||
if (request.hasMajor() && request.getMajor()) {
|
|
||||||
allFiles = true;
|
|
||||||
}
|
|
||||||
String familyLogMsg = (family != null) ? Bytes.toString(family) : "";
|
|
||||||
if (LOG.isTraceEnabled()) {
|
|
||||||
LOG.trace("User-triggered mob compaction requested for table: "
|
|
||||||
+ tableName.getNameAsString() + " for column family: " + familyLogMsg);
|
|
||||||
}
|
|
||||||
master.requestMobCompaction(tableName, compactedColumns, allFiles);
|
|
||||||
return CompactRegionResponse.newBuilder().build();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public IsBalancerEnabledResponse isBalancerEnabled(RpcController controller,
|
public IsBalancerEnabledResponse isBalancerEnabled(RpcController controller,
|
||||||
|
|
|
@ -1,96 +0,0 @@
|
||||||
/**
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
package org.apache.hadoop.hbase.master;
|
|
||||||
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.concurrent.ExecutorService;
|
|
||||||
import java.util.concurrent.TimeUnit;
|
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.ScheduledChore;
|
|
||||||
import org.apache.hadoop.hbase.TableDescriptors;
|
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
|
||||||
import org.slf4j.Logger;
|
|
||||||
import org.slf4j.LoggerFactory;
|
|
||||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
|
||||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
|
||||||
import org.apache.hadoop.hbase.client.TableState;
|
|
||||||
import org.apache.hadoop.hbase.master.locking.LockManager;
|
|
||||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
|
||||||
import org.apache.hadoop.hbase.procedure2.LockType;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The Class MobCompactChore for running compaction regularly to merge small mob files.
|
|
||||||
*/
|
|
||||||
@InterfaceAudience.Private
|
|
||||||
public class MobCompactionChore extends ScheduledChore {
|
|
||||||
|
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(MobCompactionChore.class);
|
|
||||||
private HMaster master;
|
|
||||||
private ExecutorService pool;
|
|
||||||
|
|
||||||
public MobCompactionChore(HMaster master, int period) {
|
|
||||||
// use the period as initial delay.
|
|
||||||
super(master.getServerName() + "-MobCompactionChore", master, period, period, TimeUnit.SECONDS);
|
|
||||||
this.master = master;
|
|
||||||
this.pool = MobUtils.createMobCompactorThreadPool(master.getConfiguration());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void chore() {
|
|
||||||
try {
|
|
||||||
TableDescriptors htds = master.getTableDescriptors();
|
|
||||||
Map<String, TableDescriptor> map = htds.getAll();
|
|
||||||
for (TableDescriptor htd : map.values()) {
|
|
||||||
if (!master.getTableStateManager().isTableState(htd.getTableName(),
|
|
||||||
TableState.State.ENABLED)) {
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
boolean reported = false;
|
|
||||||
try {
|
|
||||||
final LockManager.MasterLock lock = master.getLockManager().createMasterLock(
|
|
||||||
MobUtils.getTableLockName(htd.getTableName()), LockType.EXCLUSIVE,
|
|
||||||
this.getClass().getName() + ": mob compaction");
|
|
||||||
for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
|
|
||||||
if (!hcd.isMobEnabled()) {
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
if (!reported) {
|
|
||||||
master.reportMobCompactionStart(htd.getTableName());
|
|
||||||
reported = true;
|
|
||||||
}
|
|
||||||
MobUtils.doMobCompaction(master.getConfiguration(), master.getFileSystem(),
|
|
||||||
htd.getTableName(), hcd, pool, false, lock);
|
|
||||||
}
|
|
||||||
} finally {
|
|
||||||
if (reported) {
|
|
||||||
master.reportMobCompactionEnd(htd.getTableName());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} catch (Exception e) {
|
|
||||||
LOG.error("Failed to compact mob files", e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected synchronized void cleanup() {
|
|
||||||
super.cleanup();
|
|
||||||
pool.shutdown();
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -17,18 +17,26 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.mob;
|
package org.apache.hadoop.hbase.mob;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.hbase.regionserver.ScanType.COMPACT_DROP_DELETES;
|
||||||
|
import static org.apache.hadoop.hbase.regionserver.ScanType.COMPACT_RETAIN_DELETES;
|
||||||
|
|
||||||
|
import java.io.FileNotFoundException;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InterruptedIOException;
|
import java.io.InterruptedIOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Date;
|
import java.util.Date;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hbase.Cell;
|
import org.apache.hadoop.hbase.Cell;
|
||||||
import org.apache.hadoop.hbase.CellUtil;
|
|
||||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
|
||||||
import org.apache.hadoop.hbase.KeyValue;
|
import org.apache.hadoop.hbase.KeyValue;
|
||||||
|
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||||
import org.apache.hadoop.hbase.regionserver.CellSink;
|
import org.apache.hadoop.hbase.regionserver.CellSink;
|
||||||
import org.apache.hadoop.hbase.regionserver.HMobStore;
|
import org.apache.hadoop.hbase.regionserver.HMobStore;
|
||||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||||
|
@ -52,6 +60,8 @@ import org.apache.yetus.audience.InterfaceAudience;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Compact passed set of files in the mob-enabled column family.
|
* Compact passed set of files in the mob-enabled column family.
|
||||||
*/
|
*/
|
||||||
|
@ -59,15 +69,68 @@ import org.slf4j.LoggerFactory;
|
||||||
public class DefaultMobStoreCompactor extends DefaultCompactor {
|
public class DefaultMobStoreCompactor extends DefaultCompactor {
|
||||||
|
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(DefaultMobStoreCompactor.class);
|
private static final Logger LOG = LoggerFactory.getLogger(DefaultMobStoreCompactor.class);
|
||||||
private long mobSizeThreshold;
|
protected long mobSizeThreshold;
|
||||||
private HMobStore mobStore;
|
protected HMobStore mobStore;
|
||||||
|
protected boolean ioOptimizedMode = false;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* MOB file reference set thread local variable. It contains set of a MOB file names, which newly
|
||||||
|
* compacted store file has references to. This variable is populated during compaction and the
|
||||||
|
* content of it is written into meta section of a newly created store file at the final step of
|
||||||
|
* compaction process.
|
||||||
|
*/
|
||||||
|
|
||||||
|
static ThreadLocal<Set<String>> mobRefSet = new ThreadLocal<Set<String>>() {
|
||||||
|
@Override
|
||||||
|
protected Set<String> initialValue() {
|
||||||
|
return new HashSet<String>();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Is it user or system-originated request.
|
||||||
|
*/
|
||||||
|
|
||||||
|
static ThreadLocal<Boolean> userRequest = new ThreadLocal<Boolean>() {
|
||||||
|
@Override
|
||||||
|
protected Boolean initialValue() {
|
||||||
|
return Boolean.FALSE;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Disable IO mode. IO mode can be forcefully disabled if compactor finds
|
||||||
|
* old MOB file (pre-distributed compaction). This means that migration has not
|
||||||
|
* been completed yet. During data migration (upgrade) process only general compaction
|
||||||
|
* is allowed.
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
|
||||||
|
static ThreadLocal<Boolean> disableIO = new ThreadLocal<Boolean>() {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Boolean initialValue() {
|
||||||
|
return Boolean.FALSE;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Map : MOB file name - file length Can be expensive for large amount of MOB files.
|
||||||
|
*/
|
||||||
|
static ThreadLocal<HashMap<String, Long>> mobLengthMap =
|
||||||
|
new ThreadLocal<HashMap<String, Long>>() {
|
||||||
|
@Override
|
||||||
|
protected HashMap<String, Long> initialValue() {
|
||||||
|
return new HashMap<String, Long>();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
private final InternalScannerFactory scannerFactory = new InternalScannerFactory() {
|
private final InternalScannerFactory scannerFactory = new InternalScannerFactory() {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ScanType getScanType(CompactionRequestImpl request) {
|
public ScanType getScanType(CompactionRequestImpl request) {
|
||||||
// retain the delete markers until they are expired.
|
return request.isAllFiles() ? COMPACT_DROP_DELETES : COMPACT_RETAIN_DELETES;
|
||||||
return ScanType.COMPACT_RETAIN_DELETES;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -99,59 +162,90 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
|
||||||
if (!(store instanceof HMobStore)) {
|
if (!(store instanceof HMobStore)) {
|
||||||
throw new IllegalArgumentException("The store " + store + " is not a HMobStore");
|
throw new IllegalArgumentException("The store " + store + " is not a HMobStore");
|
||||||
}
|
}
|
||||||
mobStore = (HMobStore) store;
|
this.mobStore = (HMobStore) store;
|
||||||
mobSizeThreshold = store.getColumnFamilyDescriptor().getMobThreshold();
|
this.mobSizeThreshold = store.getColumnFamilyDescriptor().getMobThreshold();
|
||||||
|
this.ioOptimizedMode = conf.get(MobConstants.MOB_COMPACTION_TYPE_KEY,
|
||||||
|
MobConstants.DEFAULT_MOB_COMPACTION_TYPE).
|
||||||
|
equals(MobConstants.OPTIMIZED_MOB_COMPACTION_TYPE);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<Path> compact(CompactionRequestImpl request, ThroughputController throughputController,
|
public List<Path> compact(CompactionRequestImpl request,
|
||||||
User user) throws IOException {
|
ThroughputController throughputController, User user) throws IOException {
|
||||||
|
String tableName = store.getTableName().toString();
|
||||||
|
String regionName = store.getRegionInfo().getRegionNameAsString();
|
||||||
|
String familyName = store.getColumnFamilyName();
|
||||||
|
LOG.info("MOB compaction: major={} isAll={} priority={} throughput controller={}" +
|
||||||
|
" table={} cf={} region={}",
|
||||||
|
request.isMajor(), request.isAllFiles(), request.getPriority(),
|
||||||
|
throughputController, tableName, familyName, regionName);
|
||||||
|
if (request.getPriority() == HStore.PRIORITY_USER) {
|
||||||
|
userRequest.set(Boolean.TRUE);
|
||||||
|
} else {
|
||||||
|
userRequest.set(Boolean.FALSE);
|
||||||
|
}
|
||||||
|
LOG.debug("MOB compaction table={} cf={} region={} files: {}", tableName, familyName,
|
||||||
|
regionName, request.getFiles());
|
||||||
|
// Check if I/O optimized MOB compaction
|
||||||
|
if (ioOptimizedMode) {
|
||||||
|
if (request.isMajor() && request.getPriority() == HStore.PRIORITY_USER) {
|
||||||
|
Path mobDir =
|
||||||
|
MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName());
|
||||||
|
List<Path> mobFiles = MobUtils.getReferencedMobFiles(request.getFiles(), mobDir);
|
||||||
|
//reset disableIO
|
||||||
|
disableIO.set(Boolean.FALSE);
|
||||||
|
if (mobFiles.size() > 0) {
|
||||||
|
calculateMobLengthMap(mobFiles);
|
||||||
|
}
|
||||||
|
LOG.info("Table={} cf={} region={}. I/O optimized MOB compaction. "+
|
||||||
|
"Total referenced MOB files: {}", tableName, familyName, regionName, mobFiles.size());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
return compact(request, scannerFactory, writerFactory, throughputController, user);
|
return compact(request, scannerFactory, writerFactory, throughputController, user);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void calculateMobLengthMap(List<Path> mobFiles) throws IOException {
|
||||||
|
FileSystem fs = store.getFileSystem();
|
||||||
|
HashMap<String, Long> map = mobLengthMap.get();
|
||||||
|
map.clear();
|
||||||
|
for (Path p : mobFiles) {
|
||||||
|
if (MobFileName.isOldMobFileName(p.getName())) {
|
||||||
|
disableIO.set(Boolean.TRUE);
|
||||||
|
}
|
||||||
|
FileStatus st = fs.getFileStatus(p);
|
||||||
|
long size = st.getLen();
|
||||||
|
LOG.debug("Referenced MOB file={} size={}", p, size);
|
||||||
|
map.put(p.getName(), fs.getFileStatus(p).getLen());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Performs compaction on a column family with the mob flag enabled.
|
* Performs compaction on a column family with the mob flag enabled. This works only when MOB
|
||||||
* This is for when the mob threshold size has changed or if the mob
|
* compaction is explicitly requested (by User), or by Master There are two modes of a MOB
|
||||||
* column family mode has been toggled via an alter table statement.
|
* compaction:<br>
|
||||||
* Compacts the files by the following rules.
|
* <p>
|
||||||
|
* <ul>
|
||||||
|
* <li>1. Full mode - when all MOB data for a region is compacted into a single MOB file.
|
||||||
|
* <li>2. I/O optimized mode - for use cases with no or infrequent updates/deletes of a <br>
|
||||||
|
* MOB data. The main idea behind i/o optimized compaction is to limit maximum size of a MOB file
|
||||||
|
* produced during compaction and to limit I/O write/read amplification.
|
||||||
|
* </ul>
|
||||||
|
* The basic algorithm of compaction is the following: <br>
|
||||||
* 1. If the Put cell has a mob reference tag, the cell's value is the path of the mob file.
|
* 1. If the Put cell has a mob reference tag, the cell's value is the path of the mob file.
|
||||||
* <ol>
|
* <ol>
|
||||||
* <li>
|
* <li>If the value size of a cell is larger than the threshold, this cell is regarded as a mob,
|
||||||
* If the value size of a cell is larger than the threshold, this cell is regarded as a mob,
|
* directly copy the (with mob tag) cell into the new store file.</li>
|
||||||
* directly copy the (with mob tag) cell into the new store file.
|
* <li>Otherwise, retrieve the mob cell from the mob file, and writes a copy of the cell into the
|
||||||
* </li>
|
* new store file.</li>
|
||||||
* <li>
|
|
||||||
* Otherwise, retrieve the mob cell from the mob file, and writes a copy of the cell into
|
|
||||||
* the new store file.
|
|
||||||
* </li>
|
|
||||||
* </ol>
|
* </ol>
|
||||||
* 2. If the Put cell doesn't have a reference tag.
|
* 2. If the Put cell doesn't have a reference tag.
|
||||||
* <ol>
|
* <ol>
|
||||||
* <li>
|
* <li>If the value size of a cell is larger than the threshold, this cell is regarded as a mob,
|
||||||
* If the value size of a cell is larger than the threshold, this cell is regarded as a mob,
|
* write this cell to a mob file, and write the path of this mob file to the store file.</li>
|
||||||
* write this cell to a mob file, and write the path of this mob file to the store file.
|
* <li>Otherwise, directly write this cell into the store file.</li>
|
||||||
* </li>
|
|
||||||
* <li>
|
|
||||||
* Otherwise, directly write this cell into the store file.
|
|
||||||
* </li>
|
|
||||||
* </ol>
|
* </ol>
|
||||||
* 3. Decide how to write a Delete cell.
|
|
||||||
* <ol>
|
|
||||||
* <li>
|
|
||||||
* If a Delete cell does not have a mob reference tag which means this delete marker have not
|
|
||||||
* been written to the mob del file, write this cell to the mob del file, and write this cell
|
|
||||||
* with a ref tag to a store file.
|
|
||||||
* </li>
|
|
||||||
* <li>
|
|
||||||
* Otherwise, directly write it to a store file.
|
|
||||||
* </li>
|
|
||||||
* </ol>
|
|
||||||
* After the major compaction on the normal hfiles, we have a guarantee that we have purged all
|
|
||||||
* deleted or old version mob refs, and the delete markers are written to a del file with the
|
|
||||||
* suffix _del. Because of this, it is safe to use the del file in the mob compaction.
|
|
||||||
* The mob compaction doesn't take place in the normal hfiles, it occurs directly in the
|
|
||||||
* mob files. When the small mob files are merged into bigger ones, the del file is added into
|
|
||||||
* the scanner to filter the deleted cells.
|
|
||||||
* @param fd File details
|
* @param fd File details
|
||||||
* @param scanner Where to read from.
|
* @param scanner Where to read from.
|
||||||
* @param writer Where to write to.
|
* @param writer Where to write to.
|
||||||
|
@ -169,6 +263,23 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
|
||||||
long bytesWrittenProgressForCloseCheck = 0;
|
long bytesWrittenProgressForCloseCheck = 0;
|
||||||
long bytesWrittenProgressForLog = 0;
|
long bytesWrittenProgressForLog = 0;
|
||||||
long bytesWrittenProgressForShippedCall = 0;
|
long bytesWrittenProgressForShippedCall = 0;
|
||||||
|
// Clear old mob references
|
||||||
|
mobRefSet.get().clear();
|
||||||
|
boolean isUserRequest = userRequest.get();
|
||||||
|
boolean compactMOBs = major && isUserRequest;
|
||||||
|
boolean discardMobMiss = conf.getBoolean(MobConstants.MOB_UNSAFE_DISCARD_MISS_KEY,
|
||||||
|
MobConstants.DEFAULT_MOB_DISCARD_MISS);
|
||||||
|
if (discardMobMiss) {
|
||||||
|
LOG.warn("{}=true. This is unsafe setting recommended only when first upgrading to a version"+
|
||||||
|
" with the distributed mob compaction feature on a cluster that has experienced MOB data " +
|
||||||
|
"corruption.", MobConstants.MOB_UNSAFE_DISCARD_MISS_KEY);
|
||||||
|
}
|
||||||
|
long maxMobFileSize = conf.getLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY,
|
||||||
|
MobConstants.DEFAULT_MOB_COMPACTION_MAX_FILE_SIZE);
|
||||||
|
boolean ioOptimizedMode = this.ioOptimizedMode && !disableIO.get();
|
||||||
|
LOG.info("Compact MOB={} optimized configured={} optimized enabled={} maximum MOB file size={}"
|
||||||
|
+ " major={} store={}", compactMOBs,
|
||||||
|
this.ioOptimizedMode, ioOptimizedMode, maxMobFileSize, major, getStoreInfo());
|
||||||
// Since scanner.next() can return 'false' but still be delivering data,
|
// Since scanner.next() can return 'false' but still be delivering data,
|
||||||
// we have to use a do/while loop.
|
// we have to use a do/while loop.
|
||||||
List<Cell> cells = new ArrayList<>();
|
List<Cell> cells = new ArrayList<>();
|
||||||
|
@ -181,93 +292,164 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
|
||||||
String compactionName = ThroughputControlUtil.getNameForThrottling(store, "compaction");
|
String compactionName = ThroughputControlUtil.getNameForThrottling(store, "compaction");
|
||||||
long now = 0;
|
long now = 0;
|
||||||
boolean hasMore;
|
boolean hasMore;
|
||||||
Path path = MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName());
|
|
||||||
byte[] fileName = null;
|
byte[] fileName = null;
|
||||||
StoreFileWriter mobFileWriter = null, delFileWriter = null;
|
StoreFileWriter mobFileWriter = null;
|
||||||
long mobCells = 0, deleteMarkersCount = 0;
|
/*
|
||||||
|
* mobCells are used only to decide if we need to commit or abort current MOB output file.
|
||||||
|
*/
|
||||||
|
long mobCells = 0;
|
||||||
long cellsCountCompactedToMob = 0, cellsCountCompactedFromMob = 0;
|
long cellsCountCompactedToMob = 0, cellsCountCompactedFromMob = 0;
|
||||||
long cellsSizeCompactedToMob = 0, cellsSizeCompactedFromMob = 0;
|
long cellsSizeCompactedToMob = 0, cellsSizeCompactedFromMob = 0;
|
||||||
boolean finished = false;
|
boolean finished = false;
|
||||||
|
|
||||||
ScannerContext scannerContext =
|
ScannerContext scannerContext =
|
||||||
ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build();
|
ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build();
|
||||||
throughputController.start(compactionName);
|
throughputController.start(compactionName);
|
||||||
KeyValueScanner kvs = (scanner instanceof KeyValueScanner)? (KeyValueScanner)scanner : null;
|
KeyValueScanner kvs = (scanner instanceof KeyValueScanner) ? (KeyValueScanner) scanner : null;
|
||||||
long shippedCallSizeLimit = (long) numofFilesToCompact * this.store.getColumnFamilyDescriptor().getBlocksize();
|
long shippedCallSizeLimit =
|
||||||
|
(long) numofFilesToCompact * this.store.getColumnFamilyDescriptor().getBlocksize();
|
||||||
|
|
||||||
|
Cell mobCell = null;
|
||||||
try {
|
try {
|
||||||
try {
|
|
||||||
// If the mob file writer could not be created, directly write the cell to the store file.
|
mobFileWriter = newMobWriter(fd);
|
||||||
mobFileWriter = mobStore.createWriterInTmp(new Date(fd.latestPutTs), fd.maxKeyCount,
|
fileName = Bytes.toBytes(mobFileWriter.getPath().getName());
|
||||||
compactionCompression, store.getRegionInfo().getStartKey(), true);
|
|
||||||
fileName = Bytes.toBytes(mobFileWriter.getPath().getName());
|
|
||||||
} catch (IOException e) {
|
|
||||||
LOG.warn("Failed to create mob writer, "
|
|
||||||
+ "we will continue the compaction by writing MOB cells directly in store files", e);
|
|
||||||
}
|
|
||||||
if (major) {
|
|
||||||
try {
|
|
||||||
delFileWriter = mobStore.createDelFileWriterInTmp(new Date(fd.latestPutTs),
|
|
||||||
fd.maxKeyCount, compactionCompression, store.getRegionInfo().getStartKey());
|
|
||||||
} catch (IOException e) {
|
|
||||||
LOG.warn(
|
|
||||||
"Failed to create del writer, "
|
|
||||||
+ "we will continue the compaction by writing delete markers directly in store files",
|
|
||||||
e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
do {
|
do {
|
||||||
hasMore = scanner.next(cells, scannerContext);
|
hasMore = scanner.next(cells, scannerContext);
|
||||||
if (LOG.isDebugEnabled()) {
|
now = EnvironmentEdgeManager.currentTime();
|
||||||
now = EnvironmentEdgeManager.currentTime();
|
|
||||||
}
|
|
||||||
for (Cell c : cells) {
|
for (Cell c : cells) {
|
||||||
if (major && CellUtil.isDelete(c)) {
|
if (compactMOBs) {
|
||||||
if (MobUtils.isMobReferenceCell(c) || delFileWriter == null) {
|
if (MobUtils.isMobReferenceCell(c)) {
|
||||||
// Directly write it to a store file
|
String fName = MobUtils.getMobFileName(c);
|
||||||
writer.append(c);
|
// Added to support migration
|
||||||
|
try {
|
||||||
|
mobCell = mobStore.resolve(c, true, false).getCell();
|
||||||
|
} catch (FileNotFoundException fnfe) {
|
||||||
|
if (discardMobMiss) {
|
||||||
|
LOG.error("Missing MOB cell: file={} not found cell={}", fName, c);
|
||||||
|
continue;
|
||||||
|
} else {
|
||||||
|
throw fnfe;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (discardMobMiss && mobCell.getValueLength() == 0) {
|
||||||
|
LOG.error("Missing MOB cell value: file={} mob cell={} cell={}", fName,
|
||||||
|
mobCell, c);
|
||||||
|
continue;
|
||||||
|
} else if (mobCell.getValueLength() == 0) {
|
||||||
|
String errMsg = String.format("Found 0 length MOB cell in a file=%s mob cell=%s "
|
||||||
|
+ " cell=%s",
|
||||||
|
fName, mobCell, c);
|
||||||
|
throw new IOException(errMsg);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (mobCell.getValueLength() > mobSizeThreshold) {
|
||||||
|
// put the mob data back to the MOB store file
|
||||||
|
PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId());
|
||||||
|
if (!ioOptimizedMode) {
|
||||||
|
mobFileWriter.append(mobCell);
|
||||||
|
mobCells++;
|
||||||
|
writer.append(
|
||||||
|
MobUtils.createMobRefCell(mobCell, fileName, this.mobStore.getRefCellTags()));
|
||||||
|
} else {
|
||||||
|
// I/O optimized mode
|
||||||
|
// Check if MOB cell origin file size is
|
||||||
|
// greater than threshold
|
||||||
|
Long size = mobLengthMap.get().get(fName);
|
||||||
|
if (size == null) {
|
||||||
|
// FATAL error (we should never get here though), abort compaction
|
||||||
|
// This error means that meta section of store file does not contain
|
||||||
|
// MOB file, which has references in at least one cell from this store file
|
||||||
|
String msg = String.format(
|
||||||
|
"Found an unexpected MOB file during compaction %s, aborting compaction %s",
|
||||||
|
fName, getStoreInfo());
|
||||||
|
throw new IOException(msg);
|
||||||
|
}
|
||||||
|
// Can not be null
|
||||||
|
if (size < maxMobFileSize) {
|
||||||
|
// If MOB cell origin file is below threshold
|
||||||
|
// it is get compacted
|
||||||
|
mobFileWriter.append(mobCell);
|
||||||
|
// Update number of mobCells in a current mob writer
|
||||||
|
mobCells++;
|
||||||
|
writer.append(
|
||||||
|
MobUtils.createMobRefCell(mobCell, fileName, this.mobStore.getRefCellTags()));
|
||||||
|
// Update total size of the output (we do not take into account
|
||||||
|
// file compression yet)
|
||||||
|
long len = mobFileWriter.getPos();
|
||||||
|
if (len > maxMobFileSize) {
|
||||||
|
LOG.debug("Closing output MOB File, length={} file={}, store={}", len,
|
||||||
|
mobFileWriter.getPath().getName(), getStoreInfo());
|
||||||
|
commitOrAbortMobWriter(mobFileWriter, fd.maxSeqId, mobCells, major);
|
||||||
|
mobFileWriter = newMobWriter(fd);
|
||||||
|
fileName = Bytes.toBytes(mobFileWriter.getPath().getName());
|
||||||
|
mobCells = 0;
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
// We leave large MOB file as is (is not compacted),
|
||||||
|
// then we update set of MOB file references
|
||||||
|
// and append mob cell directly to the store's writer
|
||||||
|
mobRefSet.get().add(fName);
|
||||||
|
writer.append(mobCell);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
// If MOB value is less than threshold, append it directly to a store file
|
||||||
|
PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId());
|
||||||
|
writer.append(mobCell);
|
||||||
|
cellsCountCompactedFromMob++;
|
||||||
|
cellsSizeCompactedFromMob += mobCell.getValueLength();
|
||||||
|
}
|
||||||
} else {
|
} else {
|
||||||
// Add a ref tag to this cell and write it to a store file.
|
// Not a MOB reference cell
|
||||||
writer.append(MobUtils.createMobRefDeleteMarker(c));
|
int size = c.getValueLength();
|
||||||
// Write the cell to a del file
|
if (size > mobSizeThreshold) {
|
||||||
delFileWriter.append(c);
|
// This MOB cell comes from a regular store file
|
||||||
deleteMarkersCount++;
|
// therefore we store it into original mob output
|
||||||
|
mobFileWriter.append(c);
|
||||||
|
writer
|
||||||
|
.append(MobUtils.createMobRefCell(c, fileName, this.mobStore.getRefCellTags()));
|
||||||
|
mobCells++;
|
||||||
|
cellsCountCompactedToMob++;
|
||||||
|
cellsSizeCompactedToMob += c.getValueLength();
|
||||||
|
if (ioOptimizedMode) {
|
||||||
|
// Update total size of the output (we do not take into account
|
||||||
|
// file compression yet)
|
||||||
|
long len = mobFileWriter.getPos();
|
||||||
|
if (len > maxMobFileSize) {
|
||||||
|
commitOrAbortMobWriter(mobFileWriter, fd.maxSeqId, mobCells, major);
|
||||||
|
mobFileWriter = newMobWriter(fd);
|
||||||
|
fileName = Bytes.toBytes(mobFileWriter.getPath().getName());
|
||||||
|
mobCells = 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
// Not a MOB cell, write it directly to a store file
|
||||||
|
writer.append(c);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
} else if (mobFileWriter == null || c.getTypeByte() != KeyValue.Type.Put.getCode()) {
|
} else if (c.getTypeByte() != KeyValue.Type.Put.getCode()) {
|
||||||
// If the mob file writer is null or the kv type is not put, directly write the cell
|
// Not a major compaction or major with MOB disabled
|
||||||
|
// If the kv type is not put, directly write the cell
|
||||||
// to the store file.
|
// to the store file.
|
||||||
writer.append(c);
|
writer.append(c);
|
||||||
} else if (MobUtils.isMobReferenceCell(c)) {
|
} else if (MobUtils.isMobReferenceCell(c)) {
|
||||||
|
// Not a major MOB compaction, Put MOB reference
|
||||||
if (MobUtils.hasValidMobRefCellValue(c)) {
|
if (MobUtils.hasValidMobRefCellValue(c)) {
|
||||||
int size = MobUtils.getMobValueLength(c);
|
// We do not check mobSizeThreshold during normal compaction,
|
||||||
if (size > mobSizeThreshold) {
|
// leaving it to a MOB compaction run
|
||||||
// If the value size is larger than the threshold, it's regarded as a mob. Since
|
|
||||||
// its value is already in the mob file, directly write this cell to the store file
|
|
||||||
writer.append(c);
|
|
||||||
} else {
|
|
||||||
// If the value is not larger than the threshold, it's not regarded a mob. Retrieve
|
|
||||||
// the mob cell from the mob file, and write it back to the store file. Must
|
|
||||||
// close the mob scanner once the life cycle finished.
|
|
||||||
try (MobCell mobCell = mobStore.resolve(c, false)) {
|
|
||||||
if (mobCell.getCell().getValueLength() != 0) {
|
|
||||||
// put the mob data back to the store file
|
|
||||||
PrivateCellUtil.setSequenceId(mobCell.getCell(), c.getSequenceId());
|
|
||||||
writer.append(mobCell.getCell());
|
|
||||||
cellsCountCompactedFromMob++;
|
|
||||||
cellsSizeCompactedFromMob += mobCell.getCell().getValueLength();
|
|
||||||
} else {
|
|
||||||
// If the value of a file is empty, there might be issues when retrieving,
|
|
||||||
// directly write the cell to the store file, and leave it to be handled by the
|
|
||||||
// next compaction.
|
|
||||||
writer.append(c);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
LOG.warn("The value format of the KeyValue " + c
|
|
||||||
+ " is wrong, its length is less than " + Bytes.SIZEOF_INT);
|
|
||||||
writer.append(c);
|
writer.append(c);
|
||||||
|
// Add MOB reference to a MOB reference set
|
||||||
|
mobRefSet.get().add(MobUtils.getMobFileName(c));
|
||||||
|
} else {
|
||||||
|
String errMsg = String.format("Corrupted MOB reference: %s", c.toString());
|
||||||
|
throw new IOException(errMsg);
|
||||||
}
|
}
|
||||||
} else if (c.getValueLength() <= mobSizeThreshold) {
|
} else if (c.getValueLength() <= mobSizeThreshold) {
|
||||||
//If value size of a cell is not larger than the threshold, directly write to store file
|
// If the value size of a cell is not larger than the threshold, directly write it to
|
||||||
|
// the store file.
|
||||||
writer.append(c);
|
writer.append(c);
|
||||||
} else {
|
} else {
|
||||||
// If the value size of a cell is larger than the threshold, it's regarded as a mob,
|
// If the value size of a cell is larger than the threshold, it's regarded as a mob,
|
||||||
|
@ -275,13 +457,22 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
|
||||||
mobCells++;
|
mobCells++;
|
||||||
// append the original keyValue in the mob file.
|
// append the original keyValue in the mob file.
|
||||||
mobFileWriter.append(c);
|
mobFileWriter.append(c);
|
||||||
Cell reference = MobUtils.createMobRefCell(c, fileName,
|
Cell reference = MobUtils.createMobRefCell(c, fileName, this.mobStore.getRefCellTags());
|
||||||
this.mobStore.getRefCellTags());
|
|
||||||
// write the cell whose value is the path of a mob file to the store file.
|
// write the cell whose value is the path of a mob file to the store file.
|
||||||
writer.append(reference);
|
writer.append(reference);
|
||||||
cellsCountCompactedToMob++;
|
cellsCountCompactedToMob++;
|
||||||
cellsSizeCompactedToMob += c.getValueLength();
|
cellsSizeCompactedToMob += c.getValueLength();
|
||||||
|
if (ioOptimizedMode) {
|
||||||
|
long len = mobFileWriter.getPos();
|
||||||
|
if (len > maxMobFileSize) {
|
||||||
|
commitOrAbortMobWriter(mobFileWriter, fd.maxSeqId, mobCells, major);
|
||||||
|
mobFileWriter = newMobWriter(fd);
|
||||||
|
fileName = Bytes.toBytes(mobFileWriter.getPath().getName());
|
||||||
|
mobCells = 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
int len = c.getSerializedSize();
|
int len = c.getSerializedSize();
|
||||||
++progress.currentCompactedKVs;
|
++progress.currentCompactedKVs;
|
||||||
progress.totalCompactedSize += len;
|
progress.totalCompactedSize += len;
|
||||||
|
@ -302,7 +493,7 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (kvs != null && bytesWrittenProgressForShippedCall > shippedCallSizeLimit) {
|
if (kvs != null && bytesWrittenProgressForShippedCall > shippedCallSizeLimit) {
|
||||||
((ShipperListener)writer).beforeShipped();
|
((ShipperListener) writer).beforeShipped();
|
||||||
kvs.shipped();
|
kvs.shipped();
|
||||||
bytesWrittenProgressForShippedCall = 0;
|
bytesWrittenProgressForShippedCall = 0;
|
||||||
}
|
}
|
||||||
|
@ -326,6 +517,10 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
|
||||||
progress.cancel();
|
progress.cancel();
|
||||||
throw new InterruptedIOException(
|
throw new InterruptedIOException(
|
||||||
"Interrupted while control throughput of compacting " + compactionName);
|
"Interrupted while control throughput of compacting " + compactionName);
|
||||||
|
} catch (IOException t) {
|
||||||
|
String msg = "Mob compaction failed for region: " +
|
||||||
|
store.getRegionInfo().getEncodedName();
|
||||||
|
throw new IOException(msg, t);
|
||||||
} finally {
|
} finally {
|
||||||
// Clone last cell in the final because writer will append last cell when committing. If
|
// Clone last cell in the final because writer will append last cell when committing. If
|
||||||
// don't clone here and once the scanner get closed, then the memory of last cell will be
|
// don't clone here and once the scanner get closed, then the memory of last cell will be
|
||||||
|
@ -333,35 +528,17 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
|
||||||
((ShipperListener) writer).beforeShipped();
|
((ShipperListener) writer).beforeShipped();
|
||||||
throughputController.finish(compactionName);
|
throughputController.finish(compactionName);
|
||||||
if (!finished && mobFileWriter != null) {
|
if (!finished && mobFileWriter != null) {
|
||||||
abortWriter(mobFileWriter);
|
// Remove all MOB references because compaction failed
|
||||||
}
|
mobRefSet.get().clear();
|
||||||
if (!finished && delFileWriter != null) {
|
// Abort writer
|
||||||
abortWriter(delFileWriter);
|
LOG.debug("Aborting writer for {} because of a compaction failure, Store {}",
|
||||||
}
|
mobFileWriter.getPath(), getStoreInfo());
|
||||||
}
|
|
||||||
if (delFileWriter != null) {
|
|
||||||
if (deleteMarkersCount > 0) {
|
|
||||||
// If the del file is not empty, commit it.
|
|
||||||
// If the commit fails, the compaction is re-performed again.
|
|
||||||
delFileWriter.appendMetadata(fd.maxSeqId, major, deleteMarkersCount);
|
|
||||||
delFileWriter.close();
|
|
||||||
mobStore.commitFile(delFileWriter.getPath(), path);
|
|
||||||
} else {
|
|
||||||
// If the del file is empty, delete it instead of committing.
|
|
||||||
abortWriter(delFileWriter);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
if (mobFileWriter != null) {
|
|
||||||
if (mobCells > 0) {
|
|
||||||
// If the mob file is not empty, commit it.
|
|
||||||
mobFileWriter.appendMetadata(fd.maxSeqId, major, mobCells);
|
|
||||||
mobFileWriter.close();
|
|
||||||
mobStore.commitFile(mobFileWriter.getPath(), path);
|
|
||||||
} else {
|
|
||||||
// If the mob file is empty, delete it instead of committing.
|
|
||||||
abortWriter(mobFileWriter);
|
abortWriter(mobFileWriter);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Commit last MOB writer
|
||||||
|
commitOrAbortMobWriter(mobFileWriter, fd.maxSeqId, mobCells, major);
|
||||||
mobStore.updateCellsCountCompactedFromMob(cellsCountCompactedFromMob);
|
mobStore.updateCellsCountCompactedFromMob(cellsCountCompactedFromMob);
|
||||||
mobStore.updateCellsCountCompactedToMob(cellsCountCompactedToMob);
|
mobStore.updateCellsCountCompactedToMob(cellsCountCompactedToMob);
|
||||||
mobStore.updateCellsSizeCompactedFromMob(cellsSizeCompactedFromMob);
|
mobStore.updateCellsSizeCompactedFromMob(cellsSizeCompactedFromMob);
|
||||||
|
@ -369,4 +546,85 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
|
||||||
progress.complete();
|
progress.complete();
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private String getStoreInfo() {
|
||||||
|
return String.format("[table=%s family=%s region=%s]", store.getTableName().getNameAsString(),
|
||||||
|
store.getColumnFamilyName(), store.getRegionInfo().getEncodedName()) ;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void clearThreadLocals() {
|
||||||
|
Set<String> set = mobRefSet.get();
|
||||||
|
if (set != null) {
|
||||||
|
set.clear();
|
||||||
|
}
|
||||||
|
HashMap<String, Long> map = mobLengthMap.get();
|
||||||
|
if (map != null) {
|
||||||
|
map.clear();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private StoreFileWriter newMobWriter(FileDetails fd)
|
||||||
|
throws IOException {
|
||||||
|
try {
|
||||||
|
StoreFileWriter mobFileWriter = mobStore.createWriterInTmp(new Date(fd.latestPutTs),
|
||||||
|
fd.maxKeyCount, compactionCompression, store.getRegionInfo().getStartKey(), true);
|
||||||
|
LOG.debug("New MOB writer created={} store={}", mobFileWriter.getPath().getName(),
|
||||||
|
getStoreInfo());
|
||||||
|
// Add reference we get for compact MOB
|
||||||
|
mobRefSet.get().add(mobFileWriter.getPath().getName());
|
||||||
|
return mobFileWriter;
|
||||||
|
} catch (IOException e) {
|
||||||
|
// Bailing out
|
||||||
|
throw new IOException(String.format("Failed to create mob writer, store=%s",
|
||||||
|
getStoreInfo()), e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void commitOrAbortMobWriter(StoreFileWriter mobFileWriter, long maxSeqId, long mobCells,
|
||||||
|
boolean major) throws IOException {
|
||||||
|
// Commit or abort major mob writer
|
||||||
|
// If IOException happens during below operation, some
|
||||||
|
// MOB files can be committed partially, but corresponding
|
||||||
|
// store file won't be committed, therefore these MOB files
|
||||||
|
// become orphans and will be deleted during next MOB cleaning chore cycle
|
||||||
|
|
||||||
|
if (mobFileWriter != null) {
|
||||||
|
LOG.debug("Commit or abort size={} mobCells={} major={} file={}, store={}",
|
||||||
|
mobFileWriter.getPos(), mobCells, major, mobFileWriter.getPath().getName(),
|
||||||
|
getStoreInfo());
|
||||||
|
Path path =
|
||||||
|
MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName());
|
||||||
|
if (mobCells > 0) {
|
||||||
|
// If the mob file is not empty, commit it.
|
||||||
|
mobFileWriter.appendMetadata(maxSeqId, major, mobCells);
|
||||||
|
mobFileWriter.close();
|
||||||
|
mobStore.commitFile(mobFileWriter.getPath(), path);
|
||||||
|
} else {
|
||||||
|
// If the mob file is empty, delete it instead of committing.
|
||||||
|
LOG.debug("Aborting writer for {} because there are no MOB cells, store={}",
|
||||||
|
mobFileWriter.getPath(), getStoreInfo());
|
||||||
|
// Remove MOB file from reference set
|
||||||
|
mobRefSet.get().remove(mobFileWriter.getPath().getName());
|
||||||
|
abortWriter(mobFileWriter);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
LOG.debug("Mob file writer is null, skipping commit/abort, store=",
|
||||||
|
getStoreInfo());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected List<Path> commitWriter(StoreFileWriter writer, FileDetails fd,
|
||||||
|
CompactionRequestImpl request) throws IOException {
|
||||||
|
List<Path> newFiles = Lists.newArrayList(writer.getPath());
|
||||||
|
writer.appendMetadata(fd.maxSeqId, request.isAllFiles(), request.getFiles());
|
||||||
|
// Append MOB references
|
||||||
|
Set<String> refSet = mobRefSet.get();
|
||||||
|
writer.appendMobMetadata(refSet);
|
||||||
|
writer.close();
|
||||||
|
clearThreadLocals();
|
||||||
|
return newFiles;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,7 +22,9 @@ import java.io.IOException;
|
||||||
import java.io.InterruptedIOException;
|
import java.io.InterruptedIOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Date;
|
import java.util.Date;
|
||||||
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
|
@ -69,6 +71,13 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher {
|
||||||
private long mobCellValueSizeThreshold = 0;
|
private long mobCellValueSizeThreshold = 0;
|
||||||
private Path targetPath;
|
private Path targetPath;
|
||||||
private HMobStore mobStore;
|
private HMobStore mobStore;
|
||||||
|
// MOB file reference set
|
||||||
|
static ThreadLocal<Set<String>> mobRefSet = new ThreadLocal<Set<String>>() {
|
||||||
|
@Override
|
||||||
|
protected Set<String> initialValue() {
|
||||||
|
return new HashSet<String>();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
public DefaultMobStoreFlusher(Configuration conf, HStore store) throws IOException {
|
public DefaultMobStoreFlusher(Configuration conf, HStore store) throws IOException {
|
||||||
super(conf, store);
|
super(conf, store);
|
||||||
|
@ -188,6 +197,8 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher {
|
||||||
throughputController.start(flushName);
|
throughputController.start(flushName);
|
||||||
}
|
}
|
||||||
IOException ioe = null;
|
IOException ioe = null;
|
||||||
|
// Clear all past MOB references
|
||||||
|
mobRefSet.get().clear();
|
||||||
try {
|
try {
|
||||||
do {
|
do {
|
||||||
hasMore = scanner.next(cells, scannerContext);
|
hasMore = scanner.next(cells, scannerContext);
|
||||||
|
@ -204,7 +215,6 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher {
|
||||||
mobFileWriter.append(c);
|
mobFileWriter.append(c);
|
||||||
mobSize += c.getValueLength();
|
mobSize += c.getValueLength();
|
||||||
mobCount++;
|
mobCount++;
|
||||||
|
|
||||||
// append the tags to the KeyValue.
|
// append the tags to the KeyValue.
|
||||||
// The key is same, the value is the filename of the mob file
|
// The key is same, the value is the filename of the mob file
|
||||||
Cell reference = MobUtils.createMobRefCell(c, fileName,
|
Cell reference = MobUtils.createMobRefCell(c, fileName,
|
||||||
|
@ -244,9 +254,12 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher {
|
||||||
status.setStatus("Flushing mob file " + store + ": closing flushed file");
|
status.setStatus("Flushing mob file " + store + ": closing flushed file");
|
||||||
mobFileWriter.close();
|
mobFileWriter.close();
|
||||||
mobStore.commitFile(mobFileWriter.getPath(), targetPath);
|
mobStore.commitFile(mobFileWriter.getPath(), targetPath);
|
||||||
|
LOG.debug("Flush store file: {}, store: {}", writer.getPath(), getStoreInfo());
|
||||||
mobStore.updateMobFlushCount();
|
mobStore.updateMobFlushCount();
|
||||||
mobStore.updateMobFlushedCellsCount(mobCount);
|
mobStore.updateMobFlushedCellsCount(mobCount);
|
||||||
mobStore.updateMobFlushedCellsSize(mobSize);
|
mobStore.updateMobFlushedCellsSize(mobSize);
|
||||||
|
// Add mob reference to store file metadata
|
||||||
|
mobRefSet.get().add(mobFileWriter.getPath().getName());
|
||||||
} else {
|
} else {
|
||||||
try {
|
try {
|
||||||
status.setStatus("Flushing mob file " + store + ": no mob cells, closing flushed file");
|
status.setStatus("Flushing mob file " + store + ": no mob cells, closing flushed file");
|
||||||
|
@ -258,4 +271,22 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void finalizeWriter(StoreFileWriter writer, long cacheFlushSeqNum,
|
||||||
|
MonitoredTask status) throws IOException {
|
||||||
|
// Write out the log sequence number that corresponds to this output
|
||||||
|
// hfile. Also write current time in metadata as minFlushTime.
|
||||||
|
// The hfile is current up to and including cacheFlushSeqNum.
|
||||||
|
status.setStatus("Flushing " + store + ": appending metadata");
|
||||||
|
writer.appendMetadata(cacheFlushSeqNum, false);
|
||||||
|
writer.appendMobMetadata(mobRefSet.get());
|
||||||
|
status.setStatus("Flushing " + store + ": closing flushed file");
|
||||||
|
writer.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
private String getStoreInfo() {
|
||||||
|
return String.format("[table=%s family=%s region=%s]", store.getTableName().getNameAsString(),
|
||||||
|
store.getColumnFamilyName(), store.getRegionInfo().getEncodedName()) ;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -57,7 +57,8 @@ public class ExpiredMobFileCleaner extends Configured implements Tool {
|
||||||
* @param tableName The current table name.
|
* @param tableName The current table name.
|
||||||
* @param family The current family.
|
* @param family The current family.
|
||||||
*/
|
*/
|
||||||
public void cleanExpiredMobFiles(String tableName, ColumnFamilyDescriptor family) throws IOException {
|
public void cleanExpiredMobFiles(String tableName, ColumnFamilyDescriptor family)
|
||||||
|
throws IOException {
|
||||||
Configuration conf = getConf();
|
Configuration conf = getConf();
|
||||||
TableName tn = TableName.valueOf(tableName);
|
TableName tn = TableName.valueOf(tableName);
|
||||||
FileSystem fs = FileSystem.get(conf);
|
FileSystem fs = FileSystem.get(conf);
|
||||||
|
@ -67,7 +68,7 @@ public class ExpiredMobFileCleaner extends Configured implements Tool {
|
||||||
copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f);
|
copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f);
|
||||||
CacheConfig cacheConfig = new CacheConfig(copyOfConf);
|
CacheConfig cacheConfig = new CacheConfig(copyOfConf);
|
||||||
MobUtils.cleanExpiredMobFiles(fs, conf, tn, family, cacheConfig,
|
MobUtils.cleanExpiredMobFiles(fs, conf, tn, family, cacheConfig,
|
||||||
EnvironmentEdgeManager.currentTime());
|
EnvironmentEdgeManager.currentTime());
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void main(String[] args) throws Exception {
|
public static void main(String[] args) throws Exception {
|
||||||
|
|
|
@ -35,26 +35,88 @@ public final class MobConstants {
|
||||||
public static final String MOB_CACHE_BLOCKS = "hbase.mob.cache.blocks";
|
public static final String MOB_CACHE_BLOCKS = "hbase.mob.cache.blocks";
|
||||||
public static final String MOB_SCAN_REF_ONLY = "hbase.mob.scan.ref.only";
|
public static final String MOB_SCAN_REF_ONLY = "hbase.mob.scan.ref.only";
|
||||||
public static final String EMPTY_VALUE_ON_MOBCELL_MISS = "empty.value.on.mobcell.miss";
|
public static final String EMPTY_VALUE_ON_MOBCELL_MISS = "empty.value.on.mobcell.miss";
|
||||||
|
|
||||||
public static final String MOB_FILE_CACHE_SIZE_KEY = "hbase.mob.file.cache.size";
|
public static final String MOB_FILE_CACHE_SIZE_KEY = "hbase.mob.file.cache.size";
|
||||||
public static final int DEFAULT_MOB_FILE_CACHE_SIZE = 1000;
|
public static final int DEFAULT_MOB_FILE_CACHE_SIZE = 1000;
|
||||||
|
|
||||||
public static final String MOB_DIR_NAME = "mobdir";
|
public static final String MOB_DIR_NAME = "mobdir";
|
||||||
public static final String MOB_REGION_NAME = ".mob";
|
public static final String MOB_REGION_NAME = ".mob";
|
||||||
public static final byte[] MOB_REGION_NAME_BYTES = Bytes.toBytes(MOB_REGION_NAME);
|
public static final byte[] MOB_REGION_NAME_BYTES = Bytes.toBytes(MOB_REGION_NAME);
|
||||||
|
public static final String MOB_CLEANER_PERIOD = "hbase.master.mob.cleaner.period";
|
||||||
public static final String MOB_CLEANER_PERIOD = "hbase.master.mob.ttl.cleaner.period";
|
public static final String DEPRECATED_MOB_CLEANER_PERIOD = "hbase.master.mob.ttl.cleaner.period";
|
||||||
public static final int DEFAULT_MOB_CLEANER_PERIOD = 24 * 60 * 60; // one day
|
public static final int DEFAULT_MOB_CLEANER_PERIOD = 24 * 60 * 60; // one day
|
||||||
|
|
||||||
public static final String MOB_CACHE_EVICT_PERIOD = "hbase.mob.cache.evict.period";
|
public static final String MOB_CACHE_EVICT_PERIOD = "hbase.mob.cache.evict.period";
|
||||||
public static final String MOB_CACHE_EVICT_REMAIN_RATIO = "hbase.mob.cache.evict.remain.ratio";
|
public static final String MOB_CACHE_EVICT_REMAIN_RATIO = "hbase.mob.cache.evict.remain.ratio";
|
||||||
public static final Tag MOB_REF_TAG = new ArrayBackedTag(TagType.MOB_REFERENCE_TAG_TYPE,
|
public static final Tag MOB_REF_TAG =
|
||||||
HConstants.EMPTY_BYTE_ARRAY);
|
new ArrayBackedTag(TagType.MOB_REFERENCE_TAG_TYPE, HConstants.EMPTY_BYTE_ARRAY);
|
||||||
|
|
||||||
public static final float DEFAULT_EVICT_REMAIN_RATIO = 0.5f;
|
public static final float DEFAULT_EVICT_REMAIN_RATIO = 0.5f;
|
||||||
public static final long DEFAULT_MOB_CACHE_EVICT_PERIOD = 3600L;
|
public static final long DEFAULT_MOB_CACHE_EVICT_PERIOD = 3600L;
|
||||||
|
|
||||||
public final static String TEMP_DIR_NAME = ".tmp";
|
public final static String TEMP_DIR_NAME = ".tmp";
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The max number of a MOB table regions that is allowed in a batch of the mob compaction. By
|
||||||
|
* setting this number to a custom value, users can control the overall effect of a major
|
||||||
|
* compaction of a large MOB-enabled table.
|
||||||
|
*/
|
||||||
|
|
||||||
|
public static final String MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE =
|
||||||
|
"hbase.mob.major.compaction.region.batch.size";
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Default is 0 - means no limit - all regions of a MOB table will be compacted at once
|
||||||
|
*/
|
||||||
|
|
||||||
|
public static final int DEFAULT_MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE = 0;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The period that MobCompactionChore runs. The unit is second. The default value is one week.
|
||||||
|
*/
|
||||||
|
public static final String MOB_COMPACTION_CHORE_PERIOD = "hbase.mob.compaction.chore.period";
|
||||||
|
public static final int DEFAULT_MOB_COMPACTION_CHORE_PERIOD = 24 * 60 * 60 * 7; // a week
|
||||||
|
public static final String MOB_COMPACTOR_CLASS_KEY = "hbase.mob.compactor.class";
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Mob compaction type: "full", "optimized" "full" - run full major compaction (during migration)
|
||||||
|
* "optimized" - optimized version for use case with infrequent updates/deletes
|
||||||
|
*/
|
||||||
|
|
||||||
|
public final static String OPTIMIZED_MOB_COMPACTION_TYPE = "optimized";
|
||||||
|
|
||||||
|
public final static String FULL_MOB_COMPACTION_TYPE = "full";
|
||||||
|
|
||||||
|
public final static String MOB_COMPACTION_TYPE_KEY = "hbase.mob.compaction.type";
|
||||||
|
|
||||||
|
public final static String DEFAULT_MOB_COMPACTION_TYPE = FULL_MOB_COMPACTION_TYPE;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Maximum size of a MOB compaction selection
|
||||||
|
*/
|
||||||
|
public static final String MOB_COMPACTION_MAX_FILE_SIZE_KEY =
|
||||||
|
"hbase.mob.compactions.max.file.size";
|
||||||
|
/**
|
||||||
|
* Default maximum selection size = 1GB
|
||||||
|
*/
|
||||||
|
public static final long DEFAULT_MOB_COMPACTION_MAX_FILE_SIZE = 1024 * 1024 * 1024;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Use this configuration option with caution, only during upgrade procedure to handle missing MOB
|
||||||
|
* cells during compaction.
|
||||||
|
*/
|
||||||
|
public static final String MOB_UNSAFE_DISCARD_MISS_KEY = "hbase.unsafe.mob.discard.miss";
|
||||||
|
|
||||||
|
public static final boolean DEFAULT_MOB_DISCARD_MISS = false;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Minimum MOB file age to archive, default (3600000 - 1h)
|
||||||
|
*/
|
||||||
|
public static final String MIN_AGE_TO_ARCHIVE_KEY = "hbase.mob.min.age.archive";
|
||||||
|
|
||||||
|
public static final long DEFAULT_MIN_AGE_TO_ARCHIVE = 3600000; // 1h
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Old configuration parameters (obsolete)
|
||||||
|
*/
|
||||||
|
|
||||||
public final static String BULKLOAD_DIR_NAME = ".bulkload";
|
public final static String BULKLOAD_DIR_NAME = ".bulkload";
|
||||||
public final static byte[] MOB_TABLE_LOCK_SUFFIX = Bytes.toBytes(".mobLock");
|
public final static byte[] MOB_TABLE_LOCK_SUFFIX = Bytes.toBytes(".mobLock");
|
||||||
public final static String EMPTY_STRING = "";
|
public final static String EMPTY_STRING = "";
|
||||||
|
@ -63,40 +125,23 @@ public final class MobConstants {
|
||||||
* be merged in mob compaction. The default value is 1280MB.
|
* be merged in mob compaction. The default value is 1280MB.
|
||||||
*/
|
*/
|
||||||
public static final String MOB_COMPACTION_MERGEABLE_THRESHOLD =
|
public static final String MOB_COMPACTION_MERGEABLE_THRESHOLD =
|
||||||
"hbase.mob.compaction.mergeable.threshold";
|
"hbase.mob.compaction.mergeable.threshold";
|
||||||
public static final long DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD = 10 * 128 * 1024 * 1024;
|
public static final long DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD = 10 * 128 * 1024 * 1024;
|
||||||
/**
|
|
||||||
* The max number of del files that is allowed in the mob file compaction. In the mob
|
|
||||||
* compaction, when the number of existing del files is larger than this value, they are merged
|
|
||||||
* until number of del files is not larger this value. The default value is 3.
|
|
||||||
*/
|
|
||||||
public static final String MOB_DELFILE_MAX_COUNT = "hbase.mob.delfile.max.count";
|
public static final String MOB_DELFILE_MAX_COUNT = "hbase.mob.delfile.max.count";
|
||||||
public static final int DEFAULT_MOB_DELFILE_MAX_COUNT = 3;
|
public static final int DEFAULT_MOB_DELFILE_MAX_COUNT = 3;
|
||||||
/**
|
/**
|
||||||
* The max number of the mob files that is allowed in a batch of the mob compaction.
|
* The max number of the mob files that is allowed in a batch of the mob compaction. The mob
|
||||||
* The mob compaction merges the small mob files to bigger ones. If the number of the
|
* compaction merges the small mob files to bigger ones. If the number of the small files is very
|
||||||
* small files is very large, it could lead to a "too many opened file handlers" in the merge.
|
* large, it could lead to a "too many opened file handlers" in the merge. And the merge has to be
|
||||||
* And the merge has to be split into batches. This value limits the number of mob files
|
* split into batches. This value limits the number of mob files that are selected in a batch of
|
||||||
* that are selected in a batch of the mob compaction. The default value is 100.
|
* the mob compaction. The default value is 100. Default is 0 - means no limit - all regions of a
|
||||||
|
* MOB table will be compacted at once
|
||||||
*/
|
*/
|
||||||
public static final String MOB_COMPACTION_BATCH_SIZE =
|
public static final String MOB_COMPACTION_BATCH_SIZE = "hbase.mob.compaction.batch.size";
|
||||||
"hbase.mob.compaction.batch.size";
|
|
||||||
public static final int DEFAULT_MOB_COMPACTION_BATCH_SIZE = 100;
|
public static final int DEFAULT_MOB_COMPACTION_BATCH_SIZE = 100;
|
||||||
/**
|
public static final String MOB_COMPACTION_THREADS_MAX = "hbase.mob.compaction.threads.max";
|
||||||
* The period that MobCompactionChore runs. The unit is second.
|
|
||||||
* The default value is one week.
|
|
||||||
*/
|
|
||||||
public static final String MOB_COMPACTION_CHORE_PERIOD =
|
|
||||||
"hbase.mob.compaction.chore.period";
|
|
||||||
public static final int DEFAULT_MOB_COMPACTION_CHORE_PERIOD =
|
|
||||||
24 * 60 * 60 * 7; // a week
|
|
||||||
public static final String MOB_COMPACTOR_CLASS_KEY = "hbase.mob.compactor.class";
|
|
||||||
/**
|
|
||||||
* The max number of threads used in MobCompactor.
|
|
||||||
*/
|
|
||||||
public static final String MOB_COMPACTION_THREADS_MAX =
|
|
||||||
"hbase.mob.compaction.threads.max";
|
|
||||||
public static final int DEFAULT_MOB_COMPACTION_THREADS_MAX = 1;
|
public static final int DEFAULT_MOB_COMPACTION_THREADS_MAX = 1;
|
||||||
|
|
||||||
private MobConstants() {
|
private MobConstants() {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,322 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.hadoop.hbase.mob;
|
||||||
|
|
||||||
|
import java.io.FileNotFoundException;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.fs.RemoteIterator;
|
||||||
|
import org.apache.hadoop.hbase.ScheduledChore;
|
||||||
|
import org.apache.hadoop.hbase.TableDescriptors;
|
||||||
|
import org.apache.hadoop.hbase.TableName;
|
||||||
|
import org.apache.hadoop.hbase.backup.HFileArchiver;
|
||||||
|
import org.apache.hadoop.hbase.client.Admin;
|
||||||
|
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.client.Connection;
|
||||||
|
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||||
|
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||||
|
import org.apache.hadoop.hbase.master.HMaster;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.HStoreFile;
|
||||||
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
|
import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The class MobFileCleanerChore for running cleaner regularly to remove the expired
|
||||||
|
* and obsolete (files which have no active references to) mob files.
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public class MobFileCleanerChore extends ScheduledChore {
|
||||||
|
|
||||||
|
private static final Logger LOG = LoggerFactory.getLogger(MobFileCleanerChore.class);
|
||||||
|
private final HMaster master;
|
||||||
|
private ExpiredMobFileCleaner cleaner;
|
||||||
|
|
||||||
|
static {
|
||||||
|
Configuration.addDeprecation(MobConstants.DEPRECATED_MOB_CLEANER_PERIOD,
|
||||||
|
MobConstants.MOB_CLEANER_PERIOD);
|
||||||
|
}
|
||||||
|
|
||||||
|
public MobFileCleanerChore(HMaster master) {
|
||||||
|
super(master.getServerName() + "-MobFileCleanerChore", master,
|
||||||
|
master.getConfiguration().getInt(MobConstants.MOB_CLEANER_PERIOD,
|
||||||
|
MobConstants.DEFAULT_MOB_CLEANER_PERIOD),
|
||||||
|
master.getConfiguration().getInt(MobConstants.MOB_CLEANER_PERIOD,
|
||||||
|
MobConstants.DEFAULT_MOB_CLEANER_PERIOD),
|
||||||
|
TimeUnit.SECONDS);
|
||||||
|
this.master = master;
|
||||||
|
cleaner = new ExpiredMobFileCleaner();
|
||||||
|
cleaner.setConf(master.getConfiguration());
|
||||||
|
checkObsoleteConfigurations();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void checkObsoleteConfigurations() {
|
||||||
|
Configuration conf = master.getConfiguration();
|
||||||
|
|
||||||
|
if (conf.get("hbase.mob.compaction.mergeable.threshold") != null) {
|
||||||
|
LOG.warn("'hbase.mob.compaction.mergeable.threshold' is obsolete and not used anymore.");
|
||||||
|
}
|
||||||
|
if (conf.get("hbase.mob.delfile.max.count") != null) {
|
||||||
|
LOG.warn("'hbase.mob.delfile.max.count' is obsolete and not used anymore.");
|
||||||
|
}
|
||||||
|
if (conf.get("hbase.mob.compaction.threads.max") != null) {
|
||||||
|
LOG.warn("'hbase.mob.compaction.threads.max' is obsolete and not used anymore.");
|
||||||
|
}
|
||||||
|
if (conf.get("hbase.mob.compaction.batch.size") != null) {
|
||||||
|
LOG.warn("'hbase.mob.compaction.batch.size' is obsolete and not used anymore.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
public MobFileCleanerChore() {
|
||||||
|
this.master = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void chore() {
|
||||||
|
TableDescriptors htds = master.getTableDescriptors();
|
||||||
|
|
||||||
|
Map<String, TableDescriptor> map = null;
|
||||||
|
try {
|
||||||
|
map = htds.getAll();
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.error("MobFileCleanerChore failed", e);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
for (TableDescriptor htd : map.values()) {
|
||||||
|
for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
|
||||||
|
if (hcd.isMobEnabled() && hcd.getMinVersions() == 0) {
|
||||||
|
try {
|
||||||
|
cleaner.cleanExpiredMobFiles(htd.getTableName().getNameAsString(), hcd);
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.error("Failed to clean the expired mob files table={} family={}",
|
||||||
|
htd.getTableName().getNameAsString(), hcd.getNameAsString(), e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
// Now clean obsolete files for a table
|
||||||
|
LOG.info("Cleaning obsolete MOB files from table={}", htd.getTableName());
|
||||||
|
cleanupObsoleteMobFiles(master.getConfiguration(), htd.getTableName());
|
||||||
|
LOG.info("Cleaning obsolete MOB files finished for table={}", htd.getTableName());
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.error("Failed to clean the obsolete mob files for table={}",htd.getTableName(), e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Performs housekeeping file cleaning (called by MOB Cleaner chore)
|
||||||
|
* @param conf configuration
|
||||||
|
* @param table table name
|
||||||
|
* @throws IOException exception
|
||||||
|
*/
|
||||||
|
public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws IOException {
|
||||||
|
|
||||||
|
long minAgeToArchive =
|
||||||
|
conf.getLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, MobConstants.DEFAULT_MIN_AGE_TO_ARCHIVE);
|
||||||
|
// We check only those MOB files, which creation time is less
|
||||||
|
// than maxCreationTimeToArchive. This is a current time - 1h. 1 hour gap
|
||||||
|
// gives us full confidence that all corresponding store files will
|
||||||
|
// exist at the time cleaning procedure begins and will be examined.
|
||||||
|
// So, if MOB file creation time is greater than this maxTimeToArchive,
|
||||||
|
// this will be skipped and won't be archived.
|
||||||
|
long maxCreationTimeToArchive = EnvironmentEdgeManager.currentTime() - minAgeToArchive;
|
||||||
|
try (final Connection conn = ConnectionFactory.createConnection(conf);
|
||||||
|
final Admin admin = conn.getAdmin();) {
|
||||||
|
TableDescriptor htd = admin.getDescriptor(table);
|
||||||
|
List<ColumnFamilyDescriptor> list = MobUtils.getMobColumnFamilies(htd);
|
||||||
|
if (list.size() == 0) {
|
||||||
|
LOG.info("Skipping non-MOB table [{}]", table);
|
||||||
|
return;
|
||||||
|
} else {
|
||||||
|
LOG.info("Only MOB files whose creation time older than {} will be archived, table={}",
|
||||||
|
maxCreationTimeToArchive, table);
|
||||||
|
}
|
||||||
|
|
||||||
|
Path rootDir = FSUtils.getRootDir(conf);
|
||||||
|
Path tableDir = FSUtils.getTableDir(rootDir, table);
|
||||||
|
// How safe is this call?
|
||||||
|
List<Path> regionDirs = FSUtils.getRegionDirs(FileSystem.get(conf), tableDir);
|
||||||
|
|
||||||
|
Set<String> allActiveMobFileName = new HashSet<String>();
|
||||||
|
FileSystem fs = FileSystem.get(conf);
|
||||||
|
for (Path regionPath : regionDirs) {
|
||||||
|
for (ColumnFamilyDescriptor hcd : list) {
|
||||||
|
String family = hcd.getNameAsString();
|
||||||
|
Path storePath = new Path(regionPath, family);
|
||||||
|
boolean succeed = false;
|
||||||
|
Set<String> regionMobs = new HashSet<String>();
|
||||||
|
|
||||||
|
while (!succeed) {
|
||||||
|
if (!fs.exists(storePath)) {
|
||||||
|
String errMsg =
|
||||||
|
String.format("Directory %s was deleted during MOB file cleaner chore"
|
||||||
|
+ " execution, aborting MOB file cleaner chore.",
|
||||||
|
storePath);
|
||||||
|
throw new IOException(errMsg);
|
||||||
|
}
|
||||||
|
RemoteIterator<LocatedFileStatus> rit = fs.listLocatedStatus(storePath);
|
||||||
|
List<Path> storeFiles = new ArrayList<Path>();
|
||||||
|
// Load list of store files first
|
||||||
|
while (rit.hasNext()) {
|
||||||
|
Path p = rit.next().getPath();
|
||||||
|
if (fs.isFile(p)) {
|
||||||
|
storeFiles.add(p);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
LOG.info("Found {} store files in: {}", storeFiles.size(), storePath);
|
||||||
|
Path currentPath = null;
|
||||||
|
try {
|
||||||
|
for (Path pp : storeFiles) {
|
||||||
|
currentPath = pp;
|
||||||
|
LOG.trace("Store file: {}", pp);
|
||||||
|
HStoreFile sf =
|
||||||
|
new HStoreFile(fs, pp, conf, CacheConfig.DISABLED, BloomType.NONE, true);
|
||||||
|
sf.initReader();
|
||||||
|
byte[] mobRefData = sf.getMetadataValue(HStoreFile.MOB_FILE_REFS);
|
||||||
|
byte[] bulkloadMarkerData = sf.getMetadataValue(HStoreFile.BULKLOAD_TASK_KEY);
|
||||||
|
// close store file to avoid memory leaks
|
||||||
|
sf.closeStoreFile(true);
|
||||||
|
if (mobRefData == null && bulkloadMarkerData == null) {
|
||||||
|
LOG.warn("Found old store file with no MOB_FILE_REFS: {} - "
|
||||||
|
+ "can not proceed until all old files will be MOB-compacted.",
|
||||||
|
pp);
|
||||||
|
return;
|
||||||
|
} else if (mobRefData == null && bulkloadMarkerData != null) {
|
||||||
|
LOG.debug("Skipping file without MOB references (bulkloaded file):{}", pp);
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
// mobRefData will never be null here, but to make FindBugs happy
|
||||||
|
if (mobRefData != null && mobRefData.length > 1) {
|
||||||
|
// if length = 1 means NULL, that there are no MOB references
|
||||||
|
// in this store file, but the file was created by new MOB code
|
||||||
|
String[] mobs = new String(mobRefData).split(",");
|
||||||
|
if (LOG.isTraceEnabled()) {
|
||||||
|
LOG.trace("Found: {} mob references: {}", mobs.length, Arrays.toString(mobs));
|
||||||
|
} else {
|
||||||
|
LOG.debug("Found: {} mob references", mobs.length);
|
||||||
|
}
|
||||||
|
regionMobs.addAll(Arrays.asList(mobs));
|
||||||
|
} else {
|
||||||
|
LOG.debug("File {} does not have mob references", currentPath);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} catch (FileNotFoundException e) {
|
||||||
|
LOG.warn("Missing file:{} Starting MOB cleaning cycle from the beginning"+
|
||||||
|
" due to error", currentPath, e);
|
||||||
|
regionMobs.clear();
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
succeed = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Add MOB references for current region/family
|
||||||
|
allActiveMobFileName.addAll(regionMobs);
|
||||||
|
} // END column families
|
||||||
|
} // END regions
|
||||||
|
// Check if number of MOB files too big (over 1M)
|
||||||
|
if (allActiveMobFileName.size() > 1000000) {
|
||||||
|
LOG.warn("Found too many active MOB files: {}, table={}, "+
|
||||||
|
"this may result in high memory pressure.",
|
||||||
|
allActiveMobFileName.size(), table);
|
||||||
|
}
|
||||||
|
LOG.debug("Found: {} active mob refs for table={}",
|
||||||
|
allActiveMobFileName.size(), table);
|
||||||
|
allActiveMobFileName.stream().forEach(LOG::trace);
|
||||||
|
|
||||||
|
// Now scan MOB directories and find MOB files with no references to them
|
||||||
|
for (ColumnFamilyDescriptor hcd : list) {
|
||||||
|
List<Path> toArchive = new ArrayList<Path>();
|
||||||
|
String family = hcd.getNameAsString();
|
||||||
|
Path dir = MobUtils.getMobFamilyPath(conf, table, family);
|
||||||
|
RemoteIterator<LocatedFileStatus> rit = fs.listLocatedStatus(dir);
|
||||||
|
while (rit.hasNext()) {
|
||||||
|
LocatedFileStatus lfs = rit.next();
|
||||||
|
Path p = lfs.getPath();
|
||||||
|
if (!allActiveMobFileName.contains(p.getName())) {
|
||||||
|
// MOB is not in a list of active references, but it can be too
|
||||||
|
// fresh, skip it in this case
|
||||||
|
long creationTime = fs.getFileStatus(p).getModificationTime();
|
||||||
|
if (creationTime < maxCreationTimeToArchive) {
|
||||||
|
LOG.trace("Archiving MOB file {} creation time={}", p,
|
||||||
|
(fs.getFileStatus(p).getModificationTime()));
|
||||||
|
toArchive.add(p);
|
||||||
|
} else {
|
||||||
|
LOG.trace("Skipping fresh file: {}. Creation time={}", p,
|
||||||
|
fs.getFileStatus(p).getModificationTime());
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
LOG.trace("Keeping active MOB file: {}", p);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
LOG.info(" MOB Cleaner found {} files to archive for table={} family={}",
|
||||||
|
toArchive.size(), table, family);
|
||||||
|
archiveMobFiles(conf, table, family.getBytes(), toArchive);
|
||||||
|
LOG.info(" MOB Cleaner archived {} files, table={} family={}",
|
||||||
|
toArchive.size(), table, family);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Archives the mob files.
|
||||||
|
* @param conf The current configuration.
|
||||||
|
* @param tableName The table name.
|
||||||
|
* @param family The name of the column family.
|
||||||
|
* @param storeFiles The files to be archived.
|
||||||
|
* @throws IOException exception
|
||||||
|
*/
|
||||||
|
public void archiveMobFiles(Configuration conf, TableName tableName, byte[] family,
|
||||||
|
List<Path> storeFiles) throws IOException {
|
||||||
|
|
||||||
|
if (storeFiles.size() == 0) {
|
||||||
|
// nothing to remove
|
||||||
|
LOG.debug("Skipping archiving old MOB files - no files found for table={} cf={}",
|
||||||
|
tableName, Bytes.toString(family));
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
Path mobTableDir = FSUtils.getTableDir(MobUtils.getMobHome(conf), tableName);
|
||||||
|
FileSystem fs = storeFiles.get(0).getFileSystem(conf);
|
||||||
|
|
||||||
|
for (Path p : storeFiles) {
|
||||||
|
LOG.debug("MOB Cleaner is archiving: {}", p);
|
||||||
|
HFileArchiver.archiveStoreFile(conf, fs, MobUtils.getMobRegionInfo(tableName),
|
||||||
|
mobTableDir, family, p);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,245 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.hbase.mob;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.ScheduledChore;
|
||||||
|
import org.apache.hadoop.hbase.TableDescriptors;
|
||||||
|
import org.apache.hadoop.hbase.TableName;
|
||||||
|
import org.apache.hadoop.hbase.client.Admin;
|
||||||
|
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.client.CompactionState;
|
||||||
|
import org.apache.hadoop.hbase.client.Connection;
|
||||||
|
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||||
|
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.client.TableState;
|
||||||
|
import org.apache.hadoop.hbase.master.HMaster;
|
||||||
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Periodic MOB compaction chore.
|
||||||
|
* It runs MOB compaction on region servers in parallel, thus
|
||||||
|
* utilizing distributed cluster resources. To avoid possible major
|
||||||
|
* compaction storms, one can specify maximum number regions to be compacted
|
||||||
|
* in parallel by setting configuration parameter: <br>
|
||||||
|
* 'hbase.mob.major.compaction.region.batch.size', which by default is 0 (unlimited).
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public class MobFileCompactionChore extends ScheduledChore {
|
||||||
|
|
||||||
|
private static final Logger LOG = LoggerFactory.getLogger(MobFileCompactionChore.class);
|
||||||
|
private HMaster master;
|
||||||
|
private int regionBatchSize = 0;// not set - compact all
|
||||||
|
|
||||||
|
public MobFileCompactionChore(HMaster master) {
|
||||||
|
super(master.getServerName() + "-MobFileCompactionChore", master,
|
||||||
|
master.getConfiguration().getInt(MobConstants.MOB_COMPACTION_CHORE_PERIOD,
|
||||||
|
MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD),
|
||||||
|
master.getConfiguration().getInt(MobConstants.MOB_COMPACTION_CHORE_PERIOD,
|
||||||
|
MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD),
|
||||||
|
TimeUnit.SECONDS);
|
||||||
|
this.master = master;
|
||||||
|
this.regionBatchSize =
|
||||||
|
master.getConfiguration().getInt(MobConstants.MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE,
|
||||||
|
MobConstants.DEFAULT_MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
public MobFileCompactionChore(Configuration conf, int batchSize) {
|
||||||
|
this.regionBatchSize = batchSize;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void chore() {
|
||||||
|
|
||||||
|
boolean reported = false;
|
||||||
|
|
||||||
|
try (Connection conn = master.getConnection();
|
||||||
|
Admin admin = conn.getAdmin();) {
|
||||||
|
|
||||||
|
TableDescriptors htds = master.getTableDescriptors();
|
||||||
|
Map<String, TableDescriptor> map = htds.getAll();
|
||||||
|
for (TableDescriptor htd : map.values()) {
|
||||||
|
if (!master.getTableStateManager().isTableState(htd.getTableName(),
|
||||||
|
TableState.State.ENABLED)) {
|
||||||
|
LOG.info("Skipping MOB compaction on table {} because it is not ENABLED",
|
||||||
|
htd.getTableName());
|
||||||
|
continue;
|
||||||
|
} else {
|
||||||
|
LOG.info("Starting MOB compaction on table {}, checking {} column families",
|
||||||
|
htd.getTableName(), htd.getColumnFamilyCount());
|
||||||
|
}
|
||||||
|
for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
|
||||||
|
try {
|
||||||
|
if (hcd.isMobEnabled()) {
|
||||||
|
if (!reported) {
|
||||||
|
master.reportMobCompactionStart(htd.getTableName());
|
||||||
|
reported = true;
|
||||||
|
}
|
||||||
|
LOG.info("Major MOB compacting table={} cf={}", htd.getTableName(),
|
||||||
|
hcd.getNameAsString());
|
||||||
|
if (regionBatchSize == MobConstants.DEFAULT_MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE) {
|
||||||
|
LOG.debug("Table={} cf ={}: batch MOB compaction is disabled, {}=0 -"+
|
||||||
|
" all regions will be compacted in parallel", htd.getTableName(),
|
||||||
|
hcd.getNameAsString(), "hbase.mob.compaction.batch.size");
|
||||||
|
admin.majorCompact(htd.getTableName(), hcd.getName());
|
||||||
|
} else {
|
||||||
|
LOG.info("Table={} cf={}: performing MOB major compaction in batches "+
|
||||||
|
"'hbase.mob.compaction.batch.size'={}", htd.getTableName(),
|
||||||
|
hcd.getNameAsString(), regionBatchSize);
|
||||||
|
performMajorCompactionInBatches(admin, htd, hcd);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
LOG.debug("Skipping table={} column family={} because it is not MOB-enabled",
|
||||||
|
htd.getTableName(), hcd.getNameAsString());
|
||||||
|
}
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.error("Failed to compact table={} cf={}",
|
||||||
|
htd.getTableName(), hcd.getNameAsString(), e);
|
||||||
|
} catch (InterruptedException ee) {
|
||||||
|
Thread.currentThread().interrupt();
|
||||||
|
master.reportMobCompactionEnd(htd.getTableName());
|
||||||
|
LOG.warn("Failed to compact table={} cf={}",
|
||||||
|
htd.getTableName(), hcd.getNameAsString(), ee);
|
||||||
|
// Quit the chore
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (reported) {
|
||||||
|
master.reportMobCompactionEnd(htd.getTableName());
|
||||||
|
reported = false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.error("Failed to compact", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
public void performMajorCompactionInBatches(Admin admin, TableDescriptor htd,
|
||||||
|
ColumnFamilyDescriptor hcd) throws IOException, InterruptedException {
|
||||||
|
|
||||||
|
List<RegionInfo> regions = admin.getRegions(htd.getTableName());
|
||||||
|
if (regions.size() <= this.regionBatchSize) {
|
||||||
|
LOG.debug(
|
||||||
|
"Table={} cf={} - performing major MOB compaction in non-batched mode,"
|
||||||
|
+ "regions={}, batch size={}",
|
||||||
|
htd.getTableName(), hcd.getNameAsString(), regions.size(), regionBatchSize);
|
||||||
|
admin.majorCompact(htd.getTableName(), hcd.getName());
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
// Shuffle list of regions in case if they come ordered by region server
|
||||||
|
Collections.shuffle(regions);
|
||||||
|
// Create first batch
|
||||||
|
List<RegionInfo> toCompact = new ArrayList<RegionInfo>(this.regionBatchSize);
|
||||||
|
for (int i = 0; i < this.regionBatchSize; i++) {
|
||||||
|
toCompact.add(regions.remove(0));
|
||||||
|
}
|
||||||
|
|
||||||
|
// Start compaction now
|
||||||
|
for (RegionInfo ri : toCompact) {
|
||||||
|
startCompaction(admin, htd.getTableName(), ri, hcd.getName());
|
||||||
|
}
|
||||||
|
|
||||||
|
List<RegionInfo> compacted = new ArrayList<RegionInfo>(toCompact.size());
|
||||||
|
List<RegionInfo> failed = new ArrayList<RegionInfo>();
|
||||||
|
int totalCompacted = 0;
|
||||||
|
while (!toCompact.isEmpty()) {
|
||||||
|
// Check status of active compactions
|
||||||
|
for (RegionInfo ri : toCompact) {
|
||||||
|
try {
|
||||||
|
if (admin.getCompactionStateForRegion(ri.getRegionName()) == CompactionState.NONE) {
|
||||||
|
totalCompacted++;
|
||||||
|
LOG.info(
|
||||||
|
"Finished major MOB compaction: table={} cf={} region={} compacted regions={}",
|
||||||
|
htd.getTableName(), hcd.getNameAsString(), ri.getRegionNameAsString(),
|
||||||
|
totalCompacted);
|
||||||
|
compacted.add(ri);
|
||||||
|
}
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.error("Could not get compaction state for table={} cf={} region={}, compaction will"+
|
||||||
|
" aborted for the region.",
|
||||||
|
htd.getTableName(), hcd.getNameAsString(), ri.getEncodedName());
|
||||||
|
LOG.error("Because of:", e);
|
||||||
|
failed.add(ri);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// Remove failed regions to avoid
|
||||||
|
// endless compaction loop
|
||||||
|
toCompact.removeAll(failed);
|
||||||
|
failed.clear();
|
||||||
|
// Update batch: remove compacted regions and add new ones
|
||||||
|
for (RegionInfo ri : compacted) {
|
||||||
|
toCompact.remove(ri);
|
||||||
|
if (regions.size() > 0) {
|
||||||
|
RegionInfo region = regions.remove(0);
|
||||||
|
toCompact.add(region);
|
||||||
|
startCompaction(admin, htd.getTableName(), region, hcd.getName());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
compacted.clear();
|
||||||
|
|
||||||
|
LOG.debug(
|
||||||
|
"Table={} cf={}. Wait for 10 sec, toCompact size={} regions left={}"
|
||||||
|
+ " compacted so far={}", htd.getTableName(), hcd.getNameAsString(), toCompact.size(),
|
||||||
|
regions.size(), totalCompacted);
|
||||||
|
Thread.sleep(10000);
|
||||||
|
}
|
||||||
|
LOG.info("Finished major MOB compacting table={}. cf={}", htd.getTableName(),
|
||||||
|
hcd.getNameAsString());
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
private void startCompaction(Admin admin, TableName table, RegionInfo region, byte[] cf)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
|
||||||
|
LOG.info("Started major compaction: table={} cf={} region={}", table,
|
||||||
|
Bytes.toString(cf), region.getRegionNameAsString());
|
||||||
|
admin.majorCompactRegion(region.getRegionName(), cf);
|
||||||
|
// Wait until it really starts
|
||||||
|
// but with finite timeout
|
||||||
|
long waitTime = 300000; // 5 min
|
||||||
|
long startTime = EnvironmentEdgeManager.currentTime();
|
||||||
|
while (admin.getCompactionStateForRegion(region.getRegionName()) == CompactionState.NONE) {
|
||||||
|
// Is 1 second too aggressive?
|
||||||
|
Thread.sleep(1000);
|
||||||
|
if (EnvironmentEdgeManager.currentTime() - startTime > waitTime) {
|
||||||
|
LOG.warn("Waited for {} ms to start major MOB compaction on table={} cf={} region={}."+
|
||||||
|
" Stopped waiting for request confirmation. This is not an ERROR, continue next region."
|
||||||
|
, waitTime, table.getNameAsString(), Bytes.toString(cf),region.getRegionNameAsString());
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -23,8 +23,8 @@ import org.apache.hadoop.hbase.util.MD5Hash;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The mob file name.
|
* The mob file name.
|
||||||
* It consists of a md5 of a start key, a date and an uuid.
|
* It consists of a md5 of a start key, a date, uuid and encoded region name.
|
||||||
* It looks like md5(start) + date + uuid.
|
* It looks like md5(start) + date + uuid+ "_" + encoded region name.
|
||||||
* <ol>
|
* <ol>
|
||||||
* <li>characters 0-31: md5 hex string of a start key. Since the length of the start key is not
|
* <li>characters 0-31: md5 hex string of a start key. Since the length of the start key is not
|
||||||
* fixed, have to use the md5 instead which has a fix length.</li>
|
* fixed, have to use the md5 instead which has a fix length.</li>
|
||||||
|
@ -40,15 +40,18 @@ import org.apache.hadoop.hbase.util.MD5Hash;
|
||||||
* TTL easily. If this timestamp is older than the TTL, it's regarded as expired.
|
* TTL easily. If this timestamp is older than the TTL, it's regarded as expired.
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public final class MobFileName {
|
public final class MobFileName{
|
||||||
private final String date;
|
private final String date;
|
||||||
private final String startKey;
|
private final String startKey;
|
||||||
private final String uuid;
|
private final String uuid;
|
||||||
private final String fileName;
|
private final String fileName;
|
||||||
|
// Name of a region this MOB file belongs to
|
||||||
|
private final String regionName;
|
||||||
|
|
||||||
private static final int STARTKEY_END_INDEX = 32;
|
private static final int STARTKEY_END_INDEX = 32;
|
||||||
private static final int DATE_END_INDEX = 40;
|
private static final int DATE_END_INDEX = 40;
|
||||||
private static final int UUID_END_INDEX = 72;
|
private static final int UUID_END_INDEX = 72;
|
||||||
|
public static final String REGION_SEP = "_";
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param startKey
|
* @param startKey
|
||||||
|
@ -57,12 +60,14 @@ public final class MobFileName {
|
||||||
* The string of the latest timestamp of cells in this file, the format is yyyymmdd.
|
* The string of the latest timestamp of cells in this file, the format is yyyymmdd.
|
||||||
* @param uuid
|
* @param uuid
|
||||||
* The uuid
|
* The uuid
|
||||||
|
* @param regionName name of a region, where this file was created during flush or compaction.
|
||||||
*/
|
*/
|
||||||
private MobFileName(byte[] startKey, String date, String uuid) {
|
private MobFileName(byte[] startKey, String date, String uuid, String regionName) {
|
||||||
this.startKey = MD5Hash.getMD5AsHex(startKey, 0, startKey.length);
|
this.startKey = MD5Hash.getMD5AsHex(startKey, 0, startKey.length);
|
||||||
this.uuid = uuid;
|
this.uuid = uuid;
|
||||||
this.date = date;
|
this.date = date;
|
||||||
this.fileName = this.startKey + this.date + this.uuid;
|
this.regionName = regionName;
|
||||||
|
this.fileName = this.startKey + this.date + this.uuid + REGION_SEP + this.regionName;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -72,12 +77,14 @@ public final class MobFileName {
|
||||||
* The string of the latest timestamp of cells in this file, the format is yyyymmdd.
|
* The string of the latest timestamp of cells in this file, the format is yyyymmdd.
|
||||||
* @param uuid
|
* @param uuid
|
||||||
* The uuid
|
* The uuid
|
||||||
|
* @param regionName name of a region, where this file was created during flush or compaction.
|
||||||
*/
|
*/
|
||||||
private MobFileName(String startKey, String date, String uuid) {
|
private MobFileName(String startKey, String date, String uuid, String regionName) {
|
||||||
this.startKey = startKey;
|
this.startKey = startKey;
|
||||||
this.uuid = uuid;
|
this.uuid = uuid;
|
||||||
this.date = date;
|
this.date = date;
|
||||||
this.fileName = this.startKey + this.date + this.uuid;
|
this.regionName = regionName;
|
||||||
|
this.fileName = this.startKey + this.date + this.uuid + REGION_SEP + this.regionName;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -88,10 +95,11 @@ public final class MobFileName {
|
||||||
* @param date
|
* @param date
|
||||||
* The string of the latest timestamp of cells in this file, the format is yyyymmdd.
|
* The string of the latest timestamp of cells in this file, the format is yyyymmdd.
|
||||||
* @param uuid The uuid.
|
* @param uuid The uuid.
|
||||||
|
* @param regionName name of a region, where this file was created during flush or compaction.
|
||||||
* @return An instance of a MobFileName.
|
* @return An instance of a MobFileName.
|
||||||
*/
|
*/
|
||||||
public static MobFileName create(byte[] startKey, String date, String uuid) {
|
public static MobFileName create(byte[] startKey, String date, String uuid, String regionName) {
|
||||||
return new MobFileName(startKey, date, uuid);
|
return new MobFileName(startKey, date, uuid, regionName);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -102,10 +110,11 @@ public final class MobFileName {
|
||||||
* @param date
|
* @param date
|
||||||
* The string of the latest timestamp of cells in this file, the format is yyyymmdd.
|
* The string of the latest timestamp of cells in this file, the format is yyyymmdd.
|
||||||
* @param uuid The uuid.
|
* @param uuid The uuid.
|
||||||
|
* @param regionName name of a region, where this file was created during flush or compaction.
|
||||||
* @return An instance of a MobFileName.
|
* @return An instance of a MobFileName.
|
||||||
*/
|
*/
|
||||||
public static MobFileName create(String startKey, String date, String uuid) {
|
public static MobFileName create(String startKey, String date, String uuid, String regionName) {
|
||||||
return new MobFileName(startKey, date, uuid);
|
return new MobFileName(startKey, date, uuid, regionName);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -115,11 +124,17 @@ public final class MobFileName {
|
||||||
*/
|
*/
|
||||||
public static MobFileName create(String fileName) {
|
public static MobFileName create(String fileName) {
|
||||||
// The format of a file name is md5HexString(0-31bytes) + date(32-39bytes) + UUID
|
// The format of a file name is md5HexString(0-31bytes) + date(32-39bytes) + UUID
|
||||||
|
// + "_" + region
|
||||||
// The date format is yyyyMMdd
|
// The date format is yyyyMMdd
|
||||||
String startKey = fileName.substring(0, STARTKEY_END_INDEX);
|
String startKey = fileName.substring(0, STARTKEY_END_INDEX);
|
||||||
String date = fileName.substring(STARTKEY_END_INDEX, DATE_END_INDEX);
|
String date = fileName.substring(STARTKEY_END_INDEX, DATE_END_INDEX);
|
||||||
String uuid = fileName.substring(DATE_END_INDEX, UUID_END_INDEX);
|
String uuid = fileName.substring(DATE_END_INDEX, UUID_END_INDEX);
|
||||||
return new MobFileName(startKey, date, uuid);
|
String regionName = fileName.substring(UUID_END_INDEX+1);
|
||||||
|
return new MobFileName(startKey, date, uuid, regionName);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static boolean isOldMobFileName(String name) {
|
||||||
|
return name.indexOf(REGION_SEP) < 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -148,6 +163,13 @@ public final class MobFileName {
|
||||||
return startKey;
|
return startKey;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets region name
|
||||||
|
* @return name of a region, where this file was created during flush or compaction.
|
||||||
|
*/
|
||||||
|
public String getRegionName() {
|
||||||
|
return regionName;
|
||||||
|
}
|
||||||
/**
|
/**
|
||||||
* Gets the date string. Its format is yyyymmdd.
|
* Gets the date string. Its format is yyyymmdd.
|
||||||
* @return The date string.
|
* @return The date string.
|
||||||
|
|
|
@ -22,6 +22,7 @@ import java.io.IOException;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
|
import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
|
||||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||||
|
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -29,7 +30,7 @@ import org.apache.yetus.audience.InterfaceAudience;
|
||||||
*/
|
*/
|
||||||
@InterfaceAudience.Private
|
@InterfaceAudience.Private
|
||||||
public class MobStoreEngine extends DefaultStoreEngine {
|
public class MobStoreEngine extends DefaultStoreEngine {
|
||||||
|
public final static String MOB_COMPACTOR_CLASS_KEY = "hbase.hstore.mobengine.compactor.class";
|
||||||
@Override
|
@Override
|
||||||
protected void createStoreFlusher(Configuration conf, HStore store) throws IOException {
|
protected void createStoreFlusher(Configuration conf, HStore store) throws IOException {
|
||||||
// When using MOB, we use DefaultMobStoreFlusher always
|
// When using MOB, we use DefaultMobStoreFlusher always
|
||||||
|
@ -43,6 +44,13 @@ public class MobStoreEngine extends DefaultStoreEngine {
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
protected void createCompactor(Configuration conf, HStore store) throws IOException {
|
protected void createCompactor(Configuration conf, HStore store) throws IOException {
|
||||||
compactor = new DefaultMobStoreCompactor(conf, store);
|
String className =
|
||||||
|
conf.get(MOB_COMPACTOR_CLASS_KEY, DefaultMobStoreCompactor.class.getName());
|
||||||
|
try {
|
||||||
|
compactor = ReflectionUtils.instantiateWithCustomCtor(className,
|
||||||
|
new Class[] { Configuration.class, HStore.class }, new Object[] { conf, store });
|
||||||
|
} catch (RuntimeException e) {
|
||||||
|
throw new IOException("Unable to load configured compactor '" + className + "'", e);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,16 +25,14 @@ import java.text.SimpleDateFormat;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Calendar;
|
import java.util.Calendar;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.Date;
|
import java.util.Date;
|
||||||
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
import java.util.Set;
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
import java.util.concurrent.ExecutorService;
|
|
||||||
import java.util.concurrent.RejectedExecutionException;
|
|
||||||
import java.util.concurrent.RejectedExecutionHandler;
|
|
||||||
import java.util.concurrent.SynchronousQueue;
|
|
||||||
import java.util.concurrent.ThreadPoolExecutor;
|
|
||||||
import java.util.concurrent.TimeUnit;
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
@ -48,7 +46,6 @@ import org.apache.hadoop.hbase.TagType;
|
||||||
import org.apache.hadoop.hbase.TagUtil;
|
import org.apache.hadoop.hbase.TagUtil;
|
||||||
import org.apache.hadoop.hbase.backup.HFileArchiver;
|
import org.apache.hadoop.hbase.backup.HFileArchiver;
|
||||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||||
import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy;
|
|
||||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||||
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
|
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
|
||||||
import org.apache.hadoop.hbase.client.Scan;
|
import org.apache.hadoop.hbase.client.Scan;
|
||||||
|
@ -60,10 +57,6 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||||
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
|
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
|
||||||
import org.apache.hadoop.hbase.master.locking.LockManager;
|
|
||||||
import org.apache.hadoop.hbase.mob.compactions.MobCompactor;
|
|
||||||
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId;
|
|
||||||
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactor;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||||
import org.apache.hadoop.hbase.regionserver.HStoreFile;
|
import org.apache.hadoop.hbase.regionserver.HStoreFile;
|
||||||
|
@ -72,8 +65,6 @@ import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.ChecksumType;
|
import org.apache.hadoop.hbase.util.ChecksumType;
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
|
||||||
import org.apache.hadoop.hbase.util.Threads;
|
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
@ -85,23 +76,15 @@ import org.slf4j.LoggerFactory;
|
||||||
public final class MobUtils {
|
public final class MobUtils {
|
||||||
|
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(MobUtils.class);
|
private static final Logger LOG = LoggerFactory.getLogger(MobUtils.class);
|
||||||
private final static long WEEKLY_THRESHOLD_MULTIPLIER = 7;
|
public static final String SEP = "_";
|
||||||
private final static long MONTHLY_THRESHOLD_MULTIPLIER = 4 * WEEKLY_THRESHOLD_MULTIPLIER;
|
|
||||||
|
|
||||||
private static final ThreadLocal<SimpleDateFormat> LOCAL_FORMAT =
|
private static final ThreadLocal<SimpleDateFormat> LOCAL_FORMAT =
|
||||||
new ThreadLocal<SimpleDateFormat>() {
|
new ThreadLocal<SimpleDateFormat>() {
|
||||||
@Override
|
@Override
|
||||||
protected SimpleDateFormat initialValue() {
|
protected SimpleDateFormat initialValue() {
|
||||||
return new SimpleDateFormat("yyyyMMdd");
|
return new SimpleDateFormat("yyyyMMdd");
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
private static final byte[] REF_DELETE_MARKER_TAG_BYTES;
|
|
||||||
static {
|
|
||||||
List<Tag> tags = new ArrayList<>();
|
|
||||||
tags.add(MobConstants.MOB_REF_TAG);
|
|
||||||
REF_DELETE_MARKER_TAG_BYTES = TagUtil.fromList(tags);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Private constructor to keep this class from being instantiated.
|
* Private constructor to keep this class from being instantiated.
|
||||||
|
@ -127,45 +110,6 @@ public final class MobUtils {
|
||||||
return LOCAL_FORMAT.get().parse(dateString);
|
return LOCAL_FORMAT.get().parse(dateString);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Get the first day of the input date's month
|
|
||||||
* @param calendar Calendar object
|
|
||||||
* @param date The date to find out its first day of that month
|
|
||||||
* @return The first day in the month
|
|
||||||
*/
|
|
||||||
public static Date getFirstDayOfMonth(final Calendar calendar, final Date date) {
|
|
||||||
|
|
||||||
calendar.setTime(date);
|
|
||||||
calendar.set(Calendar.HOUR_OF_DAY, 0);
|
|
||||||
calendar.set(Calendar.MINUTE, 0);
|
|
||||||
calendar.set(Calendar.SECOND, 0);
|
|
||||||
calendar.set(Calendar.MILLISECOND, 0);
|
|
||||||
calendar.set(Calendar.DAY_OF_MONTH, 1);
|
|
||||||
|
|
||||||
Date firstDayInMonth = calendar.getTime();
|
|
||||||
return firstDayInMonth;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Get the first day of the input date's week
|
|
||||||
* @param calendar Calendar object
|
|
||||||
* @param date The date to find out its first day of that week
|
|
||||||
* @return The first day in the week
|
|
||||||
*/
|
|
||||||
public static Date getFirstDayOfWeek(final Calendar calendar, final Date date) {
|
|
||||||
|
|
||||||
calendar.setTime(date);
|
|
||||||
calendar.set(Calendar.HOUR_OF_DAY, 0);
|
|
||||||
calendar.set(Calendar.MINUTE, 0);
|
|
||||||
calendar.set(Calendar.SECOND, 0);
|
|
||||||
calendar.set(Calendar.MILLISECOND, 0);
|
|
||||||
calendar.setFirstDayOfWeek(Calendar.MONDAY);
|
|
||||||
calendar.set(Calendar.DAY_OF_WEEK, Calendar.MONDAY);
|
|
||||||
|
|
||||||
Date firstDayInWeek = calendar.getTime();
|
|
||||||
return firstDayInWeek;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Whether the current cell is a mob reference cell.
|
* Whether the current cell is a mob reference cell.
|
||||||
* @param cell The current cell.
|
* @param cell The current cell.
|
||||||
|
@ -213,11 +157,10 @@ public final class MobUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Indicates whether it's a raw scan.
|
* Indicates whether it's a raw scan. The information is set in the attribute "hbase.mob.scan.raw"
|
||||||
* The information is set in the attribute "hbase.mob.scan.raw" of scan.
|
* of scan. For a mob cell, in a normal scan the scanners retrieves the mob cell from the mob
|
||||||
* For a mob cell, in a normal scan the scanners retrieves the mob cell from the mob file.
|
* file. In a raw scan, the scanner directly returns cell in HBase without retrieve the one in the
|
||||||
* In a raw scan, the scanner directly returns cell in HBase without retrieve the one in
|
* mob file.
|
||||||
* the mob file.
|
|
||||||
* @param scan The current scan.
|
* @param scan The current scan.
|
||||||
* @return True if it's a raw scan.
|
* @return True if it's a raw scan.
|
||||||
*/
|
*/
|
||||||
|
@ -231,9 +174,9 @@ public final class MobUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Indicates whether it's a reference only scan.
|
* Indicates whether it's a reference only scan. The information is set in the attribute
|
||||||
* The information is set in the attribute "hbase.mob.scan.ref.only" of scan.
|
* "hbase.mob.scan.ref.only" of scan. If it's a ref only scan, only the cells with ref tag are
|
||||||
* If it's a ref only scan, only the cells with ref tag are returned.
|
* returned.
|
||||||
* @param scan The current scan.
|
* @param scan The current scan.
|
||||||
* @return True if it's a ref only scan.
|
* @return True if it's a ref only scan.
|
||||||
*/
|
*/
|
||||||
|
@ -247,8 +190,8 @@ public final class MobUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Indicates whether the scan contains the information of caching blocks.
|
* Indicates whether the scan contains the information of caching blocks. The information is set
|
||||||
* The information is set in the attribute "hbase.mob.cache.blocks" of scan.
|
* in the attribute "hbase.mob.cache.blocks" of scan.
|
||||||
* @param scan The current scan.
|
* @param scan The current scan.
|
||||||
* @return True when the Scan attribute specifies to cache the MOB blocks.
|
* @return True when the Scan attribute specifies to cache the MOB blocks.
|
||||||
*/
|
*/
|
||||||
|
@ -263,22 +206,17 @@ public final class MobUtils {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Sets the attribute of caching blocks in the scan.
|
* Sets the attribute of caching blocks in the scan.
|
||||||
*
|
* @param scan The current scan.
|
||||||
* @param scan
|
* @param cacheBlocks True, set the attribute of caching blocks into the scan, the scanner with
|
||||||
* The current scan.
|
* this scan caches blocks. False, the scanner doesn't cache blocks for this scan.
|
||||||
* @param 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 setCacheMobBlocks(Scan scan, boolean cacheBlocks) {
|
public static void setCacheMobBlocks(Scan scan, boolean cacheBlocks) {
|
||||||
scan.setAttribute(MobConstants.MOB_CACHE_BLOCKS, Bytes.toBytes(cacheBlocks));
|
scan.setAttribute(MobConstants.MOB_CACHE_BLOCKS, Bytes.toBytes(cacheBlocks));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Cleans the expired mob files.
|
* Cleans the expired mob files. Cleans the files whose creation date is older than (current -
|
||||||
* Cleans the files whose creation date is older than (current - columnFamily.ttl), and
|
* columnFamily.ttl), and the minVersions of that column family is 0.
|
||||||
* the minVersions of that column family is 0.
|
|
||||||
* @param fs The current file system.
|
* @param fs The current file system.
|
||||||
* @param conf The current configuration.
|
* @param conf The current configuration.
|
||||||
* @param tableName The current table name.
|
* @param tableName The current table name.
|
||||||
|
@ -330,11 +268,11 @@ public final class MobUtils {
|
||||||
Date fileDate = parseDate(MobFileName.getDateFromName(fileName));
|
Date fileDate = parseDate(MobFileName.getDateFromName(fileName));
|
||||||
|
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug("Checking file " + fileName);
|
LOG.debug("Checking file {}", fileName);
|
||||||
}
|
}
|
||||||
if (fileDate.getTime() < expireDate.getTime()) {
|
if (fileDate.getTime() < expireDate.getTime()) {
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug(fileName + " is an expired file");
|
LOG.debug("{} is an expired file", fileName);
|
||||||
}
|
}
|
||||||
filesToClean
|
filesToClean
|
||||||
.add(new HStoreFile(fs, file.getPath(), conf, cacheConfig, BloomType.NONE, true));
|
.add(new HStoreFile(fs, file.getPath(), conf, cacheConfig, BloomType.NONE, true));
|
||||||
|
@ -345,19 +283,17 @@ public final class MobUtils {
|
||||||
}
|
}
|
||||||
if (!filesToClean.isEmpty()) {
|
if (!filesToClean.isEmpty()) {
|
||||||
try {
|
try {
|
||||||
removeMobFiles(conf, fs, tableName, mobTableDir, columnDescriptor.getName(),
|
removeMobFiles(conf, fs, tableName, mobTableDir, columnDescriptor.getName(), filesToClean);
|
||||||
filesToClean);
|
|
||||||
deletedFileCount = filesToClean.size();
|
deletedFileCount = filesToClean.size();
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
LOG.error("Failed to delete the mob files " + filesToClean, e);
|
LOG.error("Failed to delete the mob files " + filesToClean, e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
LOG.info(deletedFileCount + " expired mob files are deleted");
|
LOG.info("{} expired mob files are deleted", deletedFileCount);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Gets the root dir of the mob files.
|
* Gets the root dir of the mob files. It's {HBASE_DIR}/mobdir.
|
||||||
* It's {HBASE_DIR}/mobdir.
|
|
||||||
* @param conf The current configuration.
|
* @param conf The current configuration.
|
||||||
* @return the root dir of the mob file.
|
* @return the root dir of the mob file.
|
||||||
*/
|
*/
|
||||||
|
@ -367,8 +303,7 @@ public final class MobUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Gets the root dir of the mob files under the qualified HBase root dir.
|
* Gets the root dir of the mob files under the qualified HBase root dir. It's {rootDir}/mobdir.
|
||||||
* It's {rootDir}/mobdir.
|
|
||||||
* @param rootDir The qualified path of HBase root directory.
|
* @param rootDir The qualified path of HBase root directory.
|
||||||
* @return The root dir of the mob file.
|
* @return The root dir of the mob file.
|
||||||
*/
|
*/
|
||||||
|
@ -389,8 +324,8 @@ public final class MobUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Gets the table dir of the mob files under the qualified HBase root dir.
|
* Gets the table dir of the mob files under the qualified HBase root dir. It's
|
||||||
* It's {rootDir}/mobdir/data/${namespace}/${tableName}
|
* {rootDir}/mobdir/data/${namespace}/${tableName}
|
||||||
* @param rootDir The qualified path of HBase root directory.
|
* @param rootDir The qualified path of HBase root directory.
|
||||||
* @param tableName The name of table.
|
* @param tableName The name of table.
|
||||||
* @return The table dir of the mob file.
|
* @return The table dir of the mob file.
|
||||||
|
@ -400,8 +335,8 @@ public final class MobUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Gets the region dir of the mob files.
|
* Gets the region dir of the mob files. It's
|
||||||
* It's {HBASE_DIR}/mobdir/data/{namespace}/{tableName}/{regionEncodedName}.
|
* {HBASE_DIR}/mobdir/data/{namespace}/{tableName}/{regionEncodedName}.
|
||||||
* @param conf The current configuration.
|
* @param conf The current configuration.
|
||||||
* @param tableName The current table name.
|
* @param tableName The current table name.
|
||||||
* @return The region dir of the mob files.
|
* @return The region dir of the mob files.
|
||||||
|
@ -411,8 +346,8 @@ public final class MobUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Gets the region dir of the mob files under the specified root dir.
|
* Gets the region dir of the mob files under the specified root dir. It's
|
||||||
* It's {rootDir}/mobdir/data/{namespace}/{tableName}/{regionEncodedName}.
|
* {rootDir}/mobdir/data/{namespace}/{tableName}/{regionEncodedName}.
|
||||||
* @param rootDir The qualified path of HBase root directory.
|
* @param rootDir The qualified path of HBase root directory.
|
||||||
* @param tableName The current table name.
|
* @param tableName The current table name.
|
||||||
* @return The region dir of the mob files.
|
* @return The region dir of the mob files.
|
||||||
|
@ -424,8 +359,8 @@ public final class MobUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Gets the family dir of the mob files.
|
* Gets the family dir of the mob files. It's
|
||||||
* It's {HBASE_DIR}/mobdir/{namespace}/{tableName}/{regionEncodedName}/{columnFamilyName}.
|
* {HBASE_DIR}/mobdir/{namespace}/{tableName}/{regionEncodedName}/{columnFamilyName}.
|
||||||
* @param conf The current configuration.
|
* @param conf The current configuration.
|
||||||
* @param tableName The current table name.
|
* @param tableName The current table name.
|
||||||
* @param familyName The current family name.
|
* @param familyName The current family name.
|
||||||
|
@ -436,8 +371,8 @@ public final class MobUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Gets the family dir of the mob files.
|
* Gets the family dir of the mob files. It's
|
||||||
* It's {HBASE_DIR}/mobdir/{namespace}/{tableName}/{regionEncodedName}/{columnFamilyName}.
|
* {HBASE_DIR}/mobdir/{namespace}/{tableName}/{regionEncodedName}/{columnFamilyName}.
|
||||||
* @param regionPath The path of mob region which is a dummy one.
|
* @param regionPath The path of mob region which is a dummy one.
|
||||||
* @param familyName The current family name.
|
* @param familyName The current family name.
|
||||||
* @return The family dir of the mob files.
|
* @return The family dir of the mob files.
|
||||||
|
@ -447,19 +382,14 @@ public final class MobUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Gets the RegionInfo of the mob files.
|
* Gets the RegionInfo of the mob files. This is a dummy region. The mob files are not saved in a
|
||||||
* This is a dummy region. The mob files are not saved in a region in HBase.
|
* region in HBase. This is only used in mob snapshot. It's internally used only.
|
||||||
* This is only used in mob snapshot. It's internally used only.
|
|
||||||
* @param tableName
|
* @param tableName
|
||||||
* @return A dummy mob region info.
|
* @return A dummy mob region info.
|
||||||
*/
|
*/
|
||||||
public static RegionInfo getMobRegionInfo(TableName tableName) {
|
public static RegionInfo getMobRegionInfo(TableName tableName) {
|
||||||
return RegionInfoBuilder.newBuilder(tableName)
|
return RegionInfoBuilder.newBuilder(tableName).setStartKey(MobConstants.MOB_REGION_NAME_BYTES)
|
||||||
.setStartKey(MobConstants.MOB_REGION_NAME_BYTES)
|
.setEndKey(HConstants.EMPTY_END_ROW).setSplit(false).setRegionId(0).build();
|
||||||
.setEndKey(HConstants.EMPTY_END_ROW)
|
|
||||||
.setSplit(false)
|
|
||||||
.setRegionId(0)
|
|
||||||
.build();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -468,8 +398,9 @@ public final class MobUtils {
|
||||||
* @return If true, the current RegionInfo is a mob one.
|
* @return If true, the current RegionInfo is a mob one.
|
||||||
*/
|
*/
|
||||||
public static boolean isMobRegionInfo(RegionInfo regionInfo) {
|
public static boolean isMobRegionInfo(RegionInfo regionInfo) {
|
||||||
return regionInfo == null ? false : getMobRegionInfo(regionInfo.getTable()).getEncodedName()
|
return regionInfo == null ? false
|
||||||
.equals(regionInfo.getEncodedName());
|
: getMobRegionInfo(regionInfo.getTable()).getEncodedName()
|
||||||
|
.equals(regionInfo.getEncodedName());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -482,16 +413,6 @@ public final class MobUtils {
|
||||||
return Bytes.equals(regionName, getMobRegionInfo(tableName).getRegionName());
|
return Bytes.equals(regionName, getMobRegionInfo(tableName).getRegionName());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Gets the working directory of the mob compaction.
|
|
||||||
* @param root The root directory of the mob compaction.
|
|
||||||
* @param jobName The current job name.
|
|
||||||
* @return The directory of the mob compaction for the current job.
|
|
||||||
*/
|
|
||||||
public static Path getCompactionWorkingPath(Path root, String jobName) {
|
|
||||||
return new Path(root, jobName);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Archives the mob files.
|
* Archives the mob files.
|
||||||
* @param conf The current configuration.
|
* @param conf The current configuration.
|
||||||
|
@ -504,16 +425,16 @@ public final class MobUtils {
|
||||||
public static void removeMobFiles(Configuration conf, FileSystem fs, TableName tableName,
|
public static void removeMobFiles(Configuration conf, FileSystem fs, TableName tableName,
|
||||||
Path tableDir, byte[] family, Collection<HStoreFile> storeFiles) throws IOException {
|
Path tableDir, byte[] family, Collection<HStoreFile> storeFiles) throws IOException {
|
||||||
HFileArchiver.archiveStoreFiles(conf, fs, getMobRegionInfo(tableName), tableDir, family,
|
HFileArchiver.archiveStoreFiles(conf, fs, getMobRegionInfo(tableName), tableDir, family,
|
||||||
storeFiles);
|
storeFiles);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a mob reference KeyValue.
|
* Creates a mob reference KeyValue. The value of the mob reference KeyValue is mobCellValueSize +
|
||||||
* The value of the mob reference KeyValue is mobCellValueSize + mobFileName.
|
* mobFileName.
|
||||||
* @param cell The original Cell.
|
* @param cell The original Cell.
|
||||||
* @param fileName The mob file name where the mob reference KeyValue is written.
|
* @param fileName The mob file name where the mob reference KeyValue is written.
|
||||||
* @param tableNameTag The tag of the current table name. It's very important in
|
* @param tableNameTag The tag of the current table name. It's very important in cloning the
|
||||||
* cloning the snapshot.
|
* snapshot.
|
||||||
* @return The mob reference KeyValue.
|
* @return The mob reference KeyValue.
|
||||||
*/
|
*/
|
||||||
public static Cell createMobRefCell(Cell cell, byte[] fileName, Tag tableNameTag) {
|
public static Cell createMobRefCell(Cell cell, byte[] fileName, Tag tableNameTag) {
|
||||||
|
@ -554,89 +475,14 @@ public final class MobUtils {
|
||||||
public static StoreFileWriter createWriter(Configuration conf, FileSystem fs,
|
public static StoreFileWriter createWriter(Configuration conf, FileSystem fs,
|
||||||
ColumnFamilyDescriptor family, String date, Path basePath, long maxKeyCount,
|
ColumnFamilyDescriptor family, String date, Path basePath, long maxKeyCount,
|
||||||
Compression.Algorithm compression, String startKey, CacheConfig cacheConfig,
|
Compression.Algorithm compression, String startKey, CacheConfig cacheConfig,
|
||||||
Encryption.Context cryptoContext, boolean isCompaction)
|
Encryption.Context cryptoContext, boolean isCompaction, String regionName)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
MobFileName mobFileName = MobFileName.create(startKey, date,
|
MobFileName mobFileName = MobFileName.create(startKey, date,
|
||||||
UUID.randomUUID().toString().replaceAll("-", ""));
|
UUID.randomUUID().toString().replaceAll("-", ""), regionName);
|
||||||
return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression,
|
|
||||||
cacheConfig, cryptoContext, isCompaction);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Creates a writer for the ref file in temp directory.
|
|
||||||
* @param conf The current configuration.
|
|
||||||
* @param fs The current file system.
|
|
||||||
* @param family The descriptor of the current column family.
|
|
||||||
* @param basePath The basic path for a temp directory.
|
|
||||||
* @param maxKeyCount The key count.
|
|
||||||
* @param cacheConfig The current cache config.
|
|
||||||
* @param cryptoContext The encryption context.
|
|
||||||
* @param isCompaction If the writer is used in compaction.
|
|
||||||
* @return The writer for the mob file.
|
|
||||||
*/
|
|
||||||
public static StoreFileWriter createRefFileWriter(Configuration conf, FileSystem fs,
|
|
||||||
ColumnFamilyDescriptor family, Path basePath, long maxKeyCount, CacheConfig cacheConfig,
|
|
||||||
Encryption.Context cryptoContext, boolean isCompaction)
|
|
||||||
throws IOException {
|
|
||||||
return createWriter(conf, fs, family,
|
|
||||||
new Path(basePath, UUID.randomUUID().toString().replaceAll("-", "")), maxKeyCount,
|
|
||||||
family.getCompactionCompressionType(), cacheConfig, cryptoContext,
|
|
||||||
HStore.getChecksumType(conf), HStore.getBytesPerChecksum(conf), family.getBlocksize(),
|
|
||||||
family.getBloomFilterType(), isCompaction);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Creates a writer for the mob file in temp directory.
|
|
||||||
* @param conf The current configuration.
|
|
||||||
* @param fs The current file system.
|
|
||||||
* @param family The descriptor of the current column family.
|
|
||||||
* @param date The date string, its format is yyyymmmdd.
|
|
||||||
* @param basePath The basic path for a temp directory.
|
|
||||||
* @param maxKeyCount The key count.
|
|
||||||
* @param compression The compression algorithm.
|
|
||||||
* @param startKey The start key.
|
|
||||||
* @param cacheConfig The current cache config.
|
|
||||||
* @param cryptoContext The encryption context.
|
|
||||||
* @param isCompaction If the writer is used in compaction.
|
|
||||||
* @return The writer for the mob file.
|
|
||||||
*/
|
|
||||||
public static StoreFileWriter createWriter(Configuration conf, FileSystem fs,
|
|
||||||
ColumnFamilyDescriptor family, String date, Path basePath, long maxKeyCount,
|
|
||||||
Compression.Algorithm compression, byte[] startKey, CacheConfig cacheConfig,
|
|
||||||
Encryption.Context cryptoContext, boolean isCompaction)
|
|
||||||
throws IOException {
|
|
||||||
MobFileName mobFileName = MobFileName.create(startKey, date,
|
|
||||||
UUID.randomUUID().toString().replaceAll("-", ""));
|
|
||||||
return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression,
|
return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression,
|
||||||
cacheConfig, cryptoContext, isCompaction);
|
cacheConfig, cryptoContext, isCompaction);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Creates a writer for the del file in temp directory.
|
|
||||||
* @param conf The current configuration.
|
|
||||||
* @param fs The current file system.
|
|
||||||
* @param family The descriptor of the current column family.
|
|
||||||
* @param date The date string, its format is yyyymmmdd.
|
|
||||||
* @param basePath The basic path for a temp directory.
|
|
||||||
* @param maxKeyCount The key count.
|
|
||||||
* @param compression The compression algorithm.
|
|
||||||
* @param startKey The start key.
|
|
||||||
* @param cacheConfig The current cache config.
|
|
||||||
* @param cryptoContext The encryption context.
|
|
||||||
* @return The writer for the del file.
|
|
||||||
*/
|
|
||||||
public static StoreFileWriter createDelFileWriter(Configuration conf, FileSystem fs,
|
|
||||||
ColumnFamilyDescriptor family, String date, Path basePath, long maxKeyCount,
|
|
||||||
Compression.Algorithm compression, byte[] startKey, CacheConfig cacheConfig,
|
|
||||||
Encryption.Context cryptoContext)
|
|
||||||
throws IOException {
|
|
||||||
String suffix = UUID
|
|
||||||
.randomUUID().toString().replaceAll("-", "") + "_del";
|
|
||||||
MobFileName mobFileName = MobFileName.create(startKey, date, suffix);
|
|
||||||
return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression,
|
|
||||||
cacheConfig, cryptoContext, true);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a writer for the mob file in temp directory.
|
* Creates a writer for the mob file in temp directory.
|
||||||
* @param conf The current configuration.
|
* @param conf The current configuration.
|
||||||
|
@ -654,12 +500,10 @@ public final class MobUtils {
|
||||||
public static StoreFileWriter createWriter(Configuration conf, FileSystem fs,
|
public static StoreFileWriter createWriter(Configuration conf, FileSystem fs,
|
||||||
ColumnFamilyDescriptor family, MobFileName mobFileName, Path basePath, long maxKeyCount,
|
ColumnFamilyDescriptor family, MobFileName mobFileName, Path basePath, long maxKeyCount,
|
||||||
Compression.Algorithm compression, CacheConfig cacheConfig, Encryption.Context cryptoContext,
|
Compression.Algorithm compression, CacheConfig cacheConfig, Encryption.Context cryptoContext,
|
||||||
boolean isCompaction)
|
boolean isCompaction) throws IOException {
|
||||||
throws IOException {
|
return createWriter(conf, fs, family, new Path(basePath, mobFileName.getFileName()),
|
||||||
return createWriter(conf, fs, family,
|
maxKeyCount, compression, cacheConfig, cryptoContext, HStore.getChecksumType(conf),
|
||||||
new Path(basePath, mobFileName.getFileName()), maxKeyCount, compression, cacheConfig,
|
HStore.getBytesPerChecksum(conf), family.getBlocksize(), BloomType.NONE, isCompaction);
|
||||||
cryptoContext, HStore.getChecksumType(conf), HStore.getBytesPerChecksum(conf),
|
|
||||||
family.getBlocksize(), BloomType.NONE, isCompaction);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -680,10 +524,9 @@ public final class MobUtils {
|
||||||
* @return The writer for the mob file.
|
* @return The writer for the mob file.
|
||||||
*/
|
*/
|
||||||
public static StoreFileWriter createWriter(Configuration conf, FileSystem fs,
|
public static StoreFileWriter createWriter(Configuration conf, FileSystem fs,
|
||||||
ColumnFamilyDescriptor family, Path path, long maxKeyCount,
|
ColumnFamilyDescriptor family, Path path, long maxKeyCount, Compression.Algorithm compression,
|
||||||
Compression.Algorithm compression, CacheConfig cacheConfig, Encryption.Context cryptoContext,
|
CacheConfig cacheConfig, Encryption.Context cryptoContext, ChecksumType checksumType,
|
||||||
ChecksumType checksumType, int bytesPerChecksum, int blocksize, BloomType bloomType,
|
int bytesPerChecksum, int blocksize, BloomType bloomType, boolean isCompaction)
|
||||||
boolean isCompaction)
|
|
||||||
throws IOException {
|
throws IOException {
|
||||||
if (compression == null) {
|
if (compression == null) {
|
||||||
compression = HFile.DEFAULT_COMPRESSION_ALGORITHM;
|
compression = HFile.DEFAULT_COMPRESSION_ALGORITHM;
|
||||||
|
@ -696,13 +539,10 @@ public final class MobUtils {
|
||||||
writerCacheConf = cacheConfig;
|
writerCacheConf = cacheConfig;
|
||||||
}
|
}
|
||||||
HFileContext hFileContext = new HFileContextBuilder().withCompression(compression)
|
HFileContext hFileContext = new HFileContextBuilder().withCompression(compression)
|
||||||
.withIncludesMvcc(true).withIncludesTags(true)
|
.withIncludesMvcc(true).withIncludesTags(true).withCompressTags(family.isCompressTags())
|
||||||
.withCompressTags(family.isCompressTags())
|
.withChecksumType(checksumType).withBytesPerCheckSum(bytesPerChecksum)
|
||||||
.withChecksumType(checksumType)
|
.withBlockSize(blocksize).withHBaseCheckSum(true)
|
||||||
.withBytesPerCheckSum(bytesPerChecksum)
|
.withDataBlockEncoding(family.getDataBlockEncoding()).withEncryptionContext(cryptoContext)
|
||||||
.withBlockSize(blocksize)
|
|
||||||
.withHBaseCheckSum(true).withDataBlockEncoding(family.getDataBlockEncoding())
|
|
||||||
.withEncryptionContext(cryptoContext)
|
|
||||||
.withCreateTime(EnvironmentEdgeManager.currentTime()).build();
|
.withCreateTime(EnvironmentEdgeManager.currentTime()).build();
|
||||||
|
|
||||||
StoreFileWriter w = new StoreFileWriter.Builder(conf, writerCacheConf, fs)
|
StoreFileWriter w = new StoreFileWriter.Builder(conf, writerCacheConf, fs)
|
||||||
|
@ -712,62 +552,9 @@ public final class MobUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Commits the mob file.
|
* Indicates whether the current mob ref cell has a valid value. A mob ref cell has a mob
|
||||||
* @param conf The current configuration.
|
* reference tag. The value of a mob ref cell consists of two parts, real mob value length and mob
|
||||||
* @param fs The current file system.
|
* file name. The real mob value length takes 4 bytes. The remaining part is the mob file name.
|
||||||
* @param sourceFile The path where the mob file is saved.
|
|
||||||
* @param targetPath The directory path where the source file is renamed to.
|
|
||||||
* @param cacheConfig The current cache config.
|
|
||||||
* @return The target file path the source file is renamed to.
|
|
||||||
*/
|
|
||||||
public static Path commitFile(Configuration conf, FileSystem fs, final Path sourceFile,
|
|
||||||
Path targetPath, CacheConfig cacheConfig) throws IOException {
|
|
||||||
if (sourceFile == null) {
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
Path dstPath = new Path(targetPath, sourceFile.getName());
|
|
||||||
validateMobFile(conf, fs, sourceFile, cacheConfig, true);
|
|
||||||
String msg = "Renaming flushed file from " + sourceFile + " to " + dstPath;
|
|
||||||
LOG.info(msg);
|
|
||||||
Path parent = dstPath.getParent();
|
|
||||||
if (!fs.exists(parent)) {
|
|
||||||
fs.mkdirs(parent);
|
|
||||||
}
|
|
||||||
if (!fs.rename(sourceFile, dstPath)) {
|
|
||||||
throw new IOException("Failed rename of " + sourceFile + " to " + dstPath);
|
|
||||||
}
|
|
||||||
return dstPath;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Validates a mob file by opening and closing it.
|
|
||||||
* @param conf The current configuration.
|
|
||||||
* @param fs The current file system.
|
|
||||||
* @param path The path where the mob file is saved.
|
|
||||||
* @param cacheConfig The current cache config.
|
|
||||||
*/
|
|
||||||
private static void validateMobFile(Configuration conf, FileSystem fs, Path path,
|
|
||||||
CacheConfig cacheConfig, boolean primaryReplica) throws IOException {
|
|
||||||
HStoreFile storeFile = null;
|
|
||||||
try {
|
|
||||||
storeFile = new HStoreFile(fs, path, conf, cacheConfig, BloomType.NONE, primaryReplica);
|
|
||||||
storeFile.initReader();
|
|
||||||
} catch (IOException e) {
|
|
||||||
LOG.error("Failed to open mob file[" + path + "], keep it in temp directory.", e);
|
|
||||||
throw e;
|
|
||||||
} finally {
|
|
||||||
if (storeFile != null) {
|
|
||||||
storeFile.closeStoreFile(false);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Indicates whether the current mob ref cell has a valid value.
|
|
||||||
* A mob ref cell has a mob reference tag.
|
|
||||||
* The value of a mob ref cell consists of two parts, real mob value length and mob file name.
|
|
||||||
* The real mob value length takes 4 bytes.
|
|
||||||
* The remaining part is the mob file name.
|
|
||||||
* @param cell The mob ref cell.
|
* @param cell The mob ref cell.
|
||||||
* @return True if the cell has a valid value.
|
* @return True if the cell has a valid value.
|
||||||
*/
|
*/
|
||||||
|
@ -776,11 +563,9 @@ public final class MobUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Gets the mob value length from the mob ref cell.
|
* Gets the mob value length from the mob ref cell. A mob ref cell has a mob reference tag. The
|
||||||
* A mob ref cell has a mob reference tag.
|
* value of a mob ref cell consists of two parts, real mob value length and mob file name. The
|
||||||
* The value of a mob ref cell consists of two parts, real mob value length and mob file name.
|
* real mob value length takes 4 bytes. The remaining part is the mob file name.
|
||||||
* The real mob value length takes 4 bytes.
|
|
||||||
* The remaining part is the mob file name.
|
|
||||||
* @param cell The mob ref cell.
|
* @param cell The mob ref cell.
|
||||||
* @return The real mob value length.
|
* @return The real mob value length.
|
||||||
*/
|
*/
|
||||||
|
@ -789,98 +574,15 @@ public final class MobUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Gets the mob file name from the mob ref cell.
|
* Gets the mob file name from the mob ref cell. A mob ref cell has a mob reference tag. The value
|
||||||
* A mob ref cell has a mob reference tag.
|
* of a mob ref cell consists of two parts, real mob value length and mob file name. The real mob
|
||||||
* The value of a mob ref cell consists of two parts, real mob value length and mob file name.
|
* value length takes 4 bytes. The remaining part is the mob file name.
|
||||||
* The real mob value length takes 4 bytes.
|
|
||||||
* The remaining part is the mob file name.
|
|
||||||
* @param cell The mob ref cell.
|
* @param cell The mob ref cell.
|
||||||
* @return The mob file name.
|
* @return The mob file name.
|
||||||
*/
|
*/
|
||||||
public static String getMobFileName(Cell cell) {
|
public static String getMobFileName(Cell cell) {
|
||||||
return Bytes.toString(cell.getValueArray(), cell.getValueOffset() + Bytes.SIZEOF_INT,
|
return Bytes.toString(cell.getValueArray(), cell.getValueOffset() + Bytes.SIZEOF_INT,
|
||||||
cell.getValueLength() - Bytes.SIZEOF_INT);
|
cell.getValueLength() - Bytes.SIZEOF_INT);
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Gets the table name used in the table lock.
|
|
||||||
* The table lock name is a dummy one, it's not a table name. It's tableName + ".mobLock".
|
|
||||||
* @param tn The table name.
|
|
||||||
* @return The table name used in table lock.
|
|
||||||
*/
|
|
||||||
public static TableName getTableLockName(TableName tn) {
|
|
||||||
byte[] tableName = tn.getName();
|
|
||||||
return TableName.valueOf(Bytes.add(tableName, MobConstants.MOB_TABLE_LOCK_SUFFIX));
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Performs the mob compaction.
|
|
||||||
* @param conf the Configuration
|
|
||||||
* @param fs the file system
|
|
||||||
* @param tableName the table the compact
|
|
||||||
* @param hcd the column descriptor
|
|
||||||
* @param pool the thread pool
|
|
||||||
* @param allFiles Whether add all mob files into the compaction.
|
|
||||||
*/
|
|
||||||
public static void doMobCompaction(Configuration conf, FileSystem fs, TableName tableName,
|
|
||||||
ColumnFamilyDescriptor hcd, ExecutorService pool, boolean allFiles,
|
|
||||||
LockManager.MasterLock lock)
|
|
||||||
throws IOException {
|
|
||||||
String className = conf.get(MobConstants.MOB_COMPACTOR_CLASS_KEY,
|
|
||||||
PartitionedMobCompactor.class.getName());
|
|
||||||
// instantiate the mob compactor.
|
|
||||||
MobCompactor compactor = null;
|
|
||||||
try {
|
|
||||||
compactor = ReflectionUtils.instantiateWithCustomCtor(className, new Class[] {
|
|
||||||
Configuration.class, FileSystem.class, TableName.class, ColumnFamilyDescriptor.class,
|
|
||||||
ExecutorService.class }, new Object[] { conf, fs, tableName, hcd, pool });
|
|
||||||
} catch (Exception e) {
|
|
||||||
throw new IOException("Unable to load configured mob file compactor '" + className + "'", e);
|
|
||||||
}
|
|
||||||
// compact only for mob-enabled column.
|
|
||||||
// obtain a write table lock before performing compaction to avoid race condition
|
|
||||||
// with major compaction in mob-enabled column.
|
|
||||||
try {
|
|
||||||
lock.acquire();
|
|
||||||
LOG.info("start MOB compaction of files for table='{}', column='{}', allFiles={}, " +
|
|
||||||
"compactor='{}'", tableName, hcd.getNameAsString(), allFiles, compactor.getClass());
|
|
||||||
compactor.compact(allFiles);
|
|
||||||
} catch (Exception e) {
|
|
||||||
LOG.error("Failed to compact the mob files for the column " + hcd.getNameAsString()
|
|
||||||
+ " in the table " + tableName.getNameAsString(), e);
|
|
||||||
} finally {
|
|
||||||
LOG.info("end MOB compaction of files for table='{}', column='{}', allFiles={}, " +
|
|
||||||
"compactor='{}'", tableName, hcd.getNameAsString(), allFiles, compactor.getClass());
|
|
||||||
lock.release();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Creates a thread pool.
|
|
||||||
* @param conf the Configuration
|
|
||||||
* @return A thread pool.
|
|
||||||
*/
|
|
||||||
public static ExecutorService createMobCompactorThreadPool(Configuration conf) {
|
|
||||||
int maxThreads = conf.getInt(MobConstants.MOB_COMPACTION_THREADS_MAX,
|
|
||||||
MobConstants.DEFAULT_MOB_COMPACTION_THREADS_MAX);
|
|
||||||
if (maxThreads == 0) {
|
|
||||||
maxThreads = 1;
|
|
||||||
}
|
|
||||||
final SynchronousQueue<Runnable> queue = new SynchronousQueue<>();
|
|
||||||
ThreadPoolExecutor pool = new ThreadPoolExecutor(1, maxThreads, 60, TimeUnit.SECONDS, queue,
|
|
||||||
Threads.newDaemonThreadFactory("MobCompactor"), new RejectedExecutionHandler() {
|
|
||||||
@Override
|
|
||||||
public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) {
|
|
||||||
try {
|
|
||||||
// waiting for a thread to pick up instead of throwing exceptions.
|
|
||||||
queue.put(r);
|
|
||||||
} catch (InterruptedException e) {
|
|
||||||
throw new RejectedExecutionException(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
});
|
|
||||||
((ThreadPoolExecutor) pool).allowCoreThreadTimeOut(true);
|
|
||||||
return pool;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -899,14 +601,31 @@ public final class MobUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Indicates whether return null value when the mob file is missing or corrupt.
|
* Get list of Mob column families (if any exists)
|
||||||
* The information is set in the attribute "empty.value.on.mobcell.miss" of scan.
|
* @param htd table descriptor
|
||||||
|
* @return list of Mob column families
|
||||||
|
*/
|
||||||
|
public static List<ColumnFamilyDescriptor> getMobColumnFamilies(TableDescriptor htd) {
|
||||||
|
|
||||||
|
List<ColumnFamilyDescriptor> fams = new ArrayList<ColumnFamilyDescriptor>();
|
||||||
|
ColumnFamilyDescriptor[] hcds = htd.getColumnFamilies();
|
||||||
|
for (ColumnFamilyDescriptor hcd : hcds) {
|
||||||
|
if (hcd.isMobEnabled()) {
|
||||||
|
fams.add(hcd);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return fams;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Indicates whether return null value when the mob file is missing or corrupt. The information is
|
||||||
|
* set in the attribute "empty.value.on.mobcell.miss" of scan.
|
||||||
* @param scan The current scan.
|
* @param scan The current scan.
|
||||||
* @return True if the readEmptyValueOnMobCellMiss is enabled.
|
* @return True if the readEmptyValueOnMobCellMiss is enabled.
|
||||||
*/
|
*/
|
||||||
public static boolean isReadEmptyValueOnMobCellMiss(Scan scan) {
|
public static boolean isReadEmptyValueOnMobCellMiss(Scan scan) {
|
||||||
byte[] readEmptyValueOnMobCellMiss =
|
byte[] readEmptyValueOnMobCellMiss =
|
||||||
scan.getAttribute(MobConstants.EMPTY_VALUE_ON_MOBCELL_MISS);
|
scan.getAttribute(MobConstants.EMPTY_VALUE_ON_MOBCELL_MISS);
|
||||||
try {
|
try {
|
||||||
return readEmptyValueOnMobCellMiss != null && Bytes.toBoolean(readEmptyValueOnMobCellMiss);
|
return readEmptyValueOnMobCellMiss != null && Bytes.toBoolean(readEmptyValueOnMobCellMiss);
|
||||||
} catch (IllegalArgumentException e) {
|
} catch (IllegalArgumentException e) {
|
||||||
|
@ -914,15 +633,6 @@ public final class MobUtils {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Creates a mob ref delete marker.
|
|
||||||
* @param cell The current delete marker.
|
|
||||||
* @return A delete marker with the ref tag.
|
|
||||||
*/
|
|
||||||
public static Cell createMobRefDeleteMarker(Cell cell) {
|
|
||||||
return PrivateCellUtil.createCell(cell, TagUtil.concatTags(REF_DELETE_MARKER_TAG_BYTES, cell));
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Checks if the mob file is expired.
|
* Checks if the mob file is expired.
|
||||||
* @param column The descriptor of the current column family.
|
* @param column The descriptor of the current column family.
|
||||||
|
@ -955,85 +665,27 @@ public final class MobUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* fill out partition id based on compaction policy and date, threshold...
|
* Get list of referenced MOB files from a given collection of store files
|
||||||
* @param id Partition id to be filled out
|
* @param storeFiles store files
|
||||||
* @param firstDayOfCurrentMonth The first day in the current month
|
* @param mobDir MOB file directory
|
||||||
* @param firstDayOfCurrentWeek The first day in the current week
|
* @return list of MOB file paths
|
||||||
* @param dateStr Date string from the mob file
|
|
||||||
* @param policy Mob compaction policy
|
|
||||||
* @param calendar Calendar object
|
|
||||||
* @param threshold Mob compaciton threshold configured
|
|
||||||
* @return true if the file needs to be excluded from compaction
|
|
||||||
*/
|
*/
|
||||||
public static boolean fillPartitionId(final CompactionPartitionId id,
|
|
||||||
final Date firstDayOfCurrentMonth, final Date firstDayOfCurrentWeek, final String dateStr,
|
|
||||||
final MobCompactPartitionPolicy policy, final Calendar calendar, final long threshold) {
|
|
||||||
|
|
||||||
boolean skipCompcation = false;
|
public static List<Path> getReferencedMobFiles(Collection<HStoreFile> storeFiles, Path mobDir) {
|
||||||
id.setThreshold(threshold);
|
|
||||||
if (threshold <= 0) {
|
|
||||||
id.setDate(dateStr);
|
|
||||||
return skipCompcation;
|
|
||||||
}
|
|
||||||
|
|
||||||
long finalThreshold;
|
Set<String> mobSet = new HashSet<String>();
|
||||||
Date date;
|
for (HStoreFile sf : storeFiles) {
|
||||||
try {
|
byte[] value = sf.getMetadataValue(HStoreFile.MOB_FILE_REFS);
|
||||||
date = MobUtils.parseDate(dateStr);
|
if (value != null && value.length > 1) {
|
||||||
} catch (ParseException e) {
|
String s = Bytes.toString(value);
|
||||||
LOG.warn("Failed to parse date " + dateStr, e);
|
String[] all = s.split(",");
|
||||||
id.setDate(dateStr);
|
Collections.addAll(mobSet, all);
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
/* The algorithm works as follows:
|
|
||||||
* For monthly policy:
|
|
||||||
* 1). If the file's date is in past months, apply 4 * 7 * threshold
|
|
||||||
* 2). If the file's date is in past weeks, apply 7 * threshold
|
|
||||||
* 3). If the file's date is in current week, exclude it from the compaction
|
|
||||||
* For weekly policy:
|
|
||||||
* 1). If the file's date is in past weeks, apply 7 * threshold
|
|
||||||
* 2). If the file's date in currently, apply threshold
|
|
||||||
* For daily policy:
|
|
||||||
* 1). apply threshold
|
|
||||||
*/
|
|
||||||
if (policy == MobCompactPartitionPolicy.MONTHLY) {
|
|
||||||
if (date.before(firstDayOfCurrentMonth)) {
|
|
||||||
// Check overflow
|
|
||||||
if (threshold < (Long.MAX_VALUE / MONTHLY_THRESHOLD_MULTIPLIER)) {
|
|
||||||
finalThreshold = MONTHLY_THRESHOLD_MULTIPLIER * threshold;
|
|
||||||
} else {
|
|
||||||
finalThreshold = Long.MAX_VALUE;
|
|
||||||
}
|
|
||||||
id.setThreshold(finalThreshold);
|
|
||||||
|
|
||||||
// set to the date for the first day of that month
|
|
||||||
id.setDate(MobUtils.formatDate(MobUtils.getFirstDayOfMonth(calendar, date)));
|
|
||||||
return skipCompcation;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
List<Path> retList = new ArrayList<Path>();
|
||||||
if ((policy == MobCompactPartitionPolicy.MONTHLY) ||
|
for (String name : mobSet) {
|
||||||
(policy == MobCompactPartitionPolicy.WEEKLY)) {
|
retList.add(new Path(mobDir, name));
|
||||||
// Check if it needs to apply weekly multiplier
|
|
||||||
if (date.before(firstDayOfCurrentWeek)) {
|
|
||||||
// Check overflow
|
|
||||||
if (threshold < (Long.MAX_VALUE / WEEKLY_THRESHOLD_MULTIPLIER)) {
|
|
||||||
finalThreshold = WEEKLY_THRESHOLD_MULTIPLIER * threshold;
|
|
||||||
} else {
|
|
||||||
finalThreshold = Long.MAX_VALUE;
|
|
||||||
}
|
|
||||||
id.setThreshold(finalThreshold);
|
|
||||||
|
|
||||||
id.setDate(MobUtils.formatDate(MobUtils.getFirstDayOfWeek(calendar, date)));
|
|
||||||
return skipCompcation;
|
|
||||||
} else if (policy == MobCompactPartitionPolicy.MONTHLY) {
|
|
||||||
skipCompcation = true;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
return retList;
|
||||||
// Rest is daily
|
|
||||||
id.setDate(dateStr);
|
|
||||||
return skipCompcation;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,64 +0,0 @@
|
||||||
/**
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
package org.apache.hadoop.hbase.mob.compactions;
|
|
||||||
|
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The compaction request for mob files.
|
|
||||||
*/
|
|
||||||
@InterfaceAudience.Private
|
|
||||||
public abstract class MobCompactionRequest {
|
|
||||||
|
|
||||||
protected long selectionTime;
|
|
||||||
protected CompactionType type = CompactionType.PART_FILES;
|
|
||||||
|
|
||||||
public void setCompactionType(CompactionType type) {
|
|
||||||
this.type = type;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Gets the selection time.
|
|
||||||
* @return The selection time.
|
|
||||||
*/
|
|
||||||
public long getSelectionTime() {
|
|
||||||
return this.selectionTime;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Gets the compaction type.
|
|
||||||
* @return The compaction type.
|
|
||||||
*/
|
|
||||||
public CompactionType getCompactionType() {
|
|
||||||
return type;
|
|
||||||
}
|
|
||||||
|
|
||||||
protected enum CompactionType {
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Part of mob files are selected.
|
|
||||||
*/
|
|
||||||
PART_FILES,
|
|
||||||
|
|
||||||
/**
|
|
||||||
* All of mob files are selected.
|
|
||||||
*/
|
|
||||||
ALL_FILES
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -1,90 +0,0 @@
|
||||||
/**
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
package org.apache.hadoop.hbase.mob.compactions;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.concurrent.ExecutorService;
|
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.hadoop.hbase.TableName;
|
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
|
||||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
|
||||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* A mob compactor to directly compact the mob files.
|
|
||||||
*/
|
|
||||||
@InterfaceAudience.Private
|
|
||||||
public abstract class MobCompactor {
|
|
||||||
|
|
||||||
protected FileSystem fs;
|
|
||||||
protected Configuration conf;
|
|
||||||
protected TableName tableName;
|
|
||||||
protected ColumnFamilyDescriptor column;
|
|
||||||
|
|
||||||
protected Path mobTableDir;
|
|
||||||
protected Path mobFamilyDir;
|
|
||||||
protected ExecutorService pool;
|
|
||||||
|
|
||||||
public MobCompactor(Configuration conf, FileSystem fs, TableName tableName,
|
|
||||||
ColumnFamilyDescriptor column, ExecutorService pool) {
|
|
||||||
this.conf = conf;
|
|
||||||
this.fs = fs;
|
|
||||||
this.tableName = tableName;
|
|
||||||
this.column = column;
|
|
||||||
this.pool = pool;
|
|
||||||
mobTableDir = FSUtils.getTableDir(MobUtils.getMobHome(conf), tableName);
|
|
||||||
mobFamilyDir = MobUtils.getMobFamilyPath(conf, tableName, column.getNameAsString());
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Compacts the mob files for the current column family.
|
|
||||||
* @return The paths of new mob files generated in the compaction.
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
public List<Path> compact() throws IOException {
|
|
||||||
return compact(false);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Compacts the mob files by compaction type for the current column family.
|
|
||||||
* @param allFiles Whether add all mob files into the compaction.
|
|
||||||
* @return The paths of new mob files generated in the compaction.
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
public List<Path> compact(boolean allFiles) throws IOException {
|
|
||||||
return compact(Arrays.asList(fs.listStatus(mobFamilyDir)), allFiles);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Compacts the candidate mob files.
|
|
||||||
* @param files The candidate mob files.
|
|
||||||
* @param allFiles Whether add all mob files into the compaction.
|
|
||||||
* @return The paths of new mob files generated in the compaction.
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
public abstract List<Path> compact(List<FileStatus> files, boolean allFiles)
|
|
||||||
throws IOException;
|
|
||||||
}
|
|
|
@ -1,333 +0,0 @@
|
||||||
/**
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
package org.apache.hadoop.hbase.mob.compactions;
|
|
||||||
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Collection;
|
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.HStoreFile;
|
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* An implementation of {@link MobCompactionRequest} that is used in
|
|
||||||
* {@link PartitionedMobCompactor}.
|
|
||||||
* The mob files that have the same start key and date in their names belong to
|
|
||||||
* the same partition.
|
|
||||||
*/
|
|
||||||
@InterfaceAudience.Private
|
|
||||||
public class PartitionedMobCompactionRequest extends MobCompactionRequest {
|
|
||||||
|
|
||||||
protected List<CompactionDelPartition> delPartitions;
|
|
||||||
protected Collection<CompactionPartition> compactionPartitions;
|
|
||||||
|
|
||||||
public PartitionedMobCompactionRequest(Collection<CompactionPartition> compactionPartitions,
|
|
||||||
List<CompactionDelPartition> delPartitions) {
|
|
||||||
this.selectionTime = EnvironmentEdgeManager.currentTime();
|
|
||||||
this.compactionPartitions = compactionPartitions;
|
|
||||||
this.delPartitions = delPartitions;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Gets the compaction partitions.
|
|
||||||
* @return The compaction partitions.
|
|
||||||
*/
|
|
||||||
public Collection<CompactionPartition> getCompactionPartitions() {
|
|
||||||
return this.compactionPartitions;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Gets the del files.
|
|
||||||
* @return The del files.
|
|
||||||
*/
|
|
||||||
public List<CompactionDelPartition> getDelPartitions() {
|
|
||||||
return this.delPartitions;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The partition in the mob compaction.
|
|
||||||
* The mob files that have the same start key and date in their names belong to
|
|
||||||
* the same partition.
|
|
||||||
*/
|
|
||||||
protected static class CompactionPartition {
|
|
||||||
private List<FileStatus> files = new ArrayList<>();
|
|
||||||
private CompactionPartitionId partitionId;
|
|
||||||
|
|
||||||
// The startKey and endKey of this partition, both are inclusive.
|
|
||||||
private byte[] startKey;
|
|
||||||
private byte[] endKey;
|
|
||||||
|
|
||||||
public CompactionPartition(CompactionPartitionId partitionId) {
|
|
||||||
this.partitionId = partitionId;
|
|
||||||
}
|
|
||||||
|
|
||||||
public CompactionPartitionId getPartitionId() {
|
|
||||||
return this.partitionId;
|
|
||||||
}
|
|
||||||
|
|
||||||
public void addFile(FileStatus file) {
|
|
||||||
files.add(file);
|
|
||||||
}
|
|
||||||
|
|
||||||
public List<FileStatus> listFiles() {
|
|
||||||
return Collections.unmodifiableList(files);
|
|
||||||
}
|
|
||||||
|
|
||||||
public int getFileCount () {
|
|
||||||
return files.size();
|
|
||||||
}
|
|
||||||
|
|
||||||
public byte[] getStartKey() {
|
|
||||||
return startKey;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Set start key of this partition, only if the input startKey is less than
|
|
||||||
* the current start key.
|
|
||||||
*/
|
|
||||||
public void setStartKey(final byte[] startKey) {
|
|
||||||
if ((this.startKey == null) || (Bytes.compareTo(startKey, this.startKey) < 0)) {
|
|
||||||
this.startKey = startKey;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public byte[] getEndKey() {
|
|
||||||
return endKey;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Set end key of this partition, only if the input endKey is greater than
|
|
||||||
* the current end key.
|
|
||||||
*/
|
|
||||||
public void setEndKey(final byte[] endKey) {
|
|
||||||
if ((this.endKey == null) || (Bytes.compareTo(endKey, this.endKey) > 0)) {
|
|
||||||
this.endKey = endKey;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The partition id that consists of start key and date of the mob file name.
|
|
||||||
*/
|
|
||||||
public static class CompactionPartitionId {
|
|
||||||
private String startKey;
|
|
||||||
private String date;
|
|
||||||
private String latestDate;
|
|
||||||
private long threshold;
|
|
||||||
|
|
||||||
public CompactionPartitionId() {
|
|
||||||
// initialize these fields to empty string
|
|
||||||
this.startKey = "";
|
|
||||||
this.date = "";
|
|
||||||
this.latestDate = "";
|
|
||||||
this.threshold = 0;
|
|
||||||
}
|
|
||||||
|
|
||||||
public CompactionPartitionId(String startKey, String date) {
|
|
||||||
if (startKey == null || date == null) {
|
|
||||||
throw new IllegalArgumentException("Neither of start key and date could be null");
|
|
||||||
}
|
|
||||||
this.startKey = startKey;
|
|
||||||
this.date = date;
|
|
||||||
this.latestDate = "";
|
|
||||||
this.threshold = 0;
|
|
||||||
}
|
|
||||||
|
|
||||||
public void setThreshold (final long threshold) {
|
|
||||||
this.threshold = threshold;
|
|
||||||
}
|
|
||||||
|
|
||||||
public long getThreshold () {
|
|
||||||
return this.threshold;
|
|
||||||
}
|
|
||||||
|
|
||||||
public String getStartKey() {
|
|
||||||
return this.startKey;
|
|
||||||
}
|
|
||||||
|
|
||||||
public void setStartKey(final String startKey) {
|
|
||||||
this.startKey = startKey;
|
|
||||||
}
|
|
||||||
|
|
||||||
public String getDate() {
|
|
||||||
return this.date;
|
|
||||||
}
|
|
||||||
|
|
||||||
public void setDate(final String date) {
|
|
||||||
this.date = date;
|
|
||||||
}
|
|
||||||
|
|
||||||
public String getLatestDate () { return this.latestDate; }
|
|
||||||
|
|
||||||
public void updateLatestDate(final String latestDate) {
|
|
||||||
if (this.latestDate.compareTo(latestDate) < 0) {
|
|
||||||
this.latestDate = latestDate;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int hashCode() {
|
|
||||||
int result = 17;
|
|
||||||
result = 31 * result + startKey.hashCode();
|
|
||||||
result = 31 * result + date.hashCode();
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean equals(Object obj) {
|
|
||||||
if (this == obj) {
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
if (!(obj instanceof CompactionPartitionId)) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
CompactionPartitionId another = (CompactionPartitionId) obj;
|
|
||||||
if (!this.startKey.equals(another.startKey)) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
if (!this.date.equals(another.date)) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String toString() {
|
|
||||||
return new StringBuilder(startKey).append(date).toString();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The delete file partition in the mob compaction.
|
|
||||||
* The delete partition is defined as [startKey, endKey] pair.
|
|
||||||
* The mob delete files that have the same start key and end key belong to
|
|
||||||
* the same partition.
|
|
||||||
*/
|
|
||||||
protected static class CompactionDelPartition {
|
|
||||||
private List<Path> delFiles = new ArrayList<Path>();
|
|
||||||
private List<HStoreFile> storeFiles = new ArrayList<>();
|
|
||||||
private CompactionDelPartitionId id;
|
|
||||||
|
|
||||||
public CompactionDelPartition(CompactionDelPartitionId id) {
|
|
||||||
this.id = id;
|
|
||||||
}
|
|
||||||
|
|
||||||
public CompactionDelPartitionId getId() {
|
|
||||||
return this.id;
|
|
||||||
}
|
|
||||||
|
|
||||||
void addDelFile(FileStatus file) {
|
|
||||||
delFiles.add(file.getPath());
|
|
||||||
}
|
|
||||||
public void addStoreFile(HStoreFile file) {
|
|
||||||
storeFiles.add(file);
|
|
||||||
}
|
|
||||||
|
|
||||||
public List<HStoreFile> getStoreFiles() {
|
|
||||||
return storeFiles;
|
|
||||||
}
|
|
||||||
|
|
||||||
List<Path> listDelFiles() {
|
|
||||||
return Collections.unmodifiableList(delFiles);
|
|
||||||
}
|
|
||||||
|
|
||||||
void addDelFileList(final Collection<Path> list) {
|
|
||||||
delFiles.addAll(list);
|
|
||||||
}
|
|
||||||
|
|
||||||
int getDelFileCount () {
|
|
||||||
return delFiles.size();
|
|
||||||
}
|
|
||||||
|
|
||||||
void cleanDelFiles() {
|
|
||||||
delFiles.clear();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The delete partition id that consists of start key and end key
|
|
||||||
*/
|
|
||||||
public static class CompactionDelPartitionId implements Comparable<CompactionDelPartitionId> {
|
|
||||||
private byte[] startKey;
|
|
||||||
private byte[] endKey;
|
|
||||||
|
|
||||||
public CompactionDelPartitionId() {
|
|
||||||
}
|
|
||||||
|
|
||||||
public CompactionDelPartitionId(final byte[] startKey, final byte[] endKey) {
|
|
||||||
this.startKey = startKey;
|
|
||||||
this.endKey = endKey;
|
|
||||||
}
|
|
||||||
|
|
||||||
public byte[] getStartKey() {
|
|
||||||
return this.startKey;
|
|
||||||
}
|
|
||||||
public void setStartKey(final byte[] startKey) {
|
|
||||||
this.startKey = startKey;
|
|
||||||
}
|
|
||||||
|
|
||||||
public byte[] getEndKey() {
|
|
||||||
return this.endKey;
|
|
||||||
}
|
|
||||||
public void setEndKey(final byte[] endKey) {
|
|
||||||
this.endKey = endKey;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int compareTo(CompactionDelPartitionId o) {
|
|
||||||
/*
|
|
||||||
* 1). Compare the start key, if the k1 < k2, then k1 is less
|
|
||||||
* 2). If start Key is same, check endKey, k1 < k2, k1 is less
|
|
||||||
* If both are same, then they are equal.
|
|
||||||
*/
|
|
||||||
int result = Bytes.compareTo(this.startKey, o.getStartKey());
|
|
||||||
if (result != 0) {
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
return Bytes.compareTo(this.endKey, o.getEndKey());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int hashCode() {
|
|
||||||
int result = 17;
|
|
||||||
result = 31 * result + java.util.Arrays.hashCode(startKey);
|
|
||||||
result = 31 * result + java.util.Arrays.hashCode(endKey);
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean equals(Object obj) {
|
|
||||||
if (this == obj) {
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
if (!(obj instanceof CompactionDelPartitionId)) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
CompactionDelPartitionId another = (CompactionDelPartitionId) obj;
|
|
||||||
|
|
||||||
return (this.compareTo(another) == 0);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -1,949 +0,0 @@
|
||||||
/**
|
|
||||||
*
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
package org.apache.hadoop.hbase.mob.compactions;
|
|
||||||
|
|
||||||
import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TIME_KEY;
|
|
||||||
import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_CELLS_COUNT;
|
|
||||||
import static org.apache.hadoop.hbase.regionserver.HStoreFile.SKIP_RESET_SEQ_ID;
|
|
||||||
|
|
||||||
import java.io.FileNotFoundException;
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Calendar;
|
|
||||||
import java.util.Collection;
|
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.Comparator;
|
|
||||||
import java.util.Date;
|
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Map.Entry;
|
|
||||||
import java.util.NavigableMap;
|
|
||||||
import java.util.Objects;
|
|
||||||
import java.util.TreeMap;
|
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
import java.util.concurrent.ExecutorService;
|
|
||||||
import java.util.concurrent.Future;
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.hadoop.hbase.ArrayBackedTag;
|
|
||||||
import org.apache.hadoop.hbase.Cell;
|
|
||||||
import org.apache.hadoop.hbase.CellComparator;
|
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
|
||||||
import org.apache.hadoop.hbase.TableName;
|
|
||||||
import org.apache.hadoop.hbase.Tag;
|
|
||||||
import org.apache.hadoop.hbase.TagType;
|
|
||||||
import org.apache.hadoop.hbase.TagUtil;
|
|
||||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
|
||||||
import org.apache.hadoop.hbase.client.Connection;
|
|
||||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
|
||||||
import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy;
|
|
||||||
import org.apache.hadoop.hbase.client.Table;
|
|
||||||
import org.apache.hadoop.hbase.io.HFileLink;
|
|
||||||
import org.apache.hadoop.hbase.io.crypto.Encryption;
|
|
||||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
|
||||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
|
||||||
import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
|
|
||||||
import org.apache.hadoop.hbase.mob.MobConstants;
|
|
||||||
import org.apache.hadoop.hbase.mob.MobFileName;
|
|
||||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
|
||||||
import org.apache.hadoop.hbase.mob.compactions.MobCompactionRequest.CompactionType;
|
|
||||||
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionDelPartition;
|
|
||||||
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionDelPartitionId;
|
|
||||||
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartition;
|
|
||||||
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.HStoreFile;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.ScanInfo;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.ScanType;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.ScannerContext;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.StoreScanner;
|
|
||||||
import org.apache.hadoop.hbase.security.EncryptionUtil;
|
|
||||||
import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
|
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
|
||||||
import org.slf4j.Logger;
|
|
||||||
import org.slf4j.LoggerFactory;
|
|
||||||
|
|
||||||
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* An implementation of {@link MobCompactor} that compacts the mob files in partitions.
|
|
||||||
*/
|
|
||||||
@InterfaceAudience.Private
|
|
||||||
public class PartitionedMobCompactor extends MobCompactor {
|
|
||||||
|
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(PartitionedMobCompactor.class);
|
|
||||||
protected long mergeableSize;
|
|
||||||
protected int delFileMaxCount;
|
|
||||||
/** The number of files compacted in a batch */
|
|
||||||
protected int compactionBatchSize;
|
|
||||||
protected int compactionKVMax;
|
|
||||||
|
|
||||||
private final Path tempPath;
|
|
||||||
private final Path bulkloadPath;
|
|
||||||
private final CacheConfig compactionCacheConfig;
|
|
||||||
private final byte[] refCellTags;
|
|
||||||
private Encryption.Context cryptoContext = Encryption.Context.NONE;
|
|
||||||
|
|
||||||
public PartitionedMobCompactor(Configuration conf, FileSystem fs, TableName tableName,
|
|
||||||
ColumnFamilyDescriptor column, ExecutorService pool) throws IOException {
|
|
||||||
super(conf, fs, tableName, column, pool);
|
|
||||||
mergeableSize = conf.getLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD,
|
|
||||||
MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD);
|
|
||||||
delFileMaxCount = conf.getInt(MobConstants.MOB_DELFILE_MAX_COUNT,
|
|
||||||
MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT);
|
|
||||||
// default is 100
|
|
||||||
compactionBatchSize = conf.getInt(MobConstants.MOB_COMPACTION_BATCH_SIZE,
|
|
||||||
MobConstants.DEFAULT_MOB_COMPACTION_BATCH_SIZE);
|
|
||||||
tempPath = new Path(MobUtils.getMobHome(conf), MobConstants.TEMP_DIR_NAME);
|
|
||||||
bulkloadPath = new Path(tempPath, new Path(MobConstants.BULKLOAD_DIR_NAME, new Path(
|
|
||||||
tableName.getNamespaceAsString(), tableName.getQualifierAsString())));
|
|
||||||
compactionKVMax = this.conf.getInt(HConstants.COMPACTION_KV_MAX,
|
|
||||||
HConstants.COMPACTION_KV_MAX_DEFAULT);
|
|
||||||
Configuration copyOfConf = new Configuration(conf);
|
|
||||||
copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f);
|
|
||||||
compactionCacheConfig = new CacheConfig(copyOfConf);
|
|
||||||
List<Tag> tags = new ArrayList<>(2);
|
|
||||||
tags.add(MobConstants.MOB_REF_TAG);
|
|
||||||
Tag tableNameTag = new ArrayBackedTag(TagType.MOB_TABLE_NAME_TAG_TYPE, tableName.getName());
|
|
||||||
tags.add(tableNameTag);
|
|
||||||
this.refCellTags = TagUtil.fromList(tags);
|
|
||||||
cryptoContext = EncryptionUtil.createEncryptionContext(copyOfConf, column);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public List<Path> compact(List<FileStatus> files, boolean allFiles) throws IOException {
|
|
||||||
if (files == null || files.isEmpty()) {
|
|
||||||
LOG.info("No candidate mob files");
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
LOG.info("is allFiles: " + allFiles);
|
|
||||||
|
|
||||||
// find the files to compact.
|
|
||||||
PartitionedMobCompactionRequest request = select(files, allFiles);
|
|
||||||
// compact the files.
|
|
||||||
return performCompaction(request);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Selects the compacted mob/del files.
|
|
||||||
* Iterates the candidates to find out all the del files and small mob files.
|
|
||||||
* @param candidates All the candidates.
|
|
||||||
* @param allFiles Whether add all mob files into the compaction.
|
|
||||||
* @return A compaction request.
|
|
||||||
* @throws IOException if IO failure is encountered
|
|
||||||
*/
|
|
||||||
protected PartitionedMobCompactionRequest select(List<FileStatus> candidates,
|
|
||||||
boolean allFiles) throws IOException {
|
|
||||||
final Map<CompactionPartitionId, CompactionPartition> filesToCompact = new HashMap<>();
|
|
||||||
final CompactionPartitionId id = new CompactionPartitionId();
|
|
||||||
final NavigableMap<CompactionDelPartitionId, CompactionDelPartition> delFilesToCompact = new TreeMap<>();
|
|
||||||
final CompactionDelPartitionId delId = new CompactionDelPartitionId();
|
|
||||||
final ArrayList<CompactionDelPartition> allDelPartitions = new ArrayList<>();
|
|
||||||
int selectedFileCount = 0;
|
|
||||||
int irrelevantFileCount = 0;
|
|
||||||
int totalDelFiles = 0;
|
|
||||||
MobCompactPartitionPolicy policy = column.getMobCompactPartitionPolicy();
|
|
||||||
|
|
||||||
Calendar calendar = Calendar.getInstance();
|
|
||||||
Date currentDate = new Date();
|
|
||||||
Date firstDayOfCurrentMonth = null;
|
|
||||||
Date firstDayOfCurrentWeek = null;
|
|
||||||
|
|
||||||
if (policy == MobCompactPartitionPolicy.MONTHLY) {
|
|
||||||
firstDayOfCurrentMonth = MobUtils.getFirstDayOfMonth(calendar, currentDate);
|
|
||||||
firstDayOfCurrentWeek = MobUtils.getFirstDayOfWeek(calendar, currentDate);
|
|
||||||
} else if (policy == MobCompactPartitionPolicy.WEEKLY) {
|
|
||||||
firstDayOfCurrentWeek = MobUtils.getFirstDayOfWeek(calendar, currentDate);
|
|
||||||
}
|
|
||||||
|
|
||||||
// We check if there is any del files so the logic can be optimized for the following processing
|
|
||||||
// First step is to check if there is any delete files. If there is any delete files,
|
|
||||||
// For each Partition, it needs to read its startKey and endKey from files.
|
|
||||||
// If there is no delete file, there is no need to read startKey and endKey from files, this
|
|
||||||
// is an optimization.
|
|
||||||
boolean withDelFiles = false;
|
|
||||||
for (FileStatus file : candidates) {
|
|
||||||
if (!file.isFile()) {
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
// group the del files and small files.
|
|
||||||
FileStatus linkedFile = file;
|
|
||||||
if (HFileLink.isHFileLink(file.getPath())) {
|
|
||||||
HFileLink link = HFileLink.buildFromHFileLinkPattern(conf, file.getPath());
|
|
||||||
linkedFile = getLinkedFileStatus(link);
|
|
||||||
if (linkedFile == null) {
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
if (StoreFileInfo.isDelFile(linkedFile.getPath())) {
|
|
||||||
withDelFiles = true;
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
for (FileStatus file : candidates) {
|
|
||||||
if (!file.isFile()) {
|
|
||||||
irrelevantFileCount++;
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
// group the del files and small files.
|
|
||||||
FileStatus linkedFile = file;
|
|
||||||
if (HFileLink.isHFileLink(file.getPath())) {
|
|
||||||
HFileLink link = HFileLink.buildFromHFileLinkPattern(conf, file.getPath());
|
|
||||||
linkedFile = getLinkedFileStatus(link);
|
|
||||||
if (linkedFile == null) {
|
|
||||||
// If the linked file cannot be found, regard it as an irrelevantFileCount file
|
|
||||||
irrelevantFileCount++;
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
if (withDelFiles && StoreFileInfo.isDelFile(linkedFile.getPath())) {
|
|
||||||
// File in the Del Partition List
|
|
||||||
|
|
||||||
// Get delId from the file
|
|
||||||
try (Reader reader = HFile.createReader(fs, linkedFile.getPath(), conf)) {
|
|
||||||
delId.setStartKey(reader.getFirstRowKey().get());
|
|
||||||
delId.setEndKey(reader.getLastRowKey().get());
|
|
||||||
}
|
|
||||||
CompactionDelPartition delPartition = delFilesToCompact.get(delId);
|
|
||||||
if (delPartition == null) {
|
|
||||||
CompactionDelPartitionId newDelId =
|
|
||||||
new CompactionDelPartitionId(delId.getStartKey(), delId.getEndKey());
|
|
||||||
delPartition = new CompactionDelPartition(newDelId);
|
|
||||||
delFilesToCompact.put(newDelId, delPartition);
|
|
||||||
}
|
|
||||||
delPartition.addDelFile(file);
|
|
||||||
totalDelFiles ++;
|
|
||||||
} else {
|
|
||||||
String fileName = linkedFile.getPath().getName();
|
|
||||||
String date = MobFileName.getDateFromName(fileName);
|
|
||||||
boolean skipCompaction = MobUtils
|
|
||||||
.fillPartitionId(id, firstDayOfCurrentMonth, firstDayOfCurrentWeek, date, policy,
|
|
||||||
calendar, mergeableSize);
|
|
||||||
if (allFiles || (!skipCompaction && (linkedFile.getLen() < id.getThreshold()))) {
|
|
||||||
// add all files if allFiles is true,
|
|
||||||
// otherwise add the small files to the merge pool
|
|
||||||
// filter out files which are not supposed to be compacted with the
|
|
||||||
// current policy
|
|
||||||
|
|
||||||
id.setStartKey(MobFileName.getStartKeyFromName(fileName));
|
|
||||||
CompactionPartition compactionPartition = filesToCompact.get(id);
|
|
||||||
if (compactionPartition == null) {
|
|
||||||
CompactionPartitionId newId = new CompactionPartitionId(id.getStartKey(), id.getDate());
|
|
||||||
compactionPartition = new CompactionPartition(newId);
|
|
||||||
compactionPartition.addFile(file);
|
|
||||||
filesToCompact.put(newId, compactionPartition);
|
|
||||||
newId.updateLatestDate(date);
|
|
||||||
} else {
|
|
||||||
compactionPartition.addFile(file);
|
|
||||||
compactionPartition.getPartitionId().updateLatestDate(date);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (withDelFiles) {
|
|
||||||
// get startKey and endKey from the file and update partition
|
|
||||||
// TODO: is it possible to skip read of most hfiles?
|
|
||||||
try (Reader reader = HFile.createReader(fs, linkedFile.getPath(), conf)) {
|
|
||||||
compactionPartition.setStartKey(reader.getFirstRowKey().get());
|
|
||||||
compactionPartition.setEndKey(reader.getLastRowKey().get());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
selectedFileCount++;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/*
|
|
||||||
* Merge del files so there are only non-overlapped del file lists
|
|
||||||
*/
|
|
||||||
for(Map.Entry<CompactionDelPartitionId, CompactionDelPartition> entry : delFilesToCompact.entrySet()) {
|
|
||||||
if (allDelPartitions.size() > 0) {
|
|
||||||
// check if the current key range overlaps the previous one
|
|
||||||
CompactionDelPartition prev = allDelPartitions.get(allDelPartitions.size() - 1);
|
|
||||||
if (Bytes.compareTo(prev.getId().getEndKey(), entry.getKey().getStartKey()) >= 0) {
|
|
||||||
// merge them together
|
|
||||||
prev.getId().setEndKey(entry.getValue().getId().getEndKey());
|
|
||||||
prev.addDelFileList(entry.getValue().listDelFiles());
|
|
||||||
|
|
||||||
} else {
|
|
||||||
allDelPartitions.add(entry.getValue());
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
allDelPartitions.add(entry.getValue());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
PartitionedMobCompactionRequest request = new PartitionedMobCompactionRequest(
|
|
||||||
filesToCompact.values(), allDelPartitions);
|
|
||||||
if (candidates.size() == (totalDelFiles + selectedFileCount + irrelevantFileCount)) {
|
|
||||||
// all the files are selected
|
|
||||||
request.setCompactionType(CompactionType.ALL_FILES);
|
|
||||||
}
|
|
||||||
LOG.info("The compaction type is {}, the request has {} del files, {} selected files, and {} " +
|
|
||||||
"irrelevant files table '{}' and column '{}'", request.getCompactionType(), totalDelFiles,
|
|
||||||
selectedFileCount, irrelevantFileCount, tableName, column.getNameAsString());
|
|
||||||
return request;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Performs the compaction on the selected files.
|
|
||||||
* <ol>
|
|
||||||
* <li>Compacts the del files.</li>
|
|
||||||
* <li>Compacts the selected small mob files and all the del files.</li>
|
|
||||||
* <li>If all the candidates are selected, delete the del files.</li>
|
|
||||||
* </ol>
|
|
||||||
* @param request The compaction request.
|
|
||||||
* @return The paths of new mob files generated in the compaction.
|
|
||||||
* @throws IOException if IO failure is encountered
|
|
||||||
*/
|
|
||||||
protected List<Path> performCompaction(PartitionedMobCompactionRequest request)
|
|
||||||
throws IOException {
|
|
||||||
|
|
||||||
// merge the del files, it is per del partition
|
|
||||||
for (CompactionDelPartition delPartition : request.getDelPartitions()) {
|
|
||||||
if (delPartition.getDelFileCount() <= 1) continue;
|
|
||||||
List<Path> newDelPaths = compactDelFiles(request, delPartition.listDelFiles());
|
|
||||||
delPartition.cleanDelFiles();
|
|
||||||
delPartition.addDelFileList(newDelPaths);
|
|
||||||
}
|
|
||||||
|
|
||||||
List<Path> paths = null;
|
|
||||||
int totalDelFileCount = 0;
|
|
||||||
try {
|
|
||||||
for (CompactionDelPartition delPartition : request.getDelPartitions()) {
|
|
||||||
for (Path newDelPath : delPartition.listDelFiles()) {
|
|
||||||
HStoreFile sf =
|
|
||||||
new HStoreFile(fs, newDelPath, conf, compactionCacheConfig, BloomType.NONE, true);
|
|
||||||
// pre-create reader of a del file to avoid race condition when opening the reader in each
|
|
||||||
// partition.
|
|
||||||
sf.initReader();
|
|
||||||
delPartition.addStoreFile(sf);
|
|
||||||
totalDelFileCount++;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
LOG.info("After merging, there are {} del files. table='{}' column='{}'", totalDelFileCount,
|
|
||||||
tableName, column.getNameAsString());
|
|
||||||
// compact the mob files by partitions.
|
|
||||||
paths = compactMobFiles(request);
|
|
||||||
LOG.info("After compaction, there are {} mob files. table='{}' column='{}'", paths.size(),
|
|
||||||
tableName, column.getNameAsString());
|
|
||||||
} finally {
|
|
||||||
for (CompactionDelPartition delPartition : request.getDelPartitions()) {
|
|
||||||
closeStoreFileReaders(delPartition.getStoreFiles());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// archive the del files if all the mob files are selected.
|
|
||||||
if (request.type == CompactionType.ALL_FILES && !request.getDelPartitions().isEmpty()) {
|
|
||||||
LOG.info("After a mob compaction with all files selected, archiving the del files for " +
|
|
||||||
"table='{}' and column='{}'", tableName, column.getNameAsString());
|
|
||||||
for (CompactionDelPartition delPartition : request.getDelPartitions()) {
|
|
||||||
LOG.info(Objects.toString(delPartition.listDelFiles()));
|
|
||||||
try {
|
|
||||||
MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(),
|
|
||||||
delPartition.getStoreFiles());
|
|
||||||
} catch (IOException e) {
|
|
||||||
LOG.error("Failed to archive the del files {} for partition {} table='{}' and " +
|
|
||||||
"column='{}'", delPartition.getStoreFiles(), delPartition.getId(), tableName,
|
|
||||||
column.getNameAsString(), e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return paths;
|
|
||||||
}
|
|
||||||
|
|
||||||
static class DelPartitionComparator implements Comparator<CompactionDelPartition> {
|
|
||||||
private boolean compareStartKey;
|
|
||||||
|
|
||||||
DelPartitionComparator(boolean compareStartKey) {
|
|
||||||
this.compareStartKey = compareStartKey;
|
|
||||||
}
|
|
||||||
|
|
||||||
public boolean getCompareStartKey() {
|
|
||||||
return this.compareStartKey;
|
|
||||||
}
|
|
||||||
|
|
||||||
public void setCompareStartKey(final boolean compareStartKey) {
|
|
||||||
this.compareStartKey = compareStartKey;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int compare(CompactionDelPartition o1, CompactionDelPartition o2) {
|
|
||||||
|
|
||||||
if (compareStartKey) {
|
|
||||||
return Bytes.compareTo(o1.getId().getStartKey(), o2.getId().getStartKey());
|
|
||||||
} else {
|
|
||||||
return Bytes.compareTo(o1.getId().getEndKey(), o2.getId().getEndKey());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@VisibleForTesting
|
|
||||||
List<HStoreFile> getListOfDelFilesForPartition(final CompactionPartition partition,
|
|
||||||
final List<CompactionDelPartition> delPartitions) {
|
|
||||||
// Binary search for startKey and endKey
|
|
||||||
|
|
||||||
List<HStoreFile> result = new ArrayList<>();
|
|
||||||
|
|
||||||
DelPartitionComparator comparator = new DelPartitionComparator(false);
|
|
||||||
CompactionDelPartitionId id = new CompactionDelPartitionId(null, partition.getStartKey());
|
|
||||||
CompactionDelPartition target = new CompactionDelPartition(id);
|
|
||||||
int start = Collections.binarySearch(delPartitions, target, comparator);
|
|
||||||
|
|
||||||
// Get the start index for partition
|
|
||||||
if (start < 0) {
|
|
||||||
// Calculate the insert point
|
|
||||||
start = (start + 1) * (-1);
|
|
||||||
if (start == delPartitions.size()) {
|
|
||||||
// no overlap
|
|
||||||
return result;
|
|
||||||
} else {
|
|
||||||
// Check another case which has no overlap
|
|
||||||
if (Bytes.compareTo(partition.getEndKey(), delPartitions.get(start).getId().getStartKey()) < 0) {
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Search for end index for the partition
|
|
||||||
comparator.setCompareStartKey(true);
|
|
||||||
id.setStartKey(partition.getEndKey());
|
|
||||||
int end = Collections.binarySearch(delPartitions, target, comparator);
|
|
||||||
|
|
||||||
if (end < 0) {
|
|
||||||
end = (end + 1) * (-1);
|
|
||||||
if (end == 0) {
|
|
||||||
return result;
|
|
||||||
} else {
|
|
||||||
--end;
|
|
||||||
if (Bytes.compareTo(partition.getStartKey(), delPartitions.get(end).getId().getEndKey()) > 0) {
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
for (int i = start; i <= end; ++i) {
|
|
||||||
result.addAll(delPartitions.get(i).getStoreFiles());
|
|
||||||
}
|
|
||||||
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Compacts the selected small mob files and all the del files.
|
|
||||||
* @param request The compaction request.
|
|
||||||
* @return The paths of new mob files after compactions.
|
|
||||||
* @throws IOException if IO failure is encountered
|
|
||||||
*/
|
|
||||||
protected List<Path> compactMobFiles(final PartitionedMobCompactionRequest request)
|
|
||||||
throws IOException {
|
|
||||||
Collection<CompactionPartition> partitions = request.compactionPartitions;
|
|
||||||
if (partitions == null || partitions.isEmpty()) {
|
|
||||||
LOG.info("No partitions of mob files in table='{}' and column='{}'", tableName,
|
|
||||||
column.getNameAsString());
|
|
||||||
return Collections.emptyList();
|
|
||||||
}
|
|
||||||
List<Path> paths = new ArrayList<>();
|
|
||||||
final Connection c = ConnectionFactory.createConnection(conf);
|
|
||||||
final Table table = c.getTable(tableName);
|
|
||||||
|
|
||||||
try {
|
|
||||||
Map<CompactionPartitionId, Future<List<Path>>> results = new HashMap<>();
|
|
||||||
// compact the mob files by partitions in parallel.
|
|
||||||
for (final CompactionPartition partition : partitions) {
|
|
||||||
|
|
||||||
// How to efficiently come up a list of delFiles for one partition?
|
|
||||||
// Search the delPartitions and collect all the delFiles for the partition
|
|
||||||
// One optimization can do is that if there is no del file, we do not need to
|
|
||||||
// come up with startKey/endKey.
|
|
||||||
List<HStoreFile> delFiles = getListOfDelFilesForPartition(partition,
|
|
||||||
request.getDelPartitions());
|
|
||||||
|
|
||||||
results.put(partition.getPartitionId(), pool.submit(new Callable<List<Path>>() {
|
|
||||||
@Override
|
|
||||||
public List<Path> call() throws Exception {
|
|
||||||
LOG.info("Compacting mob files for partition {} for table='{}' and column='{}'",
|
|
||||||
partition.getPartitionId(), tableName, column.getNameAsString());
|
|
||||||
return compactMobFilePartition(request, partition, delFiles, c, table);
|
|
||||||
}
|
|
||||||
}));
|
|
||||||
}
|
|
||||||
// compact the partitions in parallel.
|
|
||||||
List<CompactionPartitionId> failedPartitions = new ArrayList<>();
|
|
||||||
for (Entry<CompactionPartitionId, Future<List<Path>>> result : results.entrySet()) {
|
|
||||||
try {
|
|
||||||
paths.addAll(result.getValue().get());
|
|
||||||
} catch (Exception e) {
|
|
||||||
// just log the error
|
|
||||||
LOG.error("Failed to compact the partition {} for table='{}' and column='{}'",
|
|
||||||
result.getKey(), tableName, column.getNameAsString(), e);
|
|
||||||
failedPartitions.add(result.getKey());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
if (!failedPartitions.isEmpty()) {
|
|
||||||
// if any partition fails in the compaction, directly throw an exception.
|
|
||||||
throw new IOException("Failed to compact the partitions " + failedPartitions +
|
|
||||||
" for table='" + tableName + "' column='" + column.getNameAsString() + "'");
|
|
||||||
}
|
|
||||||
} finally {
|
|
||||||
try {
|
|
||||||
table.close();
|
|
||||||
} catch (IOException e) {
|
|
||||||
LOG.error("Failed to close the Table", e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return paths;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Compacts a partition of selected small mob files and all the del files.
|
|
||||||
* @param request The compaction request.
|
|
||||||
* @param partition A compaction partition.
|
|
||||||
* @param delFiles The del files.
|
|
||||||
* @param connection The connection to use.
|
|
||||||
* @param table The current table.
|
|
||||||
* @return The paths of new mob files after compactions.
|
|
||||||
* @throws IOException if IO failure is encountered
|
|
||||||
*/
|
|
||||||
private List<Path> compactMobFilePartition(PartitionedMobCompactionRequest request,
|
|
||||||
CompactionPartition partition,
|
|
||||||
List<HStoreFile> delFiles,
|
|
||||||
Connection connection,
|
|
||||||
Table table) throws IOException {
|
|
||||||
if (MobUtils.isMobFileExpired(column, EnvironmentEdgeManager.currentTime(),
|
|
||||||
partition.getPartitionId().getDate())) {
|
|
||||||
// If the files in the partition are expired, do not compact them and directly
|
|
||||||
// return an empty list.
|
|
||||||
return Collections.emptyList();
|
|
||||||
}
|
|
||||||
List<Path> newFiles = new ArrayList<>();
|
|
||||||
List<FileStatus> files = partition.listFiles();
|
|
||||||
int offset = 0;
|
|
||||||
Path bulkloadPathOfPartition = new Path(bulkloadPath, partition.getPartitionId().toString());
|
|
||||||
Path bulkloadColumnPath = new Path(bulkloadPathOfPartition, column.getNameAsString());
|
|
||||||
while (offset < files.size()) {
|
|
||||||
int batch = compactionBatchSize;
|
|
||||||
if (files.size() - offset < compactionBatchSize) {
|
|
||||||
batch = files.size() - offset;
|
|
||||||
}
|
|
||||||
if (batch == 1 && delFiles.isEmpty()) {
|
|
||||||
// only one file left and no del files, do not compact it,
|
|
||||||
// and directly add it to the new files.
|
|
||||||
newFiles.add(files.get(offset).getPath());
|
|
||||||
offset++;
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
// clean the bulkload directory to avoid loading old files.
|
|
||||||
fs.delete(bulkloadPathOfPartition, true);
|
|
||||||
// add the selected mob files and del files into filesToCompact
|
|
||||||
List<HStoreFile> filesToCompact = new ArrayList<>();
|
|
||||||
for (int i = offset; i < batch + offset; i++) {
|
|
||||||
HStoreFile sf = new HStoreFile(fs, files.get(i).getPath(), conf, compactionCacheConfig,
|
|
||||||
BloomType.NONE, true);
|
|
||||||
filesToCompact.add(sf);
|
|
||||||
}
|
|
||||||
filesToCompact.addAll(delFiles);
|
|
||||||
// compact the mob files in a batch.
|
|
||||||
compactMobFilesInBatch(request, partition, connection, table, filesToCompact, batch,
|
|
||||||
bulkloadPathOfPartition, bulkloadColumnPath, newFiles);
|
|
||||||
// move to the next batch.
|
|
||||||
offset += batch;
|
|
||||||
}
|
|
||||||
LOG.info("Compaction is finished. The number of mob files is changed from {} to {} for " +
|
|
||||||
"partition={} for table='{}' and column='{}'", files.size(), newFiles.size(),
|
|
||||||
partition.getPartitionId(), tableName, column.getNameAsString());
|
|
||||||
return newFiles;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Closes the readers of store files.
|
|
||||||
* @param storeFiles The store files to be closed.
|
|
||||||
*/
|
|
||||||
private void closeStoreFileReaders(List<HStoreFile> storeFiles) {
|
|
||||||
for (HStoreFile storeFile : storeFiles) {
|
|
||||||
try {
|
|
||||||
storeFile.closeStoreFile(true);
|
|
||||||
} catch (IOException e) {
|
|
||||||
LOG.warn("Failed to close the reader on store file " + storeFile.getPath(), e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Compacts a partition of selected small mob files and all the del files in a batch.
|
|
||||||
* @param request The compaction request.
|
|
||||||
* @param partition A compaction partition.
|
|
||||||
* @param connection To use for transport
|
|
||||||
* @param table The current table.
|
|
||||||
* @param filesToCompact The files to be compacted.
|
|
||||||
* @param batch The number of mob files to be compacted in a batch.
|
|
||||||
* @param bulkloadPathOfPartition The directory where the bulkload column of the current
|
|
||||||
* partition is saved.
|
|
||||||
* @param bulkloadColumnPath The directory where the bulkload files of current partition
|
|
||||||
* are saved.
|
|
||||||
* @param newFiles The paths of new mob files after compactions.
|
|
||||||
* @throws IOException if IO failure is encountered
|
|
||||||
*/
|
|
||||||
private void compactMobFilesInBatch(PartitionedMobCompactionRequest request,
|
|
||||||
CompactionPartition partition,
|
|
||||||
Connection connection, Table table,
|
|
||||||
List<HStoreFile> filesToCompact, int batch,
|
|
||||||
Path bulkloadPathOfPartition, Path bulkloadColumnPath,
|
|
||||||
List<Path> newFiles)
|
|
||||||
throws IOException {
|
|
||||||
// open scanner to the selected mob files and del files.
|
|
||||||
StoreScanner scanner = createScanner(filesToCompact, ScanType.COMPACT_DROP_DELETES);
|
|
||||||
// the mob files to be compacted, not include the del files.
|
|
||||||
List<HStoreFile> mobFilesToCompact = filesToCompact.subList(0, batch);
|
|
||||||
// Pair(maxSeqId, cellsCount)
|
|
||||||
Pair<Long, Long> fileInfo = getFileInfo(mobFilesToCompact);
|
|
||||||
// open writers for the mob files and new ref store files.
|
|
||||||
StoreFileWriter writer = null;
|
|
||||||
StoreFileWriter refFileWriter = null;
|
|
||||||
Path filePath = null;
|
|
||||||
long mobCells = 0;
|
|
||||||
boolean cleanupTmpMobFile = false;
|
|
||||||
boolean cleanupBulkloadDirOfPartition = false;
|
|
||||||
boolean cleanupCommittedMobFile = false;
|
|
||||||
boolean closeReaders= true;
|
|
||||||
|
|
||||||
try {
|
|
||||||
try {
|
|
||||||
writer = MobUtils
|
|
||||||
.createWriter(conf, fs, column, partition.getPartitionId().getLatestDate(), tempPath,
|
|
||||||
Long.MAX_VALUE, column.getCompactionCompressionType(),
|
|
||||||
partition.getPartitionId().getStartKey(), compactionCacheConfig, cryptoContext,
|
|
||||||
true);
|
|
||||||
cleanupTmpMobFile = true;
|
|
||||||
filePath = writer.getPath();
|
|
||||||
byte[] fileName = Bytes.toBytes(filePath.getName());
|
|
||||||
// create a temp file and open a writer for it in the bulkloadPath
|
|
||||||
refFileWriter = MobUtils.createRefFileWriter(conf, fs, column, bulkloadColumnPath,
|
|
||||||
fileInfo.getSecond().longValue(), compactionCacheConfig, cryptoContext, true);
|
|
||||||
cleanupBulkloadDirOfPartition = true;
|
|
||||||
List<Cell> cells = new ArrayList<>();
|
|
||||||
boolean hasMore;
|
|
||||||
ScannerContext scannerContext =
|
|
||||||
ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build();
|
|
||||||
do {
|
|
||||||
hasMore = scanner.next(cells, scannerContext);
|
|
||||||
for (Cell cell : cells) {
|
|
||||||
// write the mob cell to the mob file.
|
|
||||||
writer.append(cell);
|
|
||||||
// write the new reference cell to the store file.
|
|
||||||
Cell reference = MobUtils.createMobRefCell(cell, fileName, this.refCellTags);
|
|
||||||
refFileWriter.append(reference);
|
|
||||||
mobCells++;
|
|
||||||
}
|
|
||||||
cells.clear();
|
|
||||||
} while (hasMore);
|
|
||||||
} finally {
|
|
||||||
// close the scanner.
|
|
||||||
scanner.close();
|
|
||||||
|
|
||||||
if (cleanupTmpMobFile) {
|
|
||||||
// append metadata to the mob file, and close the mob file writer.
|
|
||||||
closeMobFileWriter(writer, fileInfo.getFirst(), mobCells);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (cleanupBulkloadDirOfPartition) {
|
|
||||||
// append metadata and bulkload info to the ref mob file, and close the writer.
|
|
||||||
closeRefFileWriter(refFileWriter, fileInfo.getFirst(), request.selectionTime);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (mobCells > 0) {
|
|
||||||
// commit mob file
|
|
||||||
MobUtils.commitFile(conf, fs, filePath, mobFamilyDir, compactionCacheConfig);
|
|
||||||
cleanupTmpMobFile = false;
|
|
||||||
cleanupCommittedMobFile = true;
|
|
||||||
// bulkload the ref file
|
|
||||||
LOG.info("start MOB ref bulkload for partition {} table='{}' column='{}'",
|
|
||||||
partition.getPartitionId(), tableName, column.getNameAsString());
|
|
||||||
bulkloadRefFile(table.getName(), bulkloadPathOfPartition, filePath.getName());
|
|
||||||
cleanupCommittedMobFile = false;
|
|
||||||
LOG.info("end MOB ref bulkload for partition {} table='{}' column='{}'",
|
|
||||||
partition.getPartitionId(), tableName, column.getNameAsString());
|
|
||||||
newFiles.add(new Path(mobFamilyDir, filePath.getName()));
|
|
||||||
}
|
|
||||||
|
|
||||||
// archive the old mob files, do not archive the del files.
|
|
||||||
try {
|
|
||||||
closeStoreFileReaders(mobFilesToCompact);
|
|
||||||
closeReaders = false;
|
|
||||||
MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), mobFilesToCompact);
|
|
||||||
} catch (IOException e) {
|
|
||||||
LOG.error("Failed to archive the files " + mobFilesToCompact, e);
|
|
||||||
}
|
|
||||||
} finally {
|
|
||||||
if (closeReaders) {
|
|
||||||
closeStoreFileReaders(mobFilesToCompact);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (cleanupTmpMobFile) {
|
|
||||||
deletePath(filePath);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (cleanupBulkloadDirOfPartition) {
|
|
||||||
// delete the bulkload files in bulkloadPath
|
|
||||||
deletePath(bulkloadPathOfPartition);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (cleanupCommittedMobFile) {
|
|
||||||
LOG.error("failed MOB ref bulkload for partition {} table='{}' column='{}'",
|
|
||||||
partition.getPartitionId(), tableName, column.getNameAsString());
|
|
||||||
MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(),
|
|
||||||
Collections.singletonList(new HStoreFile(fs, new Path(mobFamilyDir, filePath.getName()),
|
|
||||||
conf, compactionCacheConfig, BloomType.NONE, true)));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Compacts the del files in batches which avoids opening too many files.
|
|
||||||
* @param request The compaction request.
|
|
||||||
* @param delFilePaths Del file paths to compact
|
|
||||||
* @return The paths of new del files after merging or the original files if no merging
|
|
||||||
* is necessary.
|
|
||||||
* @throws IOException if IO failure is encountered
|
|
||||||
*/
|
|
||||||
protected List<Path> compactDelFiles(PartitionedMobCompactionRequest request,
|
|
||||||
List<Path> delFilePaths) throws IOException {
|
|
||||||
if (delFilePaths.size() <= delFileMaxCount) {
|
|
||||||
return delFilePaths;
|
|
||||||
}
|
|
||||||
// when there are more del files than the number that is allowed, merge it firstly.
|
|
||||||
int offset = 0;
|
|
||||||
List<Path> paths = new ArrayList<>();
|
|
||||||
while (offset < delFilePaths.size()) {
|
|
||||||
// get the batch
|
|
||||||
int batch = compactionBatchSize;
|
|
||||||
if (delFilePaths.size() - offset < compactionBatchSize) {
|
|
||||||
batch = delFilePaths.size() - offset;
|
|
||||||
}
|
|
||||||
List<HStoreFile> batchedDelFiles = new ArrayList<>();
|
|
||||||
if (batch == 1) {
|
|
||||||
// only one file left, do not compact it, directly add it to the new files.
|
|
||||||
paths.add(delFilePaths.get(offset));
|
|
||||||
offset++;
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
for (int i = offset; i < batch + offset; i++) {
|
|
||||||
batchedDelFiles.add(new HStoreFile(fs, delFilePaths.get(i), conf, compactionCacheConfig,
|
|
||||||
BloomType.NONE, true));
|
|
||||||
}
|
|
||||||
// compact the del files in a batch.
|
|
||||||
paths.add(compactDelFilesInBatch(request, batchedDelFiles));
|
|
||||||
// move to the next batch.
|
|
||||||
offset += batch;
|
|
||||||
}
|
|
||||||
return compactDelFiles(request, paths);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Compacts the del file in a batch.
|
|
||||||
* @param request The compaction request.
|
|
||||||
* @param delFiles The del files.
|
|
||||||
* @return The path of new del file after merging.
|
|
||||||
* @throws IOException if IO failure is encountered
|
|
||||||
*/
|
|
||||||
private Path compactDelFilesInBatch(PartitionedMobCompactionRequest request,
|
|
||||||
List<HStoreFile> delFiles) throws IOException {
|
|
||||||
// create a scanner for the del files.
|
|
||||||
StoreScanner scanner = createScanner(delFiles, ScanType.COMPACT_RETAIN_DELETES);
|
|
||||||
StoreFileWriter writer = null;
|
|
||||||
Path filePath = null;
|
|
||||||
try {
|
|
||||||
writer = MobUtils.createDelFileWriter(conf, fs, column,
|
|
||||||
MobUtils.formatDate(new Date(request.selectionTime)), tempPath, Long.MAX_VALUE,
|
|
||||||
column.getCompactionCompressionType(), HConstants.EMPTY_START_ROW, compactionCacheConfig,
|
|
||||||
cryptoContext);
|
|
||||||
filePath = writer.getPath();
|
|
||||||
List<Cell> cells = new ArrayList<>();
|
|
||||||
boolean hasMore;
|
|
||||||
ScannerContext scannerContext =
|
|
||||||
ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build();
|
|
||||||
do {
|
|
||||||
hasMore = scanner.next(cells, scannerContext);
|
|
||||||
for (Cell cell : cells) {
|
|
||||||
writer.append(cell);
|
|
||||||
}
|
|
||||||
cells.clear();
|
|
||||||
} while (hasMore);
|
|
||||||
} finally {
|
|
||||||
scanner.close();
|
|
||||||
if (writer != null) {
|
|
||||||
try {
|
|
||||||
writer.close();
|
|
||||||
} catch (IOException e) {
|
|
||||||
LOG.error("Failed to close the writer of the file " + filePath, e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// commit the new del file
|
|
||||||
Path path = MobUtils.commitFile(conf, fs, filePath, mobFamilyDir, compactionCacheConfig);
|
|
||||||
// archive the old del files
|
|
||||||
try {
|
|
||||||
MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), delFiles);
|
|
||||||
} catch (IOException e) {
|
|
||||||
LOG.error("Failed to archive the old del files " + delFiles, e);
|
|
||||||
}
|
|
||||||
return path;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Creates a store scanner.
|
|
||||||
* @param filesToCompact The files to be compacted.
|
|
||||||
* @param scanType The scan type.
|
|
||||||
* @return The store scanner.
|
|
||||||
* @throws IOException if IO failure is encountered
|
|
||||||
*/
|
|
||||||
private StoreScanner createScanner(List<HStoreFile> filesToCompact, ScanType scanType)
|
|
||||||
throws IOException {
|
|
||||||
List<StoreFileScanner> scanners = StoreFileScanner.getScannersForStoreFiles(filesToCompact,
|
|
||||||
false, true, false, false, HConstants.LATEST_TIMESTAMP);
|
|
||||||
long ttl = HStore.determineTTLFromFamily(column);
|
|
||||||
ScanInfo scanInfo = new ScanInfo(conf, column, ttl, 0, CellComparator.getInstance());
|
|
||||||
return new StoreScanner(scanInfo, scanType, scanners);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Bulkloads the current file.
|
|
||||||
* @param tableName The table to load into.
|
|
||||||
* @param bulkloadDirectory The path of bulkload directory.
|
|
||||||
* @param fileName The current file name.
|
|
||||||
* @throws IOException if IO failure is encountered
|
|
||||||
*/
|
|
||||||
private void bulkloadRefFile(TableName tableName, Path bulkloadDirectory, String fileName)
|
|
||||||
throws IOException {
|
|
||||||
// bulkload the ref file
|
|
||||||
try {
|
|
||||||
BulkLoadHFiles bulkLoader = BulkLoadHFiles.create(conf);
|
|
||||||
bulkLoader.disableReplication();
|
|
||||||
bulkLoader.bulkLoad(tableName, bulkloadDirectory);
|
|
||||||
} catch (Exception e) {
|
|
||||||
throw new IOException(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Closes the mob file writer.
|
|
||||||
* @param writer The mob file writer.
|
|
||||||
* @param maxSeqId Maximum sequence id.
|
|
||||||
* @param mobCellsCount The number of mob cells.
|
|
||||||
* @throws IOException if IO failure is encountered
|
|
||||||
*/
|
|
||||||
private void closeMobFileWriter(StoreFileWriter writer, long maxSeqId, long mobCellsCount)
|
|
||||||
throws IOException {
|
|
||||||
if (writer != null) {
|
|
||||||
writer.appendMetadata(maxSeqId, false, mobCellsCount);
|
|
||||||
try {
|
|
||||||
writer.close();
|
|
||||||
} catch (IOException e) {
|
|
||||||
LOG.error("Failed to close the writer of the file " + writer.getPath(), e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Closes the ref file writer.
|
|
||||||
* @param writer The ref file writer.
|
|
||||||
* @param maxSeqId Maximum sequence id.
|
|
||||||
* @param bulkloadTime The timestamp at which the bulk load file is created.
|
|
||||||
* @throws IOException if IO failure is encountered
|
|
||||||
*/
|
|
||||||
private void closeRefFileWriter(StoreFileWriter writer, long maxSeqId, long bulkloadTime)
|
|
||||||
throws IOException {
|
|
||||||
if (writer != null) {
|
|
||||||
writer.appendMetadata(maxSeqId, false);
|
|
||||||
writer.appendFileInfo(BULKLOAD_TIME_KEY, Bytes.toBytes(bulkloadTime));
|
|
||||||
writer.appendFileInfo(SKIP_RESET_SEQ_ID, Bytes.toBytes(true));
|
|
||||||
try {
|
|
||||||
writer.close();
|
|
||||||
} catch (IOException e) {
|
|
||||||
LOG.error("Failed to close the writer of the ref file " + writer.getPath(), e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Gets the max seqId and number of cells of the store files.
|
|
||||||
* @param storeFiles The store files.
|
|
||||||
* @return The pair of the max seqId and number of cells of the store files.
|
|
||||||
* @throws IOException if IO failure is encountered
|
|
||||||
*/
|
|
||||||
private Pair<Long, Long> getFileInfo(List<HStoreFile> storeFiles) throws IOException {
|
|
||||||
long maxSeqId = 0;
|
|
||||||
long maxKeyCount = 0;
|
|
||||||
for (HStoreFile sf : storeFiles) {
|
|
||||||
// the readers will be closed later after the merge.
|
|
||||||
maxSeqId = Math.max(maxSeqId, sf.getMaxSequenceId());
|
|
||||||
sf.initReader();
|
|
||||||
byte[] count = sf.getReader().loadFileInfo().get(MOB_CELLS_COUNT);
|
|
||||||
if (count != null) {
|
|
||||||
maxKeyCount += Bytes.toLong(count);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return new Pair<>(maxSeqId, maxKeyCount);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Deletes a file.
|
|
||||||
* @param path The path of the file to be deleted.
|
|
||||||
*/
|
|
||||||
private void deletePath(Path path) {
|
|
||||||
LOG.debug("Cleanup, delete path '{}'", path);
|
|
||||||
try {
|
|
||||||
if (path != null) {
|
|
||||||
fs.delete(path, true);
|
|
||||||
}
|
|
||||||
} catch (IOException e) {
|
|
||||||
LOG.error("Failed to delete the file " + path, e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private FileStatus getLinkedFileStatus(HFileLink link) throws IOException {
|
|
||||||
Path[] locations = link.getLocations();
|
|
||||||
FileStatus file;
|
|
||||||
for (Path location : locations) {
|
|
||||||
|
|
||||||
if (location != null) {
|
|
||||||
try {
|
|
||||||
file = fs.getFileStatus(location);
|
|
||||||
if (file != null) {
|
|
||||||
return file;
|
|
||||||
}
|
|
||||||
} catch (FileNotFoundException e) {
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
LOG.warn("The file " + link + " links to can not be found");
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -193,28 +193,6 @@ public class HMobStore extends HStore {
|
||||||
isCompaction);
|
isCompaction);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Creates the writer for the del file in temp directory.
|
|
||||||
* The del file keeps tracking the delete markers. Its name has a suffix _del,
|
|
||||||
* the format is [0-9a-f]+(_del)?.
|
|
||||||
* @param date The latest date of written cells.
|
|
||||||
* @param maxKeyCount The key count.
|
|
||||||
* @param compression The compression algorithm.
|
|
||||||
* @param startKey The start key.
|
|
||||||
* @return The writer for the del file.
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
public StoreFileWriter createDelFileWriterInTmp(Date date, long maxKeyCount,
|
|
||||||
Compression.Algorithm compression, byte[] startKey) throws IOException {
|
|
||||||
if (startKey == null) {
|
|
||||||
startKey = HConstants.EMPTY_START_ROW;
|
|
||||||
}
|
|
||||||
Path path = getTempDir();
|
|
||||||
String suffix = UUID
|
|
||||||
.randomUUID().toString().replaceAll("-", "") + "_del";
|
|
||||||
MobFileName mobFileName = MobFileName.create(startKey, MobUtils.formatDate(date), suffix);
|
|
||||||
return createWriterInTmp(mobFileName, path, maxKeyCount, compression, true);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates the writer for the mob file in temp directory.
|
* Creates the writer for the mob file in temp directory.
|
||||||
|
@ -231,7 +209,7 @@ public class HMobStore extends HStore {
|
||||||
Compression.Algorithm compression, byte[] startKey,
|
Compression.Algorithm compression, byte[] startKey,
|
||||||
boolean isCompaction) throws IOException {
|
boolean isCompaction) throws IOException {
|
||||||
MobFileName mobFileName = MobFileName.create(startKey, date, UUID.randomUUID()
|
MobFileName mobFileName = MobFileName.create(startKey, date, UUID.randomUUID()
|
||||||
.toString().replaceAll("-", ""));
|
.toString().replaceAll("-", ""), region.getRegionInfo().getEncodedName());
|
||||||
return createWriterInTmp(mobFileName, basePath, maxKeyCount, compression, isCompaction);
|
return createWriterInTmp(mobFileName, basePath, maxKeyCount, compression, isCompaction);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -265,8 +243,7 @@ public class HMobStore extends HStore {
|
||||||
}
|
}
|
||||||
Path dstPath = new Path(targetPath, sourceFile.getName());
|
Path dstPath = new Path(targetPath, sourceFile.getName());
|
||||||
validateMobFile(sourceFile);
|
validateMobFile(sourceFile);
|
||||||
String msg = "Renaming flushed file from " + sourceFile + " to " + dstPath;
|
LOG.info(" FLUSH Renaming flushed file from {} to {}", sourceFile, dstPath);
|
||||||
LOG.info(msg);
|
|
||||||
Path parent = dstPath.getParent();
|
Path parent = dstPath.getParent();
|
||||||
if (!region.getFilesystem().exists(parent)) {
|
if (!region.getFilesystem().exists(parent)) {
|
||||||
region.getFilesystem().mkdirs(parent);
|
region.getFilesystem().mkdirs(parent);
|
||||||
|
@ -309,6 +286,20 @@ public class HMobStore extends HStore {
|
||||||
return resolve(reference, cacheBlocks, -1, true);
|
return resolve(reference, cacheBlocks, -1, true);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Reads the cell from the mob file with readEmptyValueOnMobCellMiss
|
||||||
|
* @param reference The cell found in the HBase, its value is a path to a mob file.
|
||||||
|
* @param cacheBlocks Whether the scanner should cache blocks.
|
||||||
|
* @param readEmptyValueOnMobCellMiss should return empty mob cell if reference
|
||||||
|
* can not be resolved.
|
||||||
|
* @return The cell found in the mob file.
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public MobCell resolve(Cell reference, boolean cacheBlocks, boolean readEmptyValueOnMobCellMiss)
|
||||||
|
throws IOException {
|
||||||
|
return resolve(reference, cacheBlocks, -1, readEmptyValueOnMobCellMiss);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Reads the cell from the mob file.
|
* Reads the cell from the mob file.
|
||||||
* @param reference The cell found in the HBase, its value is a path to a mob file.
|
* @param reference The cell found in the HBase, its value is a path to a mob file.
|
||||||
|
@ -520,4 +511,6 @@ public class HMobStore extends HStore {
|
||||||
public byte[] getRefCellTags() {
|
public byte[] getRefCellTags() {
|
||||||
return this.refCellTags;
|
return this.refCellTags;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -1901,7 +1901,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
}
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
void setTableDescriptor(TableDescriptor desc) {
|
public void setTableDescriptor(TableDescriptor desc) {
|
||||||
htableDescriptor = desc;
|
htableDescriptor = desc;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -1973,7 +1973,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
|
||||||
finishCompactionRequest(compaction.getRequest());
|
finishCompactionRequest(compaction.getRequest());
|
||||||
}
|
}
|
||||||
|
|
||||||
private void finishCompactionRequest(CompactionRequestImpl cr) {
|
protected void finishCompactionRequest(CompactionRequestImpl cr) {
|
||||||
this.region.reportCompactionRequestEnd(cr.isMajor(), cr.getFiles().size(), cr.getSize());
|
this.region.reportCompactionRequestEnd(cr.isMajor(), cr.getFiles().size(), cr.getSize());
|
||||||
if (cr.isOffPeak()) {
|
if (cr.isOffPeak()) {
|
||||||
offPeakCompactionTracker.set(false);
|
offPeakCompactionTracker.set(false);
|
||||||
|
|
|
@ -107,6 +107,12 @@ public class HStoreFile implements StoreFile {
|
||||||
/** Key for the number of mob cells in metadata */
|
/** Key for the number of mob cells in metadata */
|
||||||
public static final byte[] MOB_CELLS_COUNT = Bytes.toBytes("MOB_CELLS_COUNT");
|
public static final byte[] MOB_CELLS_COUNT = Bytes.toBytes("MOB_CELLS_COUNT");
|
||||||
|
|
||||||
|
/** Null data */
|
||||||
|
public static final byte[] NULL_VALUE = new byte[] {0};
|
||||||
|
|
||||||
|
/** Key for the list of MOB file references */
|
||||||
|
public static final byte[] MOB_FILE_REFS = Bytes.toBytes("MOB_FILE_REFS");
|
||||||
|
|
||||||
/** Meta key set when store file is a result of a bulk load */
|
/** Meta key set when store file is a result of a bulk load */
|
||||||
public static final byte[] BULKLOAD_TASK_KEY = Bytes.toBytes("BULKLOAD_SOURCE_TASK");
|
public static final byte[] BULKLOAD_TASK_KEY = Bytes.toBytes("BULKLOAD_SOURCE_TASK");
|
||||||
public static final byte[] BULKLOAD_TIME_KEY = Bytes.toBytes("BULKLOAD_TIMESTAMP");
|
public static final byte[] BULKLOAD_TIME_KEY = Bytes.toBytes("BULKLOAD_TIMESTAMP");
|
||||||
|
|
|
@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.io.hfile.HFileInfo;
|
||||||
import org.apache.hadoop.hbase.io.hfile.ReaderContext;
|
import org.apache.hadoop.hbase.io.hfile.ReaderContext;
|
||||||
import org.apache.hadoop.hbase.io.hfile.ReaderContext.ReaderType;
|
import org.apache.hadoop.hbase.io.hfile.ReaderContext.ReaderType;
|
||||||
import org.apache.hadoop.hbase.io.hfile.ReaderContextBuilder;
|
import org.apache.hadoop.hbase.io.hfile.ReaderContextBuilder;
|
||||||
|
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
|
@ -52,36 +53,25 @@ public class StoreFileInfo {
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(StoreFileInfo.class);
|
private static final Logger LOG = LoggerFactory.getLogger(StoreFileInfo.class);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A non-capture group, for hfiles, so that this can be embedded.
|
* A non-capture group, for hfiles, so that this can be embedded. HFiles are uuid ([0-9a-z]+).
|
||||||
* HFiles are uuid ([0-9a-z]+). Bulk loaded hfiles has (_SeqId_[0-9]+_) has suffix.
|
* Bulk loaded hfiles has (_SeqId_[0-9]+_) has suffix. The mob del file has (_del) as suffix.
|
||||||
* The mob del file has (_del) as suffix.
|
|
||||||
*/
|
*/
|
||||||
public static final String HFILE_NAME_REGEX = "[0-9a-f]+(?:(?:_SeqId_[0-9]+_)|(?:_del))?";
|
public static final String HFILE_NAME_REGEX = "[0-9a-f]+(?:(?:_SeqId_[0-9]+_)|(?:_del))?";
|
||||||
|
|
||||||
/** Regex that will work for hfiles */
|
/** Regex that will work for hfiles */
|
||||||
private static final Pattern HFILE_NAME_PATTERN =
|
private static final Pattern HFILE_NAME_PATTERN = Pattern.compile("^(" + HFILE_NAME_REGEX + ")");
|
||||||
Pattern.compile("^(" + HFILE_NAME_REGEX + ")");
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A non-capture group, for del files, so that this can be embedded.
|
* Regex that will work for straight reference names ({@code <hfile>.<parentEncRegion>}) and
|
||||||
* A del file has (_del) as suffix.
|
* hfilelink reference names ({@code
|
||||||
|
* <table>
|
||||||
|
* =<region>-<hfile>.<parentEncRegion>}) If reference, then the regex has more than just one
|
||||||
|
* group. Group 1, hfile/hfilelink pattern, is this file's id. Group 2 '(.+)' is the reference's
|
||||||
|
* parent region name.
|
||||||
*/
|
*/
|
||||||
public static final String DELFILE_NAME_REGEX = "[0-9a-f]+(?:_del)";
|
private static final Pattern REF_NAME_PATTERN = Pattern
|
||||||
|
.compile(String.format("^(%s|%s)\\.(.+)$", HFILE_NAME_REGEX, HFileLink.LINK_NAME_REGEX));
|
||||||
/** Regex that will work for del files */
|
|
||||||
private static final Pattern DELFILE_NAME_PATTERN =
|
|
||||||
Pattern.compile("^(" + DELFILE_NAME_REGEX + ")");
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Regex that will work for straight reference names ({@code <hfile>.<parentEncRegion>})
|
|
||||||
* and hfilelink reference names ({@code <table>=<region>-<hfile>.<parentEncRegion>})
|
|
||||||
* If reference, then the regex has more than just one group.
|
|
||||||
* Group 1, hfile/hfilelink pattern, is this file's id.
|
|
||||||
* Group 2 '(.+)' is the reference's parent region name.
|
|
||||||
*/
|
|
||||||
private static final Pattern REF_NAME_PATTERN =
|
|
||||||
Pattern.compile(String.format("^(%s|%s)\\.(.+)$",
|
|
||||||
HFILE_NAME_REGEX, HFileLink.LINK_NAME_REGEX));
|
|
||||||
|
|
||||||
public static final String STORE_FILE_READER_NO_READAHEAD = "hbase.store.reader.no-readahead";
|
public static final String STORE_FILE_READER_NO_READAHEAD = "hbase.store.reader.no-readahead";
|
||||||
public static final boolean DEFAULT_STORE_FILE_READER_NO_READAHEAD = false;
|
public static final boolean DEFAULT_STORE_FILE_READER_NO_READAHEAD = false;
|
||||||
|
@ -144,7 +134,7 @@ public class StoreFileInfo {
|
||||||
this.initialPath = initialPath;
|
this.initialPath = initialPath;
|
||||||
this.primaryReplica = primaryReplica;
|
this.primaryReplica = primaryReplica;
|
||||||
this.noReadahead = this.conf.getBoolean(STORE_FILE_READER_NO_READAHEAD,
|
this.noReadahead = this.conf.getBoolean(STORE_FILE_READER_NO_READAHEAD,
|
||||||
DEFAULT_STORE_FILE_READER_NO_READAHEAD);
|
DEFAULT_STORE_FILE_READER_NO_READAHEAD);
|
||||||
Path p = initialPath;
|
Path p = initialPath;
|
||||||
if (HFileLink.isHFileLink(p)) {
|
if (HFileLink.isHFileLink(p)) {
|
||||||
// HFileLink
|
// HFileLink
|
||||||
|
@ -162,7 +152,7 @@ public class StoreFileInfo {
|
||||||
this.link = null;
|
this.link = null;
|
||||||
}
|
}
|
||||||
LOG.trace("{} is a {} reference to {}", p, reference.getFileRegion(), referencePath);
|
LOG.trace("{} is a {} reference to {}", p, reference.getFileRegion(), referencePath);
|
||||||
} else if (isHFile(p)) {
|
} else if (isHFile(p) || isMobFile(p) || isMobRefFile(p)) {
|
||||||
// HFile
|
// HFile
|
||||||
if (fileStatus != null) {
|
if (fileStatus != null) {
|
||||||
this.createdTimestamp = fileStatus.getModificationTime();
|
this.createdTimestamp = fileStatus.getModificationTime();
|
||||||
|
@ -227,11 +217,11 @@ public class StoreFileInfo {
|
||||||
this.conf = conf;
|
this.conf = conf;
|
||||||
this.primaryReplica = false;
|
this.primaryReplica = false;
|
||||||
this.initialPath = (fileStatus == null) ? null : fileStatus.getPath();
|
this.initialPath = (fileStatus == null) ? null : fileStatus.getPath();
|
||||||
this.createdTimestamp = (fileStatus == null) ? 0 :fileStatus.getModificationTime();
|
this.createdTimestamp = (fileStatus == null) ? 0 : fileStatus.getModificationTime();
|
||||||
this.reference = reference;
|
this.reference = reference;
|
||||||
this.link = link;
|
this.link = link;
|
||||||
this.noReadahead = this.conf.getBoolean(STORE_FILE_READER_NO_READAHEAD,
|
this.noReadahead = this.conf.getBoolean(STORE_FILE_READER_NO_READAHEAD,
|
||||||
DEFAULT_STORE_FILE_READER_NO_READAHEAD);
|
DEFAULT_STORE_FILE_READER_NO_READAHEAD);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -250,9 +240,10 @@ public class StoreFileInfo {
|
||||||
this.coprocessorHost = coprocessorHost;
|
this.coprocessorHost = coprocessorHost;
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
/**
|
||||||
* @return the Reference object associated to this StoreFileInfo.
|
* @return the Reference object associated to this StoreFileInfo.
|
||||||
* null if the StoreFile is not a reference.
|
* null if the StoreFile is not a
|
||||||
|
* reference.
|
||||||
*/
|
*/
|
||||||
public Reference getReference() {
|
public Reference getReference() {
|
||||||
return this.reference;
|
return this.reference;
|
||||||
|
@ -278,8 +269,7 @@ public class StoreFileInfo {
|
||||||
return this.hdfsBlocksDistribution;
|
return this.hdfsBlocksDistribution;
|
||||||
}
|
}
|
||||||
|
|
||||||
StoreFileReader createReader(ReaderContext context, CacheConfig cacheConf)
|
StoreFileReader createReader(ReaderContext context, CacheConfig cacheConf) throws IOException {
|
||||||
throws IOException {
|
|
||||||
StoreFileReader reader = null;
|
StoreFileReader reader = null;
|
||||||
if (this.reference != null) {
|
if (this.reference != null) {
|
||||||
reader = new HalfStoreFileReader(context, hfileInfo, cacheConf, reference, refCount, conf);
|
reader = new HalfStoreFileReader(context, hfileInfo, cacheConf, reference, refCount, conf);
|
||||||
|
@ -316,12 +306,9 @@ public class StoreFileInfo {
|
||||||
status = fs.getFileStatus(initialPath);
|
status = fs.getFileStatus(initialPath);
|
||||||
}
|
}
|
||||||
long length = status.getLen();
|
long length = status.getLen();
|
||||||
ReaderContextBuilder contextBuilder = new ReaderContextBuilder()
|
ReaderContextBuilder contextBuilder =
|
||||||
.withInputStreamWrapper(in)
|
new ReaderContextBuilder().withInputStreamWrapper(in).withFileSize(length)
|
||||||
.withFileSize(length)
|
.withPrimaryReplicaReader(this.primaryReplica).withReaderType(type).withFileSystem(fs);
|
||||||
.withPrimaryReplicaReader(this.primaryReplica)
|
|
||||||
.withReaderType(type)
|
|
||||||
.withFileSystem(fs);
|
|
||||||
if (this.reference != null) {
|
if (this.reference != null) {
|
||||||
contextBuilder.withFilePath(this.getPath());
|
contextBuilder.withFilePath(this.getPath());
|
||||||
} else {
|
} else {
|
||||||
|
@ -425,8 +412,8 @@ public class StoreFileInfo {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return this.getPath() +
|
return this.getPath()
|
||||||
(isReference() ? "->" + getReferredToFile(this.getPath()) + "-" + reference : "");
|
+ (isReference() ? "->" + getReferredToFile(this.getPath()) + "-" + reference : "");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -443,22 +430,42 @@ public class StoreFileInfo {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param path Path to check.
|
* Checks if the file is a MOB file
|
||||||
* @return True if the path has format of a del file.
|
* @param path path to a file
|
||||||
|
* @return true, if - yes, false otherwise
|
||||||
*/
|
*/
|
||||||
public static boolean isDelFile(final Path path) {
|
public static boolean isMobFile(final Path path) {
|
||||||
return isDelFile(path.getName());
|
String fileName = path.getName();
|
||||||
|
String[] parts = fileName.split(MobUtils.SEP);
|
||||||
|
if (parts.length != 2) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
Matcher m = HFILE_NAME_PATTERN.matcher(parts[0]);
|
||||||
|
Matcher mm = HFILE_NAME_PATTERN.matcher(parts[1]);
|
||||||
|
return m.matches() && mm.matches();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param fileName Sting version of path to validate.
|
* Checks if the file is a MOB reference file,
|
||||||
* @return True if the file name has format of a del file.
|
* created by snapshot
|
||||||
|
* @param path path to a file
|
||||||
|
* @return true, if - yes, false otherwise
|
||||||
*/
|
*/
|
||||||
public static boolean isDelFile(final String fileName) {
|
public static boolean isMobRefFile(final Path path) {
|
||||||
Matcher m = DELFILE_NAME_PATTERN.matcher(fileName);
|
String fileName = path.getName();
|
||||||
return m.matches() && m.groupCount() > 0;
|
int lastIndex = fileName.lastIndexOf(MobUtils.SEP);
|
||||||
|
if (lastIndex < 0) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
String[] parts = new String[2];
|
||||||
|
parts[0] = fileName.substring(0, lastIndex);
|
||||||
|
parts[1] = fileName.substring(lastIndex + 1);
|
||||||
|
String name = parts[0] + "." + parts[1];
|
||||||
|
Matcher m = REF_NAME_PATTERN.matcher(name);
|
||||||
|
return m.matches() && m.groupCount() > 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param path Path to check.
|
* @param path Path to check.
|
||||||
* @return True if the path has format of a HStoreFile reference.
|
* @return True if the path has format of a HStoreFile reference.
|
||||||
|
@ -484,8 +491,8 @@ public class StoreFileInfo {
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Return path to the file referred to by a Reference. Presumes a directory
|
* Return path to the file referred to by a Reference. Presumes a directory hierarchy of
|
||||||
* hierarchy of <code>${hbase.rootdir}/data/${namespace}/tablename/regionname/familyname</code>.
|
* <code>${hbase.rootdir}/data/${namespace}/tablename/regionname/familyname</code>.
|
||||||
* @param p Path to a Reference file.
|
* @param p Path to a Reference file.
|
||||||
* @return Calculated path to parent region file.
|
* @return Calculated path to parent region file.
|
||||||
* @throws IllegalArgumentException when path regex fails to match.
|
* @throws IllegalArgumentException when path regex fails to match.
|
||||||
|
@ -494,8 +501,7 @@ public class StoreFileInfo {
|
||||||
Matcher m = REF_NAME_PATTERN.matcher(p.getName());
|
Matcher m = REF_NAME_PATTERN.matcher(p.getName());
|
||||||
if (m == null || !m.matches()) {
|
if (m == null || !m.matches()) {
|
||||||
LOG.warn("Failed match of store file name {}", p.toString());
|
LOG.warn("Failed match of store file name {}", p.toString());
|
||||||
throw new IllegalArgumentException("Failed match of store file name " +
|
throw new IllegalArgumentException("Failed match of store file name " + p.toString());
|
||||||
p.toString());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Other region name is suffix on the passed Reference file name
|
// Other region name is suffix on the passed Reference file name
|
||||||
|
@ -506,9 +512,9 @@ public class StoreFileInfo {
|
||||||
LOG.trace("reference {} to region={} hfile={}", p, otherRegion, nameStrippedOfSuffix);
|
LOG.trace("reference {} to region={} hfile={}", p, otherRegion, nameStrippedOfSuffix);
|
||||||
|
|
||||||
// Build up new path with the referenced region in place of our current
|
// Build up new path with the referenced region in place of our current
|
||||||
// region in the reference path. Also strip regionname suffix from name.
|
// region in the reference path. Also strip regionname suffix from name.
|
||||||
return new Path(new Path(new Path(tableDir, otherRegion),
|
return new Path(new Path(new Path(tableDir, otherRegion), p.getParent().getName()),
|
||||||
p.getParent().getName()), nameStrippedOfSuffix);
|
nameStrippedOfSuffix);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -517,8 +523,9 @@ public class StoreFileInfo {
|
||||||
* @return <tt>true</tt> if the file could be a valid store file, <tt>false</tt> otherwise
|
* @return <tt>true</tt> if the file could be a valid store file, <tt>false</tt> otherwise
|
||||||
*/
|
*/
|
||||||
public static boolean validateStoreFileName(final String fileName) {
|
public static boolean validateStoreFileName(final String fileName) {
|
||||||
if (HFileLink.isHFileLink(fileName) || isReference(fileName))
|
if (HFileLink.isHFileLink(fileName) || isReference(fileName)) {
|
||||||
return(true);
|
return true;
|
||||||
|
}
|
||||||
return !fileName.contains("-");
|
return !fileName.contains("-");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -527,12 +534,12 @@ public class StoreFileInfo {
|
||||||
* @param fileStatus The {@link FileStatus} of the file
|
* @param fileStatus The {@link FileStatus} of the file
|
||||||
* @return <tt>true</tt> if the file is valid
|
* @return <tt>true</tt> if the file is valid
|
||||||
*/
|
*/
|
||||||
public static boolean isValid(final FileStatus fileStatus)
|
public static boolean isValid(final FileStatus fileStatus) throws IOException {
|
||||||
throws IOException {
|
|
||||||
final Path p = fileStatus.getPath();
|
final Path p = fileStatus.getPath();
|
||||||
|
|
||||||
if (fileStatus.isDirectory())
|
if (fileStatus.isDirectory()) {
|
||||||
return false;
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
// Check for empty hfile. Should never be the case but can happen
|
// Check for empty hfile. Should never be the case but can happen
|
||||||
// after data loss in hdfs for whatever reason (upgrade, etc.): HBASE-646
|
// after data loss in hdfs for whatever reason (upgrade, etc.): HBASE-646
|
||||||
|
@ -546,21 +553,19 @@ public class StoreFileInfo {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* helper function to compute HDFS blocks distribution of a given reference
|
* helper function to compute HDFS blocks distribution of a given reference file.For reference
|
||||||
* file.For reference file, we don't compute the exact value. We use some
|
* file, we don't compute the exact value. We use some estimate instead given it might be good
|
||||||
* estimate instead given it might be good enough. we assume bottom part
|
* enough. we assume bottom part takes the first half of reference file, top part takes the second
|
||||||
* takes the first half of reference file, top part takes the second half
|
* half of the reference file. This is just estimate, given midkey ofregion != midkey of HFile,
|
||||||
* of the reference file. This is just estimate, given
|
* also the number and size of keys vary. If this estimate isn't good enough, we can improve it
|
||||||
* midkey ofregion != midkey of HFile, also the number and size of keys vary.
|
* later.
|
||||||
* If this estimate isn't good enough, we can improve it later.
|
* @param fs The FileSystem
|
||||||
* @param fs The FileSystem
|
* @param reference The reference
|
||||||
* @param reference The reference
|
* @param status The reference FileStatus
|
||||||
* @param status The reference FileStatus
|
|
||||||
* @return HDFS blocks distribution
|
* @return HDFS blocks distribution
|
||||||
*/
|
*/
|
||||||
private static HDFSBlocksDistribution computeRefFileHDFSBlockDistribution(
|
private static HDFSBlocksDistribution computeRefFileHDFSBlockDistribution(final FileSystem fs,
|
||||||
final FileSystem fs, final Reference reference, final FileStatus status)
|
final Reference reference, final FileStatus status) throws IOException {
|
||||||
throws IOException {
|
|
||||||
if (status == null) {
|
if (status == null) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
@ -569,36 +574,59 @@ public class StoreFileInfo {
|
||||||
long length = 0;
|
long length = 0;
|
||||||
|
|
||||||
if (Reference.isTopFileRegion(reference.getFileRegion())) {
|
if (Reference.isTopFileRegion(reference.getFileRegion())) {
|
||||||
start = status.getLen()/2;
|
start = status.getLen() / 2;
|
||||||
length = status.getLen() - status.getLen()/2;
|
length = status.getLen() - status.getLen() / 2;
|
||||||
} else {
|
} else {
|
||||||
start = 0;
|
start = 0;
|
||||||
length = status.getLen()/2;
|
length = status.getLen() / 2;
|
||||||
}
|
}
|
||||||
return FSUtils.computeHDFSBlocksDistribution(fs, status, start, length);
|
return FSUtils.computeHDFSBlocksDistribution(fs, status, start, length);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean equals(Object that) {
|
public boolean equals(Object that) {
|
||||||
if (this == that) return true;
|
if (this == that) {
|
||||||
if (that == null) return false;
|
return true;
|
||||||
|
}
|
||||||
|
if (that == null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
if (!(that instanceof StoreFileInfo)) return false;
|
if (!(that instanceof StoreFileInfo)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
StoreFileInfo o = (StoreFileInfo)that;
|
StoreFileInfo o = (StoreFileInfo) that;
|
||||||
if (initialPath != null && o.initialPath == null) return false;
|
if (initialPath != null && o.initialPath == null) {
|
||||||
if (initialPath == null && o.initialPath != null) return false;
|
return false;
|
||||||
|
}
|
||||||
|
if (initialPath == null && o.initialPath != null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
if (initialPath != o.initialPath && initialPath != null
|
if (initialPath != o.initialPath && initialPath != null
|
||||||
&& !initialPath.equals(o.initialPath)) return false;
|
&& !initialPath.equals(o.initialPath)) {
|
||||||
|
return false;
|
||||||
if (reference != null && o.reference == null) return false;
|
}
|
||||||
if (reference == null && o.reference != null) return false;
|
if (reference != null && o.reference == null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (reference == null && o.reference != null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
if (reference != o.reference && reference != null
|
if (reference != o.reference && reference != null
|
||||||
&& !reference.equals(o.reference)) return false;
|
&& !reference.equals(o.reference)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
if (link != null && o.link == null) return false;
|
if (link != null && o.link == null) {
|
||||||
if (link == null && o.link != null) return false;
|
return false;
|
||||||
if (link != o.link && link != null && !link.equals(o.link)) return false;
|
}
|
||||||
|
if (link == null && o.link != null) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (link != o.link && link != null && !link.equals(o.link)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
@ -607,9 +635,9 @@ public class StoreFileInfo {
|
||||||
public int hashCode() {
|
public int hashCode() {
|
||||||
int hash = 17;
|
int hash = 17;
|
||||||
hash = hash * 31 + ((reference == null) ? 0 : reference.hashCode());
|
hash = hash * 31 + ((reference == null) ? 0 : reference.hashCode());
|
||||||
hash = hash * 31 + ((initialPath == null) ? 0 : initialPath.hashCode());
|
hash = hash * 31 + ((initialPath == null) ? 0 : initialPath.hashCode());
|
||||||
hash = hash * 31 + ((link == null) ? 0 : link.hashCode());
|
hash = hash * 31 + ((link == null) ? 0 : link.hashCode());
|
||||||
return hash;
|
return hash;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -652,8 +680,7 @@ public class StoreFileInfo {
|
||||||
StoreFileReader reader = null;
|
StoreFileReader reader = null;
|
||||||
if (this.coprocessorHost != null) {
|
if (this.coprocessorHost != null) {
|
||||||
reader = this.coprocessorHost.preStoreFileReaderOpen(fs, this.getPath(),
|
reader = this.coprocessorHost.preStoreFileReaderOpen(fs, this.getPath(),
|
||||||
context.getInputStreamWrapper(), context.getFileSize(),
|
context.getInputStreamWrapper(), context.getFileSize(), cacheConf, reference);
|
||||||
cacheConf, reference);
|
|
||||||
}
|
}
|
||||||
return reader;
|
return reader;
|
||||||
}
|
}
|
||||||
|
@ -663,8 +690,7 @@ public class StoreFileInfo {
|
||||||
StoreFileReader res = reader;
|
StoreFileReader res = reader;
|
||||||
if (this.coprocessorHost != null) {
|
if (this.coprocessorHost != null) {
|
||||||
res = this.coprocessorHost.postStoreFileReaderOpen(fs, this.getPath(),
|
res = this.coprocessorHost.postStoreFileReaderOpen(fs, this.getPath(),
|
||||||
context.getInputStreamWrapper(), context.getFileSize(),
|
context.getInputStreamWrapper(), context.getFileSize(), cacheConf, reference, reader);
|
||||||
cacheConf, reference, reader);
|
|
||||||
}
|
}
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,6 +25,8 @@ import static org.apache.hadoop.hbase.regionserver.HStoreFile.EARLIEST_PUT_TS;
|
||||||
import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY;
|
import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY;
|
||||||
import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAX_SEQ_ID_KEY;
|
import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAX_SEQ_ID_KEY;
|
||||||
import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_CELLS_COUNT;
|
import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_CELLS_COUNT;
|
||||||
|
import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_FILE_REFS;
|
||||||
|
import static org.apache.hadoop.hbase.regionserver.HStoreFile.NULL_VALUE;
|
||||||
import static org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY;
|
import static org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.net.InetSocketAddress;
|
import java.net.InetSocketAddress;
|
||||||
|
@ -36,6 +38,8 @@ import java.util.UUID;
|
||||||
import java.util.function.Supplier;
|
import java.util.function.Supplier;
|
||||||
import java.util.regex.Pattern;
|
import java.util.regex.Pattern;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
import org.apache.commons.lang3.StringUtils;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
|
@ -47,6 +51,7 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||||
|
import org.apache.hadoop.hbase.io.hfile.HFileWriterImpl;
|
||||||
import org.apache.hadoop.hbase.util.BloomContext;
|
import org.apache.hadoop.hbase.util.BloomContext;
|
||||||
import org.apache.hadoop.hbase.util.BloomFilterFactory;
|
import org.apache.hadoop.hbase.util.BloomFilterFactory;
|
||||||
import org.apache.hadoop.hbase.util.BloomFilterUtil;
|
import org.apache.hadoop.hbase.util.BloomFilterUtil;
|
||||||
|
@ -165,6 +170,9 @@ public class StoreFileWriter implements CellSink, ShipperListener {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public long getPos() throws IOException {
|
||||||
|
return ((HFileWriterImpl) writer).getPos();
|
||||||
|
}
|
||||||
/**
|
/**
|
||||||
* Writes meta data.
|
* Writes meta data.
|
||||||
* Call before {@link #close()} since its written as meta data to this file.
|
* Call before {@link #close()} since its written as meta data to this file.
|
||||||
|
@ -238,6 +246,21 @@ public class StoreFileWriter implements CellSink, ShipperListener {
|
||||||
appendTrackedTimestampsToMetadata();
|
appendTrackedTimestampsToMetadata();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Appends MOB - specific metadata (even if it is empty)
|
||||||
|
* @param mobRefSet - set of MOB file names
|
||||||
|
* @throws IOException problem writing to FS
|
||||||
|
*/
|
||||||
|
public void appendMobMetadata(Set<String> mobRefSet) throws IOException {
|
||||||
|
if (mobRefSet != null && mobRefSet.size() > 0) {
|
||||||
|
String sb = StringUtils.join(mobRefSet, ",");
|
||||||
|
byte[] bytes = Bytes.toBytes(sb.toString());
|
||||||
|
writer.appendFileInfo(MOB_FILE_REFS, bytes);
|
||||||
|
} else {
|
||||||
|
writer.appendFileInfo(MOB_FILE_REFS, NULL_VALUE);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Add TimestampRange and earliest put timestamp to Metadata
|
* Add TimestampRange and earliest put timestamp to Metadata
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -1141,7 +1141,7 @@ public abstract class FSUtils extends CommonFSUtils {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected boolean accept(Path p, @CheckForNull Boolean isDir) {
|
protected boolean accept(Path p, @CheckForNull Boolean isDir) {
|
||||||
if (!StoreFileInfo.isHFile(p)) {
|
if (!StoreFileInfo.isHFile(p) && !StoreFileInfo.isMobFile(p)) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -263,9 +263,9 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
|
||||||
byte[][] families = { Bytes.toBytes("mob") };
|
byte[][] families = { Bytes.toBytes("mob") };
|
||||||
loadData(tableName, families, 3000, 8);
|
loadData(tableName, families, 3000, 8);
|
||||||
|
|
||||||
admin.majorCompact(tableName, CompactType.MOB).get();
|
admin.majorCompact(tableName).get();
|
||||||
|
|
||||||
CompactionState state = admin.getCompactionState(tableName, CompactType.MOB).get();
|
CompactionState state = admin.getCompactionState(tableName).get();
|
||||||
assertNotEquals(CompactionState.NONE, state);
|
assertNotEquals(CompactionState.NONE, state);
|
||||||
|
|
||||||
waitUntilMobCompactionFinished(tableName);
|
waitUntilMobCompactionFinished(tableName);
|
||||||
|
|
|
@ -0,0 +1,370 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.hbase.mob;
|
||||||
|
|
||||||
|
import java.io.FileNotFoundException;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.InterruptedIOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Date;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Random;
|
||||||
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.hbase.Cell;
|
||||||
|
import org.apache.hadoop.hbase.KeyValue;
|
||||||
|
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||||
|
import org.apache.hadoop.hbase.io.hfile.CorruptHFileException;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.CellSink;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.ScannerContext;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.ShipperListener;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputControlUtil;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
|
||||||
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
/**
|
||||||
|
* This class is used for testing only. The main purpose is to emulate
|
||||||
|
* random failures during MOB compaction process.
|
||||||
|
* Example of usage:
|
||||||
|
* <pre>{@code
|
||||||
|
* public class SomeTest {
|
||||||
|
*
|
||||||
|
* public void initConfiguration(Configuration conf){
|
||||||
|
* conf.set(MobStoreEngine.DEFAULT_MOB_COMPACTOR_CLASS_KEY,
|
||||||
|
FaultyMobStoreCompactor.class.getName());
|
||||||
|
conf.setDouble("hbase.mob.compaction.fault.probability", 0.1);
|
||||||
|
* }
|
||||||
|
* }
|
||||||
|
* }</pre>
|
||||||
|
* @see org.apache.hadoop.hbase.mob.MobStressToolRunner on how to use and configure
|
||||||
|
* this class.
|
||||||
|
*
|
||||||
|
*/
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
public class FaultyMobStoreCompactor extends DefaultMobStoreCompactor {
|
||||||
|
|
||||||
|
private static final Logger LOG = LoggerFactory.getLogger(FaultyMobStoreCompactor.class);
|
||||||
|
|
||||||
|
public static AtomicLong mobCounter = new AtomicLong();
|
||||||
|
public static AtomicLong totalFailures = new AtomicLong();
|
||||||
|
public static AtomicLong totalCompactions = new AtomicLong();
|
||||||
|
public static AtomicLong totalMajorCompactions = new AtomicLong();
|
||||||
|
|
||||||
|
static double failureProb = 0.1d;
|
||||||
|
static Random rnd = new Random();
|
||||||
|
|
||||||
|
public FaultyMobStoreCompactor(Configuration conf, HStore store) {
|
||||||
|
super(conf, store);
|
||||||
|
failureProb = conf.getDouble("hbase.mob.compaction.fault.probability", 0.1);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected boolean performCompaction(FileDetails fd, InternalScanner scanner, CellSink writer,
|
||||||
|
long smallestReadPoint, boolean cleanSeqId, ThroughputController throughputController,
|
||||||
|
boolean major, int numofFilesToCompact) throws IOException {
|
||||||
|
|
||||||
|
totalCompactions.incrementAndGet();
|
||||||
|
if (major) {
|
||||||
|
totalMajorCompactions.incrementAndGet();
|
||||||
|
}
|
||||||
|
long bytesWrittenProgressForCloseCheck = 0;
|
||||||
|
long bytesWrittenProgressForLog = 0;
|
||||||
|
long bytesWrittenProgressForShippedCall = 0;
|
||||||
|
// Clear old mob references
|
||||||
|
mobRefSet.get().clear();
|
||||||
|
boolean isUserRequest = userRequest.get();
|
||||||
|
boolean compactMOBs = major && isUserRequest;
|
||||||
|
boolean discardMobMiss = conf.getBoolean(MobConstants.MOB_UNSAFE_DISCARD_MISS_KEY,
|
||||||
|
MobConstants.DEFAULT_MOB_DISCARD_MISS);
|
||||||
|
|
||||||
|
boolean mustFail = false;
|
||||||
|
if (compactMOBs) {
|
||||||
|
mobCounter.incrementAndGet();
|
||||||
|
double dv = rnd.nextDouble();
|
||||||
|
if (dv < failureProb) {
|
||||||
|
mustFail = true;
|
||||||
|
totalFailures.incrementAndGet();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
FileSystem fs = store.getFileSystem();
|
||||||
|
|
||||||
|
// Since scanner.next() can return 'false' but still be delivering data,
|
||||||
|
// we have to use a do/while loop.
|
||||||
|
List<Cell> cells = new ArrayList<>();
|
||||||
|
// Limit to "hbase.hstore.compaction.kv.max" (default 10) to avoid OOME
|
||||||
|
int closeCheckSizeLimit = HStore.getCloseCheckInterval();
|
||||||
|
long lastMillis = 0;
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
lastMillis = EnvironmentEdgeManager.currentTime();
|
||||||
|
}
|
||||||
|
String compactionName = ThroughputControlUtil.getNameForThrottling(store, "compaction");
|
||||||
|
long now = 0;
|
||||||
|
boolean hasMore;
|
||||||
|
Path path = MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName());
|
||||||
|
byte[] fileName = null;
|
||||||
|
StoreFileWriter mobFileWriter = null;
|
||||||
|
long mobCells = 0;
|
||||||
|
long cellsCountCompactedToMob = 0, cellsCountCompactedFromMob = 0;
|
||||||
|
long cellsSizeCompactedToMob = 0, cellsSizeCompactedFromMob = 0;
|
||||||
|
boolean finished = false;
|
||||||
|
|
||||||
|
ScannerContext scannerContext =
|
||||||
|
ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build();
|
||||||
|
throughputController.start(compactionName);
|
||||||
|
KeyValueScanner kvs = (scanner instanceof KeyValueScanner) ? (KeyValueScanner) scanner : null;
|
||||||
|
long shippedCallSizeLimit =
|
||||||
|
(long) numofFilesToCompact * this.store.getColumnFamilyDescriptor().getBlocksize();
|
||||||
|
|
||||||
|
Cell mobCell = null;
|
||||||
|
|
||||||
|
long counter = 0;
|
||||||
|
long countFailAt = -1;
|
||||||
|
if (mustFail) {
|
||||||
|
countFailAt = rnd.nextInt(100); // randomly fail fast
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
try {
|
||||||
|
mobFileWriter = mobStore.createWriterInTmp(new Date(fd.latestPutTs), fd.maxKeyCount,
|
||||||
|
compactionCompression, store.getRegionInfo().getStartKey(), true);
|
||||||
|
fileName = Bytes.toBytes(mobFileWriter.getPath().getName());
|
||||||
|
} catch (IOException e) {
|
||||||
|
// Bailing out
|
||||||
|
LOG.error("Failed to create mob writer, ", e);
|
||||||
|
throw e;
|
||||||
|
}
|
||||||
|
if (compactMOBs) {
|
||||||
|
// Add the only reference we get for compact MOB case
|
||||||
|
// because new store file will have only one MOB reference
|
||||||
|
// in this case - of newly compacted MOB file
|
||||||
|
mobRefSet.get().add(mobFileWriter.getPath().getName());
|
||||||
|
}
|
||||||
|
do {
|
||||||
|
hasMore = scanner.next(cells, scannerContext);
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
now = EnvironmentEdgeManager.currentTime();
|
||||||
|
}
|
||||||
|
for (Cell c : cells) {
|
||||||
|
counter++;
|
||||||
|
if (compactMOBs) {
|
||||||
|
if (MobUtils.isMobReferenceCell(c)) {
|
||||||
|
if (counter == countFailAt) {
|
||||||
|
LOG.warn("INJECTED FAULT mobCounter={}", mobCounter.get());
|
||||||
|
throw new CorruptHFileException("injected fault");
|
||||||
|
}
|
||||||
|
String fName = MobUtils.getMobFileName(c);
|
||||||
|
// Added to support migration
|
||||||
|
try {
|
||||||
|
mobCell = mobStore.resolve(c, true, false).getCell();
|
||||||
|
} catch (FileNotFoundException fnfe) {
|
||||||
|
if (discardMobMiss) {
|
||||||
|
LOG.error("Missing MOB cell: file={} not found", fName);
|
||||||
|
continue;
|
||||||
|
} else {
|
||||||
|
throw fnfe;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (discardMobMiss && mobCell.getValueLength() == 0) {
|
||||||
|
LOG.error("Missing MOB cell value: file={} cell={}", fName, mobCell);
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (mobCell.getValueLength() > mobSizeThreshold) {
|
||||||
|
// put the mob data back to the store file
|
||||||
|
PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId());
|
||||||
|
mobFileWriter.append(mobCell);
|
||||||
|
writer.append(
|
||||||
|
MobUtils.createMobRefCell(mobCell, fileName, this.mobStore.getRefCellTags()));
|
||||||
|
mobCells++;
|
||||||
|
} else {
|
||||||
|
// If MOB value is less than threshold, append it directly to a store file
|
||||||
|
PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId());
|
||||||
|
writer.append(mobCell);
|
||||||
|
cellsCountCompactedFromMob++;
|
||||||
|
cellsSizeCompactedFromMob += mobCell.getValueLength();
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
// Not a MOB reference cell
|
||||||
|
int size = c.getValueLength();
|
||||||
|
if (size > mobSizeThreshold) {
|
||||||
|
mobFileWriter.append(c);
|
||||||
|
writer
|
||||||
|
.append(MobUtils.createMobRefCell(c, fileName, this.mobStore.getRefCellTags()));
|
||||||
|
mobCells++;
|
||||||
|
cellsCountCompactedToMob++;
|
||||||
|
cellsSizeCompactedToMob += c.getValueLength();
|
||||||
|
} else {
|
||||||
|
writer.append(c);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} else if (c.getTypeByte() != KeyValue.Type.Put.getCode()) {
|
||||||
|
// Not a major compaction or major with MOB disabled
|
||||||
|
// If the kv type is not put, directly write the cell
|
||||||
|
// to the store file.
|
||||||
|
writer.append(c);
|
||||||
|
} else if (MobUtils.isMobReferenceCell(c)) {
|
||||||
|
// Not a major MOB compaction, Put MOB reference
|
||||||
|
if (MobUtils.hasValidMobRefCellValue(c)) {
|
||||||
|
int size = MobUtils.getMobValueLength(c);
|
||||||
|
if (size > mobSizeThreshold) {
|
||||||
|
// If the value size is larger than the threshold, it's regarded as a mob. Since
|
||||||
|
// its value is already in the mob file, directly write this cell to the store file
|
||||||
|
writer.append(c);
|
||||||
|
// Add MOB reference to a set
|
||||||
|
mobRefSet.get().add(MobUtils.getMobFileName(c));
|
||||||
|
} else {
|
||||||
|
// If the value is not larger than the threshold, it's not regarded a mob. Retrieve
|
||||||
|
// the mob cell from the mob file, and write it back to the store file.
|
||||||
|
mobCell = mobStore.resolve(c, true, false).getCell();
|
||||||
|
if (mobCell.getValueLength() != 0) {
|
||||||
|
// put the mob data back to the store file
|
||||||
|
PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId());
|
||||||
|
writer.append(mobCell);
|
||||||
|
cellsCountCompactedFromMob++;
|
||||||
|
cellsSizeCompactedFromMob += mobCell.getValueLength();
|
||||||
|
} else {
|
||||||
|
// If the value of a file is empty, there might be issues when retrieving,
|
||||||
|
// directly write the cell to the store file, and leave it to be handled by the
|
||||||
|
// next compaction.
|
||||||
|
LOG.error("Empty value for: " + c);
|
||||||
|
writer.append(c);
|
||||||
|
// Add MOB reference to a set
|
||||||
|
mobRefSet.get().add(MobUtils.getMobFileName(c));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
LOG.error("Corrupted MOB reference: {}", c);
|
||||||
|
writer.append(c);
|
||||||
|
}
|
||||||
|
} else if (c.getValueLength() <= mobSizeThreshold) {
|
||||||
|
// If the value size of a cell is not larger than the threshold, directly write it to
|
||||||
|
// the store file.
|
||||||
|
writer.append(c);
|
||||||
|
} else {
|
||||||
|
// If the value size of a cell is larger than the threshold, it's regarded as a mob,
|
||||||
|
// write this cell to a mob file, and write the path to the store file.
|
||||||
|
mobCells++;
|
||||||
|
// append the original keyValue in the mob file.
|
||||||
|
mobFileWriter.append(c);
|
||||||
|
Cell reference = MobUtils.createMobRefCell(c, fileName, this.mobStore.getRefCellTags());
|
||||||
|
// write the cell whose value is the path of a mob file to the store file.
|
||||||
|
writer.append(reference);
|
||||||
|
cellsCountCompactedToMob++;
|
||||||
|
cellsSizeCompactedToMob += c.getValueLength();
|
||||||
|
// Add ref we get for compact MOB case
|
||||||
|
mobRefSet.get().add(mobFileWriter.getPath().getName());
|
||||||
|
}
|
||||||
|
|
||||||
|
int len = c.getSerializedSize();
|
||||||
|
++progress.currentCompactedKVs;
|
||||||
|
progress.totalCompactedSize += len;
|
||||||
|
bytesWrittenProgressForShippedCall += len;
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
bytesWrittenProgressForLog += len;
|
||||||
|
}
|
||||||
|
throughputController.control(compactionName, len);
|
||||||
|
// check periodically to see if a system stop is requested
|
||||||
|
if (closeCheckSizeLimit > 0) {
|
||||||
|
bytesWrittenProgressForCloseCheck += len;
|
||||||
|
if (bytesWrittenProgressForCloseCheck > closeCheckSizeLimit) {
|
||||||
|
bytesWrittenProgressForCloseCheck = 0;
|
||||||
|
if (!store.areWritesEnabled()) {
|
||||||
|
progress.cancel();
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (kvs != null && bytesWrittenProgressForShippedCall > shippedCallSizeLimit) {
|
||||||
|
((ShipperListener) writer).beforeShipped();
|
||||||
|
kvs.shipped();
|
||||||
|
bytesWrittenProgressForShippedCall = 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// Log the progress of long running compactions every minute if
|
||||||
|
// logging at DEBUG level
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
if ((now - lastMillis) >= COMPACTION_PROGRESS_LOG_INTERVAL) {
|
||||||
|
String rate = String.format("%.2f",
|
||||||
|
(bytesWrittenProgressForLog / 1024.0) / ((now - lastMillis) / 1000.0));
|
||||||
|
LOG.debug("Compaction progress: {} {}, rate={} KB/sec, throughputController is {}",
|
||||||
|
compactionName, progress, rate, throughputController);
|
||||||
|
lastMillis = now;
|
||||||
|
bytesWrittenProgressForLog = 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
cells.clear();
|
||||||
|
} while (hasMore);
|
||||||
|
finished = true;
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
progress.cancel();
|
||||||
|
throw new InterruptedIOException(
|
||||||
|
"Interrupted while control throughput of compacting " + compactionName);
|
||||||
|
} catch (FileNotFoundException e) {
|
||||||
|
LOG.error("MOB Stress Test FAILED, region: " + store.getRegionInfo().getEncodedName(), e);
|
||||||
|
System.exit(-1);
|
||||||
|
} catch (IOException t) {
|
||||||
|
LOG.error("Mob compaction failed for region: " + store.getRegionInfo().getEncodedName());
|
||||||
|
throw t;
|
||||||
|
} finally {
|
||||||
|
// Clone last cell in the final because writer will append last cell when committing. If
|
||||||
|
// don't clone here and once the scanner get closed, then the memory of last cell will be
|
||||||
|
// released. (HBASE-22582)
|
||||||
|
((ShipperListener) writer).beforeShipped();
|
||||||
|
throughputController.finish(compactionName);
|
||||||
|
if (!finished && mobFileWriter != null) {
|
||||||
|
// Remove all MOB references because compaction failed
|
||||||
|
mobRefSet.get().clear();
|
||||||
|
// Abort writer
|
||||||
|
abortWriter(mobFileWriter);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (mobFileWriter != null) {
|
||||||
|
if (mobCells > 0) {
|
||||||
|
// If the mob file is not empty, commit it.
|
||||||
|
mobFileWriter.appendMetadata(fd.maxSeqId, major, mobCells);
|
||||||
|
mobFileWriter.close();
|
||||||
|
mobStore.commitFile(mobFileWriter.getPath(), path);
|
||||||
|
} else {
|
||||||
|
// If the mob file is empty, delete it instead of committing.
|
||||||
|
abortWriter(mobFileWriter);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
mobStore.updateCellsCountCompactedFromMob(cellsCountCompactedFromMob);
|
||||||
|
mobStore.updateCellsCountCompactedToMob(cellsCountCompactedToMob);
|
||||||
|
mobStore.updateCellsSizeCompactedFromMob(cellsSizeCompactedFromMob);
|
||||||
|
mobStore.updateCellsSizeCompactedToMob(cellsSizeCompactedToMob);
|
||||||
|
progress.complete();
|
||||||
|
return true;
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,81 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.hbase.mob;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||||
|
import org.apache.hadoop.hbase.util.AbstractHBaseTool;
|
||||||
|
import org.apache.hadoop.util.ToolRunner;
|
||||||
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
import org.apache.yetus.audience.InterfaceStability;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
|
||||||
|
|
||||||
|
|
||||||
|
@InterfaceAudience.Private
|
||||||
|
@InterfaceStability.Evolving
|
||||||
|
public class MobStressTool extends AbstractHBaseTool {
|
||||||
|
private static final Logger LOG = LoggerFactory.getLogger(MobStressTool.class);
|
||||||
|
private CommandLine cmd;
|
||||||
|
|
||||||
|
public MobStressTool() throws IOException {
|
||||||
|
init();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void init() throws IOException {
|
||||||
|
// define supported options
|
||||||
|
addOptWithArg("n", "Number of MOB key-values to insert, default - 10000000");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void addOptions() {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void processOptions(CommandLine cmd) {
|
||||||
|
this.cmd = cmd;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected int doWork() throws Exception {
|
||||||
|
long numRowsToInsert = 10000000;
|
||||||
|
if (cmd.hasOption("n")) {
|
||||||
|
numRowsToInsert = Long.parseLong(cmd.getOptionValue("n"));
|
||||||
|
if (numRowsToInsert < 0) {
|
||||||
|
LOG.warn("Ignore wrong option '-n'");
|
||||||
|
numRowsToInsert = 10000000;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
MobStressToolRunner runner = new MobStressToolRunner();
|
||||||
|
runner.init(getConf(), numRowsToInsert);
|
||||||
|
runner.runStressTest();
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void main(String[] args) throws Exception {
|
||||||
|
Configuration conf = HBaseConfiguration.create();
|
||||||
|
int ret = ToolRunner.run(conf, new MobStressTool(), args);
|
||||||
|
System.exit(ret);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,312 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.hbase.mob;
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Arrays;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||||
|
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.TableName;
|
||||||
|
import org.apache.hadoop.hbase.client.Admin;
|
||||||
|
import org.apache.hadoop.hbase.client.Connection;
|
||||||
|
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||||
|
import org.apache.hadoop.hbase.client.Put;
|
||||||
|
import org.apache.hadoop.hbase.client.Result;
|
||||||
|
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||||
|
import org.apache.hadoop.hbase.client.Table;
|
||||||
|
import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner;
|
||||||
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
/**
|
||||||
|
Reproduction for MOB data loss
|
||||||
|
|
||||||
|
1. Settings: Region Size 200 MB, Flush threshold 800 KB.
|
||||||
|
2. Insert 10 Million records
|
||||||
|
3. MOB Compaction and Archiver
|
||||||
|
a) Trigger MOB Compaction (every 2 minutes)
|
||||||
|
b) Trigger major compaction (every 2 minutes)
|
||||||
|
c) Trigger archive cleaner (every 3 minutes)
|
||||||
|
4. Validate MOB data after complete data load.
|
||||||
|
|
||||||
|
This class is used by MobStressTool only. This is not a unit test
|
||||||
|
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("deprecation")
|
||||||
|
public class MobStressToolRunner {
|
||||||
|
private static final Logger LOG = LoggerFactory.getLogger(MobStressToolRunner.class);
|
||||||
|
|
||||||
|
|
||||||
|
private HBaseTestingUtility HTU;
|
||||||
|
|
||||||
|
private final static String famStr = "f1";
|
||||||
|
private final static byte[] fam = Bytes.toBytes(famStr);
|
||||||
|
private final static byte[] qualifier = Bytes.toBytes("q1");
|
||||||
|
private final static long mobLen = 10;
|
||||||
|
private final static byte[] mobVal = Bytes
|
||||||
|
.toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789");
|
||||||
|
|
||||||
|
private Configuration conf;
|
||||||
|
private HTableDescriptor hdt;
|
||||||
|
private HColumnDescriptor hcd;
|
||||||
|
private Admin admin;
|
||||||
|
private long count = 500000;
|
||||||
|
private double failureProb = 0.1;
|
||||||
|
private Table table = null;
|
||||||
|
private MobFileCleanerChore chore = new MobFileCleanerChore();
|
||||||
|
|
||||||
|
private static volatile boolean run = true;
|
||||||
|
|
||||||
|
public MobStressToolRunner() {
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
public void init(Configuration conf, long numRows) throws IOException {
|
||||||
|
this.conf = conf;
|
||||||
|
this.count = numRows;
|
||||||
|
initConf();
|
||||||
|
printConf();
|
||||||
|
hdt = new HTableDescriptor(TableName.valueOf("testMobCompactTable"));
|
||||||
|
Connection conn = ConnectionFactory.createConnection(this.conf);
|
||||||
|
this.admin = conn.getAdmin();
|
||||||
|
this.hcd = new HColumnDescriptor(fam);
|
||||||
|
this.hcd.setMobEnabled(true);
|
||||||
|
this.hcd.setMobThreshold(mobLen);
|
||||||
|
this.hcd.setMaxVersions(1);
|
||||||
|
this.hdt.addFamily(hcd);
|
||||||
|
if (admin.tableExists(hdt.getTableName())) {
|
||||||
|
admin.disableTable(hdt.getTableName());
|
||||||
|
admin.deleteTable(hdt.getTableName());
|
||||||
|
}
|
||||||
|
admin.createTable(hdt);
|
||||||
|
table = conn.getTable(hdt.getTableName());
|
||||||
|
}
|
||||||
|
|
||||||
|
private void printConf() {
|
||||||
|
LOG.info("Please ensure the following HBase configuration is set:");
|
||||||
|
LOG.info("hfile.format.version=3");
|
||||||
|
LOG.info("hbase.master.hfilecleaner.ttl=0");
|
||||||
|
LOG.info("hbase.hregion.max.filesize=200000000");
|
||||||
|
LOG.info("hbase.client.retries.number=100");
|
||||||
|
LOG.info("hbase.hregion.memstore.flush.size=800000");
|
||||||
|
LOG.info("hbase.hstore.blockingStoreFiles=150");
|
||||||
|
LOG.info("hbase.hstore.compaction.throughput.lower.bound=50000000");
|
||||||
|
LOG.info("hbase.hstore.compaction.throughput.higher.bound=100000000");
|
||||||
|
LOG.info("hbase.master.mob.cleaner.period=0");
|
||||||
|
LOG.info("hbase.mob.default.compactor=org.apache.hadoop.hbase.mob.FaultyMobStoreCompactor");
|
||||||
|
LOG.warn("hbase.mob.compaction.fault.probability=x, where x is between 0. and 1.");
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
private void initConf() {
|
||||||
|
|
||||||
|
conf.setInt("hfile.format.version", 3);
|
||||||
|
conf.setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0);
|
||||||
|
conf.setInt("hbase.client.retries.number", 100);
|
||||||
|
conf.setInt("hbase.hregion.max.filesize", 200000000);
|
||||||
|
conf.setInt("hbase.hregion.memstore.flush.size", 800000);
|
||||||
|
conf.setInt("hbase.hstore.blockingStoreFiles", 150);
|
||||||
|
conf.setInt("hbase.hstore.compaction.throughput.lower.bound", 52428800);
|
||||||
|
conf.setInt("hbase.hstore.compaction.throughput.higher.bound", 2 * 52428800);
|
||||||
|
conf.setDouble("hbase.mob.compaction.fault.probability", failureProb);
|
||||||
|
// conf.set(MobStoreEngine.DEFAULT_MOB_COMPACTOR_CLASS_KEY,
|
||||||
|
// FaultyMobStoreCompactor.class.getName());
|
||||||
|
conf.setLong(MobConstants.MOB_COMPACTION_CHORE_PERIOD, 0);
|
||||||
|
conf.setLong(MobConstants.MOB_CLEANER_PERIOD, 0);
|
||||||
|
conf.setLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, 120000);
|
||||||
|
conf.set(MobConstants.MOB_COMPACTION_TYPE_KEY,
|
||||||
|
MobConstants.OPTIMIZED_MOB_COMPACTION_TYPE);
|
||||||
|
conf.setLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, 1000000);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
class MajorCompaction implements Runnable {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
while (run) {
|
||||||
|
try {
|
||||||
|
admin.majorCompact(hdt.getTableName(), fam);
|
||||||
|
Thread.sleep(120000);
|
||||||
|
} catch (Exception e) {
|
||||||
|
LOG.error("MOB Stress Test FAILED", e);
|
||||||
|
System.exit(-1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
class CleanMobAndArchive implements Runnable {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
while (run) {
|
||||||
|
try {
|
||||||
|
LOG.info("MOB cleanup chore started ...");
|
||||||
|
chore.cleanupObsoleteMobFiles(conf, table.getName());
|
||||||
|
LOG.info("MOB cleanup chore finished");
|
||||||
|
|
||||||
|
Thread.sleep(130000);
|
||||||
|
} catch (Exception e) {
|
||||||
|
LOG.error("CleanMobAndArchive", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
class WriteData implements Runnable {
|
||||||
|
|
||||||
|
private long rows = -1;
|
||||||
|
|
||||||
|
public WriteData(long rows) {
|
||||||
|
this.rows = rows;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
try {
|
||||||
|
|
||||||
|
// Put Operation
|
||||||
|
for (int i = 0; i < rows; i++) {
|
||||||
|
byte[] key = Bytes.toBytes(i);
|
||||||
|
Put p = new Put(key);
|
||||||
|
p.addColumn(fam, qualifier, Bytes.add(key,mobVal));
|
||||||
|
table.put(p);
|
||||||
|
if (i % 10000 == 0) {
|
||||||
|
LOG.info("LOADED=" + i);
|
||||||
|
try {
|
||||||
|
Thread.sleep(500);
|
||||||
|
} catch (InterruptedException ee) {
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (i % 100000 == 0) {
|
||||||
|
printStats(i);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
admin.flush(table.getName());
|
||||||
|
run = false;
|
||||||
|
} catch (Exception e) {
|
||||||
|
LOG.error("MOB Stress Test FAILED", e);
|
||||||
|
System.exit(-1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void runStressTest() throws InterruptedException, IOException {
|
||||||
|
|
||||||
|
try {
|
||||||
|
|
||||||
|
Thread writeData = new Thread(new WriteData(count));
|
||||||
|
writeData.start();
|
||||||
|
|
||||||
|
Thread majorcompact = new Thread(new MajorCompaction());
|
||||||
|
majorcompact.start();
|
||||||
|
|
||||||
|
Thread cleaner = new Thread(new CleanMobAndArchive());
|
||||||
|
cleaner.start();
|
||||||
|
|
||||||
|
while (run) {
|
||||||
|
Thread.sleep(1000);
|
||||||
|
}
|
||||||
|
|
||||||
|
getNumberOfMobFiles(conf, table.getName(), new String(fam));
|
||||||
|
LOG.info("Waiting for write thread to finish ...");
|
||||||
|
writeData.join();
|
||||||
|
// Cleanup again
|
||||||
|
chore.cleanupObsoleteMobFiles(conf, table.getName());
|
||||||
|
getNumberOfMobFiles(conf, table.getName(), new String(fam));
|
||||||
|
|
||||||
|
if (HTU != null) {
|
||||||
|
LOG.info("Archive cleaner started ...");
|
||||||
|
// Call archive cleaner again
|
||||||
|
HTU.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting();
|
||||||
|
LOG.info("Archive cleaner finished");
|
||||||
|
}
|
||||||
|
|
||||||
|
scanTable();
|
||||||
|
|
||||||
|
} finally {
|
||||||
|
|
||||||
|
admin.disableTable(hdt.getTableName());
|
||||||
|
admin.deleteTable(hdt.getTableName());
|
||||||
|
}
|
||||||
|
LOG.info("MOB Stress Test finished OK");
|
||||||
|
printStats(count);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
private long getNumberOfMobFiles(Configuration conf, TableName tableName, String family)
|
||||||
|
throws IOException {
|
||||||
|
FileSystem fs = FileSystem.get(conf);
|
||||||
|
Path dir = MobUtils.getMobFamilyPath(conf, tableName, family);
|
||||||
|
FileStatus[] stat = fs.listStatus(dir);
|
||||||
|
long size = 0;
|
||||||
|
for (FileStatus st : stat) {
|
||||||
|
LOG.debug("MOB Directory content: {} len={}", st.getPath(), st.getLen());
|
||||||
|
size+= st.getLen();
|
||||||
|
}
|
||||||
|
LOG.debug("MOB Directory content total files: {}, total size={}", stat.length, size);
|
||||||
|
|
||||||
|
return stat.length;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void printStats(long loaded) {
|
||||||
|
LOG.info("MOB Stress Test: loaded=" + loaded + " compactions="
|
||||||
|
+ FaultyMobStoreCompactor.totalCompactions.get() + " major="
|
||||||
|
+ FaultyMobStoreCompactor.totalMajorCompactions.get() + " mob="
|
||||||
|
+ FaultyMobStoreCompactor.mobCounter.get() + " injected failures="
|
||||||
|
+ FaultyMobStoreCompactor.totalFailures.get());
|
||||||
|
}
|
||||||
|
|
||||||
|
private void scanTable() {
|
||||||
|
try {
|
||||||
|
|
||||||
|
Result result;
|
||||||
|
ResultScanner scanner = table.getScanner(fam);
|
||||||
|
int counter = 0;
|
||||||
|
while ((result = scanner.next()) != null) {
|
||||||
|
byte[] key = result.getRow();
|
||||||
|
assertTrue(Arrays.equals(result.getValue(fam, qualifier),
|
||||||
|
Bytes.add(key,mobVal)));
|
||||||
|
if (counter % 10000 == 0) {
|
||||||
|
LOG.info("GET=" + counter+" key=" + Bytes.toInt(key));
|
||||||
|
}
|
||||||
|
counter++;
|
||||||
|
}
|
||||||
|
|
||||||
|
assertEquals(count, counter);
|
||||||
|
} catch (Exception e) {
|
||||||
|
e.printStackTrace();
|
||||||
|
LOG.error("MOB Stress Test FAILED");
|
||||||
|
if (HTU != null) {
|
||||||
|
assertTrue(false);
|
||||||
|
} else {
|
||||||
|
System.exit(-1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,230 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.hbase.mob;
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
import static org.junit.Assert.fail;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Random;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||||
|
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.TableName;
|
||||||
|
import org.apache.hadoop.hbase.client.Admin;
|
||||||
|
import org.apache.hadoop.hbase.client.CompactionState;
|
||||||
|
import org.apache.hadoop.hbase.client.Put;
|
||||||
|
import org.apache.hadoop.hbase.client.Result;
|
||||||
|
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||||
|
import org.apache.hadoop.hbase.client.Table;
|
||||||
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.hadoop.hbase.util.RegionSplitter;
|
||||||
|
import org.junit.After;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Mob file compaction base test.
|
||||||
|
* 1. Enables batch mode for regular MOB compaction,
|
||||||
|
* Sets batch size to 7 regions. (Optional)
|
||||||
|
* 2. Disables periodic MOB compactions, sets minimum age to archive to 10 sec
|
||||||
|
* 3. Creates MOB table with 20 regions
|
||||||
|
* 4. Loads MOB data (randomized keys, 1000 rows), flushes data.
|
||||||
|
* 5. Repeats 4. two more times
|
||||||
|
* 6. Verifies that we have 20 *3 = 60 mob files (equals to number of regions x 3)
|
||||||
|
* 7. Runs major MOB compaction.
|
||||||
|
* 8. Verifies that number of MOB files in a mob directory is 20 x4 = 80
|
||||||
|
* 9. Waits for a period of time larger than minimum age to archive
|
||||||
|
* 10. Runs Mob cleaner chore
|
||||||
|
* 11 Verifies that number of MOB files in a mob directory is 20.
|
||||||
|
* 12 Runs scanner and checks all 3 * 1000 rows.
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("deprecation")
|
||||||
|
public abstract class TestMobCompactionBase {
|
||||||
|
private static final Logger LOG =
|
||||||
|
LoggerFactory.getLogger(TestMobCompactionBase.class);
|
||||||
|
|
||||||
|
protected HBaseTestingUtility HTU;
|
||||||
|
|
||||||
|
protected final static String famStr = "f1";
|
||||||
|
protected final static byte[] fam = Bytes.toBytes(famStr);
|
||||||
|
protected final static byte[] qualifier = Bytes.toBytes("q1");
|
||||||
|
protected final static long mobLen = 10;
|
||||||
|
protected final static byte[] mobVal = Bytes
|
||||||
|
.toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789");
|
||||||
|
|
||||||
|
protected Configuration conf;
|
||||||
|
protected HTableDescriptor hdt;
|
||||||
|
private HColumnDescriptor hcd;
|
||||||
|
protected Admin admin;
|
||||||
|
protected Table table = null;
|
||||||
|
protected long minAgeToArchive = 10000;
|
||||||
|
protected int numRegions = 20;
|
||||||
|
protected int rows = 1000;
|
||||||
|
|
||||||
|
protected MobFileCleanerChore cleanerChore;
|
||||||
|
|
||||||
|
public TestMobCompactionBase() {
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp() throws Exception {
|
||||||
|
HTU = new HBaseTestingUtility();
|
||||||
|
hdt = HTU.createTableDescriptor(getClass().getName());
|
||||||
|
conf = HTU.getConfiguration();
|
||||||
|
|
||||||
|
initConf();
|
||||||
|
|
||||||
|
HTU.startMiniCluster();
|
||||||
|
admin = HTU.getAdmin();
|
||||||
|
cleanerChore = new MobFileCleanerChore();
|
||||||
|
hcd = new HColumnDescriptor(fam);
|
||||||
|
hcd.setMobEnabled(true);
|
||||||
|
hcd.setMobThreshold(mobLen);
|
||||||
|
hcd.setMaxVersions(1);
|
||||||
|
hdt.addFamily(hcd);
|
||||||
|
RegionSplitter.UniformSplit splitAlgo = new RegionSplitter.UniformSplit();
|
||||||
|
byte[][] splitKeys = splitAlgo.split(numRegions);
|
||||||
|
table = HTU.createTable(hdt, splitKeys);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void initConf() {
|
||||||
|
|
||||||
|
conf.setInt("hfile.format.version", 3);
|
||||||
|
// Disable automatic MOB compaction
|
||||||
|
conf.setLong(MobConstants.MOB_COMPACTION_CHORE_PERIOD, 0);
|
||||||
|
// Disable automatic MOB file cleaner chore
|
||||||
|
conf.setLong(MobConstants.MOB_CLEANER_PERIOD, 0);
|
||||||
|
// Set minimum age to archive to 10 sec
|
||||||
|
conf.setLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, minAgeToArchive);
|
||||||
|
// Set compacted file discharger interval to a half minAgeToArchive
|
||||||
|
conf.setLong("hbase.hfile.compaction.discharger.interval", minAgeToArchive/2);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void loadData(int num) {
|
||||||
|
|
||||||
|
Random r = new Random();
|
||||||
|
try {
|
||||||
|
LOG.info("Started loading {} rows", num);
|
||||||
|
for (int i = 0; i < num; i++) {
|
||||||
|
byte[] key = new byte[32];
|
||||||
|
r.nextBytes(key);
|
||||||
|
Put p = new Put(key);
|
||||||
|
p.addColumn(fam, qualifier, mobVal);
|
||||||
|
table.put(p);
|
||||||
|
}
|
||||||
|
admin.flush(table.getName());
|
||||||
|
LOG.info("Finished loading {} rows", num);
|
||||||
|
} catch (Exception e) {
|
||||||
|
LOG.error("MOB file compaction chore test FAILED", e);
|
||||||
|
fail("MOB file compaction chore test FAILED");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void tearDown() throws Exception {
|
||||||
|
admin.disableTable(hdt.getTableName());
|
||||||
|
admin.deleteTable(hdt.getTableName());
|
||||||
|
HTU.shutdownMiniCluster();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
public void baseTestMobFileCompaction() throws InterruptedException, IOException {
|
||||||
|
|
||||||
|
// Load and flush data 3 times
|
||||||
|
loadData(rows);
|
||||||
|
loadData(rows);
|
||||||
|
loadData(rows);
|
||||||
|
long num = getNumberOfMobFiles(conf, table.getName(), new String(fam));
|
||||||
|
assertEquals(numRegions * 3, num);
|
||||||
|
// Major MOB compact
|
||||||
|
mobCompact(admin, hdt, hcd);
|
||||||
|
// wait until compaction is complete
|
||||||
|
while (admin.getCompactionState(hdt.getTableName()) != CompactionState.NONE) {
|
||||||
|
Thread.sleep(100);
|
||||||
|
}
|
||||||
|
|
||||||
|
num = getNumberOfMobFiles(conf, table.getName(), new String(fam));
|
||||||
|
assertEquals(numRegions * 4, num);
|
||||||
|
// We have guarantee, that compacted file discharger will run during this pause
|
||||||
|
// because it has interval less than this wait time
|
||||||
|
LOG.info("Waiting for {}ms", minAgeToArchive + 1000);
|
||||||
|
|
||||||
|
Thread.sleep(minAgeToArchive + 1000);
|
||||||
|
LOG.info("Cleaning up MOB files");
|
||||||
|
// Cleanup again
|
||||||
|
cleanerChore.cleanupObsoleteMobFiles(conf, table.getName());
|
||||||
|
|
||||||
|
num = getNumberOfMobFiles(conf, table.getName(), new String(fam));
|
||||||
|
assertEquals(numRegions, num);
|
||||||
|
|
||||||
|
long scanned = scanTable();
|
||||||
|
assertEquals(3 * rows, scanned);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
protected abstract void mobCompact(Admin admin2, HTableDescriptor hdt2, HColumnDescriptor hcd2)
|
||||||
|
throws IOException, InterruptedException;
|
||||||
|
|
||||||
|
|
||||||
|
protected long getNumberOfMobFiles(Configuration conf, TableName tableName, String family)
|
||||||
|
throws IOException {
|
||||||
|
FileSystem fs = FileSystem.get(conf);
|
||||||
|
Path dir = MobUtils.getMobFamilyPath(conf, tableName, family);
|
||||||
|
FileStatus[] stat = fs.listStatus(dir);
|
||||||
|
for (FileStatus st : stat) {
|
||||||
|
LOG.debug("MOB Directory content: {}", st.getPath());
|
||||||
|
}
|
||||||
|
LOG.debug("MOB Directory content total files: {}", stat.length);
|
||||||
|
|
||||||
|
return stat.length;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
protected long scanTable() {
|
||||||
|
try {
|
||||||
|
|
||||||
|
Result result;
|
||||||
|
ResultScanner scanner = table.getScanner(fam);
|
||||||
|
long counter = 0;
|
||||||
|
while ((result = scanner.next()) != null) {
|
||||||
|
assertTrue(Arrays.equals(result.getValue(fam, qualifier), mobVal));
|
||||||
|
counter++;
|
||||||
|
}
|
||||||
|
return counter;
|
||||||
|
} catch (Exception e) {
|
||||||
|
LOG.error("MOB file compaction test FAILED", e);
|
||||||
|
if (HTU != null) {
|
||||||
|
fail(e.getMessage());
|
||||||
|
} else {
|
||||||
|
System.exit(-1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,83 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.hbase.mob;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||||
|
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.client.Admin;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||||
|
import org.junit.ClassRule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Mob file compaction chore in a generational non-batch mode test.
|
||||||
|
* 1. Uses default (non-batch) mode for regular MOB compaction, sets generational mode ON
|
||||||
|
* 2. Disables periodic MOB compactions, sets minimum age to archive to 10 sec
|
||||||
|
* 3. Creates MOB table with 20 regions
|
||||||
|
* 4. Loads MOB data (randomized keys, 1000 rows), flushes data.
|
||||||
|
* 5. Repeats 4. two more times
|
||||||
|
* 6. Verifies that we have 20 *3 = 60 mob files (equals to number of regions x 3)
|
||||||
|
* 7. Runs major MOB compaction.
|
||||||
|
* 8. Verifies that number of MOB files in a mob directory is 20 x4 = 80
|
||||||
|
* 9. Waits for a period of time larger than minimum age to archive
|
||||||
|
* 10. Runs Mob cleaner chore
|
||||||
|
* 11 Verifies that number of MOB files in a mob directory is 20.
|
||||||
|
* 12 Runs scanner and checks all 3 * 1000 rows.
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("deprecation")
|
||||||
|
@Category(LargeTests.class)
|
||||||
|
public class TestMobCompactionOptMode extends TestMobCompactionBase{
|
||||||
|
private static final Logger LOG =
|
||||||
|
LoggerFactory.getLogger(TestMobCompactionOptMode.class);
|
||||||
|
@ClassRule
|
||||||
|
public static final HBaseClassTestRule CLASS_RULE =
|
||||||
|
HBaseClassTestRule.forClass(TestMobCompactionOptMode.class);
|
||||||
|
|
||||||
|
public TestMobCompactionOptMode() {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void initConf() {
|
||||||
|
super.initConf();
|
||||||
|
conf.set(MobConstants.MOB_COMPACTION_TYPE_KEY,
|
||||||
|
MobConstants.OPTIMIZED_MOB_COMPACTION_TYPE);
|
||||||
|
conf.setLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, 1000000);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMobFileCompactionBatchMode() throws InterruptedException, IOException {
|
||||||
|
LOG.info("MOB compaction generational (non-batch) mode started");
|
||||||
|
baseTestMobFileCompaction();
|
||||||
|
LOG.info("MOB compaction generational (non-batch) mode finished OK");
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void mobCompact(Admin admin, HTableDescriptor hdt, HColumnDescriptor hcd)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
// Major compact MOB table
|
||||||
|
admin.majorCompact(hdt.getTableName(), hcd.getName());
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,94 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.hbase.mob;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||||
|
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.client.Admin;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.ClassRule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Mob file compaction chore in a generational batch mode test.
|
||||||
|
* 1. Enables batch mode for regular MOB compaction,
|
||||||
|
* Sets batch size to 7 regions. Enables generational mode.
|
||||||
|
* 2. Disables periodic MOB compactions, sets minimum age to archive to 10 sec
|
||||||
|
* 3. Creates MOB table with 20 regions
|
||||||
|
* 4. Loads MOB data (randomized keys, 1000 rows), flushes data.
|
||||||
|
* 5. Repeats 4. two more times
|
||||||
|
* 6. Verifies that we have 20 *3 = 60 mob files (equals to number of regions x 3)
|
||||||
|
* 7. Runs major MOB compaction.
|
||||||
|
* 8. Verifies that number of MOB files in a mob directory is 20 x4 = 80
|
||||||
|
* 9. Waits for a period of time larger than minimum age to archive
|
||||||
|
* 10. Runs Mob cleaner chore
|
||||||
|
* 11 Verifies that number of MOB files in a mob directory is 20.
|
||||||
|
* 12 Runs scanner and checks all 3 * 1000 rows.
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("deprecation")
|
||||||
|
@Category(LargeTests.class)
|
||||||
|
public class TestMobCompactionOptRegionBatchMode extends TestMobCompactionBase{
|
||||||
|
private static final Logger LOG =
|
||||||
|
LoggerFactory.getLogger(TestMobCompactionOptRegionBatchMode.class);
|
||||||
|
@ClassRule
|
||||||
|
public static final HBaseClassTestRule CLASS_RULE =
|
||||||
|
HBaseClassTestRule.forClass(TestMobCompactionOptRegionBatchMode.class);
|
||||||
|
|
||||||
|
private int batchSize = 7;
|
||||||
|
private MobFileCompactionChore compactionChore;
|
||||||
|
|
||||||
|
public TestMobCompactionOptRegionBatchMode() {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp() throws Exception {
|
||||||
|
super.setUp();
|
||||||
|
compactionChore = new MobFileCompactionChore(conf, batchSize);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void initConf() {
|
||||||
|
super.initConf();
|
||||||
|
conf.setInt(MobConstants.MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE, batchSize);
|
||||||
|
conf.set(MobConstants.MOB_COMPACTION_TYPE_KEY,
|
||||||
|
MobConstants.OPTIMIZED_MOB_COMPACTION_TYPE);
|
||||||
|
conf.setLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, 1000000);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMobFileCompactionBatchMode() throws InterruptedException, IOException {
|
||||||
|
LOG.info("MOB compaction chore generational batch mode started");
|
||||||
|
baseTestMobFileCompaction();
|
||||||
|
LOG.info("MOB compaction chore generational batch mode finished OK");
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void mobCompact(Admin admin, HTableDescriptor hdt, HColumnDescriptor hcd)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
// Major compact with batch mode enabled
|
||||||
|
compactionChore.performMajorCompactionInBatches(admin, hdt, hcd);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,75 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.hbase.mob;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||||
|
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.client.Admin;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||||
|
import org.junit.ClassRule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Mob file compaction chore in a regular non-batch mode test.
|
||||||
|
* 1. Uses default (non-batch) mode for regular MOB compaction,
|
||||||
|
* 2. Disables periodic MOB compactions, sets minimum age to archive to 10 sec
|
||||||
|
* 3. Creates MOB table with 20 regions
|
||||||
|
* 4. Loads MOB data (randomized keys, 1000 rows), flushes data.
|
||||||
|
* 5. Repeats 4. two more times
|
||||||
|
* 6. Verifies that we have 20 *3 = 60 mob files (equals to number of regions x 3)
|
||||||
|
* 7. Runs major MOB compaction.
|
||||||
|
* 8. Verifies that number of MOB files in a mob directory is 20 x4 = 80
|
||||||
|
* 9. Waits for a period of time larger than minimum age to archive
|
||||||
|
* 10. Runs Mob cleaner chore
|
||||||
|
* 11 Verifies that number of MOB files in a mob directory is 20.
|
||||||
|
* 12 Runs scanner and checks all 3 * 1000 rows.
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("deprecation")
|
||||||
|
@Category(LargeTests.class)
|
||||||
|
public class TestMobCompactionRegularMode extends TestMobCompactionBase{
|
||||||
|
private static final Logger LOG =
|
||||||
|
LoggerFactory.getLogger(TestMobCompactionRegularMode.class);
|
||||||
|
@ClassRule
|
||||||
|
public static final HBaseClassTestRule CLASS_RULE =
|
||||||
|
HBaseClassTestRule.forClass(TestMobCompactionRegularMode.class);
|
||||||
|
|
||||||
|
public TestMobCompactionRegularMode() {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMobFileCompactionBatchMode() throws InterruptedException, IOException {
|
||||||
|
LOG.info("MOB compaction regular mode started");
|
||||||
|
baseTestMobFileCompaction();
|
||||||
|
LOG.info("MOB compaction regular mode finished OK");
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void mobCompact(Admin admin, HTableDescriptor hdt, HColumnDescriptor hcd)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
// Major compact MOB table
|
||||||
|
admin.majorCompact(hdt.getTableName(), hcd.getName());
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,91 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.hbase.mob;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||||
|
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.client.Admin;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.ClassRule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Mob file compaction chore in a regular batch mode test.
|
||||||
|
* 1. Enables batch mode for regular MOB compaction,
|
||||||
|
* Sets batch size to 7 regions.
|
||||||
|
* 2. Disables periodic MOB compactions, sets minimum age to archive to 10 sec
|
||||||
|
* 3. Creates MOB table with 20 regions
|
||||||
|
* 4. Loads MOB data (randomized keys, 1000 rows), flushes data.
|
||||||
|
* 5. Repeats 4. two more times
|
||||||
|
* 6. Verifies that we have 20 *3 = 60 mob files (equals to number of regions x 3)
|
||||||
|
* 7. Runs major MOB compaction.
|
||||||
|
* 8. Verifies that number of MOB files in a mob directory is 20 x4 = 80
|
||||||
|
* 9. Waits for a period of time larger than minimum age to archive
|
||||||
|
* 10. Runs Mob cleaner chore
|
||||||
|
* 11 Verifies that number of MOB files in a mob directory is 20.
|
||||||
|
* 12 Runs scanner and checks all 3 * 1000 rows.
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("deprecation")
|
||||||
|
@Category(LargeTests.class)
|
||||||
|
public class TestMobCompactionRegularRegionBatchMode extends TestMobCompactionBase{
|
||||||
|
private static final Logger LOG =
|
||||||
|
LoggerFactory.getLogger(TestMobCompactionRegularRegionBatchMode.class);
|
||||||
|
@ClassRule
|
||||||
|
public static final HBaseClassTestRule CLASS_RULE =
|
||||||
|
HBaseClassTestRule.forClass(TestMobCompactionRegularRegionBatchMode.class);
|
||||||
|
|
||||||
|
private int batchSize = 7;
|
||||||
|
private MobFileCompactionChore compactionChore;
|
||||||
|
|
||||||
|
public TestMobCompactionRegularRegionBatchMode() {
|
||||||
|
}
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp() throws Exception {
|
||||||
|
super.setUp();
|
||||||
|
compactionChore = new MobFileCompactionChore(conf, batchSize);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected void initConf() {
|
||||||
|
super.initConf();
|
||||||
|
conf.setInt(MobConstants.MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE, batchSize);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMobFileCompactionBatchMode() throws InterruptedException, IOException {
|
||||||
|
LOG.info("MOB compaction chore regular batch mode started");
|
||||||
|
baseTestMobFileCompaction();
|
||||||
|
LOG.info("MOB compaction chore regular batch mode finished OK");
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void mobCompact(Admin admin, HTableDescriptor hdt, HColumnDescriptor hcd)
|
||||||
|
throws IOException, InterruptedException {
|
||||||
|
// Major compact with batch mode enabled
|
||||||
|
compactionChore.performMajorCompactionInBatches(admin, hdt, hcd);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,223 @@
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.hbase.mob;
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Arrays;
|
||||||
|
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||||
|
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||||
|
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.TableName;
|
||||||
|
import org.apache.hadoop.hbase.client.Admin;
|
||||||
|
import org.apache.hadoop.hbase.client.CompactionState;
|
||||||
|
import org.apache.hadoop.hbase.client.Put;
|
||||||
|
import org.apache.hadoop.hbase.client.Result;
|
||||||
|
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||||
|
import org.apache.hadoop.hbase.client.Table;
|
||||||
|
import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.junit.After;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.ClassRule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.experimental.categories.Category;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Mob file cleaner chore test.
|
||||||
|
* 1. Creates MOB table
|
||||||
|
* 2. Load MOB data and flushes it N times
|
||||||
|
* 3. Runs major MOB compaction (N MOB files go to archive)
|
||||||
|
* 4. Verifies that number of MOB files in a mob directory is N+1
|
||||||
|
* 5. Waits for a period of time larger than minimum age to archive
|
||||||
|
* 6. Runs Mob cleaner chore
|
||||||
|
* 7 Verifies that number of MOB files in a mob directory is 1.
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("deprecation")
|
||||||
|
@Category(MediumTests.class)
|
||||||
|
public class TestMobFileCleanerChore {
|
||||||
|
private static final Logger LOG = LoggerFactory.getLogger(TestMobFileCleanerChore.class);
|
||||||
|
@ClassRule
|
||||||
|
public static final HBaseClassTestRule CLASS_RULE =
|
||||||
|
HBaseClassTestRule.forClass(TestMobFileCleanerChore.class);
|
||||||
|
|
||||||
|
private HBaseTestingUtility HTU;
|
||||||
|
|
||||||
|
private final static String famStr = "f1";
|
||||||
|
private final static byte[] fam = Bytes.toBytes(famStr);
|
||||||
|
private final static byte[] qualifier = Bytes.toBytes("q1");
|
||||||
|
private final static long mobLen = 10;
|
||||||
|
private final static byte[] mobVal = Bytes
|
||||||
|
.toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789");
|
||||||
|
|
||||||
|
private Configuration conf;
|
||||||
|
private HTableDescriptor hdt;
|
||||||
|
private HColumnDescriptor hcd;
|
||||||
|
private Admin admin;
|
||||||
|
private Table table = null;
|
||||||
|
private MobFileCleanerChore chore;
|
||||||
|
private long minAgeToArchive = 10000;
|
||||||
|
|
||||||
|
public TestMobFileCleanerChore() {
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp() throws Exception {
|
||||||
|
HTU = new HBaseTestingUtility();
|
||||||
|
hdt = HTU.createTableDescriptor("testMobCompactTable");
|
||||||
|
conf = HTU.getConfiguration();
|
||||||
|
|
||||||
|
initConf();
|
||||||
|
|
||||||
|
HTU.startMiniCluster();
|
||||||
|
admin = HTU.getAdmin();
|
||||||
|
chore = new MobFileCleanerChore();
|
||||||
|
hcd = new HColumnDescriptor(fam);
|
||||||
|
hcd.setMobEnabled(true);
|
||||||
|
hcd.setMobThreshold(mobLen);
|
||||||
|
hcd.setMaxVersions(1);
|
||||||
|
hdt.addFamily(hcd);
|
||||||
|
table = HTU.createTable(hdt, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void initConf() {
|
||||||
|
|
||||||
|
conf.setInt("hfile.format.version", 3);
|
||||||
|
conf.setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0);
|
||||||
|
conf.setInt("hbase.client.retries.number", 100);
|
||||||
|
conf.setInt("hbase.hregion.max.filesize", 200000000);
|
||||||
|
conf.setInt("hbase.hregion.memstore.flush.size", 800000);
|
||||||
|
conf.setInt("hbase.hstore.blockingStoreFiles", 150);
|
||||||
|
conf.setInt("hbase.hstore.compaction.throughput.lower.bound", 52428800);
|
||||||
|
conf.setInt("hbase.hstore.compaction.throughput.higher.bound", 2 * 52428800);
|
||||||
|
//conf.set(MobStoreEngine.DEFAULT_MOB_COMPACTOR_CLASS_KEY,
|
||||||
|
// FaultyMobStoreCompactor.class.getName());
|
||||||
|
// Disable automatic MOB compaction
|
||||||
|
conf.setLong(MobConstants.MOB_COMPACTION_CHORE_PERIOD, 0);
|
||||||
|
// Disable automatic MOB file cleaner chore
|
||||||
|
conf.setLong(MobConstants.MOB_CLEANER_PERIOD, 0);
|
||||||
|
// Set minimum age to archive to 10 sec
|
||||||
|
conf.setLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, minAgeToArchive);
|
||||||
|
// Set compacted file discharger interval to a half minAgeToArchive
|
||||||
|
conf.setLong("hbase.hfile.compaction.discharger.interval", minAgeToArchive/2);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void loadData(int start, int num) {
|
||||||
|
try {
|
||||||
|
|
||||||
|
for (int i = 0; i < num; i++) {
|
||||||
|
Put p = new Put(Bytes.toBytes(start + i));
|
||||||
|
p.addColumn(fam, qualifier, mobVal);
|
||||||
|
table.put(p);
|
||||||
|
}
|
||||||
|
admin.flush(table.getName());
|
||||||
|
} catch (Exception e) {
|
||||||
|
LOG.error("MOB file cleaner chore test FAILED", e);
|
||||||
|
assertTrue(false);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void tearDown() throws Exception {
|
||||||
|
admin.disableTable(hdt.getTableName());
|
||||||
|
admin.deleteTable(hdt.getTableName());
|
||||||
|
HTU.shutdownMiniCluster();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMobFileCleanerChore() throws InterruptedException, IOException {
|
||||||
|
|
||||||
|
loadData(0, 10);
|
||||||
|
loadData(10, 10);
|
||||||
|
loadData(20, 10);
|
||||||
|
long num = getNumberOfMobFiles(conf, table.getName(), new String(fam));
|
||||||
|
assertEquals(3, num);
|
||||||
|
// Major compact
|
||||||
|
admin.majorCompact(hdt.getTableName(), fam);
|
||||||
|
// wait until compaction is complete
|
||||||
|
while (admin.getCompactionState(hdt.getTableName()) != CompactionState.NONE) {
|
||||||
|
Thread.sleep(100);
|
||||||
|
}
|
||||||
|
|
||||||
|
num = getNumberOfMobFiles(conf, table.getName(), new String(fam));
|
||||||
|
assertEquals(4, num);
|
||||||
|
// We have guarantee, that compcated file discharger will run during this pause
|
||||||
|
// because it has interval less than this wait time
|
||||||
|
LOG.info("Waiting for {}ms", minAgeToArchive + 1000);
|
||||||
|
|
||||||
|
Thread.sleep(minAgeToArchive + 1000);
|
||||||
|
LOG.info("Cleaning up MOB files");
|
||||||
|
// Cleanup again
|
||||||
|
chore.cleanupObsoleteMobFiles(conf, table.getName());
|
||||||
|
|
||||||
|
num = getNumberOfMobFiles(conf, table.getName(), new String(fam));
|
||||||
|
assertEquals(1, num);
|
||||||
|
|
||||||
|
long scanned = scanTable();
|
||||||
|
assertEquals(30, scanned);
|
||||||
|
}
|
||||||
|
|
||||||
|
private long getNumberOfMobFiles(Configuration conf, TableName tableName, String family)
|
||||||
|
throws IOException {
|
||||||
|
FileSystem fs = FileSystem.get(conf);
|
||||||
|
Path dir = MobUtils.getMobFamilyPath(conf, tableName, family);
|
||||||
|
FileStatus[] stat = fs.listStatus(dir);
|
||||||
|
for (FileStatus st : stat) {
|
||||||
|
LOG.debug("DDDD MOB Directory content: {} size={}", st.getPath(), st.getLen());
|
||||||
|
}
|
||||||
|
LOG.debug("MOB Directory content total files: {}", stat.length);
|
||||||
|
|
||||||
|
return stat.length;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
private long scanTable() {
|
||||||
|
try {
|
||||||
|
|
||||||
|
Result result;
|
||||||
|
ResultScanner scanner = table.getScanner(fam);
|
||||||
|
long counter = 0;
|
||||||
|
while ((result = scanner.next()) != null) {
|
||||||
|
assertTrue(Arrays.equals(result.getValue(fam, qualifier), mobVal));
|
||||||
|
counter++;
|
||||||
|
}
|
||||||
|
return counter;
|
||||||
|
} catch (Exception e) {
|
||||||
|
e.printStackTrace();
|
||||||
|
LOG.error("MOB file cleaner chore test FAILED");
|
||||||
|
if (HTU != null) {
|
||||||
|
assertTrue(false);
|
||||||
|
} else {
|
||||||
|
System.exit(-1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
}
|
|
@ -47,6 +47,7 @@ public class TestMobFileName {
|
||||||
private Date date;
|
private Date date;
|
||||||
private String dateStr;
|
private String dateStr;
|
||||||
private byte[] startKey;
|
private byte[] startKey;
|
||||||
|
private String regionName = "region";
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setUp() {
|
public void setUp() {
|
||||||
|
@ -59,32 +60,32 @@ public class TestMobFileName {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testHashCode() {
|
public void testHashCode() {
|
||||||
assertEquals(MobFileName.create(startKey, dateStr, uuid).hashCode(),
|
assertEquals(MobFileName.create(startKey, dateStr, uuid, regionName).hashCode(),
|
||||||
MobFileName.create(startKey, dateStr, uuid).hashCode());
|
MobFileName.create(startKey, dateStr, uuid, regionName).hashCode());
|
||||||
assertNotSame(MobFileName.create(startKey, dateStr, uuid),
|
assertNotSame(MobFileName.create(startKey, dateStr, uuid, regionName),
|
||||||
MobFileName.create(startKey, dateStr, uuid));
|
MobFileName.create(startKey, dateStr, uuid, regionName));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testCreate() {
|
public void testCreate() {
|
||||||
MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid);
|
MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid, regionName);
|
||||||
assertEquals(mobFileName, MobFileName.create(mobFileName.getFileName()));
|
assertEquals(mobFileName, MobFileName.create(mobFileName.getFileName()));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testGet() {
|
public void testGet() {
|
||||||
MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid);
|
MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid, regionName);
|
||||||
assertEquals(MD5Hash.getMD5AsHex(startKey, 0, startKey.length), mobFileName.getStartKey());
|
assertEquals(MD5Hash.getMD5AsHex(startKey, 0, startKey.length), mobFileName.getStartKey());
|
||||||
assertEquals(dateStr, mobFileName.getDate());
|
assertEquals(dateStr, mobFileName.getDate());
|
||||||
assertEquals(mobFileName.getFileName(), MD5Hash.getMD5AsHex(startKey, 0, startKey.length)
|
assertEquals(mobFileName.getFileName(), MD5Hash.getMD5AsHex(startKey, 0, startKey.length)
|
||||||
+ dateStr + uuid);
|
+ dateStr + uuid+"_"+regionName);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testEquals() {
|
public void testEquals() {
|
||||||
MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid);
|
MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid, regionName);
|
||||||
assertTrue(mobFileName.equals(mobFileName));
|
assertTrue(mobFileName.equals(mobFileName));
|
||||||
assertFalse(mobFileName.equals(this));
|
assertFalse(mobFileName.equals(this));
|
||||||
assertTrue(mobFileName.equals(MobFileName.create(startKey, dateStr, uuid)));
|
assertTrue(mobFileName.equals(MobFileName.create(startKey, dateStr, uuid, regionName)));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -15,7 +15,7 @@
|
||||||
* See the License for the specific language governing permissions and
|
* See the License for the specific language governing permissions and
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.regionserver;
|
package org.apache.hadoop.hbase.mob;
|
||||||
|
|
||||||
import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY;
|
import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY;
|
||||||
import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
|
import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
|
||||||
|
@ -29,6 +29,7 @@ import java.util.ArrayList;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Optional;
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
|
@ -37,8 +38,6 @@ import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hbase.Cell;
|
import org.apache.hadoop.hbase.Cell;
|
||||||
import org.apache.hadoop.hbase.CellComparatorImpl;
|
|
||||||
import org.apache.hadoop.hbase.CellUtil;
|
|
||||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||||
|
@ -60,10 +59,17 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||||
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
|
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
|
||||||
import org.apache.hadoop.hbase.mob.MobConstants;
|
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||||
import org.apache.hadoop.hbase.mob.MobFileCache;
|
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
import org.apache.hadoop.hbase.regionserver.HStoreFile;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.RegionAsTable;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
|
||||||
|
import org.apache.hadoop.hbase.security.User;
|
||||||
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
|
@ -79,7 +85,7 @@ import org.slf4j.LoggerFactory;
|
||||||
/**
|
/**
|
||||||
* Test mob store compaction
|
* Test mob store compaction
|
||||||
*/
|
*/
|
||||||
@Category(SmallTests.class)
|
@Category(MediumTests.class)
|
||||||
public class TestMobStoreCompaction {
|
public class TestMobStoreCompaction {
|
||||||
|
|
||||||
@ClassRule
|
@ClassRule
|
||||||
|
@ -178,7 +184,20 @@ public class TestMobStoreCompaction {
|
||||||
// Change the threshold larger than the data size
|
// Change the threshold larger than the data size
|
||||||
setMobThreshold(region, COLUMN_FAMILY, 500);
|
setMobThreshold(region, COLUMN_FAMILY, 500);
|
||||||
region.initialize();
|
region.initialize();
|
||||||
region.compactStores();
|
|
||||||
|
List<HStore> stores = region.getStores();
|
||||||
|
for (HStore store: stores) {
|
||||||
|
// Force major compaction
|
||||||
|
store.triggerMajorCompaction();
|
||||||
|
Optional<CompactionContext> context =
|
||||||
|
store.requestCompaction(HStore.PRIORITY_USER, CompactionLifeCycleTracker.DUMMY,
|
||||||
|
User.getCurrent());
|
||||||
|
if (!context.isPresent()) {
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
region.compact(context.get(), store,
|
||||||
|
NoLimitThroughputController.INSTANCE, User.getCurrent());
|
||||||
|
}
|
||||||
|
|
||||||
assertEquals("After compaction: store files", 1, countStoreFiles());
|
assertEquals("After compaction: store files", 1, countStoreFiles());
|
||||||
assertEquals("After compaction: mob file count", compactionThreshold, countMobFiles());
|
assertEquals("After compaction: mob file count", compactionThreshold, countMobFiles());
|
||||||
|
@ -270,30 +289,6 @@ public class TestMobStoreCompaction {
|
||||||
// region.compactStores();
|
// region.compactStores();
|
||||||
region.compact(true);
|
region.compact(true);
|
||||||
assertEquals("After compaction: store files", 1, countStoreFiles());
|
assertEquals("After compaction: store files", 1, countStoreFiles());
|
||||||
// still have original mob hfiles and now added a mob del file
|
|
||||||
assertEquals("After compaction: mob files", numHfiles + 1, countMobFiles());
|
|
||||||
|
|
||||||
Scan scan = new Scan();
|
|
||||||
scan.setRaw(true);
|
|
||||||
InternalScanner scanner = region.getScanner(scan);
|
|
||||||
List<Cell> results = new ArrayList<>();
|
|
||||||
scanner.next(results);
|
|
||||||
int deleteCount = 0;
|
|
||||||
while (!results.isEmpty()) {
|
|
||||||
for (Cell c : results) {
|
|
||||||
if (c.getTypeByte() == KeyValue.Type.DeleteFamily.getCode()) {
|
|
||||||
deleteCount++;
|
|
||||||
assertTrue(Bytes.equals(CellUtil.cloneRow(c), deleteRow));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
results.clear();
|
|
||||||
scanner.next(results);
|
|
||||||
}
|
|
||||||
// assert the delete mark is retained after the major compaction
|
|
||||||
assertEquals(1, deleteCount);
|
|
||||||
scanner.close();
|
|
||||||
// assert the deleted cell is not counted
|
|
||||||
assertEquals("The cells in mob files", numHfiles - 1, countMobCellsInMobFiles(1));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private int countStoreFiles() throws IOException {
|
private int countStoreFiles() throws IOException {
|
||||||
|
@ -424,38 +419,4 @@ public class TestMobStoreCompaction {
|
||||||
return files.size();
|
return files.size();
|
||||||
}
|
}
|
||||||
|
|
||||||
private int countMobCellsInMobFiles(int expectedNumDelfiles) throws IOException {
|
|
||||||
Configuration copyOfConf = new Configuration(conf);
|
|
||||||
copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f);
|
|
||||||
CacheConfig cacheConfig = new CacheConfig(copyOfConf);
|
|
||||||
Path mobDirPath = MobUtils.getMobFamilyPath(conf, htd.getTableName(), hcd.getNameAsString());
|
|
||||||
List<HStoreFile> sfs = new ArrayList<>();
|
|
||||||
int numDelfiles = 0;
|
|
||||||
int size = 0;
|
|
||||||
if (fs.exists(mobDirPath)) {
|
|
||||||
for (FileStatus f : fs.listStatus(mobDirPath)) {
|
|
||||||
HStoreFile sf = new HStoreFile(fs, f.getPath(), conf, cacheConfig, BloomType.NONE, true);
|
|
||||||
sfs.add(sf);
|
|
||||||
if (StoreFileInfo.isDelFile(sf.getPath())) {
|
|
||||||
numDelfiles++;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
List<StoreFileScanner> scanners = StoreFileScanner.getScannersForStoreFiles(sfs, false, true,
|
|
||||||
false, false, HConstants.LATEST_TIMESTAMP);
|
|
||||||
long timeToPurgeDeletes = Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0);
|
|
||||||
long ttl = HStore.determineTTLFromFamily(hcd);
|
|
||||||
ScanInfo scanInfo = new ScanInfo(copyOfConf, hcd, ttl, timeToPurgeDeletes,
|
|
||||||
CellComparatorImpl.COMPARATOR);
|
|
||||||
StoreScanner scanner = new StoreScanner(scanInfo, ScanType.COMPACT_DROP_DELETES, scanners);
|
|
||||||
try {
|
|
||||||
size += UTIL.countRows(scanner);
|
|
||||||
} finally {
|
|
||||||
scanner.close();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// assert the number of the existing del files
|
|
||||||
assertEquals(expectedNumDelfiles, numDelfiles);
|
|
||||||
return size;
|
|
||||||
}
|
|
||||||
}
|
}
|
|
@ -15,7 +15,7 @@
|
||||||
* See the License for the specific language governing permissions and
|
* See the License for the specific language governing permissions and
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.regionserver;
|
package org.apache.hadoop.hbase.mob;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -43,9 +43,7 @@ import org.apache.hadoop.hbase.client.Scan;
|
||||||
import org.apache.hadoop.hbase.client.Table;
|
import org.apache.hadoop.hbase.client.Table;
|
||||||
import org.apache.hadoop.hbase.io.hfile.CorruptHFileException;
|
import org.apache.hadoop.hbase.io.hfile.CorruptHFileException;
|
||||||
import org.apache.hadoop.hbase.io.hfile.TestHFile;
|
import org.apache.hadoop.hbase.io.hfile.TestHFile;
|
||||||
import org.apache.hadoop.hbase.mob.MobConstants;
|
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||||
import org.apache.hadoop.hbase.mob.MobTestUtil;
|
|
||||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
|
||||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
import org.apache.hadoop.hbase.util.FSUtils;
|
||||||
|
@ -141,8 +139,8 @@ public class TestMobStoreScanner {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testMobStoreScanner() throws Exception {
|
public void testMobStoreScanner() throws Exception {
|
||||||
testGetFromFiles(false);
|
testGetFromFiles(false);
|
||||||
testGetFromMemStore(false);
|
testGetFromMemStore(false);
|
||||||
testGetReferences(false);
|
testGetReferences(false);
|
||||||
testMobThreshold(false);
|
testMobThreshold(false);
|
||||||
testGetFromArchive(false);
|
testGetFromArchive(false);
|
||||||
|
@ -150,8 +148,8 @@ public class TestMobStoreScanner {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testReversedMobStoreScanner() throws Exception {
|
public void testReversedMobStoreScanner() throws Exception {
|
||||||
testGetFromFiles(true);
|
testGetFromFiles(true);
|
||||||
testGetFromMemStore(true);
|
testGetFromMemStore(true);
|
||||||
testGetReferences(true);
|
testGetReferences(true);
|
||||||
testMobThreshold(true);
|
testMobThreshold(true);
|
||||||
testGetFromArchive(true);
|
testGetFromArchive(true);
|
File diff suppressed because it is too large
Load Diff
|
@ -1,65 +0,0 @@
|
||||||
/**
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
package org.apache.hadoop.hbase.mob.compactions;
|
|
||||||
|
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
|
||||||
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartition;
|
|
||||||
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId;
|
|
||||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
|
||||||
import org.junit.Assert;
|
|
||||||
import org.junit.ClassRule;
|
|
||||||
import org.junit.Test;
|
|
||||||
import org.junit.experimental.categories.Category;
|
|
||||||
|
|
||||||
@Category(SmallTests.class)
|
|
||||||
public class TestPartitionedMobCompactionRequest {
|
|
||||||
|
|
||||||
@ClassRule
|
|
||||||
public static final HBaseClassTestRule CLASS_RULE =
|
|
||||||
HBaseClassTestRule.forClass(TestPartitionedMobCompactionRequest.class);
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCompactedPartitionId() {
|
|
||||||
String startKey1 = "startKey1";
|
|
||||||
String startKey2 = "startKey2";
|
|
||||||
String date1 = "date1";
|
|
||||||
String date2 = "date2";
|
|
||||||
CompactionPartitionId partitionId1 = new CompactionPartitionId(startKey1, date1);
|
|
||||||
CompactionPartitionId partitionId2 = new CompactionPartitionId(startKey2, date2);
|
|
||||||
CompactionPartitionId partitionId3 = new CompactionPartitionId(startKey1, date2);
|
|
||||||
|
|
||||||
Assert.assertTrue(partitionId1.equals(partitionId1));
|
|
||||||
Assert.assertFalse(partitionId1.equals(partitionId2));
|
|
||||||
Assert.assertFalse(partitionId1.equals(partitionId3));
|
|
||||||
Assert.assertFalse(partitionId2.equals(partitionId3));
|
|
||||||
|
|
||||||
Assert.assertEquals(startKey1, partitionId1.getStartKey());
|
|
||||||
Assert.assertEquals(date1, partitionId1.getDate());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCompactedPartition() {
|
|
||||||
CompactionPartitionId partitionId = new CompactionPartitionId("startKey1", "date1");
|
|
||||||
CompactionPartition partition = new CompactionPartition(partitionId);
|
|
||||||
FileStatus file = new FileStatus(1, false, 1, 1024, 1, new Path("/test"));
|
|
||||||
partition.addFile(file);
|
|
||||||
Assert.assertEquals(file, partition.listFiles().get(0));
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -1,961 +0,0 @@
|
||||||
/**
|
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
|
||||||
* or more contributor license agreements. See the NOTICE file
|
|
||||||
* distributed with this work for additional information
|
|
||||||
* regarding copyright ownership. The ASF licenses this file
|
|
||||||
* to you under the Apache License, Version 2.0 (the
|
|
||||||
* "License"); you may not use this file except in compliance
|
|
||||||
* with the License. You may obtain a copy of the License at
|
|
||||||
*
|
|
||||||
* http://www.apache.org/licenses/LICENSE-2.0
|
|
||||||
*
|
|
||||||
* Unless required by applicable law or agreed to in writing, software
|
|
||||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
|
||||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
||||||
* See the License for the specific language governing permissions and
|
|
||||||
* limitations under the License.
|
|
||||||
*/
|
|
||||||
package org.apache.hadoop.hbase.mob.compactions;
|
|
||||||
|
|
||||||
import static org.junit.Assert.assertEquals;
|
|
||||||
import static org.junit.Assert.assertTrue;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.text.ParseException;
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Calendar;
|
|
||||||
import java.util.Collection;
|
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.Date;
|
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Random;
|
|
||||||
import java.util.concurrent.ExecutorService;
|
|
||||||
import java.util.concurrent.RejectedExecutionException;
|
|
||||||
import java.util.concurrent.RejectedExecutionHandler;
|
|
||||||
import java.util.concurrent.SynchronousQueue;
|
|
||||||
import java.util.concurrent.ThreadPoolExecutor;
|
|
||||||
import java.util.concurrent.TimeUnit;
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.hadoop.hbase.Cell;
|
|
||||||
import org.apache.hadoop.hbase.CellComparatorImpl;
|
|
||||||
import org.apache.hadoop.hbase.CellUtil;
|
|
||||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
|
||||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
|
||||||
import org.apache.hadoop.hbase.KeyValue;
|
|
||||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
|
||||||
import org.apache.hadoop.hbase.TableName;
|
|
||||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
|
||||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
|
||||||
import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy;
|
|
||||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
|
||||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
|
||||||
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
|
|
||||||
import org.apache.hadoop.hbase.mob.MobConstants;
|
|
||||||
import org.apache.hadoop.hbase.mob.MobFileName;
|
|
||||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
|
||||||
import org.apache.hadoop.hbase.mob.compactions.MobCompactionRequest.CompactionType;
|
|
||||||
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionDelPartition;
|
|
||||||
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartition;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.HStoreFile;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.ScanInfo;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.ScanType;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.StoreScanner;
|
|
||||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
|
||||||
import org.apache.hadoop.hbase.util.Threads;
|
|
||||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
||||||
import org.junit.AfterClass;
|
|
||||||
import org.junit.BeforeClass;
|
|
||||||
import org.junit.ClassRule;
|
|
||||||
import org.junit.Rule;
|
|
||||||
import org.junit.Test;
|
|
||||||
import org.junit.experimental.categories.Category;
|
|
||||||
import org.junit.rules.TestName;
|
|
||||||
import org.slf4j.Logger;
|
|
||||||
import org.slf4j.LoggerFactory;
|
|
||||||
|
|
||||||
@Category(LargeTests.class)
|
|
||||||
public class TestPartitionedMobCompactor {
|
|
||||||
|
|
||||||
@ClassRule
|
|
||||||
public static final HBaseClassTestRule CLASS_RULE =
|
|
||||||
HBaseClassTestRule.forClass(TestPartitionedMobCompactor.class);
|
|
||||||
|
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(TestPartitionedMobCompactor.class);
|
|
||||||
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
|
||||||
private final static String family = "family";
|
|
||||||
private final static String qf = "qf";
|
|
||||||
private final long DAY_IN_MS = 1000 * 60 * 60 * 24;
|
|
||||||
private static byte[] KEYS = Bytes.toBytes("012");
|
|
||||||
private ColumnFamilyDescriptorBuilder cfdb =
|
|
||||||
ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family));
|
|
||||||
private Configuration conf = TEST_UTIL.getConfiguration();
|
|
||||||
private CacheConfig cacheConf = new CacheConfig(conf);
|
|
||||||
private FileSystem fs;
|
|
||||||
private List<FileStatus> mobFiles = new ArrayList<>();
|
|
||||||
private List<Path> delFiles = new ArrayList<>();
|
|
||||||
private List<FileStatus> allFiles = new ArrayList<>();
|
|
||||||
private Path basePath;
|
|
||||||
private String mobSuffix;
|
|
||||||
private String delSuffix;
|
|
||||||
private static ExecutorService pool;
|
|
||||||
|
|
||||||
@Rule
|
|
||||||
public TestName name = new TestName();
|
|
||||||
|
|
||||||
@BeforeClass
|
|
||||||
public static void setUpBeforeClass() throws Exception {
|
|
||||||
TEST_UTIL.getConfiguration().setInt("hfile.format.version", 3);
|
|
||||||
// Inject our customized DistributedFileSystem
|
|
||||||
TEST_UTIL.getConfiguration().setClass("fs.hdfs.impl", FaultyDistributedFileSystem.class,
|
|
||||||
DistributedFileSystem.class);
|
|
||||||
TEST_UTIL.startMiniCluster(1);
|
|
||||||
pool = createThreadPool();
|
|
||||||
}
|
|
||||||
|
|
||||||
@AfterClass
|
|
||||||
public static void tearDownAfterClass() throws Exception {
|
|
||||||
pool.shutdown();
|
|
||||||
TEST_UTIL.shutdownMiniCluster();
|
|
||||||
}
|
|
||||||
|
|
||||||
private void init(String tableName) throws Exception {
|
|
||||||
fs = FileSystem.get(conf);
|
|
||||||
Path testDir = FSUtils.getRootDir(conf);
|
|
||||||
Path mobTestDir = new Path(testDir, MobConstants.MOB_DIR_NAME);
|
|
||||||
basePath = new Path(new Path(mobTestDir, tableName), family);
|
|
||||||
mobSuffix = TEST_UTIL.getRandomUUID().toString().replaceAll("-", "");
|
|
||||||
delSuffix = TEST_UTIL.getRandomUUID().toString().replaceAll("-", "") + "_del";
|
|
||||||
allFiles.clear();
|
|
||||||
mobFiles.clear();
|
|
||||||
delFiles.clear();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCompactionSelectAllFilesWeeklyPolicy() throws Exception {
|
|
||||||
String tableName = "testCompactionSelectAllFilesWeeklyPolicy";
|
|
||||||
testCompactionAtMergeSize(tableName, MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD,
|
|
||||||
CompactionType.ALL_FILES, false, false, new Date(), MobCompactPartitionPolicy.WEEKLY, 1);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCompactionSelectPartFilesWeeklyPolicy() throws Exception {
|
|
||||||
String tableName = "testCompactionSelectPartFilesWeeklyPolicy";
|
|
||||||
testCompactionAtMergeSize(tableName, 4000, CompactionType.PART_FILES, false, false,
|
|
||||||
new Date(), MobCompactPartitionPolicy.WEEKLY, 1);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCompactionSelectPartFilesWeeklyPolicyWithPastWeek() throws Exception {
|
|
||||||
String tableName = "testCompactionSelectPartFilesWeeklyPolicyWithPastWeek";
|
|
||||||
Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS));
|
|
||||||
testCompactionAtMergeSize(tableName, 700, CompactionType.PART_FILES, false, false, dateLastWeek,
|
|
||||||
MobCompactPartitionPolicy.WEEKLY, 7);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCompactionSelectAllFilesWeeklyPolicyWithPastWeek() throws Exception {
|
|
||||||
String tableName = "testCompactionSelectAllFilesWeeklyPolicyWithPastWeek";
|
|
||||||
Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS));
|
|
||||||
testCompactionAtMergeSize(tableName, 3000, CompactionType.ALL_FILES,
|
|
||||||
false, false, dateLastWeek, MobCompactPartitionPolicy.WEEKLY, 7);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCompactionSelectAllFilesMonthlyPolicy() throws Exception {
|
|
||||||
String tableName = "testCompactionSelectAllFilesMonthlyPolicy";
|
|
||||||
Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS));
|
|
||||||
testCompactionAtMergeSize(tableName, MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD,
|
|
||||||
CompactionType.ALL_FILES, false, false, dateLastWeek,
|
|
||||||
MobCompactPartitionPolicy.MONTHLY, 7);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCompactionSelectNoFilesWithinCurrentWeekMonthlyPolicy() throws Exception {
|
|
||||||
String tableName = "testCompactionSelectNoFilesWithinCurrentWeekMonthlyPolicy";
|
|
||||||
testCompactionAtMergeSize(tableName, MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD,
|
|
||||||
CompactionType.PART_FILES, false, false, new Date(), MobCompactPartitionPolicy.MONTHLY, 1);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCompactionSelectPartFilesMonthlyPolicy() throws Exception {
|
|
||||||
String tableName = "testCompactionSelectPartFilesMonthlyPolicy";
|
|
||||||
testCompactionAtMergeSize(tableName, 4000, CompactionType.PART_FILES, false, false,
|
|
||||||
new Date(), MobCompactPartitionPolicy.MONTHLY, 1);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCompactionSelectPartFilesMonthlyPolicyWithPastWeek() throws Exception {
|
|
||||||
String tableName = "testCompactionSelectPartFilesMonthlyPolicyWithPastWeek";
|
|
||||||
Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS));
|
|
||||||
Calendar calendar = Calendar.getInstance();
|
|
||||||
Date firstDayOfCurrentMonth = MobUtils.getFirstDayOfMonth(calendar, new Date());
|
|
||||||
CompactionType type = CompactionType.PART_FILES;
|
|
||||||
long mergeSizeMultiFactor = 7;
|
|
||||||
|
|
||||||
|
|
||||||
// The dateLastWeek may not really be last week, suppose that it runs at 2/1/2017, it is going
|
|
||||||
// to be last month and the monthly policy is going to be applied here.
|
|
||||||
if (dateLastWeek.before(firstDayOfCurrentMonth)) {
|
|
||||||
type = CompactionType.ALL_FILES;
|
|
||||||
mergeSizeMultiFactor *= 4;
|
|
||||||
}
|
|
||||||
|
|
||||||
testCompactionAtMergeSize(tableName, 700, type, false, false, dateLastWeek,
|
|
||||||
MobCompactPartitionPolicy.MONTHLY, mergeSizeMultiFactor);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCompactionSelectAllFilesMonthlyPolicyWithPastWeek() throws Exception {
|
|
||||||
String tableName = "testCompactionSelectAllFilesMonthlyPolicyWithPastWeek";
|
|
||||||
Date dateLastWeek = new Date(System.currentTimeMillis() - (7 * DAY_IN_MS));
|
|
||||||
|
|
||||||
testCompactionAtMergeSize(tableName, 3000, CompactionType.ALL_FILES,
|
|
||||||
false, false, dateLastWeek, MobCompactPartitionPolicy.MONTHLY, 7);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCompactionSelectPartFilesMonthlyPolicyWithPastMonth() throws Exception {
|
|
||||||
String tableName = "testCompactionSelectPartFilesMonthlyPolicyWithPastMonth";
|
|
||||||
|
|
||||||
// back 5 weeks, it is going to be a past month
|
|
||||||
Date dateLastMonth = new Date(System.currentTimeMillis() - (7 * 5 * DAY_IN_MS));
|
|
||||||
testCompactionAtMergeSize(tableName, 200, CompactionType.PART_FILES, false, false, dateLastMonth,
|
|
||||||
MobCompactPartitionPolicy.MONTHLY, 28);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCompactionSelectAllFilesMonthlyPolicyWithPastMonth() throws Exception {
|
|
||||||
String tableName = "testCompactionSelectAllFilesMonthlyPolicyWithPastMonth";
|
|
||||||
|
|
||||||
// back 5 weeks, it is going to be a past month
|
|
||||||
Date dateLastMonth = new Date(System.currentTimeMillis() - (7 * 5 * DAY_IN_MS));
|
|
||||||
testCompactionAtMergeSize(tableName, 750, CompactionType.ALL_FILES,
|
|
||||||
false, false, dateLastMonth, MobCompactPartitionPolicy.MONTHLY, 28);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCompactionSelectWithAllFiles() throws Exception {
|
|
||||||
String tableName = "testCompactionSelectWithAllFiles";
|
|
||||||
// If there is only 1 file, it will not be compacted with _del files, so
|
|
||||||
// It wont be CompactionType.ALL_FILES in this case, do not create with _del files.
|
|
||||||
testCompactionAtMergeSize(tableName, MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD,
|
|
||||||
CompactionType.ALL_FILES, false, false);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCompactionSelectWithPartFiles() throws Exception {
|
|
||||||
String tableName = "testCompactionSelectWithPartFiles";
|
|
||||||
testCompactionAtMergeSize(tableName, 4000, CompactionType.PART_FILES, false);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCompactionSelectWithForceAllFiles() throws Exception {
|
|
||||||
String tableName = "testCompactionSelectWithForceAllFiles";
|
|
||||||
testCompactionAtMergeSize(tableName, Long.MAX_VALUE, CompactionType.ALL_FILES, true);
|
|
||||||
}
|
|
||||||
|
|
||||||
private void testCompactionAtMergeSize(final String tableName,
|
|
||||||
final long mergeSize, final CompactionType type, final boolean isForceAllFiles)
|
|
||||||
throws Exception {
|
|
||||||
testCompactionAtMergeSize(tableName, mergeSize, type, isForceAllFiles, true);
|
|
||||||
}
|
|
||||||
|
|
||||||
private void testCompactionAtMergeSize(final String tableName,
|
|
||||||
final long mergeSize, final CompactionType type, final boolean isForceAllFiles,
|
|
||||||
final boolean createDelFiles)
|
|
||||||
throws Exception {
|
|
||||||
Date date = new Date();
|
|
||||||
testCompactionAtMergeSize(tableName, mergeSize, type, isForceAllFiles, createDelFiles, date);
|
|
||||||
}
|
|
||||||
|
|
||||||
private void testCompactionAtMergeSize(final String tableName,
|
|
||||||
final long mergeSize, final CompactionType type, final boolean isForceAllFiles,
|
|
||||||
final boolean createDelFiles, final Date date)
|
|
||||||
throws Exception {
|
|
||||||
testCompactionAtMergeSize(tableName, mergeSize, type, isForceAllFiles, createDelFiles, date,
|
|
||||||
MobCompactPartitionPolicy.DAILY, 1);
|
|
||||||
}
|
|
||||||
|
|
||||||
private void testCompactionAtMergeSize(final String tableName,
|
|
||||||
final long mergeSize, final CompactionType type, final boolean isForceAllFiles,
|
|
||||||
final boolean createDelFiles, final Date date, final MobCompactPartitionPolicy policy,
|
|
||||||
final long mergeSizeMultiFactor)
|
|
||||||
throws Exception {
|
|
||||||
resetConf();
|
|
||||||
init(tableName);
|
|
||||||
int count = 10;
|
|
||||||
// create 10 mob files.
|
|
||||||
createStoreFiles(basePath, family, qf, count, Type.Put, date);
|
|
||||||
|
|
||||||
if (createDelFiles) {
|
|
||||||
// create 10 del files
|
|
||||||
createStoreFiles(basePath, family, qf, count, Type.Delete, date);
|
|
||||||
}
|
|
||||||
|
|
||||||
Calendar calendar = Calendar.getInstance();
|
|
||||||
Date firstDayOfCurrentWeek = MobUtils.getFirstDayOfWeek(calendar, new Date());
|
|
||||||
|
|
||||||
listFiles();
|
|
||||||
List<String> expectedStartKeys = new ArrayList<>();
|
|
||||||
for(FileStatus file : mobFiles) {
|
|
||||||
if(file.getLen() < mergeSize * mergeSizeMultiFactor) {
|
|
||||||
String fileName = file.getPath().getName();
|
|
||||||
String startKey = fileName.substring(0, 32);
|
|
||||||
|
|
||||||
// If the policy is monthly and files are in current week, they will be skipped
|
|
||||||
// in minor compcation.
|
|
||||||
boolean skipCompaction = false;
|
|
||||||
if (policy == MobCompactPartitionPolicy.MONTHLY) {
|
|
||||||
String fileDateStr = MobFileName.getDateFromName(fileName);
|
|
||||||
Date fileDate;
|
|
||||||
try {
|
|
||||||
fileDate = MobUtils.parseDate(fileDateStr);
|
|
||||||
} catch (ParseException e) {
|
|
||||||
LOG.warn("Failed to parse date " + fileDateStr, e);
|
|
||||||
fileDate = new Date();
|
|
||||||
}
|
|
||||||
if (!fileDate.before(firstDayOfCurrentWeek)) {
|
|
||||||
skipCompaction = true;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// If it is not an major mob compaction and del files are there,
|
|
||||||
// these mob files wont be compacted.
|
|
||||||
if (isForceAllFiles || (!createDelFiles && !skipCompaction)) {
|
|
||||||
expectedStartKeys.add(startKey);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Set the policy
|
|
||||||
this.cfdb.setMobCompactPartitionPolicy(policy);
|
|
||||||
// set the mob compaction mergeable threshold
|
|
||||||
conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize);
|
|
||||||
testSelectFiles(tableName, type, isForceAllFiles, expectedStartKeys);
|
|
||||||
// go back to the default daily policy
|
|
||||||
this.cfdb.setMobCompactPartitionPolicy(MobCompactPartitionPolicy.DAILY);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCompactDelFilesWithDefaultBatchSize() throws Exception {
|
|
||||||
testCompactDelFilesAtBatchSize(name.getMethodName(), MobConstants.DEFAULT_MOB_COMPACTION_BATCH_SIZE,
|
|
||||||
MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCompactDelFilesWithSmallBatchSize() throws Exception {
|
|
||||||
testCompactDelFilesAtBatchSize(name.getMethodName(), 4, MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCompactDelFilesChangeMaxDelFileCount() throws Exception {
|
|
||||||
testCompactDelFilesAtBatchSize(name.getMethodName(), 4, 2);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCompactFilesWithDstDirFull() throws Exception {
|
|
||||||
String tableName = name.getMethodName();
|
|
||||||
fs = FileSystem.get(conf);
|
|
||||||
FaultyDistributedFileSystem faultyFs = (FaultyDistributedFileSystem)fs;
|
|
||||||
Path testDir = FSUtils.getRootDir(conf);
|
|
||||||
Path mobTestDir = new Path(testDir, MobConstants.MOB_DIR_NAME);
|
|
||||||
basePath = new Path(new Path(mobTestDir, tableName), family);
|
|
||||||
|
|
||||||
try {
|
|
||||||
int count = 2;
|
|
||||||
// create 2 mob files.
|
|
||||||
createStoreFiles(basePath, family, qf, count, Type.Put, true, new Date());
|
|
||||||
listFiles();
|
|
||||||
|
|
||||||
TableName tName = TableName.valueOf(tableName);
|
|
||||||
MobCompactor compactor = new PartitionedMobCompactor(conf, faultyFs, tName,
|
|
||||||
cfdb.build(), pool);
|
|
||||||
faultyFs.setThrowException(true);
|
|
||||||
try {
|
|
||||||
compactor.compact(allFiles, true);
|
|
||||||
} catch (IOException e) {
|
|
||||||
System.out.println("Expected exception, ignore");
|
|
||||||
}
|
|
||||||
|
|
||||||
// Verify that all the files in tmp directory are cleaned up
|
|
||||||
Path tempPath = new Path(MobUtils.getMobHome(conf), MobConstants.TEMP_DIR_NAME);
|
|
||||||
FileStatus[] ls = faultyFs.listStatus(tempPath);
|
|
||||||
|
|
||||||
// Only .bulkload under this directory
|
|
||||||
assertTrue(ls.length == 1);
|
|
||||||
assertTrue(MobConstants.BULKLOAD_DIR_NAME.equalsIgnoreCase(ls[0].getPath().getName()));
|
|
||||||
|
|
||||||
Path bulkloadPath = new Path(tempPath, new Path(MobConstants.BULKLOAD_DIR_NAME, new Path(
|
|
||||||
tName.getNamespaceAsString(), tName.getQualifierAsString())));
|
|
||||||
|
|
||||||
// Nothing in bulkLoad directory
|
|
||||||
FileStatus[] lsBulkload = faultyFs.listStatus(bulkloadPath);
|
|
||||||
assertTrue(lsBulkload.length == 0);
|
|
||||||
|
|
||||||
} finally {
|
|
||||||
faultyFs.setThrowException(false);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Create mulitple partition files
|
|
||||||
*/
|
|
||||||
private void createMobFile(Path basePath) throws IOException {
|
|
||||||
HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build();
|
|
||||||
MobFileName mobFileName = null;
|
|
||||||
int ii = 0;
|
|
||||||
Date today = new Date();
|
|
||||||
for (byte k0 : KEYS) {
|
|
||||||
byte[] startRow = Bytes.toBytes(ii++);
|
|
||||||
|
|
||||||
mobFileName = MobFileName.create(startRow, MobUtils.formatDate(today), mobSuffix);
|
|
||||||
|
|
||||||
StoreFileWriter mobFileWriter =
|
|
||||||
new StoreFileWriter.Builder(conf, cacheConf, fs).withFileContext(meta)
|
|
||||||
.withFilePath(new Path(basePath, mobFileName.getFileName())).build();
|
|
||||||
|
|
||||||
long now = System.currentTimeMillis();
|
|
||||||
try {
|
|
||||||
for (int i = 0; i < 10; i++) {
|
|
||||||
byte[] key = Bytes.add(Bytes.toBytes(k0), Bytes.toBytes(i));
|
|
||||||
byte[] dummyData = new byte[5000];
|
|
||||||
new Random().nextBytes(dummyData);
|
|
||||||
mobFileWriter.append(
|
|
||||||
new KeyValue(key, Bytes.toBytes(family), Bytes.toBytes(qf), now, Type.Put, dummyData));
|
|
||||||
}
|
|
||||||
} finally {
|
|
||||||
mobFileWriter.close();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Create mulitple partition delete files
|
|
||||||
*/
|
|
||||||
private void createMobDelFile(Path basePath, int startKey) throws IOException {
|
|
||||||
HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build();
|
|
||||||
MobFileName mobFileName = null;
|
|
||||||
Date today = new Date();
|
|
||||||
|
|
||||||
byte[] startRow = Bytes.toBytes(startKey);
|
|
||||||
|
|
||||||
mobFileName = MobFileName.create(startRow, MobUtils.formatDate(today), delSuffix);
|
|
||||||
|
|
||||||
StoreFileWriter mobFileWriter =
|
|
||||||
new StoreFileWriter.Builder(conf, cacheConf, fs).withFileContext(meta)
|
|
||||||
.withFilePath(new Path(basePath, mobFileName.getFileName())).build();
|
|
||||||
|
|
||||||
long now = System.currentTimeMillis();
|
|
||||||
try {
|
|
||||||
byte[] key = Bytes.add(Bytes.toBytes(KEYS[startKey]), Bytes.toBytes(0));
|
|
||||||
byte[] dummyData = new byte[5000];
|
|
||||||
new Random().nextBytes(dummyData);
|
|
||||||
mobFileWriter.append(
|
|
||||||
new KeyValue(key, Bytes.toBytes(family), Bytes.toBytes(qf), now, Type.Delete, dummyData));
|
|
||||||
key = Bytes.add(Bytes.toBytes(KEYS[startKey]), Bytes.toBytes(2));
|
|
||||||
mobFileWriter.append(
|
|
||||||
new KeyValue(key, Bytes.toBytes(family), Bytes.toBytes(qf), now, Type.Delete, dummyData));
|
|
||||||
key = Bytes.add(Bytes.toBytes(KEYS[startKey]), Bytes.toBytes(4));
|
|
||||||
mobFileWriter.append(
|
|
||||||
new KeyValue(key, Bytes.toBytes(family), Bytes.toBytes(qf), now, Type.Delete, dummyData));
|
|
||||||
|
|
||||||
} finally {
|
|
||||||
mobFileWriter.close();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCompactFilesWithoutDelFile() throws Exception {
|
|
||||||
String tableName = "testCompactFilesWithoutDelFile";
|
|
||||||
resetConf();
|
|
||||||
init(tableName);
|
|
||||||
|
|
||||||
createMobFile(basePath);
|
|
||||||
|
|
||||||
listFiles();
|
|
||||||
|
|
||||||
PartitionedMobCompactor compactor = new PartitionedMobCompactor(conf, fs,
|
|
||||||
TableName.valueOf(tableName), cfdb.build(), pool) {
|
|
||||||
@Override
|
|
||||||
public List<Path> compact(List<FileStatus> files, boolean isForceAllFiles)
|
|
||||||
throws IOException {
|
|
||||||
if (files == null || files.isEmpty()) {
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
PartitionedMobCompactionRequest request = select(files, isForceAllFiles);
|
|
||||||
|
|
||||||
// Make sure that there is no del Partitions
|
|
||||||
assertTrue(request.getDelPartitions().size() == 0);
|
|
||||||
|
|
||||||
// Make sure that when there is no startKey/endKey for partition.
|
|
||||||
for (CompactionPartition p : request.getCompactionPartitions()) {
|
|
||||||
assertTrue(p.getStartKey() == null);
|
|
||||||
assertTrue(p.getEndKey() == null);
|
|
||||||
}
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
compactor.compact(allFiles, true);
|
|
||||||
}
|
|
||||||
|
|
||||||
static class MyPartitionedMobCompactor extends PartitionedMobCompactor {
|
|
||||||
int delPartitionSize = 0;
|
|
||||||
int PartitionsIncludeDelFiles = 0;
|
|
||||||
CacheConfig cacheConfig = null;
|
|
||||||
|
|
||||||
MyPartitionedMobCompactor(Configuration conf, FileSystem fs, TableName tableName,
|
|
||||||
ColumnFamilyDescriptor column, ExecutorService pool, final int delPartitionSize,
|
|
||||||
final CacheConfig cacheConf, final int PartitionsIncludeDelFiles)
|
|
||||||
throws IOException {
|
|
||||||
super(conf, fs, tableName, column, pool);
|
|
||||||
this.delPartitionSize = delPartitionSize;
|
|
||||||
this.cacheConfig = cacheConf;
|
|
||||||
this.PartitionsIncludeDelFiles = PartitionsIncludeDelFiles;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override public List<Path> compact(List<FileStatus> files, boolean isForceAllFiles)
|
|
||||||
throws IOException {
|
|
||||||
if (files == null || files.isEmpty()) {
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
PartitionedMobCompactionRequest request = select(files, isForceAllFiles);
|
|
||||||
|
|
||||||
assertTrue(request.getDelPartitions().size() == delPartitionSize);
|
|
||||||
if (request.getDelPartitions().size() > 0) {
|
|
||||||
for (CompactionPartition p : request.getCompactionPartitions()) {
|
|
||||||
assertTrue(p.getStartKey() != null);
|
|
||||||
assertTrue(p.getEndKey() != null);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
try {
|
|
||||||
for (CompactionDelPartition delPartition : request.getDelPartitions()) {
|
|
||||||
for (Path newDelPath : delPartition.listDelFiles()) {
|
|
||||||
HStoreFile sf =
|
|
||||||
new HStoreFile(fs, newDelPath, conf, this.cacheConfig, BloomType.NONE, true);
|
|
||||||
// pre-create reader of a del file to avoid race condition when opening the reader in
|
|
||||||
// each partition.
|
|
||||||
sf.initReader();
|
|
||||||
delPartition.addStoreFile(sf);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Make sure that CompactionDelPartitions does not overlap
|
|
||||||
CompactionDelPartition prevDelP = null;
|
|
||||||
for (CompactionDelPartition delP : request.getDelPartitions()) {
|
|
||||||
assertTrue(
|
|
||||||
Bytes.compareTo(delP.getId().getStartKey(), delP.getId().getEndKey()) <= 0);
|
|
||||||
|
|
||||||
if (prevDelP != null) {
|
|
||||||
assertTrue(
|
|
||||||
Bytes.compareTo(prevDelP.getId().getEndKey(), delP.getId().getStartKey()) < 0);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
int affectedPartitions = 0;
|
|
||||||
|
|
||||||
// Make sure that only del files within key range for a partition is included in compaction.
|
|
||||||
// compact the mob files by partitions in parallel.
|
|
||||||
for (CompactionPartition partition : request.getCompactionPartitions()) {
|
|
||||||
List<HStoreFile> delFiles = getListOfDelFilesForPartition(partition, request.getDelPartitions());
|
|
||||||
if (!request.getDelPartitions().isEmpty()) {
|
|
||||||
if (!((Bytes.compareTo(request.getDelPartitions().get(0).getId().getStartKey(),
|
|
||||||
partition.getEndKey()) > 0) || (Bytes.compareTo(
|
|
||||||
request.getDelPartitions().get(request.getDelPartitions().size() - 1).getId()
|
|
||||||
.getEndKey(), partition.getStartKey()) < 0))) {
|
|
||||||
|
|
||||||
if (delFiles.size() > 0) {
|
|
||||||
assertTrue(delFiles.size() == 1);
|
|
||||||
affectedPartitions += delFiles.size();
|
|
||||||
assertTrue(Bytes.compareTo(partition.getStartKey(),
|
|
||||||
CellUtil.cloneRow(delFiles.get(0).getLastKey().get())) <= 0);
|
|
||||||
assertTrue(Bytes.compareTo(partition.getEndKey(),
|
|
||||||
CellUtil.cloneRow(delFiles.get(delFiles.size() - 1).getFirstKey().get())) >= 0);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// The del file is only included in one partition
|
|
||||||
assertTrue(affectedPartitions == PartitionsIncludeDelFiles);
|
|
||||||
} finally {
|
|
||||||
for (CompactionDelPartition delPartition : request.getDelPartitions()) {
|
|
||||||
for (HStoreFile storeFile : delPartition.getStoreFiles()) {
|
|
||||||
try {
|
|
||||||
storeFile.closeStoreFile(true);
|
|
||||||
} catch (IOException e) {
|
|
||||||
LOG.warn("Failed to close the reader on store file " + storeFile.getPath(), e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCompactFilesWithOneDelFile() throws Exception {
|
|
||||||
String tableName = "testCompactFilesWithOneDelFile";
|
|
||||||
resetConf();
|
|
||||||
init(tableName);
|
|
||||||
|
|
||||||
// Create only del file.
|
|
||||||
createMobFile(basePath);
|
|
||||||
createMobDelFile(basePath, 2);
|
|
||||||
|
|
||||||
listFiles();
|
|
||||||
|
|
||||||
MyPartitionedMobCompactor compactor = new MyPartitionedMobCompactor(conf, fs,
|
|
||||||
TableName.valueOf(tableName), cfdb.build(), pool, 1, cacheConf, 1);
|
|
||||||
|
|
||||||
compactor.compact(allFiles, true);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testCompactFilesWithMultiDelFiles() throws Exception {
|
|
||||||
String tableName = "testCompactFilesWithMultiDelFiles";
|
|
||||||
resetConf();
|
|
||||||
init(tableName);
|
|
||||||
|
|
||||||
// Create only del file.
|
|
||||||
createMobFile(basePath);
|
|
||||||
createMobDelFile(basePath, 0);
|
|
||||||
createMobDelFile(basePath, 1);
|
|
||||||
createMobDelFile(basePath, 2);
|
|
||||||
|
|
||||||
listFiles();
|
|
||||||
|
|
||||||
MyPartitionedMobCompactor compactor = new MyPartitionedMobCompactor(conf, fs,
|
|
||||||
TableName.valueOf(tableName), cfdb.build(), pool, 3, cacheConf, 3);
|
|
||||||
compactor.compact(allFiles, true);
|
|
||||||
}
|
|
||||||
|
|
||||||
private void testCompactDelFilesAtBatchSize(String tableName, int batchSize,
|
|
||||||
int delfileMaxCount) throws Exception {
|
|
||||||
resetConf();
|
|
||||||
init(tableName);
|
|
||||||
// create 20 mob files.
|
|
||||||
createStoreFiles(basePath, family, qf, 20, Type.Put, new Date());
|
|
||||||
// create 13 del files
|
|
||||||
createStoreFiles(basePath, family, qf, 13, Type.Delete, new Date());
|
|
||||||
listFiles();
|
|
||||||
|
|
||||||
// set the max del file count
|
|
||||||
conf.setInt(MobConstants.MOB_DELFILE_MAX_COUNT, delfileMaxCount);
|
|
||||||
// set the mob compaction batch size
|
|
||||||
conf.setInt(MobConstants.MOB_COMPACTION_BATCH_SIZE, batchSize);
|
|
||||||
testCompactDelFiles(tableName, 1, 13, false);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Tests the selectFiles
|
|
||||||
* @param tableName the table name
|
|
||||||
* @param type the expected compaction type
|
|
||||||
* @param isForceAllFiles whether all the mob files are selected
|
|
||||||
* @param expected the expected start keys
|
|
||||||
*/
|
|
||||||
private void testSelectFiles(String tableName, final CompactionType type,
|
|
||||||
final boolean isForceAllFiles, final List<String> expected) throws IOException {
|
|
||||||
PartitionedMobCompactor compactor = new PartitionedMobCompactor(conf, fs,
|
|
||||||
TableName.valueOf(tableName), cfdb.build(), pool) {
|
|
||||||
@Override
|
|
||||||
public List<Path> compact(List<FileStatus> files, boolean isForceAllFiles)
|
|
||||||
throws IOException {
|
|
||||||
if (files == null || files.isEmpty()) {
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
PartitionedMobCompactionRequest request = select(files, isForceAllFiles);
|
|
||||||
|
|
||||||
// Make sure that when there is no del files, there will be no startKey/endKey for partition.
|
|
||||||
if (request.getDelPartitions().size() == 0) {
|
|
||||||
for (CompactionPartition p : request.getCompactionPartitions()) {
|
|
||||||
assertTrue(p.getStartKey() == null);
|
|
||||||
assertTrue(p.getEndKey() == null);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Make sure that CompactionDelPartitions does not overlap
|
|
||||||
CompactionDelPartition prevDelP = null;
|
|
||||||
for (CompactionDelPartition delP : request.getDelPartitions()) {
|
|
||||||
assertTrue(Bytes.compareTo(delP.getId().getStartKey(),
|
|
||||||
delP.getId().getEndKey()) <= 0);
|
|
||||||
|
|
||||||
if (prevDelP != null) {
|
|
||||||
assertTrue(Bytes.compareTo(prevDelP.getId().getEndKey(),
|
|
||||||
delP.getId().getStartKey()) < 0);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Make sure that only del files within key range for a partition is included in compaction.
|
|
||||||
// compact the mob files by partitions in parallel.
|
|
||||||
for (CompactionPartition partition : request.getCompactionPartitions()) {
|
|
||||||
List<HStoreFile> delFiles = getListOfDelFilesForPartition(partition, request.getDelPartitions());
|
|
||||||
if (!request.getDelPartitions().isEmpty()) {
|
|
||||||
if (!((Bytes.compareTo(request.getDelPartitions().get(0).getId().getStartKey(),
|
|
||||||
partition.getEndKey()) > 0) || (Bytes.compareTo(
|
|
||||||
request.getDelPartitions().get(request.getDelPartitions().size() - 1).getId()
|
|
||||||
.getEndKey(), partition.getStartKey()) < 0))) {
|
|
||||||
if (delFiles.size() > 0) {
|
|
||||||
assertTrue(Bytes.compareTo(partition.getStartKey(),
|
|
||||||
delFiles.get(0).getFirstKey().get().getRowArray()) >= 0);
|
|
||||||
assertTrue(Bytes.compareTo(partition.getEndKey(),
|
|
||||||
delFiles.get(delFiles.size() - 1).getLastKey().get().getRowArray()) <= 0);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// assert the compaction type
|
|
||||||
assertEquals(type, request.type);
|
|
||||||
// assert get the right partitions
|
|
||||||
compareCompactedPartitions(expected, request.compactionPartitions);
|
|
||||||
// assert get the right del files
|
|
||||||
compareDelFiles(request.getDelPartitions());
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
};
|
|
||||||
compactor.compact(allFiles, isForceAllFiles);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Tests the compacteDelFile
|
|
||||||
* @param tableName the table name
|
|
||||||
* @param expectedFileCount the expected file count
|
|
||||||
* @param expectedCellCount the expected cell count
|
|
||||||
* @param isForceAllFiles whether all the mob files are selected
|
|
||||||
*/
|
|
||||||
private void testCompactDelFiles(String tableName, final int expectedFileCount,
|
|
||||||
final int expectedCellCount, boolean isForceAllFiles) throws IOException {
|
|
||||||
PartitionedMobCompactor compactor = new PartitionedMobCompactor(conf, fs,
|
|
||||||
TableName.valueOf(tableName), cfdb.build(), pool) {
|
|
||||||
@Override
|
|
||||||
protected List<Path> performCompaction(PartitionedMobCompactionRequest request)
|
|
||||||
throws IOException {
|
|
||||||
List<Path> delFilePaths = new ArrayList<>();
|
|
||||||
for (CompactionDelPartition delPartition: request.getDelPartitions()) {
|
|
||||||
for (Path p : delPartition.listDelFiles()) {
|
|
||||||
delFilePaths.add(p);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
List<Path> newDelPaths = compactDelFiles(request, delFilePaths);
|
|
||||||
// assert the del files are merged.
|
|
||||||
assertEquals(expectedFileCount, newDelPaths.size());
|
|
||||||
assertEquals(expectedCellCount, countDelCellsInDelFiles(newDelPaths));
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
};
|
|
||||||
compactor.compact(allFiles, isForceAllFiles);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Lists the files in the path
|
|
||||||
*/
|
|
||||||
private void listFiles() throws IOException {
|
|
||||||
for (FileStatus file : fs.listStatus(basePath)) {
|
|
||||||
allFiles.add(file);
|
|
||||||
if (file.getPath().getName().endsWith("_del")) {
|
|
||||||
delFiles.add(file.getPath());
|
|
||||||
} else {
|
|
||||||
mobFiles.add(file);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Compares the compacted partitions.
|
|
||||||
* @param partitions the collection of CompactedPartitions
|
|
||||||
*/
|
|
||||||
private void compareCompactedPartitions(List<String> expected,
|
|
||||||
Collection<CompactionPartition> partitions) {
|
|
||||||
List<String> actualKeys = new ArrayList<>();
|
|
||||||
for (CompactionPartition partition : partitions) {
|
|
||||||
actualKeys.add(partition.getPartitionId().getStartKey());
|
|
||||||
}
|
|
||||||
Collections.sort(expected);
|
|
||||||
Collections.sort(actualKeys);
|
|
||||||
assertEquals(expected.size(), actualKeys.size());
|
|
||||||
for (int i = 0; i < expected.size(); i++) {
|
|
||||||
assertEquals(expected.get(i), actualKeys.get(i));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Compares the del files.
|
|
||||||
* @param delPartitions all del partitions
|
|
||||||
*/
|
|
||||||
private void compareDelFiles(List<CompactionDelPartition> delPartitions) {
|
|
||||||
Map<Path, Path> delMap = new HashMap<>();
|
|
||||||
for (CompactionDelPartition delPartition : delPartitions) {
|
|
||||||
for (Path f : delPartition.listDelFiles()) {
|
|
||||||
delMap.put(f, f);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
for (Path f : delFiles) {
|
|
||||||
assertTrue(delMap.containsKey(f));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Creates store files.
|
|
||||||
* @param basePath the path to create file
|
|
||||||
* @param family the column family name
|
|
||||||
* @param qualifier the column qualifier assigned to data values
|
|
||||||
* @param count the store file number
|
|
||||||
* @param type the row key type
|
|
||||||
* @param date the latest timestamp when an instance of MobFileName is created
|
|
||||||
*/
|
|
||||||
private void createStoreFiles(Path basePath, String family, String qualifier, int count,
|
|
||||||
Type type, final Date date) throws IOException {
|
|
||||||
createStoreFiles(basePath, family, qualifier, count, type, false, date);
|
|
||||||
}
|
|
||||||
|
|
||||||
private void createStoreFiles(Path basePath, String family, String qualifier, int count,
|
|
||||||
Type type, boolean sameStartKey, final Date date) throws IOException {
|
|
||||||
HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build();
|
|
||||||
String startKey = "row_";
|
|
||||||
MobFileName mobFileName = null;
|
|
||||||
for (int i = 0; i < count; i++) {
|
|
||||||
byte[] startRow;
|
|
||||||
if (sameStartKey) {
|
|
||||||
// When creating multiple files under one partition, suffix needs to be different.
|
|
||||||
startRow = Bytes.toBytes(startKey);
|
|
||||||
mobSuffix = TEST_UTIL.getRandomUUID().toString().replaceAll("-", "");
|
|
||||||
delSuffix = TEST_UTIL.getRandomUUID().toString().replaceAll("-", "") + "_del";
|
|
||||||
} else {
|
|
||||||
startRow = Bytes.toBytes(startKey + i);
|
|
||||||
}
|
|
||||||
if(type.equals(Type.Delete)) {
|
|
||||||
mobFileName = MobFileName.create(startRow, MobUtils.formatDate(date), delSuffix);
|
|
||||||
}
|
|
||||||
if(type.equals(Type.Put)){
|
|
||||||
mobFileName = MobFileName.create(startRow, MobUtils.formatDate(date), mobSuffix);
|
|
||||||
}
|
|
||||||
StoreFileWriter mobFileWriter = new StoreFileWriter.Builder(conf, cacheConf, fs)
|
|
||||||
.withFileContext(meta).withFilePath(new Path(basePath, mobFileName.getFileName())).build();
|
|
||||||
writeStoreFile(mobFileWriter, startRow, Bytes.toBytes(family), Bytes.toBytes(qualifier),
|
|
||||||
type, (i+1)*1000);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Writes data to store file.
|
|
||||||
* @param writer the store file writer
|
|
||||||
* @param row the row key
|
|
||||||
* @param family the family name
|
|
||||||
* @param qualifier the column qualifier
|
|
||||||
* @param type the key type
|
|
||||||
* @param size the size of value
|
|
||||||
*/
|
|
||||||
private static void writeStoreFile(final StoreFileWriter writer, byte[]row, byte[] family,
|
|
||||||
byte[] qualifier, Type type, int size) throws IOException {
|
|
||||||
long now = System.currentTimeMillis();
|
|
||||||
try {
|
|
||||||
byte[] dummyData = new byte[size];
|
|
||||||
new Random().nextBytes(dummyData);
|
|
||||||
writer.append(new KeyValue(row, family, qualifier, now, type, dummyData));
|
|
||||||
} finally {
|
|
||||||
writer.close();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Gets the number of del cell in the del files
|
|
||||||
* @param paths the del file paths
|
|
||||||
* @return the cell size
|
|
||||||
*/
|
|
||||||
private int countDelCellsInDelFiles(List<Path> paths) throws IOException {
|
|
||||||
List<HStoreFile> sfs = new ArrayList<>();
|
|
||||||
int size = 0;
|
|
||||||
for (Path path : paths) {
|
|
||||||
HStoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
|
|
||||||
sfs.add(sf);
|
|
||||||
}
|
|
||||||
List<KeyValueScanner> scanners = new ArrayList<>(StoreFileScanner.getScannersForStoreFiles(sfs,
|
|
||||||
false, true, false, false, HConstants.LATEST_TIMESTAMP));
|
|
||||||
long timeToPurgeDeletes = Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0);
|
|
||||||
long ttl = HStore.determineTTLFromFamily(cfdb.build());
|
|
||||||
ScanInfo scanInfo = new ScanInfo(conf, cfdb.build(), ttl, timeToPurgeDeletes,
|
|
||||||
CellComparatorImpl.COMPARATOR);
|
|
||||||
StoreScanner scanner = new StoreScanner(scanInfo, ScanType.COMPACT_RETAIN_DELETES, scanners);
|
|
||||||
List<Cell> results = new ArrayList<>();
|
|
||||||
boolean hasMore = true;
|
|
||||||
|
|
||||||
while (hasMore) {
|
|
||||||
hasMore = scanner.next(results);
|
|
||||||
size += results.size();
|
|
||||||
results.clear();
|
|
||||||
}
|
|
||||||
scanner.close();
|
|
||||||
return size;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static ExecutorService createThreadPool() {
|
|
||||||
int maxThreads = 10;
|
|
||||||
long keepAliveTime = 60;
|
|
||||||
final SynchronousQueue<Runnable> queue = new SynchronousQueue<>();
|
|
||||||
ThreadPoolExecutor pool = new ThreadPoolExecutor(1, maxThreads, keepAliveTime,
|
|
||||||
TimeUnit.SECONDS, queue, Threads.newDaemonThreadFactory("MobFileCompactionChore"),
|
|
||||||
new RejectedExecutionHandler() {
|
|
||||||
@Override
|
|
||||||
public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) {
|
|
||||||
try {
|
|
||||||
// waiting for a thread to pick up instead of throwing exceptions.
|
|
||||||
queue.put(r);
|
|
||||||
} catch (InterruptedException e) {
|
|
||||||
throw new RejectedExecutionException(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
});
|
|
||||||
((ThreadPoolExecutor) pool).allowCoreThreadTimeOut(true);
|
|
||||||
return pool;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Resets the configuration.
|
|
||||||
*/
|
|
||||||
private void resetConf() {
|
|
||||||
conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD,
|
|
||||||
MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD);
|
|
||||||
conf.setInt(MobConstants.MOB_DELFILE_MAX_COUNT, MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT);
|
|
||||||
conf.setInt(MobConstants.MOB_COMPACTION_BATCH_SIZE,
|
|
||||||
MobConstants.DEFAULT_MOB_COMPACTION_BATCH_SIZE);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The customized Distributed File System Implementation
|
|
||||||
*/
|
|
||||||
static class FaultyDistributedFileSystem extends DistributedFileSystem {
|
|
||||||
private volatile boolean throwException = false;
|
|
||||||
|
|
||||||
public FaultyDistributedFileSystem() {
|
|
||||||
super();
|
|
||||||
}
|
|
||||||
|
|
||||||
public void setThrowException(boolean throwException) {
|
|
||||||
this.throwException = throwException;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean rename(Path src, Path dst) throws IOException {
|
|
||||||
if (throwException) {
|
|
||||||
throw new IOException("No more files allowed");
|
|
||||||
}
|
|
||||||
return super.rename(src, dst);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -21,21 +21,17 @@ import static org.apache.hadoop.hbase.HConstants.REPLICATION_CLUSTER_ID;
|
||||||
import static org.apache.hadoop.hbase.HConstants.REPLICATION_CONF_DIR;
|
import static org.apache.hadoop.hbase.HConstants.REPLICATION_CONF_DIR;
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.FileOutputStream;
|
import java.io.FileOutputStream;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.Date;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.Random;
|
|
||||||
import java.util.UUID;
|
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.ExecutorService;
|
|
||||||
import java.util.concurrent.Executors;
|
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
|
@ -49,7 +45,6 @@ import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.KeyValue;
|
import org.apache.hadoop.hbase.KeyValue;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.client.Admin;
|
import org.apache.hadoop.hbase.client.Admin;
|
||||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
|
||||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||||
import org.apache.hadoop.hbase.client.Connection;
|
import org.apache.hadoop.hbase.client.Connection;
|
||||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||||
|
@ -62,21 +57,14 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
||||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
|
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
|
||||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
|
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
|
||||||
import org.apache.hadoop.hbase.coprocessor.RegionObserver;
|
import org.apache.hadoop.hbase.coprocessor.RegionObserver;
|
||||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
|
||||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
|
||||||
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
|
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
|
||||||
import org.apache.hadoop.hbase.mob.MobConstants;
|
|
||||||
import org.apache.hadoop.hbase.mob.MobFileName;
|
|
||||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
|
||||||
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactor;
|
|
||||||
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
|
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
|
||||||
import org.apache.hadoop.hbase.replication.TestReplicationBase;
|
import org.apache.hadoop.hbase.replication.TestReplicationBase;
|
||||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
|
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
|
||||||
import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool;
|
import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.hadoop.hbase.util.FSUtils;
|
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
|
@ -257,30 +245,6 @@ public class TestBulkLoadReplication extends TestReplicationBase {
|
||||||
assertEquals(9, BULK_LOADS_COUNT.get());
|
assertEquals(9, BULK_LOADS_COUNT.get());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testPartionedMOBCompactionBulkLoadDoesntReplicate() throws Exception {
|
|
||||||
Path path = createMobFiles(UTIL3);
|
|
||||||
ColumnFamilyDescriptor descriptor =
|
|
||||||
new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(famName);
|
|
||||||
ExecutorService pool = null;
|
|
||||||
try {
|
|
||||||
pool = Executors.newFixedThreadPool(1);
|
|
||||||
PartitionedMobCompactor compactor =
|
|
||||||
new PartitionedMobCompactor(UTIL3.getConfiguration(), UTIL3.getTestFileSystem(), tableName,
|
|
||||||
descriptor, pool);
|
|
||||||
BULK_LOAD_LATCH = new CountDownLatch(1);
|
|
||||||
BULK_LOADS_COUNT.set(0);
|
|
||||||
compactor.compact(Arrays.asList(UTIL3.getTestFileSystem().listStatus(path)), true);
|
|
||||||
assertTrue(BULK_LOAD_LATCH.await(1, TimeUnit.SECONDS));
|
|
||||||
Thread.sleep(400);
|
|
||||||
assertEquals(1, BULK_LOADS_COUNT.get());
|
|
||||||
} finally {
|
|
||||||
if(pool != null && !pool.isTerminated()) {
|
|
||||||
pool.shutdownNow();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
protected void assertBulkLoadConditions(TableName tableName, byte[] row, byte[] value,
|
protected void assertBulkLoadConditions(TableName tableName, byte[] row, byte[] value,
|
||||||
HBaseTestingUtility utility, Table...tables) throws Exception {
|
HBaseTestingUtility utility, Table...tables) throws Exception {
|
||||||
|
@ -348,36 +312,6 @@ public class TestBulkLoadReplication extends TestReplicationBase {
|
||||||
return hFileLocation.getAbsoluteFile().getAbsolutePath();
|
return hFileLocation.getAbsoluteFile().getAbsolutePath();
|
||||||
}
|
}
|
||||||
|
|
||||||
private Path createMobFiles(HBaseTestingUtility util) throws IOException {
|
|
||||||
Path testDir = FSUtils.getRootDir(util.getConfiguration());
|
|
||||||
Path mobTestDir = new Path(testDir, MobConstants.MOB_DIR_NAME);
|
|
||||||
Path basePath = new Path(new Path(mobTestDir, tableName.getNameAsString()), "f");
|
|
||||||
HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build();
|
|
||||||
MobFileName mobFileName = null;
|
|
||||||
byte[] mobFileStartRow = new byte[32];
|
|
||||||
for (byte rowKey : Bytes.toBytes("01234")) {
|
|
||||||
mobFileName = MobFileName.create(mobFileStartRow, MobUtils.formatDate(new Date()),
|
|
||||||
UUID.randomUUID().toString().replaceAll("-", ""));
|
|
||||||
StoreFileWriter mobFileWriter =
|
|
||||||
new StoreFileWriter.Builder(util.getConfiguration(),
|
|
||||||
new CacheConfig(util.getConfiguration()), util.getTestFileSystem()).withFileContext(meta)
|
|
||||||
.withFilePath(new Path(basePath, mobFileName.getFileName())).build();
|
|
||||||
long now = System.currentTimeMillis();
|
|
||||||
try {
|
|
||||||
for (int i = 0; i < 10; i++) {
|
|
||||||
byte[] key = Bytes.add(Bytes.toBytes(rowKey), Bytes.toBytes(i));
|
|
||||||
byte[] dummyData = new byte[5000];
|
|
||||||
new Random().nextBytes(dummyData);
|
|
||||||
mobFileWriter.append(
|
|
||||||
new KeyValue(key, famName, Bytes.toBytes("1"), now, KeyValue.Type.Put, dummyData));
|
|
||||||
}
|
|
||||||
} finally {
|
|
||||||
mobFileWriter.close();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return basePath;
|
|
||||||
}
|
|
||||||
|
|
||||||
public static class BulkReplicationTestObserver implements RegionCoprocessor {
|
public static class BulkReplicationTestObserver implements RegionCoprocessor {
|
||||||
|
|
||||||
String clusterName;
|
String clusterName;
|
||||||
|
|
|
@ -25,6 +25,8 @@ import static org.junit.Assert.assertTrue;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Optional;
|
||||||
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.CompatibilityFactory;
|
import org.apache.hadoop.hbase.CompatibilityFactory;
|
||||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||||
|
@ -53,6 +55,10 @@ import org.apache.hadoop.hbase.client.Table;
|
||||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||||
import org.apache.hadoop.hbase.master.LoadBalancer;
|
import org.apache.hadoop.hbase.master.LoadBalancer;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
|
||||||
|
import org.apache.hadoop.hbase.security.User;
|
||||||
import org.apache.hadoop.hbase.test.MetricsAssertHelper;
|
import org.apache.hadoop.hbase.test.MetricsAssertHelper;
|
||||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||||
|
@ -494,7 +500,20 @@ public class TestRegionServerMetrics {
|
||||||
setMobThreshold(region, cf, 100);
|
setMobThreshold(region, cf, 100);
|
||||||
// metrics are reset by the region initialization
|
// metrics are reset by the region initialization
|
||||||
region.initialize();
|
region.initialize();
|
||||||
region.compact(true);
|
// This is how we MOB compact region
|
||||||
|
List<HStore> stores = region.getStores();
|
||||||
|
for (HStore store: stores) {
|
||||||
|
// Force major compaction
|
||||||
|
store.triggerMajorCompaction();
|
||||||
|
Optional<CompactionContext> context =
|
||||||
|
store.requestCompaction(HStore.PRIORITY_USER, CompactionLifeCycleTracker.DUMMY,
|
||||||
|
User.getCurrent());
|
||||||
|
if (!context.isPresent()) {
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
region.compact(context.get(), store,
|
||||||
|
NoLimitThroughputController.INSTANCE, User.getCurrent());
|
||||||
|
}
|
||||||
metricsRegionServer.getRegionServerWrapper().forceRecompute();
|
metricsRegionServer.getRegionServerWrapper().forceRecompute();
|
||||||
assertCounter("cellsCountCompactedFromMob", numHfiles);
|
assertCounter("cellsCountCompactedFromMob", numHfiles);
|
||||||
assertCounter("cellsCountCompactedToMob", 0);
|
assertCounter("cellsCountCompactedToMob", 0);
|
||||||
|
|
|
@ -411,7 +411,7 @@ public class BaseTestHBaseFsck {
|
||||||
String startKey = mobFileName.getStartKey();
|
String startKey = mobFileName.getStartKey();
|
||||||
String date = mobFileName.getDate();
|
String date = mobFileName.getDate();
|
||||||
return MobFileName.create(startKey, date,
|
return MobFileName.create(startKey, date,
|
||||||
TEST_UTIL.getRandomUUID().toString().replaceAll("-", ""))
|
TEST_UTIL.getRandomUUID().toString().replaceAll("-", ""), "abcdef")
|
||||||
.getFileName();
|
.getFileName();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -98,6 +98,7 @@ public class TestHBaseFsckMOB extends BaseTestHBaseFsck {
|
||||||
/**
|
/**
|
||||||
* This creates a table and then corrupts a mob file. Hbck should quarantine the file.
|
* This creates a table and then corrupts a mob file. Hbck should quarantine the file.
|
||||||
*/
|
*/
|
||||||
|
@SuppressWarnings("deprecation")
|
||||||
@Test
|
@Test
|
||||||
public void testQuarantineCorruptMobFile() throws Exception {
|
public void testQuarantineCorruptMobFile() throws Exception {
|
||||||
TableName table = TableName.valueOf(name.getMethodName());
|
TableName table = TableName.valueOf(name.getMethodName());
|
||||||
|
|
Loading…
Reference in New Issue