diff --git a/hbase-checkstyle/src/main/resources/hbase/checkstyle-suppressions.xml b/hbase-checkstyle/src/main/resources/hbase/checkstyle-suppressions.xml index 9351ecbfe6a..2d8e880ddf4 100644 --- a/hbase-checkstyle/src/main/resources/hbase/checkstyle-suppressions.xml +++ b/hbase-checkstyle/src/main/resources/hbase/checkstyle-suppressions.xml @@ -55,4 +55,5 @@ + diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml index cafb12c0e40..6c2d3f4472e 100644 --- a/hbase-common/src/main/resources/hbase-default.xml +++ b/hbase-common/src/main/resources/hbase-default.xml @@ -1814,10 +1814,10 @@ possible configurations would overwhelm and obscure the important. - hbase.master.mob.ttl.cleaner.period + hbase.master.mob.cleaner.period 86400 - 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 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 @@ -1825,32 +1825,13 @@ possible configurations would overwhelm and obscure the important. - hbase.mob.compaction.mergeable.threshold - 1342177280 + hbase.mob.major.compaction.region.batch.size + 0 - If the size of a mob file is less than this value, it's regarded as a small - file and needs to be merged in mob compaction. The default value is 1280MB. - - - - hbase.mob.delfile.max.count - 3 - - 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. - - - - hbase.mob.compaction.batch.size - 100 - - 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. + 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. + Default is 0 - means no limit - all regions of a MOB table will be compacted at once @@ -1861,20 +1842,6 @@ possible configurations would overwhelm and obscure the important. The default value is one week. - - hbase.mob.compactor.class - org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactor - - Implementation of mob compactor, the default one is PartitionedMobCompactor. - - - - hbase.mob.compaction.threads.max - 1 - - The max number of threads used in MobCompactor. - - hbase.snapshot.master.timeout.millis 300000 diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java new file mode 100644 index 00000000000..093f9e49816 --- /dev/null +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java @@ -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 HBASE-22749 + *

+ * Sample usage: + *

+ * hbase org.apache.hadoop.hbase.IntegrationTestMobCompaction -Dservers=10 -Drows=1000000
+ * -Dfailprob=0.2
+ * 
+ */ +@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 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); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java index 604ac1f8bc6..464ea49bca5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java @@ -217,6 +217,10 @@ public class HFileWriterImpl implements HFile.Writer { 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. * diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java deleted file mode 100644 index d37a80a1c3d..00000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java +++ /dev/null @@ -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 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); - } - } - -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index edc44b8e1ce..cf5bb0959cc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -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.zksyncer.MasterAddressSyncer; 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.MonitoredTask; import org.apache.hadoop.hbase.monitoring.TaskMonitor; @@ -396,9 +397,8 @@ public class HMaster extends HRegionServer implements MasterServices { private LogCleaner logCleaner; private HFileCleaner hfileCleaner; private ReplicationBarrierCleaner replicationBarrierCleaner; - private ExpiredMobFileCleanerChore expiredMobFileCleanerChore; - private MobCompactionChore mobCompactChore; - private MasterMobCompactionThread mobCompactThread; + private MobFileCleanerChore mobFileCleanerChore; + private MobFileCompactionChore mobFileCompactionChore; // used to synchronize the mobCompactionStates private final IdLock mobCompactionLock = new IdLock(); // save the information of mob compactions in tables. @@ -1265,14 +1265,10 @@ public class HMaster extends HRegionServer implements MasterServices { } private void initMobCleaner() { - this.expiredMobFileCleanerChore = new ExpiredMobFileCleanerChore(this); - getChoreService().scheduleChore(expiredMobFileCleanerChore); - - int mobCompactionPeriod = conf.getInt(MobConstants.MOB_COMPACTION_CHORE_PERIOD, - MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD); - this.mobCompactChore = new MobCompactionChore(this, mobCompactionPeriod); - getChoreService().scheduleChore(mobCompactChore); - this.mobCompactThread = new MasterMobCompactionThread(this); + this.mobFileCleanerChore = new MobFileCleanerChore(this); + getChoreService().scheduleChore(mobFileCleanerChore); + this.mobFileCompactionChore = new MobFileCompactionChore(this); + getChoreService().scheduleChore(mobFileCompactionChore); } /** @@ -1475,9 +1471,7 @@ public class HMaster extends HRegionServer implements MasterServices { } } stopChores(); - if (this.mobCompactThread != null) { - this.mobCompactThread.close(); - } + super.stopServiceThreads(); if (cleanerPool != null) { cleanerPool.shutdownNow(); @@ -1600,8 +1594,8 @@ public class HMaster extends HRegionServer implements MasterServices { private void stopChores() { ChoreService choreService = getChoreService(); if (choreService != null) { - choreService.cancelChore(this.expiredMobFileCleanerChore); - choreService.cancelChore(this.mobCompactChore); + choreService.cancelChore(this.mobFileCleanerChore); + choreService.cancelChore(this.mobFileCompactionChore); choreService.cancelChore(this.balancerChore); choreService.cancelChore(this.normalizerChore); 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 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, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java deleted file mode 100644 index 0779eeafe8b..00000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java +++ /dev/null @@ -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 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 hcds; - private boolean allFiles; - private ExecutorService pool; - - public CompactionRunner(FileSystem fs, TableName tableName, List 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..."); - } - } - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java index 5fb5f151c86..fb19fe17d32 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java @@ -32,6 +32,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Set; import java.util.stream.Collectors; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; 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.TableName; 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.Put; 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.UserPermission; import org.apache.hadoop.hbase.security.visibility.VisibilityController; + import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; 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.wal.AbstractFSWALProvider; import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; + + import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.KeeperException; 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.UnsafeByteOperations; + import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter; 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.SnapshotProtos.SnapshotDescription; + + /** * Implements the master RPC services. */ @@ -1751,10 +1757,15 @@ public class MasterRpcServices extends RSRpcServices master.checkInitialized(); byte[] regionName = request.getRegion().getValue().toByteArray(); TableName tableName = RegionInfo.getTable(regionName); + // TODO: support CompactType.MOB // if the region is a mob region, do the mob file compaction. if (MobUtils.isMobRegionName(tableName, regionName)) { 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 { return super.compactRegion(controller, request); } @@ -1817,57 +1828,6 @@ public class MasterRpcServices extends RSRpcServices 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 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 public IsBalancerEnabledResponse isBalancerEnabled(RpcController controller, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobCompactionChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobCompactionChore.java deleted file mode 100644 index 6c5d677a86e..00000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobCompactionChore.java +++ /dev/null @@ -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 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(); - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java index e73a7d2226d..c5ef6a6b992 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java @@ -17,18 +17,26 @@ */ 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.InterruptedIOException; import java.util.ArrayList; import java.util.Date; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; +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.Cell; -import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.PrivateCellUtil; 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.HMobStore; import org.apache.hadoop.hbase.regionserver.HStore; @@ -52,6 +60,8 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; 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. */ @@ -59,15 +69,68 @@ import org.slf4j.LoggerFactory; public class DefaultMobStoreCompactor extends DefaultCompactor { private static final Logger LOG = LoggerFactory.getLogger(DefaultMobStoreCompactor.class); - private long mobSizeThreshold; - private HMobStore mobStore; + protected long mobSizeThreshold; + 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> mobRefSet = new ThreadLocal>() { + @Override + protected Set initialValue() { + return new HashSet(); + } + }; + + /* + * Is it user or system-originated request. + */ + + static ThreadLocal userRequest = new ThreadLocal() { + @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 disableIO = new ThreadLocal() { + + @Override + protected Boolean initialValue() { + return Boolean.FALSE; + } + }; + + /* + * Map : MOB file name - file length Can be expensive for large amount of MOB files. + */ + static ThreadLocal> mobLengthMap = + new ThreadLocal>() { + @Override + protected HashMap initialValue() { + return new HashMap(); + } + }; private final InternalScannerFactory scannerFactory = new InternalScannerFactory() { @Override public ScanType getScanType(CompactionRequestImpl request) { - // retain the delete markers until they are expired. - return ScanType.COMPACT_RETAIN_DELETES; + return request.isAllFiles() ? COMPACT_DROP_DELETES : COMPACT_RETAIN_DELETES; } @Override @@ -99,59 +162,90 @@ public class DefaultMobStoreCompactor extends DefaultCompactor { if (!(store instanceof HMobStore)) { throw new IllegalArgumentException("The store " + store + " is not a HMobStore"); } - mobStore = (HMobStore) store; - mobSizeThreshold = store.getColumnFamilyDescriptor().getMobThreshold(); + this.mobStore = (HMobStore) store; + 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 - public List compact(CompactionRequestImpl request, ThroughputController throughputController, - User user) throws IOException { + public List compact(CompactionRequestImpl request, + 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 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); } + private void calculateMobLengthMap(List mobFiles) throws IOException { + FileSystem fs = store.getFileSystem(); + HashMap 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. - * This is for when the mob threshold size has changed or if the mob - * column family mode has been toggled via an alter table statement. - * Compacts the files by the following rules. + * Performs compaction on a column family with the mob flag enabled. This works only when MOB + * compaction is explicitly requested (by User), or by Master There are two modes of a MOB + * compaction:
+ *

+ *

    + *
  • 1. Full mode - when all MOB data for a region is compacted into a single MOB file. + *
  • 2. I/O optimized mode - for use cases with no or infrequent updates/deletes of a
    + * 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. + *
+ * The basic algorithm of compaction is the following:
* 1. If the Put cell has a mob reference tag, the cell's value is the path of the mob file. *
    - *
  1. - * 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. - *
  2. - *
  3. - * Otherwise, retrieve the mob cell from the mob file, and writes a copy of the cell into - * the new store file. - *
  4. + *
  5. 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.
  6. + *
  7. Otherwise, retrieve the mob cell from the mob file, and writes a copy of the cell into the + * new store file.
  8. *
* 2. If the Put cell doesn't have a reference tag. *
    - *
  1. - * 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. - *
  2. - *
  3. - * Otherwise, directly write this cell into the store file. - *
  4. + *
  5. 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.
  6. + *
  7. Otherwise, directly write this cell into the store file.
  8. *
- * 3. Decide how to write a Delete cell. - *
    - *
  1. - * 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. - *
  2. - *
  3. - * Otherwise, directly write it to a store file. - *
  4. - *
- * 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 scanner Where to read from. * @param writer Where to write to. @@ -169,6 +263,23 @@ public class DefaultMobStoreCompactor extends DefaultCompactor { 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); + 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, // we have to use a do/while loop. List cells = new ArrayList<>(); @@ -181,93 +292,164 @@ public class DefaultMobStoreCompactor extends DefaultCompactor { 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, delFileWriter = null; - long mobCells = 0, deleteMarkersCount = 0; + StoreFileWriter mobFileWriter = null; + /* + * 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 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(); + KeyValueScanner kvs = (scanner instanceof KeyValueScanner) ? (KeyValueScanner) scanner : null; + long shippedCallSizeLimit = + (long) numofFilesToCompact * this.store.getColumnFamilyDescriptor().getBlocksize(); + + Cell mobCell = null; try { - try { - // If the mob file writer could not be created, directly write the cell to the store file. - mobFileWriter = mobStore.createWriterInTmp(new Date(fd.latestPutTs), fd.maxKeyCount, - 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); - } - } + + mobFileWriter = newMobWriter(fd); + fileName = Bytes.toBytes(mobFileWriter.getPath().getName()); + do { hasMore = scanner.next(cells, scannerContext); - if (LOG.isDebugEnabled()) { - now = EnvironmentEdgeManager.currentTime(); - } + now = EnvironmentEdgeManager.currentTime(); for (Cell c : cells) { - if (major && CellUtil.isDelete(c)) { - if (MobUtils.isMobReferenceCell(c) || delFileWriter == null) { - // Directly write it to a store file - writer.append(c); + if (compactMOBs) { + if (MobUtils.isMobReferenceCell(c)) { + 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 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 { - // Add a ref tag to this cell and write it to a store file. - writer.append(MobUtils.createMobRefDeleteMarker(c)); - // Write the cell to a del file - delFileWriter.append(c); - deleteMarkersCount++; + // Not a MOB reference cell + int size = c.getValueLength(); + if (size > mobSizeThreshold) { + // This MOB cell comes from a regular store file + // 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()) { - // If the mob file writer is null or the kv type is not put, directly write the cell + } 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); - } 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); + // We do not check mobSizeThreshold during normal compaction, + // leaving it to a MOB compaction run 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) { - //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); } else { // 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++; // append the original keyValue in the mob file. mobFileWriter.append(c); - Cell reference = MobUtils.createMobRefCell(c, fileName, - this.mobStore.getRefCellTags()); + 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(); + 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(); ++progress.currentCompactedKVs; progress.totalCompactedSize += len; @@ -302,7 +493,7 @@ public class DefaultMobStoreCompactor extends DefaultCompactor { } } if (kvs != null && bytesWrittenProgressForShippedCall > shippedCallSizeLimit) { - ((ShipperListener)writer).beforeShipped(); + ((ShipperListener) writer).beforeShipped(); kvs.shipped(); bytesWrittenProgressForShippedCall = 0; } @@ -326,6 +517,10 @@ public class DefaultMobStoreCompactor extends DefaultCompactor { progress.cancel(); throw new InterruptedIOException( "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 { // 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 @@ -333,35 +528,17 @@ public class DefaultMobStoreCompactor extends DefaultCompactor { ((ShipperListener) writer).beforeShipped(); throughputController.finish(compactionName); if (!finished && mobFileWriter != null) { - abortWriter(mobFileWriter); - } - if (!finished && delFileWriter != null) { - abortWriter(delFileWriter); - } - } - 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. + // Remove all MOB references because compaction failed + mobRefSet.get().clear(); + // Abort writer + LOG.debug("Aborting writer for {} because of a compaction failure, Store {}", + mobFileWriter.getPath(), getStoreInfo()); abortWriter(mobFileWriter); } } + + // Commit last MOB writer + commitOrAbortMobWriter(mobFileWriter, fd.maxSeqId, mobCells, major); mobStore.updateCellsCountCompactedFromMob(cellsCountCompactedFromMob); mobStore.updateCellsCountCompactedToMob(cellsCountCompactedToMob); mobStore.updateCellsSizeCompactedFromMob(cellsSizeCompactedFromMob); @@ -369,4 +546,85 @@ public class DefaultMobStoreCompactor extends DefaultCompactor { progress.complete(); 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 set = mobRefSet.get(); + if (set != null) { + set.clear(); + } + HashMap 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 commitWriter(StoreFileWriter writer, FileDetails fd, + CompactionRequestImpl request) throws IOException { + List newFiles = Lists.newArrayList(writer.getPath()); + writer.appendMetadata(fd.maxSeqId, request.isAllFiles(), request.getFiles()); + // Append MOB references + Set refSet = mobRefSet.get(); + writer.appendMobMetadata(refSet); + writer.close(); + clearThreadLocals(); + return newFiles; + } + } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java index 3de7992cb12..5c4c6020ae0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java @@ -22,7 +22,9 @@ import java.io.IOException; import java.io.InterruptedIOException; import java.util.ArrayList; import java.util.Date; +import java.util.HashSet; import java.util.List; +import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -69,6 +71,13 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher { private long mobCellValueSizeThreshold = 0; private Path targetPath; private HMobStore mobStore; + // MOB file reference set + static ThreadLocal> mobRefSet = new ThreadLocal>() { + @Override + protected Set initialValue() { + return new HashSet(); + } + }; public DefaultMobStoreFlusher(Configuration conf, HStore store) throws IOException { super(conf, store); @@ -188,6 +197,8 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher { throughputController.start(flushName); } IOException ioe = null; + // Clear all past MOB references + mobRefSet.get().clear(); try { do { hasMore = scanner.next(cells, scannerContext); @@ -204,7 +215,6 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher { mobFileWriter.append(c); mobSize += c.getValueLength(); mobCount++; - // append the tags to the KeyValue. // The key is same, the value is the filename of the mob file Cell reference = MobUtils.createMobRefCell(c, fileName, @@ -244,9 +254,12 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher { status.setStatus("Flushing mob file " + store + ": closing flushed file"); mobFileWriter.close(); mobStore.commitFile(mobFileWriter.getPath(), targetPath); + LOG.debug("Flush store file: {}, store: {}", writer.getPath(), getStoreInfo()); mobStore.updateMobFlushCount(); mobStore.updateMobFlushedCellsCount(mobCount); mobStore.updateMobFlushedCellsSize(mobSize); + // Add mob reference to store file metadata + mobRefSet.get().add(mobFileWriter.getPath().getName()); } else { try { 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()) ; + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java index 42e78bbd007..f82cf1a4cb1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/ExpiredMobFileCleaner.java @@ -57,7 +57,8 @@ public class ExpiredMobFileCleaner extends Configured implements Tool { * @param tableName The current table name. * @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(); TableName tn = TableName.valueOf(tableName); 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); CacheConfig cacheConfig = new CacheConfig(copyOfConf); MobUtils.cleanExpiredMobFiles(fs, conf, tn, family, cacheConfig, - EnvironmentEdgeManager.currentTime()); + EnvironmentEdgeManager.currentTime()); } public static void main(String[] args) throws Exception { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java index 4afd75b9c68..77f3dc04ff0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java @@ -35,26 +35,88 @@ public final class MobConstants { 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 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 int DEFAULT_MOB_FILE_CACHE_SIZE = 1000; - public static final String MOB_DIR_NAME = "mobdir"; public static final String MOB_REGION_NAME = ".mob"; public static final byte[] MOB_REGION_NAME_BYTES = Bytes.toBytes(MOB_REGION_NAME); - - public static final String MOB_CLEANER_PERIOD = "hbase.master.mob.ttl.cleaner.period"; + public static final String MOB_CLEANER_PERIOD = "hbase.master.mob.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 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 Tag MOB_REF_TAG = new ArrayBackedTag(TagType.MOB_REFERENCE_TAG_TYPE, - HConstants.EMPTY_BYTE_ARRAY); + public static final Tag MOB_REF_TAG = + new ArrayBackedTag(TagType.MOB_REFERENCE_TAG_TYPE, HConstants.EMPTY_BYTE_ARRAY); public static final float DEFAULT_EVICT_REMAIN_RATIO = 0.5f; public static final long DEFAULT_MOB_CACHE_EVICT_PERIOD = 3600L; 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 byte[] MOB_TABLE_LOCK_SUFFIX = Bytes.toBytes(".mobLock"); public final static String EMPTY_STRING = ""; @@ -63,40 +125,23 @@ public final class MobConstants { * be merged in mob compaction. The default value is 1280MB. */ 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; - /** - * 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 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 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. + * 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. 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 = - "hbase.mob.compaction.batch.size"; + public static final String MOB_COMPACTION_BATCH_SIZE = "hbase.mob.compaction.batch.size"; public static final int DEFAULT_MOB_COMPACTION_BATCH_SIZE = 100; - /** - * 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 String MOB_COMPACTION_THREADS_MAX = "hbase.mob.compaction.threads.max"; public static final int DEFAULT_MOB_COMPACTION_THREADS_MAX = 1; + private MobConstants() { } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerChore.java new file mode 100644 index 00000000000..7f7d90c5f64 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerChore.java @@ -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 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 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 regionDirs = FSUtils.getRegionDirs(FileSystem.get(conf), tableDir); + + Set allActiveMobFileName = new HashSet(); + 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 regionMobs = new HashSet(); + + 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 rit = fs.listLocatedStatus(storePath); + List storeFiles = new ArrayList(); + // 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 toArchive = new ArrayList(); + String family = hcd.getNameAsString(); + Path dir = MobUtils.getMobFamilyPath(conf, table, family); + RemoteIterator 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 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); + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCompactionChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCompactionChore.java new file mode 100644 index 00000000000..31472983395 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCompactionChore.java @@ -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:
+ * '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 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 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 toCompact = new ArrayList(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 compacted = new ArrayList(toCompact.size()); + List failed = new ArrayList(); + 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; + } + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileName.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileName.java index 3a29274285b..73bb07a90d4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileName.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileName.java @@ -23,8 +23,8 @@ import org.apache.hadoop.hbase.util.MD5Hash; /** * The mob file name. - * It consists of a md5 of a start key, a date and an uuid. - * It looks like md5(start) + date + uuid. + * It consists of a md5 of a start key, a date, uuid and encoded region name. + * It looks like md5(start) + date + uuid+ "_" + encoded region name. *
    *
  1. 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.
  2. @@ -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. */ @InterfaceAudience.Private -public final class MobFileName { +public final class MobFileName{ private final String date; private final String startKey; private final String uuid; 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 DATE_END_INDEX = 40; private static final int UUID_END_INDEX = 72; + public static final String REGION_SEP = "_"; /** * @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. * @param 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.uuid = uuid; 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. * @param 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.uuid = uuid; 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 * The string of the latest timestamp of cells in this file, the format is yyyymmdd. * @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. */ - public static MobFileName create(byte[] startKey, String date, String uuid) { - return new MobFileName(startKey, date, uuid); + public static MobFileName create(byte[] startKey, String date, String uuid, String regionName) { + return new MobFileName(startKey, date, uuid, regionName); } /** @@ -102,10 +110,11 @@ public final class MobFileName { * @param date * The string of the latest timestamp of cells in this file, the format is yyyymmdd. * @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. */ - public static MobFileName create(String startKey, String date, String uuid) { - return new MobFileName(startKey, date, uuid); + public static MobFileName create(String startKey, String date, String uuid, String regionName) { + return new MobFileName(startKey, date, uuid, regionName); } /** @@ -115,11 +124,17 @@ public final class MobFileName { */ public static MobFileName create(String fileName) { // The format of a file name is md5HexString(0-31bytes) + date(32-39bytes) + UUID + // + "_" + region // The date format is yyyyMMdd String startKey = fileName.substring(0, STARTKEY_END_INDEX); String date = fileName.substring(STARTKEY_END_INDEX, DATE_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; } + /** + * 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. * @return The date string. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java index ee1fe7db14f..331ecdb1b17 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java @@ -22,6 +22,7 @@ import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine; import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.yetus.audience.InterfaceAudience; /** @@ -29,7 +30,7 @@ import org.apache.yetus.audience.InterfaceAudience; */ @InterfaceAudience.Private public class MobStoreEngine extends DefaultStoreEngine { - + public final static String MOB_COMPACTOR_CLASS_KEY = "hbase.hstore.mobengine.compactor.class"; @Override protected void createStoreFlusher(Configuration conf, HStore store) throws IOException { // When using MOB, we use DefaultMobStoreFlusher always @@ -43,6 +44,13 @@ public class MobStoreEngine extends DefaultStoreEngine { */ @Override 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); + } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java index 1c00e25be3c..3a6a55da9e3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java @@ -25,16 +25,14 @@ import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Calendar; import java.util.Collection; +import java.util.Collections; import java.util.Date; +import java.util.HashSet; import java.util.List; import java.util.Optional; +import java.util.Set; 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.fs.FileStatus; 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.backup.HFileArchiver; 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.RegionInfoBuilder; 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.HFileContext; 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.HStore; 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.EnvironmentEdgeManager; 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.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -85,23 +76,15 @@ import org.slf4j.LoggerFactory; public final class MobUtils { private static final Logger LOG = LoggerFactory.getLogger(MobUtils.class); - private final static long WEEKLY_THRESHOLD_MULTIPLIER = 7; - private final static long MONTHLY_THRESHOLD_MULTIPLIER = 4 * WEEKLY_THRESHOLD_MULTIPLIER; + public static final String SEP = "_"; private static final ThreadLocal LOCAL_FORMAT = - new ThreadLocal() { - @Override - protected SimpleDateFormat initialValue() { - return new SimpleDateFormat("yyyyMMdd"); - } - }; - - private static final byte[] REF_DELETE_MARKER_TAG_BYTES; - static { - List tags = new ArrayList<>(); - tags.add(MobConstants.MOB_REF_TAG); - REF_DELETE_MARKER_TAG_BYTES = TagUtil.fromList(tags); - } + new ThreadLocal() { + @Override + protected SimpleDateFormat initialValue() { + return new SimpleDateFormat("yyyyMMdd"); + } + }; /** * Private constructor to keep this class from being instantiated. @@ -127,45 +110,6 @@ public final class MobUtils { 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. * @param cell The current cell. @@ -213,11 +157,10 @@ public final class MobUtils { } /** - * Indicates whether it's a raw scan. - * The information is set in the attribute "hbase.mob.scan.raw" of scan. - * For a mob cell, in a normal scan the scanners retrieves the mob cell from the mob file. - * In a raw scan, the scanner directly returns cell in HBase without retrieve the one in - * the mob file. + * Indicates whether it's a raw scan. The information is set in the attribute "hbase.mob.scan.raw" + * of scan. For a mob cell, in a normal scan the scanners retrieves the mob cell from the mob + * file. In a raw scan, the scanner directly returns cell in HBase without retrieve the one in the + * mob file. * @param scan The current 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. - * The information is set in the attribute "hbase.mob.scan.ref.only" of scan. - * If it's a ref only scan, only the cells with ref tag are returned. + * Indicates whether it's a reference only scan. The information is set in the attribute + * "hbase.mob.scan.ref.only" of scan. If it's a ref only scan, only the cells with ref tag are + * returned. * @param scan The current 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. - * The information is set in the attribute "hbase.mob.cache.blocks" of scan. + * Indicates whether the scan contains the information of caching blocks. The information is set + * in the attribute "hbase.mob.cache.blocks" of scan. * @param scan The current scan. * @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. - * - * @param scan - * The current 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. + * @param scan The current 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) { scan.setAttribute(MobConstants.MOB_CACHE_BLOCKS, Bytes.toBytes(cacheBlocks)); } /** - * Cleans the expired mob files. - * Cleans the files whose creation date is older than (current - columnFamily.ttl), and - * the minVersions of that column family is 0. + * Cleans the expired mob files. Cleans the files whose creation date is older than (current - + * columnFamily.ttl), and the minVersions of that column family is 0. * @param fs The current file system. * @param conf The current configuration. * @param tableName The current table name. @@ -330,11 +268,11 @@ public final class MobUtils { Date fileDate = parseDate(MobFileName.getDateFromName(fileName)); if (LOG.isDebugEnabled()) { - LOG.debug("Checking file " + fileName); + LOG.debug("Checking file {}", fileName); } if (fileDate.getTime() < expireDate.getTime()) { if (LOG.isDebugEnabled()) { - LOG.debug(fileName + " is an expired file"); + LOG.debug("{} is an expired file", fileName); } filesToClean .add(new HStoreFile(fs, file.getPath(), conf, cacheConfig, BloomType.NONE, true)); @@ -345,19 +283,17 @@ public final class MobUtils { } if (!filesToClean.isEmpty()) { try { - removeMobFiles(conf, fs, tableName, mobTableDir, columnDescriptor.getName(), - filesToClean); + removeMobFiles(conf, fs, tableName, mobTableDir, columnDescriptor.getName(), filesToClean); deletedFileCount = filesToClean.size(); } catch (IOException 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. - * It's {HBASE_DIR}/mobdir. + * Gets the root dir of the mob files. It's {HBASE_DIR}/mobdir. * @param conf The current configuration. * @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. - * It's {rootDir}/mobdir. + * Gets the root dir of the mob files under the qualified HBase root dir. It's {rootDir}/mobdir. * @param rootDir The qualified path of HBase root directory. * @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. - * It's {rootDir}/mobdir/data/${namespace}/${tableName} + * Gets the table dir of the mob files under the qualified HBase root dir. It's + * {rootDir}/mobdir/data/${namespace}/${tableName} * @param rootDir The qualified path of HBase root directory. * @param tableName The name of table. * @return The table dir of the mob file. @@ -400,8 +335,8 @@ public final class MobUtils { } /** - * Gets the region dir of the mob files. - * It's {HBASE_DIR}/mobdir/data/{namespace}/{tableName}/{regionEncodedName}. + * Gets the region dir of the mob files. It's + * {HBASE_DIR}/mobdir/data/{namespace}/{tableName}/{regionEncodedName}. * @param conf The current configuration. * @param tableName The current table name. * @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. - * It's {rootDir}/mobdir/data/{namespace}/{tableName}/{regionEncodedName}. + * Gets the region dir of the mob files under the specified root dir. It's + * {rootDir}/mobdir/data/{namespace}/{tableName}/{regionEncodedName}. * @param rootDir The qualified path of HBase root directory. * @param tableName The current table name. * @return The region dir of the mob files. @@ -424,8 +359,8 @@ public final class MobUtils { } /** - * Gets the family dir of the mob files. - * It's {HBASE_DIR}/mobdir/{namespace}/{tableName}/{regionEncodedName}/{columnFamilyName}. + * Gets the family dir of the mob files. It's + * {HBASE_DIR}/mobdir/{namespace}/{tableName}/{regionEncodedName}/{columnFamilyName}. * @param conf The current configuration. * @param tableName The current table name. * @param familyName The current family name. @@ -436,8 +371,8 @@ public final class MobUtils { } /** - * Gets the family dir of the mob files. - * It's {HBASE_DIR}/mobdir/{namespace}/{tableName}/{regionEncodedName}/{columnFamilyName}. + * Gets the family dir of the mob files. It's + * {HBASE_DIR}/mobdir/{namespace}/{tableName}/{regionEncodedName}/{columnFamilyName}. * @param regionPath The path of mob region which is a dummy one. * @param familyName The current family name. * @return The family dir of the mob files. @@ -447,19 +382,14 @@ public final class MobUtils { } /** - * Gets the RegionInfo of the mob files. - * This is a dummy region. The mob files are not saved in a region in HBase. - * This is only used in mob snapshot. It's internally used only. + * Gets the RegionInfo of the mob files. This is a dummy region. The mob files are not saved in a + * region in HBase. This is only used in mob snapshot. It's internally used only. * @param tableName * @return A dummy mob region info. */ public static RegionInfo getMobRegionInfo(TableName tableName) { - return RegionInfoBuilder.newBuilder(tableName) - .setStartKey(MobConstants.MOB_REGION_NAME_BYTES) - .setEndKey(HConstants.EMPTY_END_ROW) - .setSplit(false) - .setRegionId(0) - .build(); + return RegionInfoBuilder.newBuilder(tableName).setStartKey(MobConstants.MOB_REGION_NAME_BYTES) + .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. */ public static boolean isMobRegionInfo(RegionInfo regionInfo) { - return regionInfo == null ? false : getMobRegionInfo(regionInfo.getTable()).getEncodedName() - .equals(regionInfo.getEncodedName()); + return regionInfo == null ? false + : getMobRegionInfo(regionInfo.getTable()).getEncodedName() + .equals(regionInfo.getEncodedName()); } /** @@ -482,16 +413,6 @@ public final class MobUtils { 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. * @param conf The current configuration. @@ -504,16 +425,16 @@ public final class MobUtils { public static void removeMobFiles(Configuration conf, FileSystem fs, TableName tableName, Path tableDir, byte[] family, Collection storeFiles) throws IOException { HFileArchiver.archiveStoreFiles(conf, fs, getMobRegionInfo(tableName), tableDir, family, - storeFiles); + storeFiles); } /** - * Creates a mob reference KeyValue. - * The value of the mob reference KeyValue is mobCellValueSize + mobFileName. + * Creates a mob reference KeyValue. The value of the mob reference KeyValue is mobCellValueSize + + * mobFileName. * @param cell The original Cell. * @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 - * cloning the snapshot. + * @param tableNameTag The tag of the current table name. It's very important in cloning the + * snapshot. * @return The mob reference KeyValue. */ 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, ColumnFamilyDescriptor family, String date, Path basePath, long maxKeyCount, Compression.Algorithm compression, String startKey, CacheConfig cacheConfig, - Encryption.Context cryptoContext, boolean isCompaction) + Encryption.Context cryptoContext, boolean isCompaction, String regionName) throws IOException { MobFileName mobFileName = MobFileName.create(startKey, date, - UUID.randomUUID().toString().replaceAll("-", "")); - 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("-", "")); + UUID.randomUUID().toString().replaceAll("-", ""), regionName); return createWriter(conf, fs, family, mobFileName, basePath, maxKeyCount, compression, 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. * @param conf The current configuration. @@ -654,12 +500,10 @@ public final class MobUtils { public static StoreFileWriter createWriter(Configuration conf, FileSystem fs, ColumnFamilyDescriptor family, MobFileName mobFileName, Path basePath, long maxKeyCount, Compression.Algorithm compression, CacheConfig cacheConfig, Encryption.Context cryptoContext, - boolean isCompaction) - throws IOException { - return createWriter(conf, fs, family, - new Path(basePath, mobFileName.getFileName()), maxKeyCount, compression, cacheConfig, - cryptoContext, HStore.getChecksumType(conf), HStore.getBytesPerChecksum(conf), - family.getBlocksize(), BloomType.NONE, isCompaction); + boolean isCompaction) throws IOException { + return createWriter(conf, fs, family, new Path(basePath, mobFileName.getFileName()), + maxKeyCount, compression, cacheConfig, 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. */ public static StoreFileWriter createWriter(Configuration conf, FileSystem fs, - ColumnFamilyDescriptor family, Path path, long maxKeyCount, - Compression.Algorithm compression, CacheConfig cacheConfig, Encryption.Context cryptoContext, - ChecksumType checksumType, int bytesPerChecksum, int blocksize, BloomType bloomType, - boolean isCompaction) + ColumnFamilyDescriptor family, Path path, long maxKeyCount, Compression.Algorithm compression, + CacheConfig cacheConfig, Encryption.Context cryptoContext, ChecksumType checksumType, + int bytesPerChecksum, int blocksize, BloomType bloomType, boolean isCompaction) throws IOException { if (compression == null) { compression = HFile.DEFAULT_COMPRESSION_ALGORITHM; @@ -696,13 +539,10 @@ public final class MobUtils { writerCacheConf = cacheConfig; } HFileContext hFileContext = new HFileContextBuilder().withCompression(compression) - .withIncludesMvcc(true).withIncludesTags(true) - .withCompressTags(family.isCompressTags()) - .withChecksumType(checksumType) - .withBytesPerCheckSum(bytesPerChecksum) - .withBlockSize(blocksize) - .withHBaseCheckSum(true).withDataBlockEncoding(family.getDataBlockEncoding()) - .withEncryptionContext(cryptoContext) + .withIncludesMvcc(true).withIncludesTags(true).withCompressTags(family.isCompressTags()) + .withChecksumType(checksumType).withBytesPerCheckSum(bytesPerChecksum) + .withBlockSize(blocksize).withHBaseCheckSum(true) + .withDataBlockEncoding(family.getDataBlockEncoding()).withEncryptionContext(cryptoContext) .withCreateTime(EnvironmentEdgeManager.currentTime()).build(); StoreFileWriter w = new StoreFileWriter.Builder(conf, writerCacheConf, fs) @@ -712,62 +552,9 @@ public final class MobUtils { } /** - * Commits the mob file. - * @param conf The current configuration. - * @param fs The current file system. - * @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. + * 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. * @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. - * 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. + * Gets the mob value length from the mob ref cell. 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. * @return The real mob value length. */ @@ -789,98 +574,15 @@ public final class MobUtils { } /** - * Gets the mob file name from the mob ref cell. - * 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. + * Gets the mob file name from the mob ref cell. 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. * @return The mob file name. */ public static String getMobFileName(Cell cell) { return Bytes.toString(cell.getValueArray(), cell.getValueOffset() + 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 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; + cell.getValueLength() - Bytes.SIZEOF_INT); } /** @@ -899,14 +601,31 @@ public final class MobUtils { } /** - * 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. + * Get list of Mob column families (if any exists) + * @param htd table descriptor + * @return list of Mob column families + */ + public static List getMobColumnFamilies(TableDescriptor htd) { + + List fams = new ArrayList(); + 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. * @return True if the readEmptyValueOnMobCellMiss is enabled. */ public static boolean isReadEmptyValueOnMobCellMiss(Scan scan) { byte[] readEmptyValueOnMobCellMiss = - scan.getAttribute(MobConstants.EMPTY_VALUE_ON_MOBCELL_MISS); + scan.getAttribute(MobConstants.EMPTY_VALUE_ON_MOBCELL_MISS); try { return readEmptyValueOnMobCellMiss != null && Bytes.toBoolean(readEmptyValueOnMobCellMiss); } 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. * @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... - * @param id Partition id to be filled out - * @param firstDayOfCurrentMonth The first day in the current month - * @param firstDayOfCurrentWeek The first day in the current week - * @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 + * Get list of referenced MOB files from a given collection of store files + * @param storeFiles store files + * @param mobDir MOB file directory + * @return list of MOB file paths */ - 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; - id.setThreshold(threshold); - if (threshold <= 0) { - id.setDate(dateStr); - return skipCompcation; - } + public static List getReferencedMobFiles(Collection storeFiles, Path mobDir) { - long finalThreshold; - Date date; - try { - date = MobUtils.parseDate(dateStr); - } catch (ParseException e) { - LOG.warn("Failed to parse date " + dateStr, e); - id.setDate(dateStr); - 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; + Set mobSet = new HashSet(); + for (HStoreFile sf : storeFiles) { + byte[] value = sf.getMetadataValue(HStoreFile.MOB_FILE_REFS); + if (value != null && value.length > 1) { + String s = Bytes.toString(value); + String[] all = s.split(","); + Collections.addAll(mobSet, all); } } - - if ((policy == MobCompactPartitionPolicy.MONTHLY) || - (policy == MobCompactPartitionPolicy.WEEKLY)) { - // 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; - } + List retList = new ArrayList(); + for (String name : mobSet) { + retList.add(new Path(mobDir, name)); } - - // Rest is daily - id.setDate(dateStr); - return skipCompcation; + return retList; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactionRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactionRequest.java deleted file mode 100644 index 5025ed26122..00000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactionRequest.java +++ /dev/null @@ -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 - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java deleted file mode 100644 index c5d93ea6e06..00000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java +++ /dev/null @@ -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 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 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 compact(List files, boolean allFiles) - throws IOException; -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactionRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactionRequest.java deleted file mode 100644 index ab917a2d922..00000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactionRequest.java +++ /dev/null @@ -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 delPartitions; - protected Collection compactionPartitions; - - public PartitionedMobCompactionRequest(Collection compactionPartitions, - List delPartitions) { - this.selectionTime = EnvironmentEdgeManager.currentTime(); - this.compactionPartitions = compactionPartitions; - this.delPartitions = delPartitions; - } - - /** - * Gets the compaction partitions. - * @return The compaction partitions. - */ - public Collection getCompactionPartitions() { - return this.compactionPartitions; - } - - /** - * Gets the del files. - * @return The del files. - */ - public List 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 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 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 delFiles = new ArrayList(); - private List 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 getStoreFiles() { - return storeFiles; - } - - List listDelFiles() { - return Collections.unmodifiableList(delFiles); - } - - void addDelFileList(final Collection 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 { - 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); - } - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java deleted file mode 100644 index 669bffcb4cb..00000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java +++ /dev/null @@ -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 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 compact(List 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 candidates, - boolean allFiles) throws IOException { - final Map filesToCompact = new HashMap<>(); - final CompactionPartitionId id = new CompactionPartitionId(); - final NavigableMap delFilesToCompact = new TreeMap<>(); - final CompactionDelPartitionId delId = new CompactionDelPartitionId(); - final ArrayList 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 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. - *
      - *
    1. Compacts the del files.
    2. - *
    3. Compacts the selected small mob files and all the del files.
    4. - *
    5. If all the candidates are selected, delete the del files.
    6. - *
    - * @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 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 newDelPaths = compactDelFiles(request, delPartition.listDelFiles()); - delPartition.cleanDelFiles(); - delPartition.addDelFileList(newDelPaths); - } - - List 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 { - 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 getListOfDelFilesForPartition(final CompactionPartition partition, - final List delPartitions) { - // Binary search for startKey and endKey - - List 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 compactMobFiles(final PartitionedMobCompactionRequest request) - throws IOException { - Collection 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 paths = new ArrayList<>(); - final Connection c = ConnectionFactory.createConnection(conf); - final Table table = c.getTable(tableName); - - try { - Map>> 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 delFiles = getListOfDelFilesForPartition(partition, - request.getDelPartitions()); - - results.put(partition.getPartitionId(), pool.submit(new Callable>() { - @Override - public List 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 failedPartitions = new ArrayList<>(); - for (Entry>> 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 compactMobFilePartition(PartitionedMobCompactionRequest request, - CompactionPartition partition, - List 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 newFiles = new ArrayList<>(); - List 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 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 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 filesToCompact, int batch, - Path bulkloadPathOfPartition, Path bulkloadColumnPath, - List 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 mobFilesToCompact = filesToCompact.subList(0, batch); - // Pair(maxSeqId, cellsCount) - Pair 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 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 compactDelFiles(PartitionedMobCompactionRequest request, - List 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 paths = new ArrayList<>(); - while (offset < delFilePaths.size()) { - // get the batch - int batch = compactionBatchSize; - if (delFilePaths.size() - offset < compactionBatchSize) { - batch = delFilePaths.size() - offset; - } - List 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 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 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 filesToCompact, ScanType scanType) - throws IOException { - List 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 getFileInfo(List 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; - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java index b8ea9609e2b..d7f2ba3999f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java @@ -193,28 +193,6 @@ public class HMobStore extends HStore { 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. @@ -231,7 +209,7 @@ public class HMobStore extends HStore { Compression.Algorithm compression, byte[] startKey, boolean isCompaction) throws IOException { MobFileName mobFileName = MobFileName.create(startKey, date, UUID.randomUUID() - .toString().replaceAll("-", "")); + .toString().replaceAll("-", ""), region.getRegionInfo().getEncodedName()); return createWriterInTmp(mobFileName, basePath, maxKeyCount, compression, isCompaction); } @@ -265,8 +243,7 @@ public class HMobStore extends HStore { } Path dstPath = new Path(targetPath, sourceFile.getName()); validateMobFile(sourceFile); - String msg = "Renaming flushed file from " + sourceFile + " to " + dstPath; - LOG.info(msg); + LOG.info(" FLUSH Renaming flushed file from {} to {}", sourceFile, dstPath); Path parent = dstPath.getParent(); if (!region.getFilesystem().exists(parent)) { region.getFilesystem().mkdirs(parent); @@ -309,6 +286,20 @@ public class HMobStore extends HStore { 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. * @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() { return this.refCellTags; } + + } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 6a92be4f065..e1c55ec8b59 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -1901,7 +1901,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } @VisibleForTesting - void setTableDescriptor(TableDescriptor desc) { + public void setTableDescriptor(TableDescriptor desc) { htableDescriptor = desc; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index d77a4725951..affd43fc14b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -1973,7 +1973,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, finishCompactionRequest(compaction.getRequest()); } - private void finishCompactionRequest(CompactionRequestImpl cr) { + protected void finishCompactionRequest(CompactionRequestImpl cr) { this.region.reportCompactionRequestEnd(cr.isMajor(), cr.getFiles().size(), cr.getSize()); if (cr.isOffPeak()) { offPeakCompactionTracker.set(false); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java index fa152c5061a..0f227be472f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java @@ -107,6 +107,12 @@ public class HStoreFile implements StoreFile { /** Key for the number of mob cells in metadata */ 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 */ public static final byte[] BULKLOAD_TASK_KEY = Bytes.toBytes("BULKLOAD_SOURCE_TASK"); public static final byte[] BULKLOAD_TIME_KEY = Bytes.toBytes("BULKLOAD_TIMESTAMP"); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java index 15ed359a882..5eaab23fc6c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java @@ -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.ReaderType; 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.yetus.audience.InterfaceAudience; import org.slf4j.Logger; @@ -52,36 +53,25 @@ public class StoreFileInfo { private static final Logger LOG = LoggerFactory.getLogger(StoreFileInfo.class); /** - * A non-capture group, for hfiles, so that this can be embedded. - * HFiles are uuid ([0-9a-z]+). Bulk loaded hfiles has (_SeqId_[0-9]+_) has suffix. - * The mob del file has (_del) as suffix. + * A non-capture group, for hfiles, so that this can be embedded. HFiles are uuid ([0-9a-z]+). + * Bulk loaded hfiles has (_SeqId_[0-9]+_) has suffix. The mob del file has (_del) as suffix. */ public static final String HFILE_NAME_REGEX = "[0-9a-f]+(?:(?:_SeqId_[0-9]+_)|(?:_del))?"; /** Regex that will work for hfiles */ - private static final Pattern HFILE_NAME_PATTERN = - Pattern.compile("^(" + HFILE_NAME_REGEX + ")"); + private static final Pattern HFILE_NAME_PATTERN = Pattern.compile("^(" + HFILE_NAME_REGEX + ")"); + /** - * A non-capture group, for del files, so that this can be embedded. - * A del file has (_del) as suffix. + * Regex that will work for straight reference names ({@code .}) and + * hfilelink reference names ({@code + * + * =-.}) 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)"; - - /** 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 .}) - * and hfilelink reference names ({@code
    =-.}) - * 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)); + 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 boolean DEFAULT_STORE_FILE_READER_NO_READAHEAD = false; @@ -144,7 +134,7 @@ public class StoreFileInfo { this.initialPath = initialPath; this.primaryReplica = primaryReplica; 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; if (HFileLink.isHFileLink(p)) { // HFileLink @@ -162,7 +152,7 @@ public class StoreFileInfo { this.link = null; } LOG.trace("{} is a {} reference to {}", p, reference.getFileRegion(), referencePath); - } else if (isHFile(p)) { + } else if (isHFile(p) || isMobFile(p) || isMobRefFile(p)) { // HFile if (fileStatus != null) { this.createdTimestamp = fileStatus.getModificationTime(); @@ -227,11 +217,11 @@ public class StoreFileInfo { this.conf = conf; this.primaryReplica = false; 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.link = link; 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; } - /* + /** * @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() { return this.reference; @@ -278,8 +269,7 @@ public class StoreFileInfo { return this.hdfsBlocksDistribution; } - StoreFileReader createReader(ReaderContext context, CacheConfig cacheConf) - throws IOException { + StoreFileReader createReader(ReaderContext context, CacheConfig cacheConf) throws IOException { StoreFileReader reader = null; if (this.reference != null) { reader = new HalfStoreFileReader(context, hfileInfo, cacheConf, reference, refCount, conf); @@ -316,12 +306,9 @@ public class StoreFileInfo { status = fs.getFileStatus(initialPath); } long length = status.getLen(); - ReaderContextBuilder contextBuilder = new ReaderContextBuilder() - .withInputStreamWrapper(in) - .withFileSize(length) - .withPrimaryReplicaReader(this.primaryReplica) - .withReaderType(type) - .withFileSystem(fs); + ReaderContextBuilder contextBuilder = + new ReaderContextBuilder().withInputStreamWrapper(in).withFileSize(length) + .withPrimaryReplicaReader(this.primaryReplica).withReaderType(type).withFileSystem(fs); if (this.reference != null) { contextBuilder.withFilePath(this.getPath()); } else { @@ -425,8 +412,8 @@ public class StoreFileInfo { @Override public String toString() { - return this.getPath() + - (isReference() ? "->" + getReferredToFile(this.getPath()) + "-" + reference : ""); + return this.getPath() + + (isReference() ? "->" + getReferredToFile(this.getPath()) + "-" + reference : ""); } /** @@ -443,22 +430,42 @@ public class StoreFileInfo { } /** - * @param path Path to check. - * @return True if the path has format of a del file. + * Checks if the file is a MOB file + * @param path path to a file + * @return true, if - yes, false otherwise */ - public static boolean isDelFile(final Path path) { - return isDelFile(path.getName()); + public static boolean isMobFile(final Path path) { + 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. - * @return True if the file name has format of a del file. + * Checks if the file is a MOB reference file, + * created by snapshot + * @param path path to a file + * @return true, if - yes, false otherwise */ - public static boolean isDelFile(final String fileName) { - Matcher m = DELFILE_NAME_PATTERN.matcher(fileName); - return m.matches() && m.groupCount() > 0; + public static boolean isMobRefFile(final Path path) { + String fileName = path.getName(); + 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. * @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 - * hierarchy of ${hbase.rootdir}/data/${namespace}/tablename/regionname/familyname. + * Return path to the file referred to by a Reference. Presumes a directory hierarchy of + * ${hbase.rootdir}/data/${namespace}/tablename/regionname/familyname. * @param p Path to a Reference file. * @return Calculated path to parent region file. * @throws IllegalArgumentException when path regex fails to match. @@ -494,8 +501,7 @@ public class StoreFileInfo { Matcher m = REF_NAME_PATTERN.matcher(p.getName()); if (m == null || !m.matches()) { LOG.warn("Failed match of store file name {}", p.toString()); - throw new IllegalArgumentException("Failed match of store file name " + - p.toString()); + throw new IllegalArgumentException("Failed match of store file name " + p.toString()); } // 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); // Build up new path with the referenced region in place of our current - // region in the reference path. Also strip regionname suffix from name. - return new Path(new Path(new Path(tableDir, otherRegion), - p.getParent().getName()), nameStrippedOfSuffix); + // region in the reference path. Also strip regionname suffix from name. + return new Path(new Path(new Path(tableDir, otherRegion), p.getParent().getName()), + nameStrippedOfSuffix); } /** @@ -517,8 +523,9 @@ public class StoreFileInfo { * @return true if the file could be a valid store file, false otherwise */ public static boolean validateStoreFileName(final String fileName) { - if (HFileLink.isHFileLink(fileName) || isReference(fileName)) - return(true); + if (HFileLink.isHFileLink(fileName) || isReference(fileName)) { + return true; + } return !fileName.contains("-"); } @@ -527,12 +534,12 @@ public class StoreFileInfo { * @param fileStatus The {@link FileStatus} of the file * @return true if the file is valid */ - public static boolean isValid(final FileStatus fileStatus) - throws IOException { + public static boolean isValid(final FileStatus fileStatus) throws IOException { final Path p = fileStatus.getPath(); - if (fileStatus.isDirectory()) + if (fileStatus.isDirectory()) { return false; + } // Check for empty hfile. Should never be the case but can happen // 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 - * file.For reference file, we don't compute the exact value. We use some - * estimate instead given it might be good enough. we assume bottom part - * takes the first half of reference file, top part takes the second half - * of the reference file. This is just estimate, given - * midkey ofregion != midkey of HFile, also the number and size of keys vary. - * If this estimate isn't good enough, we can improve it later. - * @param fs The FileSystem - * @param reference The reference - * @param status The reference FileStatus + * helper function to compute HDFS blocks distribution of a given reference file.For reference + * file, we don't compute the exact value. We use some estimate instead given it might be good + * enough. we assume bottom part takes the first half of reference file, top part takes the second + * half of the reference file. This is just estimate, given midkey ofregion != midkey of HFile, + * also the number and size of keys vary. If this estimate isn't good enough, we can improve it + * later. + * @param fs The FileSystem + * @param reference The reference + * @param status The reference FileStatus * @return HDFS blocks distribution */ - private static HDFSBlocksDistribution computeRefFileHDFSBlockDistribution( - final FileSystem fs, final Reference reference, final FileStatus status) - throws IOException { + private static HDFSBlocksDistribution computeRefFileHDFSBlockDistribution(final FileSystem fs, + final Reference reference, final FileStatus status) throws IOException { if (status == null) { return null; } @@ -569,36 +574,59 @@ public class StoreFileInfo { long length = 0; if (Reference.isTopFileRegion(reference.getFileRegion())) { - start = status.getLen()/2; - length = status.getLen() - status.getLen()/2; + start = status.getLen() / 2; + length = status.getLen() - status.getLen() / 2; } else { start = 0; - length = status.getLen()/2; + length = status.getLen() / 2; } return FSUtils.computeHDFSBlocksDistribution(fs, status, start, length); } @Override public boolean equals(Object that) { - if (this == that) return true; - if (that == null) return false; + if (this == that) { + return true; + } + if (that == null) { + return false; + } - if (!(that instanceof StoreFileInfo)) return false; + if (!(that instanceof StoreFileInfo)) { + return false; + } - StoreFileInfo o = (StoreFileInfo)that; - if (initialPath != null && o.initialPath == null) return false; - if (initialPath == null && o.initialPath != null) return false; + StoreFileInfo o = (StoreFileInfo) that; + if (initialPath != null && o.initialPath == null) { + return false; + } + if (initialPath == null && o.initialPath != null) { + return false; + } if (initialPath != o.initialPath && initialPath != null - && !initialPath.equals(o.initialPath)) return false; - - if (reference != null && o.reference == null) return false; - if (reference == null && o.reference != null) 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 != 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) return false; - if (link != o.link && link != null && !link.equals(o.link)) return false; + if (link != null && o.link == null) { + return false; + } + if (link == null && o.link != null) { + return false; + } + if (link != o.link && link != null && !link.equals(o.link)) { + return false; + } return true; } @@ -607,9 +635,9 @@ public class StoreFileInfo { public int hashCode() { int hash = 17; 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()); - return hash; + return hash; } /** @@ -652,8 +680,7 @@ public class StoreFileInfo { StoreFileReader reader = null; if (this.coprocessorHost != null) { reader = this.coprocessorHost.preStoreFileReaderOpen(fs, this.getPath(), - context.getInputStreamWrapper(), context.getFileSize(), - cacheConf, reference); + context.getInputStreamWrapper(), context.getFileSize(), cacheConf, reference); } return reader; } @@ -663,8 +690,7 @@ public class StoreFileInfo { StoreFileReader res = reader; if (this.coprocessorHost != null) { res = this.coprocessorHost.postStoreFileReaderOpen(fs, this.getPath(), - context.getInputStreamWrapper(), context.getFileSize(), - cacheConf, reference, reader); + context.getInputStreamWrapper(), context.getFileSize(), cacheConf, reference, reader); } return res; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java index 02665a34541..3de97e80e19 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java @@ -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.MAX_SEQ_ID_KEY; 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 java.io.IOException; import java.net.InetSocketAddress; @@ -36,6 +38,8 @@ import java.util.UUID; import java.util.function.Supplier; import java.util.regex.Pattern; import java.util.stream.Collectors; + +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; 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.HFile; 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.BloomFilterFactory; 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. * Call before {@link #close()} since its written as meta data to this file. @@ -238,6 +246,21 @@ public class StoreFileWriter implements CellSink, ShipperListener { 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 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 */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java index 252f72b8585..6dfb79aef2e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java @@ -1141,7 +1141,7 @@ public abstract class FSUtils extends CommonFSUtils { @Override protected boolean accept(Path p, @CheckForNull Boolean isDir) { - if (!StoreFileInfo.isHFile(p)) { + if (!StoreFileInfo.isHFile(p) && !StoreFileInfo.isMobFile(p)) { return false; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java index aeff96e8964..fe79a65dbee 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java @@ -263,9 +263,9 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase { byte[][] families = { Bytes.toBytes("mob") }; 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); waitUntilMobCompactionFinished(tableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java new file mode 100644 index 00000000000..02ace93d667 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java @@ -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: + *
    {@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);
    + *   }
    + * }
    + * }
    + * @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 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; + + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java new file mode 100644 index 00000000000..e9c6969d969 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java @@ -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); + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java new file mode 100644 index 00000000000..011fcadbf86 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java @@ -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); + } + } + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java new file mode 100644 index 00000000000..2d9268c17de --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java @@ -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; + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java new file mode 100644 index 00000000000..d4f13f6cb75 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java @@ -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()); + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java new file mode 100644 index 00000000000..9871b966711 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java @@ -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); + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java new file mode 100644 index 00000000000..9145655a23b --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java @@ -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()); + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java new file mode 100644 index 00000000000..de1fecd4689 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java @@ -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); + } + +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCleanerChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCleanerChore.java new file mode 100644 index 00000000000..9e9970dfb19 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCleanerChore.java @@ -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; + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java index ae53ff21feb..a6a2ee55009 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java @@ -47,6 +47,7 @@ public class TestMobFileName { private Date date; private String dateStr; private byte[] startKey; + private String regionName = "region"; @Before public void setUp() { @@ -59,32 +60,32 @@ public class TestMobFileName { @Test public void testHashCode() { - assertEquals(MobFileName.create(startKey, dateStr, uuid).hashCode(), - MobFileName.create(startKey, dateStr, uuid).hashCode()); - assertNotSame(MobFileName.create(startKey, dateStr, uuid), - MobFileName.create(startKey, dateStr, uuid)); + assertEquals(MobFileName.create(startKey, dateStr, uuid, regionName).hashCode(), + MobFileName.create(startKey, dateStr, uuid, regionName).hashCode()); + assertNotSame(MobFileName.create(startKey, dateStr, uuid, regionName), + MobFileName.create(startKey, dateStr, uuid, regionName)); } @Test public void testCreate() { - MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid); + MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid, regionName); assertEquals(mobFileName, MobFileName.create(mobFileName.getFileName())); } @Test 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(dateStr, mobFileName.getDate()); assertEquals(mobFileName.getFileName(), MD5Hash.getMD5AsHex(startKey, 0, startKey.length) - + dateStr + uuid); + + dateStr + uuid+"_"+regionName); } @Test public void testEquals() { - MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid); + MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid, regionName); assertTrue(mobFileName.equals(mobFileName)); assertFalse(mobFileName.equals(this)); - assertTrue(mobFileName.equals(MobFileName.create(startKey, dateStr, uuid))); + assertTrue(mobFileName.equals(MobFileName.create(startKey, dateStr, uuid, regionName))); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java similarity index 84% rename from hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java rename to hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java index 74c8d7934da..99dd65312d2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * 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.fam1; @@ -29,6 +29,7 @@ import java.util.ArrayList; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Random; 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.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.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.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; -import org.apache.hadoop.hbase.mob.MobConstants; -import org.apache.hadoop.hbase.mob.MobFileCache; -import org.apache.hadoop.hbase.mob.MobUtils; -import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.HStore; +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.FSUtils; import org.apache.hadoop.hbase.util.Pair; @@ -79,7 +85,7 @@ import org.slf4j.LoggerFactory; /** * Test mob store compaction */ -@Category(SmallTests.class) +@Category(MediumTests.class) public class TestMobStoreCompaction { @ClassRule @@ -178,7 +184,20 @@ public class TestMobStoreCompaction { // Change the threshold larger than the data size setMobThreshold(region, COLUMN_FAMILY, 500); region.initialize(); - region.compactStores(); + + List stores = region.getStores(); + for (HStore store: stores) { + // Force major compaction + store.triggerMajorCompaction(); + Optional 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: mob file count", compactionThreshold, countMobFiles()); @@ -270,30 +289,6 @@ public class TestMobStoreCompaction { // region.compactStores(); region.compact(true); 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 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 { @@ -424,38 +419,4 @@ public class TestMobStoreCompaction { 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 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 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; - } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java similarity index 98% rename from hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java rename to hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java index faae36aa74f..1beeffe76de 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hbase.regionserver; +package org.apache.hadoop.hbase.mob; import java.io.IOException; 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.io.hfile.CorruptHFileException; import org.apache.hadoop.hbase.io.hfile.TestHFile; -import org.apache.hadoop.hbase.mob.MobConstants; -import org.apache.hadoop.hbase.mob.MobTestUtil; -import org.apache.hadoop.hbase.mob.MobUtils; +import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; @@ -141,8 +139,8 @@ public class TestMobStoreScanner { @Test public void testMobStoreScanner() throws Exception { - testGetFromFiles(false); - testGetFromMemStore(false); + testGetFromFiles(false); + testGetFromMemStore(false); testGetReferences(false); testMobThreshold(false); testGetFromArchive(false); @@ -150,8 +148,8 @@ public class TestMobStoreScanner { @Test public void testReversedMobStoreScanner() throws Exception { - testGetFromFiles(true); - testGetFromMemStore(true); + testGetFromFiles(true); + testGetFromMemStore(true); testGetReferences(true); testMobThreshold(true); testGetFromArchive(true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java deleted file mode 100644 index b8e3ce0d440..00000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java +++ /dev/null @@ -1,1214 +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.security.Key; -import java.security.SecureRandom; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Calendar; -import java.util.Collections; -import java.util.List; -import java.util.Objects; -import java.util.Optional; -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 javax.crypto.spec.SecretKeySpec; -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.CellUtil; -import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.NamespaceDescriptor; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.BufferedMutator; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; -import org.apache.hadoop.hbase.client.CompactType; -import org.apache.hadoop.hbase.client.CompactionState; -import org.apache.hadoop.hbase.client.Connection; -import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.Delete; -import org.apache.hadoop.hbase.client.Durability; -import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy; -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.Scan; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.client.TableDescriptorBuilder; -import org.apache.hadoop.hbase.coprocessor.ObserverContext; -import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; -import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; -import org.apache.hadoop.hbase.coprocessor.RegionObserver; -import org.apache.hadoop.hbase.io.HFileLink; -import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; -import org.apache.hadoop.hbase.io.crypto.aes.AES; -import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hadoop.hbase.io.hfile.HFile; -import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner; -import org.apache.hadoop.hbase.mob.MobConstants; -import org.apache.hadoop.hbase.mob.MobFileName; -import org.apache.hadoop.hbase.mob.MobTestUtil; -import org.apache.hadoop.hbase.mob.MobUtils; -import org.apache.hadoop.hbase.regionserver.BloomType; -import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.HStoreFile; -import org.apache.hadoop.hbase.regionserver.Store; -import org.apache.hadoop.hbase.regionserver.StoreFile; -import org.apache.hadoop.hbase.regionserver.StoreFileInfo; -import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker; -import org.apache.hadoop.hbase.security.EncryptionUtil; -import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.testclassification.LargeTests; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.Threads; -import org.junit.AfterClass; -import org.junit.Assert; -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 TestMobCompactor { - - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestMobCompactor.class); - - private static final Logger LOG = LoggerFactory.getLogger(TestMobCompactor.class); - private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private static Configuration conf = null; - private TableName tableName; - private static Connection conn; - private BufferedMutator bufMut; - private Table table; - private static Admin admin; - private TableDescriptorBuilder desc; - private ColumnFamilyDescriptorBuilder cfdb1; - private ColumnFamilyDescriptorBuilder cfdb2; - private static FileSystem fs; - private static final String family1 = "family1"; - private static final String family2 = "family2"; - private static final String qf1 = "qualifier1"; - private static final String qf2 = "qualifier2"; - - private static long tsFor20150907Monday; - private static long tsFor20151120Sunday; - private static long tsFor20151128Saturday; - private static long tsFor20151130Monday; - private static long tsFor20151201Tuesday; - private static long tsFor20151205Saturday; - private static long tsFor20151228Monday; - private static long tsFor20151231Thursday; - private static long tsFor20160101Friday; - private static long tsFor20160103Sunday; - - private static final byte[] mobKey01 = Bytes.toBytes("r01"); - private static final byte[] mobKey02 = Bytes.toBytes("r02"); - private static final byte[] mobKey03 = Bytes.toBytes("r03"); - private static final byte[] mobKey04 = Bytes.toBytes("r04"); - private static final byte[] mobKey05 = Bytes.toBytes("r05"); - private static final byte[] mobKey06 = Bytes.toBytes("r05"); - private static final byte[] mobKey1 = Bytes.toBytes("r1"); - private static final byte[] mobKey2 = Bytes.toBytes("r2"); - private static final byte[] mobKey3 = Bytes.toBytes("r3"); - private static final byte[] mobKey4 = Bytes.toBytes("r4"); - private static final byte[] mobKey5 = Bytes.toBytes("r5"); - private static final byte[] mobKey6 = Bytes.toBytes("r6"); - private static final byte[] mobKey7 = Bytes.toBytes("r7"); - private static final byte[] mobKey8 = Bytes.toBytes("r8"); - private static final String mobValue0 = "mobValue00000000000000000000000000"; - private static final String mobValue1 = "mobValue00000111111111111111111111"; - private static final String mobValue2 = "mobValue00000222222222222222222222"; - private static final String mobValue3 = "mobValue00000333333333333333333333"; - private static final String mobValue4 = "mobValue00000444444444444444444444"; - private static final String mobValue5 = "mobValue00000666666666666666666666"; - private static final String mobValue6 = "mobValue00000777777777777777777777"; - private static final String mobValue7 = "mobValue00000888888888888888888888"; - private static final String mobValue8 = "mobValue00000888888888888888888899"; - - private static byte[] KEYS = Bytes.toBytes("012"); - private static int regionNum = KEYS.length; - private static int delRowNum = 1; - private static int delCellNum = 6; - private static int cellNumPerRow = 3; - private static int rowNumPerFile = 2; - private static ExecutorService pool; - - @Rule - public TestName name = new TestName(); - - @BeforeClass - public static void setUpBeforeClass() throws Exception { - TEST_UTIL.getConfiguration().setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, 5000); - TEST_UTIL.getConfiguration() - .set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, KeyProviderForTesting.class.getName()); - TEST_UTIL.getConfiguration().set(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, "hbase"); - TEST_UTIL.getConfiguration().setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0); - TEST_UTIL.getConfiguration().setInt("hbase.client.retries.number", 1); - TEST_UTIL.getConfiguration().setInt("hbase.hfile.compaction.discharger.interval", 100); - TEST_UTIL.startMiniCluster(1); - pool = createThreadPool(TEST_UTIL.getConfiguration()); - conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration(), pool); - fs = TEST_UTIL.getTestFileSystem(); - conf = TEST_UTIL.getConfiguration(); - admin = TEST_UTIL.getAdmin(); - - // Initialize timestamps for these days - Calendar calendar = Calendar.getInstance(); - calendar.set(2015, 8, 7, 10, 20); - tsFor20150907Monday = calendar.getTimeInMillis(); - - calendar.set(2015, 10, 20, 10, 20); - tsFor20151120Sunday = calendar.getTimeInMillis(); - - calendar.set(2015, 10, 28, 10, 20); - tsFor20151128Saturday = calendar.getTimeInMillis(); - - calendar.set(2015, 10, 30, 10, 20); - tsFor20151130Monday = calendar.getTimeInMillis(); - - calendar.set(2015, 11, 1, 10, 20); - tsFor20151201Tuesday = calendar.getTimeInMillis(); - - calendar.set(2015, 11, 5, 10, 20); - tsFor20151205Saturday = calendar.getTimeInMillis(); - - calendar.set(2015, 11, 28, 10, 20); - tsFor20151228Monday = calendar.getTimeInMillis(); - - calendar.set(2015, 11, 31, 10, 20); - tsFor20151231Thursday = calendar.getTimeInMillis(); - - calendar.set(2016, 0, 1, 10, 20); - tsFor20160101Friday = calendar.getTimeInMillis(); - - calendar.set(2016, 0, 3, 10, 20); - tsFor20160103Sunday = calendar.getTimeInMillis(); - } - - @AfterClass - public static void tearDownAfterClass() throws Exception { - pool.shutdown(); - conn.close(); - TEST_UTIL.shutdownMiniCluster(); - } - - public void setUp(String tableNameAsString) throws IOException { - tableName = TableName.valueOf(tableNameAsString); - cfdb1 = ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family1)); - cfdb1.setMobEnabled(true); - cfdb1.setMobThreshold(5); - cfdb2 = ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family2)); - cfdb2.setMobEnabled(true); - cfdb2.setMobThreshold(5); - desc = TableDescriptorBuilder.newBuilder(tableName); - desc.setColumnFamily(cfdb1.build()); - desc.setColumnFamily(cfdb2.build()); - admin.createTable(desc.build(), getSplitKeys()); - table = conn.getTable(tableName); - bufMut = conn.getBufferedMutator(tableName); - } - - // Set up for mob compaction policy testing - private void setUpForPolicyTest(String tableNameAsString, MobCompactPartitionPolicy type) - throws IOException { - tableName = TableName.valueOf(tableNameAsString); - cfdb1 = ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family1)); - cfdb1.setMobEnabled(true); - cfdb1.setMobThreshold(10); - cfdb1.setMobCompactPartitionPolicy(type); - desc = TableDescriptorBuilder.newBuilder(tableName); - desc.setColumnFamily(cfdb1.build()); - admin.createTable(desc.build()); - table = conn.getTable(tableName); - bufMut = conn.getBufferedMutator(tableName); - } - - // alter mob compaction policy - private void alterForPolicyTest(final MobCompactPartitionPolicy type) - throws Exception { - - cfdb1.setMobCompactPartitionPolicy(type); - desc.modifyColumnFamily(cfdb1.build()); - admin.modifyTable(desc.build()); - LOG.info("alter status finished"); - } - - @Test - public void testMinorCompaction() throws Exception { - resetConf(); - int mergeSize = 5000; - // change the mob compaction merge size - conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize); - - // create a table with namespace - NamespaceDescriptor namespaceDescriptor = NamespaceDescriptor.create("ns").build(); - String tableNameAsString = "ns:testMinorCompaction"; - admin.createNamespace(namespaceDescriptor); - setUp(tableNameAsString); - int count = 4; - // generate mob files - loadData(admin, bufMut, tableName, count, rowNumPerFile); - int rowNumPerRegion = count * rowNumPerFile; - - assertEquals("Before deleting: mob rows count", regionNum * rowNumPerRegion, - MobTestUtil.countMobRows(table)); - assertEquals("Before deleting: mob cells count", regionNum * cellNumPerRow * rowNumPerRegion, - countMobCells(table)); - assertEquals("Before deleting: mob file count", regionNum * count, - countFiles(tableName, true, family1)); - - int largeFilesCount = countLargeFiles(mergeSize, tableName, family1); - createDelFile(table, tableName, Bytes.toBytes(family1), Bytes.toBytes(qf1)); - - assertEquals("Before compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum), - MobTestUtil.countMobRows(table)); - assertEquals("Before compaction: mob cells count", regionNum - * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table)); - assertEquals("Before compaction: family1 mob file count", regionNum * count, - countFiles(tableName, true, family1)); - assertEquals("Before compaction: family2 mob file count", regionNum * count, - countFiles(tableName, true, family2)); - assertEquals("Before compaction: family1 del file count", regionNum, - countFiles(tableName, false, family1)); - assertEquals("Before compaction: family2 del file count", regionNum, - countFiles(tableName, false, family2)); - - // do the mob file compaction - MobCompactor compactor = new PartitionedMobCompactor(conf, fs, tableName, - cfdb1.build(), pool); - compactor.compact(); - - assertEquals("After compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum), - MobTestUtil.countMobRows(table)); - assertEquals("After compaction: mob cells count", regionNum - * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table)); - // After the compaction, the files smaller than the mob compaction merge size - // is merge to one file - assertEquals("After compaction: family1 mob file count", largeFilesCount + regionNum, - countFiles(tableName, true, family1)); - assertEquals("After compaction: family2 mob file count", regionNum * count, - countFiles(tableName, true, family2)); - assertEquals("After compaction: family1 del file count", regionNum, - countFiles(tableName, false, family1)); - assertEquals("After compaction: family2 del file count", regionNum, - countFiles(tableName, false, family2)); - } - - @Test - public void testMinorCompactionWithWeeklyPolicy() throws Exception { - resetConf(); - int mergeSize = 5000; - // change the mob compaction merge size - conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize); - - commonPolicyTestLogic("testMinorCompactionWithWeeklyPolicy", - MobCompactPartitionPolicy.WEEKLY, false, 6, - new String[] { "20150907", "20151120", "20151128", "20151130", "20151205", "20160103" }, - true); - } - - @Test - public void testMajorCompactionWithWeeklyPolicy() throws Exception { - resetConf(); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyPolicy", - MobCompactPartitionPolicy.WEEKLY, true, 5, - new String[] { "20150907", "20151120", "20151128", "20151205", "20160103" }, true); - } - - @Test - public void testMinorCompactionWithMonthlyPolicy() throws Exception { - resetConf(); - int mergeSize = 5000; - // change the mob compaction merge size - conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize); - - commonPolicyTestLogic("testMinorCompactionWithMonthlyPolicy", - MobCompactPartitionPolicy.MONTHLY, false, 4, - new String[] { "20150907", "20151130", "20151231", "20160103" }, true); - } - - @Test - public void testMajorCompactionWithMonthlyPolicy() throws Exception { - resetConf(); - - commonPolicyTestLogic("testMajorCompactionWithMonthlyPolicy", - MobCompactPartitionPolicy.MONTHLY, true, 4, - new String[] {"20150907", "20151130", "20151231", "20160103"}, true); - } - - @Test - public void testMajorCompactionWithWeeklyFollowedByMonthly() throws Exception { - resetConf(); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthly", - MobCompactPartitionPolicy.WEEKLY, true, 5, - new String[] { "20150907", "20151120", "20151128", "20151205", "20160103" }, true); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthly", - MobCompactPartitionPolicy.MONTHLY, true, 4, - new String[] {"20150907", "20151128", "20151205", "20160103" }, false); - } - - @Test - public void testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByWeekly() throws Exception { - resetConf(); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByWeekly", - MobCompactPartitionPolicy.WEEKLY, true, 5, - new String[] { "20150907", "20151120", "20151128", "20151205", "20160103" }, true); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByWeekly", - MobCompactPartitionPolicy.MONTHLY, true, 4, - new String[] { "20150907", "20151128", "20151205", "20160103" }, false); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByWeekly", - MobCompactPartitionPolicy.WEEKLY, true, 4, - new String[] { "20150907", "20151128", "20151205", "20160103" }, false); - } - - @Test - public void testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByDaily() throws Exception { - resetConf(); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByDaily", - MobCompactPartitionPolicy.WEEKLY, true, 5, - new String[] { "20150907", "20151120", "20151128", "20151205", "20160103" }, true); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByDaily", - MobCompactPartitionPolicy.MONTHLY, true, 4, - new String[] { "20150907", "20151128", "20151205", "20160103" }, false); - - commonPolicyTestLogic("testMajorCompactionWithWeeklyFollowedByMonthlyFollowedByDaily", - MobCompactPartitionPolicy.DAILY, true, 4, - new String[] { "20150907", "20151128", "20151205", "20160103" }, false); - } - - @Test - public void testCompactionWithHFileLink() throws IOException, InterruptedException { - resetConf(); - String tableNameAsString = "testCompactionWithHFileLink"; - setUp(tableNameAsString); - int count = 4; - // generate mob files - loadData(admin, bufMut, tableName, count, rowNumPerFile); - int rowNumPerRegion = count * rowNumPerFile; - - long tid = System.currentTimeMillis(); - String snapshotName1 = "snaptb-" + tid; - // take a snapshot - admin.snapshot(snapshotName1, tableName); - - createDelFile(table, tableName, Bytes.toBytes(family1), Bytes.toBytes(qf1)); - - assertEquals("Before compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum), - MobTestUtil.countMobRows(table)); - assertEquals("Before compaction: mob cells count", regionNum - * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table)); - assertEquals("Before compaction: family1 mob file count", regionNum * count, - countFiles(tableName, true, family1)); - assertEquals("Before compaction: family2 mob file count", regionNum * count, - countFiles(tableName, true, family2)); - assertEquals("Before compaction: family1 del file count", regionNum, - countFiles(tableName, false, family1)); - assertEquals("Before compaction: family2 del file count", regionNum, - countFiles(tableName, false, family2)); - - // do the mob compaction - MobCompactor compactor = new PartitionedMobCompactor(conf, fs, tableName, - cfdb1.build(), pool); - compactor.compact(); - - assertEquals("After first compaction: mob rows count", regionNum - * (rowNumPerRegion - delRowNum), MobTestUtil.countMobRows(table)); - assertEquals("After first compaction: mob cells count", regionNum - * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table)); - assertEquals("After first compaction: family1 mob file count", regionNum, - countFiles(tableName, true, family1)); - assertEquals("After first compaction: family2 mob file count", regionNum * count, - countFiles(tableName, true, family2)); - assertEquals("After first compaction: family1 del file count", 0, - countFiles(tableName, false, family1)); - assertEquals("After first compaction: family2 del file count", regionNum, - countFiles(tableName, false, family2)); - assertEquals("After first compaction: family1 hfilelink count", 0, countHFileLinks(family1)); - assertEquals("After first compaction: family2 hfilelink count", 0, countHFileLinks(family2)); - - admin.disableTable(tableName); - // Restore from snapshot, the hfilelink will exist in mob dir - admin.restoreSnapshot(snapshotName1); - admin.enableTable(tableName); - - assertEquals("After restoring snapshot: mob rows count", regionNum * rowNumPerRegion, - MobTestUtil.countMobRows(table)); - assertEquals("After restoring snapshot: mob cells count", regionNum * cellNumPerRow - * rowNumPerRegion, countMobCells(table)); - assertEquals("After restoring snapshot: family1 mob file count", regionNum * count, - countFiles(tableName, true, family1)); - assertEquals("After restoring snapshot: family2 mob file count", regionNum * count, - countFiles(tableName, true, family2)); - assertEquals("After restoring snapshot: family1 del file count", 0, - countFiles(tableName, false, family1)); - assertEquals("After restoring snapshot: family2 del file count", 0, - countFiles(tableName, false, family2)); - assertEquals("After restoring snapshot: family1 hfilelink count", regionNum * count, - countHFileLinks(family1)); - assertEquals("After restoring snapshot: family2 hfilelink count", 0, countHFileLinks(family2)); - - compactor.compact(); - - assertEquals("After second compaction: mob rows count", regionNum * rowNumPerRegion, - MobTestUtil.countMobRows(table)); - assertEquals("After second compaction: mob cells count", regionNum * cellNumPerRow - * rowNumPerRegion, countMobCells(table)); - assertEquals("After second compaction: family1 mob file count", regionNum, - countFiles(tableName, true, family1)); - assertEquals("After second compaction: family2 mob file count", regionNum * count, - countFiles(tableName, true, family2)); - assertEquals("After second compaction: family1 del file count", 0, - countFiles(tableName, false, family1)); - assertEquals("After second compaction: family2 del file count", 0, - countFiles(tableName, false, family2)); - assertEquals("After second compaction: family1 hfilelink count", 0, countHFileLinks(family1)); - assertEquals("After second compaction: family2 hfilelink count", 0, countHFileLinks(family2)); - assertRefFileNameEqual(family1); - } - - @Test - public void testMajorCompactionFromAdmin() throws Exception { - resetConf(); - int mergeSize = 5000; - // change the mob compaction merge size - conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, mergeSize); - SecureRandom rng = new SecureRandom(); - byte[] keyBytes = new byte[AES.KEY_LENGTH]; - rng.nextBytes(keyBytes); - String algorithm = conf.get(HConstants.CRYPTO_KEY_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES); - Key cfKey = new SecretKeySpec(keyBytes, algorithm); - byte[] encryptionKey = EncryptionUtil.wrapKey(conf, - conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, User.getCurrent().getShortName()), cfKey); - final TableName tableName = TableName.valueOf(name.getMethodName()); - TableDescriptorBuilder tableDescriptorBuilder = - TableDescriptorBuilder.newBuilder(tableName); - ColumnFamilyDescriptor cfd1 = - ColumnFamilyDescriptorBuilder - .newBuilder(Bytes.toBytes(family1)) - .setMobEnabled(true) - .setMobThreshold(0) - .setEncryptionType(algorithm) - .setEncryptionKey(encryptionKey).build(); - ColumnFamilyDescriptor cfd2 = - ColumnFamilyDescriptorBuilder - .newBuilder(Bytes.toBytes(family2)) - .setMobEnabled(true) - .setMobThreshold(0).build(); - tableDescriptorBuilder.setColumnFamily(cfd1); - tableDescriptorBuilder.setColumnFamily(cfd2); - admin.createTable(tableDescriptorBuilder.build(), getSplitKeys()); - Table table = conn.getTable(tableName); - BufferedMutator bufMut = conn.getBufferedMutator(tableName); - int count = 4; - // generate mob files - loadData(admin, bufMut, tableName, count, rowNumPerFile); - int rowNumPerRegion = count * rowNumPerFile; - - assertEquals("Before deleting: mob rows count", regionNum * rowNumPerRegion, - MobTestUtil.countMobRows(table)); - assertEquals("Before deleting: mob cells count", regionNum * cellNumPerRow * rowNumPerRegion, - countMobCells(table)); - assertEquals("Before deleting: mob file count", regionNum * count, - countFiles(tableName, true, family1)); - - createDelFile(table, tableName, Bytes.toBytes(family1), Bytes.toBytes(qf1)); - - assertEquals("Before compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum), - MobTestUtil.countMobRows(table)); - assertEquals("Before compaction: mob cells count", regionNum - * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table)); - assertEquals("Before compaction: family1 mob file count", regionNum * count, - countFiles(tableName, true, family1)); - assertEquals("Before compaction: family2 mob file count", regionNum * count, - countFiles(tableName, true, family2)); - assertEquals("Before compaction: family1 del file count", regionNum, - countFiles(tableName, false, family1)); - assertEquals("Before compaction: family2 del file count", regionNum, - countFiles(tableName, false, family2)); - - // do the major mob compaction, it will force all files to compaction - admin.majorCompact(tableName, cfd1.getName(), CompactType.MOB); - - waitUntilMobCompactionFinished(tableName); - assertEquals("After compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum), - MobTestUtil.countMobRows(table)); - assertEquals("After compaction: mob cells count", regionNum - * (cellNumPerRow * rowNumPerRegion - delCellNum), countMobCells(table)); - assertEquals("After compaction: family1 mob file count", regionNum, - countFiles(tableName, true, family1)); - assertEquals("After compaction: family2 mob file count", regionNum * count, - countFiles(tableName, true, family2)); - assertEquals("After compaction: family1 del file count", 0, - countFiles(tableName, false, family1)); - assertEquals("After compaction: family2 del file count", regionNum, - countFiles(tableName, false, family2)); - Assert.assertTrue(verifyEncryption(tableName, family1)); - table.close(); - } - - @Test - public void testScannerOnBulkLoadRefHFiles() throws Exception { - resetConf(); - setUp("testScannerOnBulkLoadRefHFiles"); - long ts = EnvironmentEdgeManager.currentTime(); - byte[] key0 = Bytes.toBytes("k0"); - byte[] key1 = Bytes.toBytes("k1"); - String value0 = "mobValue0"; - String value1 = "mobValue1"; - String newValue0 = "new"; - Put put0 = new Put(key0); - put0.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), ts, Bytes.toBytes(value0)); - loadData(admin, bufMut, tableName, new Put[] { put0 }); - put0 = new Put(key0); - put0.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), ts, Bytes.toBytes(newValue0)); - Put put1 = new Put(key1); - put1.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), ts, Bytes.toBytes(value1)); - loadData(admin, bufMut, tableName, new Put[] { put0, put1 }); - // read the latest cell of key0. - Get get = new Get(key0); - Result result = table.get(get); - ColumnFamilyDescriptor cfd1 = cfdb1.build(); - Cell cell = result.getColumnLatestCell(cfd1.getName(), Bytes.toBytes(qf1)); - assertEquals("Before compaction: mob value of k0", newValue0, - Bytes.toString(CellUtil.cloneValue(cell))); - admin.majorCompact(tableName, cfd1.getName(), CompactType.MOB); - waitUntilMobCompactionFinished(tableName); - // read the latest cell of key0, the cell seqId in bulk loaded file is not reset in the - // scanner. The cell that has "new" value is still visible. - result = table.get(get); - cell = result.getColumnLatestCell(cfd1.getName(), Bytes.toBytes(qf1)); - assertEquals("After compaction: mob value of k0", newValue0, - Bytes.toString(CellUtil.cloneValue(cell))); - // read the ref cell, not read further to the mob cell. - get = new Get(key1); - get.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(true)); - result = table.get(get); - cell = result.getColumnLatestCell(cfd1.getName(), Bytes.toBytes(qf1)); - // the ref name is the new file - Path mobFamilyPath = - MobUtils.getMobFamilyPath(TEST_UTIL.getConfiguration(), tableName, - cfdb1.getNameAsString()); - List paths = new ArrayList<>(); - if (fs.exists(mobFamilyPath)) { - FileStatus[] files = fs.listStatus(mobFamilyPath); - for (FileStatus file : files) { - if (!StoreFileInfo.isDelFile(file.getPath())) { - paths.add(file.getPath()); - } - } - } - assertEquals("After compaction: number of mob files:", 1, paths.size()); - assertEquals("After compaction: mob file name:", MobUtils.getMobFileName(cell), paths.get(0) - .getName()); - } - - /** - * This case tests the following mob compaction and normal compaction scenario, - * after mob compaction, the mob reference in new bulkloaded hfile will win even after it - * is compacted with some other normal hfiles. This is to make sure the mvcc is included - * after compaction for mob enabled store files. - */ - @Test - public void testGetAfterCompaction() throws Exception { - resetConf(); - conf.setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0); - String famStr = "f1"; - byte[] fam = Bytes.toBytes(famStr); - byte[] qualifier = Bytes.toBytes("q1"); - byte[] mobVal = Bytes.toBytes("01234567890"); - HTableDescriptor hdt = new HTableDescriptor(TableName.valueOf(name.getMethodName())); - hdt.addCoprocessor(CompactTwoLatestHfilesCopro.class.getName()); - HColumnDescriptor hcd = new HColumnDescriptor(fam); - hcd.setMobEnabled(true); - hcd.setMobThreshold(10); - hcd.setMaxVersions(1); - hdt.addFamily(hcd); - try { - Table table = TEST_UTIL.createTable(hdt, null); - HRegion r = TEST_UTIL.getMiniHBaseCluster().getRegions(hdt.getTableName()).get(0); - Put p = new Put(Bytes.toBytes("r1")); - p.addColumn(fam, qualifier, mobVal); - table.put(p); - // Create mob file mob1 and reference file ref1 - TEST_UTIL.flush(table.getName()); - // Make sure that it is flushed. - FileSystem fs = r.getRegionFileSystem().getFileSystem(); - Path path = r.getRegionFileSystem().getStoreDir(famStr); - waitUntilFilesShowup(fs, path, 1); - - p = new Put(Bytes.toBytes("r2")); - p.addColumn(fam, qualifier, mobVal); - table.put(p); - // Create mob file mob2 and reference file ref2 - TEST_UTIL.flush(table.getName()); - waitUntilFilesShowup(fs, path, 2); - // Do mob compaction to create mob3 and ref3 - TEST_UTIL.getAdmin().compact(hdt.getTableName(), fam, CompactType.MOB); - waitUntilFilesShowup(fs, path, 3); - - // Compact ref3 and ref2 into ref4 - TEST_UTIL.getAdmin().compact(hdt.getTableName(), fam); - waitUntilFilesShowup(fs, path, 2); - - // Sleep for some time, since TimeToLiveHFileCleaner is 0, the next run of - // clean chore is guaranteed to clean up files in archive - Thread.sleep(100); - // Run cleaner to make sure that files in archive directory are cleaned up - TEST_UTIL.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting(); - - // Get "r2" - Get get = new Get(Bytes.toBytes("r2")); - try { - Result result = table.get(get); - assertTrue(Arrays.equals(result.getValue(fam, qualifier), mobVal)); - } catch (IOException e) { - assertTrue("The MOB file doesn't exist", false); - } - } finally { - TEST_UTIL.deleteTable(hdt.getTableName()); - } - } - - private void waitUntilFilesShowup(final FileSystem fs, final Path path, final int num) - throws InterruptedException, IOException { - FileStatus[] fileList = fs.listStatus(path); - while (fileList.length != num) { - Thread.sleep(50); - fileList = fs.listStatus(path); - for (FileStatus fileStatus: fileList) { - LOG.info(Objects.toString(fileStatus)); - } - } - } - - /** - * This copro overwrites the default compaction policy. It always chooses two latest hfiles and - * compacts them into a new one. - */ - public static class CompactTwoLatestHfilesCopro implements RegionCoprocessor, RegionObserver { - - @Override - public Optional getRegionObserver() { - return Optional.of(this); - } - - @Override - public void preCompactSelection(ObserverContext c, Store store, - List candidates, CompactionLifeCycleTracker tracker) - throws IOException { - int count = candidates.size(); - if (count >= 2) { - for (int i = 0; i < count - 2; i++) { - candidates.remove(0); - } - c.bypass(); - } - } - } - - private void waitUntilMobCompactionFinished(TableName tableName) throws IOException, - InterruptedException { - long finished = EnvironmentEdgeManager.currentTime() + 60000; - CompactionState state = admin.getCompactionState(tableName, CompactType.MOB); - while (EnvironmentEdgeManager.currentTime() < finished) { - if (state == CompactionState.NONE) { - break; - } - state = admin.getCompactionState(tableName, CompactType.MOB); - Thread.sleep(10); - } - assertEquals(CompactionState.NONE, state); - } - - /** - * Gets the number of cells in the given table. - * @param table to get the scanner - * @return the number of cells - */ - private int countMobCells(final Table table) throws IOException { - Scan scan = new Scan(); - // Do not retrieve the mob data when scanning - scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE)); - ResultScanner results = table.getScanner(scan); - int count = 0; - for (Result res : results) { - count += res.size(); - } - results.close(); - return count; - } - - /** - * Gets the number of files in the mob path. - * @param isMobFile gets number of the mob files or del files - * @param familyName the family name - * @return the number of the files - */ - private int countFiles(TableName tableName, boolean isMobFile, String familyName) - throws IOException { - Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, familyName); - int count = 0; - if (fs.exists(mobDirPath)) { - FileStatus[] files = fs.listStatus(mobDirPath); - for (FileStatus file : files) { - if (isMobFile == true) { - if (!StoreFileInfo.isDelFile(file.getPath())) { - count++; - } - } else { - if (StoreFileInfo.isDelFile(file.getPath())) { - count++; - } - } - } - } - return count; - } - - private boolean verifyEncryption(TableName tableName, String familyName) throws IOException { - Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, familyName); - boolean hasFiles = false; - if (fs.exists(mobDirPath)) { - FileStatus[] files = fs.listStatus(mobDirPath); - hasFiles = files != null && files.length > 0; - Assert.assertTrue(hasFiles); - Path path = files[0].getPath(); - CacheConfig cacheConf = new CacheConfig(conf); - HStoreFile sf = new HStoreFile(TEST_UTIL.getTestFileSystem(), path, conf, cacheConf, - BloomType.NONE, true); - sf.initReader(); - HFile.Reader reader = sf.getReader().getHFileReader(); - byte[] encryptionKey = reader.getTrailer().getEncryptionKey(); - Assert.assertTrue(null != encryptionKey); - Assert.assertTrue(reader.getFileContext().getEncryptionContext().getCipher().getName() - .equals(HConstants.CIPHER_AES)); - } - return hasFiles; - } - - /** - * Gets the number of HFileLink in the mob path. - * @param familyName the family name - * @return the number of the HFileLink - */ - private int countHFileLinks(String familyName) throws IOException { - Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, familyName); - int count = 0; - if (fs.exists(mobDirPath)) { - FileStatus[] files = fs.listStatus(mobDirPath); - for (FileStatus file : files) { - if (HFileLink.isHFileLink(file.getPath())) { - count++; - } - } - } - return count; - } - - /** - * Gets the number of files. - * @param size the size of the file - * @param tableName the current table name - * @param familyName the family name - * @return the number of files large than the size - */ - private int countLargeFiles(int size, TableName tableName, String familyName) throws IOException { - Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, familyName); - int count = 0; - if (fs.exists(mobDirPath)) { - FileStatus[] files = fs.listStatus(mobDirPath); - for (FileStatus file : files) { - // ignore the del files in the mob path - if ((!StoreFileInfo.isDelFile(file.getPath())) && (file.getLen() > size)) { - count++; - } - } - } - return count; - } - - /** - * loads some data to the table. - */ - private void loadData(Admin admin, BufferedMutator table, TableName tableName, int fileNum, - int rowNumPerFile) throws IOException, InterruptedException { - if (fileNum <= 0) { - throw new IllegalArgumentException(); - } - for (int i = 0; i < fileNum * rowNumPerFile; i++) { - for (byte k0 : KEYS) { - byte[] k = new byte[] { k0 }; - byte[] key = Bytes.add(k, Bytes.toBytes(i)); - byte[] mobVal = makeDummyData(10 * (i + 1)); - Put put = new Put(key); - put.setDurability(Durability.SKIP_WAL); - put.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), mobVal); - put.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf2), mobVal); - put.addColumn(Bytes.toBytes(family2), Bytes.toBytes(qf1), mobVal); - table.mutate(put); - } - if ((i + 1) % rowNumPerFile == 0) { - table.flush(); - admin.flush(tableName); - } - } - } - - private void loadData(Admin admin, BufferedMutator table, TableName tableName, Put[] puts) - throws IOException { - table.mutate(Arrays.asList(puts)); - table.flush(); - admin.flush(tableName); - } - - private void loadDataForPartitionPolicy(Admin admin, BufferedMutator table, TableName tableName) - throws IOException { - - Put[] pArray = new Put[1000]; - - for (int i = 0; i < 1000; i ++) { - Put put0 = new Put(Bytes.toBytes("r0" + i)); - put0.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), - tsFor20151130Monday, Bytes.toBytes(mobValue0)); - pArray[i] = put0; - } - loadData(admin, bufMut, tableName, pArray); - - Put put06 = new Put(mobKey06); - put06.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), - tsFor20151128Saturday, Bytes.toBytes(mobValue0)); - - loadData(admin, bufMut, tableName, new Put[] { put06 }); - - Put put1 = new Put(mobKey1); - put1.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151201Tuesday, - Bytes.toBytes(mobValue1)); - loadData(admin, bufMut, tableName, new Put[] { put1 }); - - Put put2 = new Put(mobKey2); - put2.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151205Saturday, - Bytes.toBytes(mobValue2)); - loadData(admin, bufMut, tableName, new Put[] { put2 }); - - Put put3 = new Put(mobKey3); - put3.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151228Monday, - Bytes.toBytes(mobValue3)); - loadData(admin, bufMut, tableName, new Put[] { put3 }); - - Put put4 = new Put(mobKey4); - put4.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151231Thursday, - Bytes.toBytes(mobValue4)); - loadData(admin, bufMut, tableName, new Put[] { put4 }); - - Put put5 = new Put(mobKey5); - put5.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20160101Friday, - Bytes.toBytes(mobValue5)); - loadData(admin, bufMut, tableName, new Put[] { put5 }); - - Put put6 = new Put(mobKey6); - put6.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20160103Sunday, - Bytes.toBytes(mobValue6)); - loadData(admin, bufMut, tableName, new Put[] { put6 }); - - Put put7 = new Put(mobKey7); - put7.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20150907Monday, - Bytes.toBytes(mobValue7)); - loadData(admin, bufMut, tableName, new Put[] { put7 }); - - Put put8 = new Put(mobKey8); - put8.addColumn(Bytes.toBytes(family1), Bytes.toBytes(qf1), tsFor20151120Sunday, - Bytes.toBytes(mobValue8)); - loadData(admin, bufMut, tableName, new Put[] { put8 }); - } - - - /** - * delete the row, family and cell to create the del file - */ - private void createDelFile(Table table, TableName tableName, byte[] family, byte[] qf) - throws IOException, InterruptedException { - for (byte k0 : KEYS) { - byte[] k = new byte[] { k0 }; - // delete a family - byte[] key1 = Bytes.add(k, Bytes.toBytes(0)); - Delete delete1 = new Delete(key1); - delete1.addFamily(family); - table.delete(delete1); - // delete one row - byte[] key2 = Bytes.add(k, Bytes.toBytes(2)); - Delete delete2 = new Delete(key2); - table.delete(delete2); - // delete one cell - byte[] key3 = Bytes.add(k, Bytes.toBytes(4)); - Delete delete3 = new Delete(key3); - delete3.addColumn(family, qf); - table.delete(delete3); - } - admin.flush(tableName); - List regions = TEST_UTIL.getHBaseCluster().getRegions(tableName); - for (HRegion region : regions) { - region.waitForFlushesAndCompactions(); - region.compact(true); - } - } - /** - * Creates the dummy data with a specific size. - * @param size the size of value - * @return the dummy data - */ - private byte[] makeDummyData(int size) { - byte[] dummyData = new byte[size]; - new Random().nextBytes(dummyData); - return dummyData; - } - - /** - * Gets the split keys - */ - private byte[][] getSplitKeys() { - byte[][] splitKeys = new byte[KEYS.length - 1][]; - for (int i = 0; i < splitKeys.length; ++i) { - splitKeys[i] = new byte[] { KEYS[i + 1] }; - } - return splitKeys; - } - - private static ExecutorService createThreadPool(Configuration conf) { - int maxThreads = 10; - long keepAliveTime = 60; - final SynchronousQueue 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); - } - } - }); - pool.allowCoreThreadTimeOut(true); - return pool; - } - - private void assertRefFileNameEqual(String familyName) throws IOException { - Scan scan = new Scan(); - scan.addFamily(Bytes.toBytes(familyName)); - // Do not retrieve the mob data when scanning - scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE)); - ResultScanner results = table.getScanner(scan); - Path mobFamilyPath = MobUtils.getMobFamilyPath(TEST_UTIL.getConfiguration(), - tableName, familyName); - List actualFilePaths = new ArrayList<>(); - List expectFilePaths = new ArrayList<>(); - for (Result res : results) { - for (Cell cell : res.listCells()) { - byte[] referenceValue = CellUtil.cloneValue(cell); - String fileName = Bytes.toString(referenceValue, Bytes.SIZEOF_INT, - referenceValue.length - Bytes.SIZEOF_INT); - Path targetPath = new Path(mobFamilyPath, fileName); - if(!actualFilePaths.contains(targetPath)) { - actualFilePaths.add(targetPath); - } - } - } - results.close(); - if (fs.exists(mobFamilyPath)) { - FileStatus[] files = fs.listStatus(mobFamilyPath); - for (FileStatus file : files) { - if (!StoreFileInfo.isDelFile(file.getPath())) { - expectFilePaths.add(file.getPath()); - } - } - } - Collections.sort(actualFilePaths); - Collections.sort(expectFilePaths); - assertEquals(expectFilePaths, actualFilePaths); - } - - /** - * Resets the configuration. - */ - private void resetConf() { - conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD, - MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD); - conf.setInt(MobConstants.MOB_COMPACTION_BATCH_SIZE, - MobConstants.DEFAULT_MOB_COMPACTION_BATCH_SIZE); - } - - /** - * Verify mob partition policy compaction values. - */ - private void verifyPolicyValues() throws Exception { - Get get = new Get(mobKey01); - Result result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue0))); - - get = new Get(mobKey02); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue0))); - - get = new Get(mobKey03); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue0))); - - get = new Get(mobKey04); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue0))); - - get = new Get(mobKey05); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue0))); - - get = new Get(mobKey06); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue0))); - - get = new Get(mobKey1); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue1))); - - get = new Get(mobKey2); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue2))); - - get = new Get(mobKey3); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue3))); - - get = new Get(mobKey4); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue4))); - - get = new Get(mobKey5); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue5))); - - get = new Get(mobKey6); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue6))); - - get = new Get(mobKey7); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue7))); - - get = new Get(mobKey8); - result = table.get(get); - assertTrue(Arrays.equals(result.getValue(Bytes.toBytes(family1), Bytes.toBytes(qf1)), - Bytes.toBytes(mobValue8))); - } - - private void commonPolicyTestLogic (final String tableNameAsString, - final MobCompactPartitionPolicy pType, final boolean majorCompact, - final int expectedFileNumbers, final String[] expectedFileNames, - final boolean setupAndLoadData - ) throws Exception { - if (setupAndLoadData) { - setUpForPolicyTest(tableNameAsString, pType); - - loadDataForPartitionPolicy(admin, bufMut, tableName); - } else { - alterForPolicyTest(pType); - } - - if (majorCompact) { - admin.majorCompact(tableName, cfdb1.build().getName(), CompactType.MOB); - } else { - admin.compact(tableName, cfdb1.build().getName(), CompactType.MOB); - } - - waitUntilMobCompactionFinished(tableName); - - // Run cleaner to make sure that files in archive directory are cleaned up - TEST_UTIL.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting(); - - //check the number of files - Path mobDirPath = MobUtils.getMobFamilyPath(conf, tableName, family1); - FileStatus[] fileList = fs.listStatus(mobDirPath); - - assertTrue(fileList.length == expectedFileNumbers); - - // the file names are expected - ArrayList fileNames = new ArrayList<>(expectedFileNumbers); - for (FileStatus file : fileList) { - fileNames.add(MobFileName.getDateFromName(file.getPath().getName())); - } - int index = 0; - for (String fileName : expectedFileNames) { - index = fileNames.indexOf(fileName); - assertTrue(index >= 0); - fileNames.remove(index); - } - - // Check daily mob files are removed from the mobdir, and only weekly mob files are there. - // Also check that there is no data loss. - - verifyPolicyValues(); - } - } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactionRequest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactionRequest.java deleted file mode 100644 index 27801533dc5..00000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactionRequest.java +++ /dev/null @@ -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)); - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java deleted file mode 100644 index 73ee96561e0..00000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java +++ /dev/null @@ -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 mobFiles = new ArrayList<>(); - private List delFiles = new ArrayList<>(); - private List 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 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 compact(List 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 compact(List 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 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 expected) throws IOException { - PartitionedMobCompactor compactor = new PartitionedMobCompactor(conf, fs, - TableName.valueOf(tableName), cfdb.build(), pool) { - @Override - public List compact(List 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 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 performCompaction(PartitionedMobCompactionRequest request) - throws IOException { - List delFilePaths = new ArrayList<>(); - for (CompactionDelPartition delPartition: request.getDelPartitions()) { - for (Path p : delPartition.listDelFiles()) { - delFilePaths.add(p); - } - } - List 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 expected, - Collection partitions) { - List 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 delPartitions) { - Map 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 paths) throws IOException { - List 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 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 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 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); - } - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoadReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoadReplication.java index 33ab4e3bc34..9c58c7af965 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoadReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoadReplication.java @@ -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.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; + import java.io.File; import java.io.FileOutputStream; import java.io.IOException; -import java.util.Arrays; -import java.util.Date; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Random; -import java.util.UUID; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; 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.TableName; 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.Connection; 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.RegionCoprocessorEnvironment; 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.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.PartitionedMobCompactor; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.TestReplicationBase; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.ReplicationTests; import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool; 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.hdfs.MiniDFSCluster; import org.junit.After; @@ -257,30 +245,6 @@ public class TestBulkLoadReplication extends TestReplicationBase { 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, HBaseTestingUtility utility, Table...tables) throws Exception { @@ -348,36 +312,6 @@ public class TestBulkLoadReplication extends TestReplicationBase { 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 { String clusterName; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java index bc44daa5cff..833e78ba3b0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java @@ -25,6 +25,8 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Optional; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CompatibilityFactory; 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.TableDescriptorBuilder; 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.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; @@ -494,7 +500,20 @@ public class TestRegionServerMetrics { setMobThreshold(region, cf, 100); // metrics are reset by the region initialization region.initialize(); - region.compact(true); + // This is how we MOB compact region + List stores = region.getStores(); + for (HStore store: stores) { + // Force major compaction + store.triggerMajorCompaction(); + Optional 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(); assertCounter("cellsCountCompactedFromMob", numHfiles); assertCounter("cellsCountCompactedToMob", 0); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java index b6227fc428c..8c8379080cc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java @@ -411,7 +411,7 @@ public class BaseTestHBaseFsck { String startKey = mobFileName.getStartKey(); String date = mobFileName.getDate(); return MobFileName.create(startKey, date, - TEST_UTIL.getRandomUUID().toString().replaceAll("-", "")) + TEST_UTIL.getRandomUUID().toString().replaceAll("-", ""), "abcdef") .getFileName(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java index 3a3becdeec0..9870b73f5f9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java @@ -98,6 +98,7 @@ public class TestHBaseFsckMOB extends BaseTestHBaseFsck { /** * This creates a table and then corrupts a mob file. Hbck should quarantine the file. */ + @SuppressWarnings("deprecation") @Test public void testQuarantineCorruptMobFile() throws Exception { TableName table = TableName.valueOf(name.getMethodName());