From 9d20ccaf1fb0b21f6c8be6cb93228c3260681c92 Mon Sep 17 00:00:00 2001 From: Vladimir Rodionov Date: Mon, 9 Dec 2019 12:00:37 -0800 Subject: [PATCH 01/17] HBASE-22749: Distributed MOB compactions --- .../hbase/IntegrationTestMobCompaction.java | 413 ++++++ .../hbase/io/hfile/HFileWriterImpl.java | 4 + .../master/ExpiredMobFileCleanerChore.java | 87 -- .../apache/hadoop/hbase/master/HMaster.java | 39 +- .../master/MasterMobCompactionThread.java | 181 --- .../hbase/master/MasterRpcServices.java | 90 +- .../hbase/master/MobCompactionChore.java | 96 -- .../hbase/master/MobFileCleanerChore.java | 289 ++++ .../hbase/master/MobFileCompactionChore.java | 222 +++ .../hbase/mob/DefaultMobStoreCompactor.java | 459 +++++-- .../hbase/mob/DefaultMobStoreFlusher.java | 27 +- .../hbase/mob/ExpiredMobFileCleaner.java | 5 +- .../apache/hadoop/hbase/mob/MobConstants.java | 79 +- .../apache/hadoop/hbase/mob/MobFileName.java | 39 +- .../hadoop/hbase/mob/MobStoreEngine.java | 12 +- .../org/apache/hadoop/hbase/mob/MobUtils.java | 578 ++------ .../mob/compactions/MobCompactionRequest.java | 64 - .../hbase/mob/compactions/MobCompactor.java | 90 -- .../PartitionedMobCompactionRequest.java | 333 ----- .../compactions/PartitionedMobCompactor.java | 949 ------------- .../hadoop/hbase/regionserver/HMobStore.java | 42 +- .../hadoop/hbase/regionserver/HStore.java | 2 +- .../hadoop/hbase/regionserver/HStoreFile.java | 6 + .../hbase/regionserver/StoreFileInfo.java | 114 +- .../hbase/regionserver/StoreFileWriter.java | 29 + .../regionserver/compactions/Compactor.java | 2 +- .../org/apache/hadoop/hbase/util/FSUtils.java | 2 +- .../hbase/mob/FaultyMobStoreCompactor.java | 373 +++++ .../hadoop/hbase/mob/MobStressTool.java | 79 ++ .../hbase/mob/TesMobFileCleanerChore.java | 236 ++++ .../hadoop/hbase/mob/TestMobCompaction.java | 375 +++++ .../hbase/mob/TestMobCompactionBase.java | 242 ++++ .../hbase/mob/TestMobCompactionOptMode.java | 88 ++ .../TestMobCompactionOptRegionBatchMode.java | 99 ++ .../mob/TestMobCompactionRegularMode.java | 80 ++ ...stMobCompactionRegularRegionBatchMode.java | 96 ++ .../hadoop/hbase/mob/TestMobFileName.java | 19 +- .../mob/compactions/TestMobCompactor.java | 1214 ----------------- .../TestPartitionedMobCompactionRequest.java | 65 - .../TestPartitionedMobCompactor.java | 961 ------------- .../regionserver/TestBulkLoadReplication.java | 73 +- .../regionserver/TestMobStoreCompaction.java | 58 - .../hadoop/hbase/util/BaseTestHBaseFsck.java | 2 +- .../hadoop/hbase/util/TestHBaseFsckMOB.java | 1 + 44 files changed, 3389 insertions(+), 4925 deletions(-) create mode 100644 hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/ExpiredMobFileCleanerChore.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMobCompactionThread.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobCompactionChore.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactionRequest.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/MobCompactor.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactionRequest.java delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TesMobFileCleanerChore.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactionRequest.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java 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 000000000000..3373ce98f6bd --- /dev/null +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java @@ -0,0 +1,413 @@ +/** + * 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.MobFileCleanerChore; +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.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.apache.hbase.thirdparty.com.google.common.base.MoreObjects; +import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; +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; + + +/** + * 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 + */ +@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 + } + + 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("injected.fault.probability", failureProb); + conf.set(MobStoreEngine.DEFAULT_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); + } + } + } + + /** + * + * @param args argument list + */ + 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 604ac1f8bc65..464ea49bca5f 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 @@ protected final void writeFileInfo(FixedFileTrailer trailer, DataOutputStream ou 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 d37a80a1c3db..000000000000 --- 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 6864ce699203..d578cb46f8bc 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 @@ -396,9 +396,8 @@ public void run() { 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 +1264,18 @@ public void updateConfigurationForQuotasObserver(Configuration conf) { } private void initMobCleaner() { - this.expiredMobFileCleanerChore = new ExpiredMobFileCleanerChore(this); - getChoreService().scheduleChore(expiredMobFileCleanerChore); + this.mobFileCleanerChore = new MobFileCleanerChore(this); + getChoreService().scheduleChore(mobFileCleanerChore); 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); + MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD); + + if (mobCompactionPeriod > 0) { + this.mobFileCompactionChore = new MobFileCompactionChore(this); + getChoreService().scheduleChore(mobFileCompactionChore); + } else { + LOG.info("The period is " + mobCompactionPeriod + " seconds, MobCompactionChore is disabled"); + } } /** @@ -1475,9 +1478,7 @@ protected void stopServiceThreads() { } } stopChores(); - if (this.mobCompactThread != null) { - this.mobCompactThread.close(); - } + super.stopServiceThreads(); if (cleanerPool != null) { cleanerPool.shutdownNow(); @@ -1600,8 +1601,8 @@ private void stopProcedureExecutor() { 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); @@ -3430,16 +3431,6 @@ public void reportMobCompactionEnd(TableName tableName) throws IOException { } } - /** - * 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 0779eeafe8b0..000000000000 --- 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 5fb5f151c861..6d97f5d6a45c 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.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.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,23 +104,6 @@ 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; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -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; -import org.slf4j.LoggerFactory; - -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; @@ -345,6 +328,21 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription; +import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; +import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +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.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.yetus.audience.InterfaceAudience; +import org.apache.zookeeper.KeeperException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Implements the master RPC services. @@ -1751,10 +1749,13 @@ public CompactRegionResponse compactRegion(final RpcController controller, 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 + LOG.warn("CompactType.MOB is not supported yet, will run regular compaction."); + return super.compactRegion(controller, request); } else { return super.compactRegion(controller, request); } @@ -1817,57 +1818,6 @@ public GetRegionInfoResponse getRegionInfo(final RpcController controller, 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 6c5d677a86e5..000000000000 --- 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/master/MobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java new file mode 100644 index 000000000000..0f1d0e1f156e --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java @@ -0,0 +1,289 @@ +/** + * 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.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.mob.ExpiredMobFileCleaner; +import org.apache.hadoop.hbase.mob.MobConstants; +import org.apache.hadoop.hbase.mob.MobUtils; +import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.hbase.regionserver.HStoreFile; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.util.HFileArchiveUtil; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The Class ExpiredMobFileCleanerChore 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; + private long minAgeToArchive; + + public MobFileCleanerChore(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()); + checkObsoleteConfigurations(); + } + + private void checkObsoleteConfigurations() { + Configuration conf = master.getConfiguration(); + if (conf.get("hbase.master.mob.ttl.cleaner.period") != null) { + LOG.warn("'hbase.master.mob.ttl.cleaner.period' is obsolete and not used anymore."); + } + 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 + @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) { + cleaner.cleanExpiredMobFiles(htd.getTableName().getNameAsString(), hcd); + } + } + // Now clean obsolete files for a table + LOG.info("Cleaning obsolete MOB files ..."); + cleanupObsoleteMobFiles(master.getConfiguration(), htd.getTableName()); + LOG.info("Cleaning obsolete MOB files finished"); + } + } catch (Exception e) { + LOG.error("Fail to clean the expired mob files", e); + } + } + /** + * Performs housekeeping file cleaning (called by MOB Cleaner chore) + * @param conf configuration + * @param table table name + * @throws IOException + */ + public void cleanupObsoleteMobFiles(Configuration conf, TableName table) + throws IOException { + + this.minAgeToArchive = conf.getLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, + MobConstants.DEFAULT_MIN_AGE_TO_ARCHIVE); + 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; + } + // We check only those MOB files, which creation time is less + // than maxTimeToArchive. 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; + LOG.info("Only MOB files whose creation time less than {} will be archived", + maxCreationTimeToArchive); + 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) { + + 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); + } + } + try { + for (Path pp : storeFiles) { + LOG.debug("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.info("Skipping file without MOB references (bulkloaded file):{}", pp); + continue; + } + if (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(","); + LOG.debug("Found: {} mob refs: ", mobs.length, Arrays.toString(mobs)); + regionMobs.addAll(Arrays.asList(mobs)); + } + } + } catch (FileNotFoundException e) { + LOG.warn("Starting MOB cleaning cycle from the beginning due to error:",e); + continue; + } + succeed = true; + } + // Add MOB refs 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: {}, this may result in high memory pressure.", + allActiveMobFileName.size()); + } + LOG.debug("Found: {} active mob refs", allActiveMobFileName.size()); + allActiveMobFileName.stream().forEach(LOG::debug); + + // 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) { + /* DEBUG */ LOG.info( + " Archiving MOB file{} creation time=" + (fs.getFileStatus(p).getModificationTime()), p); + toArchive.add(p); + } else { + LOG.info("Skipping fresh file: {}", p); + } + } else { + LOG.info("Keepeing active MOB file: {}", p); + } + } + LOG.info(" MOB Cleaner found {} files for family={}", toArchive.size() , family); + removeMobFiles(conf, table, family.getBytes(), toArchive); + LOG.info(" MOB Cleaner archived {} files", toArchive.size()); + } + } + } + + + /** + * 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 + */ + public void removeMobFiles(Configuration conf, TableName tableName, byte[] family, + List storeFiles) throws IOException { + + if (storeFiles.size() == 0) { + // nothing to remove + LOG.debug("Skipping archiving old MOB file: collection is empty"); + return; + } + Path mobTableDir = FSUtils.getTableDir(MobUtils.getMobHome(conf), tableName); + FileSystem fs = storeFiles.get(0).getFileSystem(conf); + Path storeArchiveDir = HFileArchiveUtil.getStoreArchivePath(conf, MobUtils.getMobRegionInfo(tableName), + mobTableDir, family); + + for (Path p : storeFiles) { + Path archiveFilePath = new Path(storeArchiveDir, p.getName()); + if (fs.exists(archiveFilePath)) { + LOG.warn("MOB Cleaner skip archiving: {} because it has been archived already", p); + continue; + } + LOG.info("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/master/MobFileCompactionChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java new file mode 100644 index 000000000000..402718ea68de --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java @@ -0,0 +1,222 @@ +/** + * 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.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.ConnectionFactory; +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.mob.MobConstants; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * 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 Configuration conf; + 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.conf = master.getConfiguration(); + 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.conf = conf; + this.regionBatchSize = batchSize; + } + + @Override + protected void chore() { + + boolean reported = false; + + try (Connection conn = ConnectionFactory.createConnection(conf); + 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.debug("Skipping MOB compaction on table {} because it is not ENABLED", + htd.getTableName()); + continue; + } else { + LOG.debug("Starting MOB compaction on table {}", htd.getTableName()); + } + for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) { + try { + if (hcd.isMobEnabled()) { + if (!reported) { + master.reportMobCompactionStart(htd.getTableName()); + reported = true; + } + LOG.info(" Major compacting {} cf={}", htd.getTableName(), hcd.getNameAsString()); + if (regionBatchSize == MobConstants.DEFAULT_MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE) { + LOG.debug("Batch compaction is disabled, {}=0", "hbase.mob.compaction.batch.size"); + admin.majorCompact(htd.getTableName(), hcd.getName()); + } else { + LOG.debug("Performing compaction in batches, {}={}", + "hbase.mob.compaction.batch.size", regionBatchSize); + performMajorCompactionInBatches(admin, htd, hcd); + } + } else { + LOG.debug("Skipping column family {} because it is not MOB-enabled", + hcd.getNameAsString()); + } + } catch (IOException e) { + LOG.error("Failed to compact table="+ htd.getTableName() +" cf="+ hcd.getNameAsString(), + e); + } catch (InterruptedException ee) { + Thread.currentThread().interrupt(); + master.reportMobCompactionEnd(htd.getTableName()); + LOG.warn("Failed to compact table="+ htd.getTableName() +" cf="+ 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("Performing compaction in non-batched mode, regions={}, batch size={}", + 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(); + 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(); + 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 compaction: table={} region={}, compacted regions={}", + htd.getTableName(),ri.getRegionNameAsString(), totalCompacted); + compacted.add(ri); + } + } catch (IOException e) { + LOG.warn("Could not get compaction state for region {}", ri.getEncodedName()); + } + } + // 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); + startCompaction(admin, htd.getTableName(),region, hcd.getName()); + toCompact.add(region); + } + } + compacted.clear(); + LOG.debug("Wait for 10 sec, toCompact size={} regions left={} compacted so far={}", + toCompact.size(), regions.size(), totalCompacted); + Thread.sleep(10000); + } + LOG.info("Finished major compacting {}. 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={} region={}", table, + region.getRegionNameAsString()); + admin.majorCompactRegion(region.getRegionName()); + // 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 compaction on table: {} region: {}. Aborted.", + waitTime, table.getNameAsString(), region.getRegionNameAsString()); + break; + } + } + } +} 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 e73a7d2226d7..d69272384600 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,27 @@ */ 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.TableName; import org.apache.hadoop.hbase.regionserver.CellSink; import org.apache.hadoop.hbase.regionserver.HMobStore; import org.apache.hadoop.hbase.regionserver.HStore; @@ -48,6 +57,7 @@ import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -59,15 +69,52 @@ 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; + + /* + * 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; + } + }; + + /* + * 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 @@ -103,17 +150,59 @@ public DefaultMobStoreCompactor(Configuration conf, HStore store) { mobSizeThreshold = store.getColumnFamilyDescriptor().getMobThreshold(); } + @Override - public List compact(CompactionRequestImpl request, ThroughputController throughputController, - User user) throws IOException { + public List compact(CompactionRequestImpl request, + ThroughputController throughputController, User user) throws IOException { + LOG.info("Mob compaction: major=" + request.isMajor() + " isAll=" + request.isAllFiles() + + " priority=" + request.getPriority()); + if (request.getPriority() == HStore.PRIORITY_USER) { + userRequest.set(Boolean.TRUE); + } else { + userRequest.set(Boolean.FALSE); + } + LOG.info("Mob compaction files: " + request.getFiles()); + // Check if I/O optimized MOB compaction + if (conf.get(MobConstants.MOB_COMPACTION_TYPE_KEY, MobConstants.DEFAULT_MOB_COMPACTION_TYPE) + .equals(MobConstants.IO_OPTIMIZED_MOB_COMPACTION_TYPE)) { + if (request.isMajor() && request.getPriority() == HStore.PRIORITY_USER) { + Path mobDir = MobUtils.getMobFamilyPath(conf, store.getTableName(), + store.getColumnFamilyName()); + List mobFiles = MobUtils.getReferencedMobFiles(request.getFiles(), mobDir); + if (mobFiles.size() > 0) { + calculateMobLengthMap(mobFiles); + } + LOG.info("I/O optimized MOB compaction. Total referenced MOB files: {}", mobFiles.size()); + } + } return compact(request, scannerFactory, writerFactory, throughputController, user); } + private void calculateMobLengthMap(List mobFiles) throws IOException { + FileSystem fs = mobFiles.get(0).getFileSystem(this.conf); + HashMap map = mobLengthMap.get(); + map.clear(); + for (Path p: mobFiles) { + FileStatus st = fs.getFileStatus(p); + long size = st.getLen(); + LOG.info("Ref 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. + * 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. @@ -135,23 +224,6 @@ public List compact(CompactionRequestImpl request, ThroughputController th * Otherwise, directly write this cell into the store file. *
  2. *
- * 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 +241,25 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel 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 ioOptimizedMode = conf.get(MobConstants.MOB_COMPACTION_TYPE_KEY, + MobConstants.DEFAULT_MOB_COMPACTION_TYPE) + .equals(MobConstants.IO_OPTIMIZED_MOB_COMPACTION_TYPE); + + boolean discardMobMiss = + conf.getBoolean(MobConstants.MOB_UNSAFE_DISCARD_MISS_KEY, + MobConstants.DEFAULT_MOB_DISCARD_MISS); + + long maxMobFileSize = conf.getLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, + MobConstants.DEFAULT_MOB_COMPACTION_MAX_FILE_SIZE); + LOG.info("Compact MOB={} optimized={} maximum MOB file size={} major={}", compactMOBs, + ioOptimizedMode, maxMobFileSize, major); + + FileSystem fs = FileSystem.get(conf); + // Since scanner.next() can return 'false' but still be delivering data, // we have to use a do/while loop. List cells = new ArrayList<>(); @@ -183,91 +274,163 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel 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); + Path pp = new Path(new Path(fs.getUri()), new Path(path, fName)); + + // Added to support migration + try { + mobCell = mobStore.resolve(c, true, false).getCell(); + } catch (FileNotFoundException fnfe) { + if (discardMobMiss) { + LOG.debug("Missing MOB cell: file={} not found cell={}", pp, c); + continue; + } else { + throw fnfe; + } + } + + if (discardMobMiss && mobCell.getValueLength() == 0) { + LOG.error("Missing MOB cell value: file=" + pp + " cell=" + mobCell); + continue; + } else if (mobCell.getValueLength() == 0) { + //TODO: what to do here? This is data corruption? + LOG.warn("Found 0 length MOB cell in a file={} cell={}", pp, mobCell); + } + + 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, abort compaction + String msg = + String.format("Found unreferenced MOB file during compaction %s, aborting.", + fName); + LOG.error(msg); + 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 = getLength(mobFileWriter); + + if (len > maxMobFileSize) { + LOG.debug("Closing output MOB File, length={} file={}", + len, Bytes.toString(fileName)); + 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 = getLength(mobFileWriter); + 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); - } - } - } + // 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 { - LOG.warn("The value format of the KeyValue " + c - + " is wrong, its length is less than " + Bytes.SIZEOF_INT); + // TODO ???? + LOG.error("Corrupted MOB reference: " + c); writer.append(c); } } 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 +438,22 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel 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 = getLength(mobFileWriter); + 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 +474,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel } } if (kvs != null && bytesWrittenProgressForShippedCall > shippedCallSizeLimit) { - ((ShipperListener)writer).beforeShipped(); + ((ShipperListener) writer).beforeShipped(); kvs.shipped(); bytesWrittenProgressForShippedCall = 0; } @@ -326,6 +498,9 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel progress.cancel(); throw new InterruptedIOException( "Interrupted while control throughput of compacting " + compactionName); + } 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 @@ -333,40 +508,90 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel ((ShipperListener) writer).beforeShipped(); throughputController.finish(compactionName); if (!finished && mobFileWriter != null) { + // Remove all MOB references because compaction failed + mobRefSet.get().clear(); + // Abort writer + LOG.debug("Aborting writer for {} because of a compaction failure", + mobFileWriter.getPath()); 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); - } + + // Commit last MOB writer + commitOrAbortMobWriter(mobFileWriter, fd.maxSeqId, mobCells, major); + + mobStore.updateCellsCountCompactedFromMob(cellsCountCompactedFromMob); + mobStore.updateCellsCountCompactedToMob(cellsCountCompactedToMob); + mobStore.updateCellsSizeCompactedFromMob(cellsSizeCompactedFromMob); + mobStore.updateCellsSizeCompactedToMob(cellsSizeCompactedToMob); + progress.complete(); + return true; + } + + private long getLength(StoreFileWriter mobFileWriter) throws IOException { + return mobFileWriter.getPos(); + } + + + private StoreFileWriter newMobWriter(FileDetails fd/*, boolean compactMOBs*/) + throws IOException { + try { + StoreFileWriter mobFileWriter = mobStore.createWriterInTmp(new Date(fd.latestPutTs), + fd.maxKeyCount, compactionCompression, store.getRegionInfo().getStartKey(), true); + LOG.debug("New MOB writer created={}", mobFileWriter.getPath().getName()); + // Add reference we get for compact MOB + mobRefSet.get().add(mobFileWriter.getPath().getName()); + return mobFileWriter; + } catch (IOException e) { + // Bailing out + LOG.error("Failed to create mob writer, ", e); + throw 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 + LOG.debug("Commit or abort size={} mobCells={} major={} file={}", + mobFileWriter.getPos(), mobCells, major, mobFileWriter.getPath().getName()); + Path path = MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName()); if (mobFileWriter != null) { if (mobCells > 0) { // If the mob file is not empty, commit it. - mobFileWriter.appendMetadata(fd.maxSeqId, major, mobCells); + 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", + mobFileWriter.getPath()); + // Remove MOB file from reference set + mobRefSet.get().remove(mobFileWriter.getPath().getName()); abortWriter(mobFileWriter); } } - mobStore.updateCellsCountCompactedFromMob(cellsCountCompactedFromMob); - mobStore.updateCellsCountCompactedToMob(cellsCountCompactedToMob); - mobStore.updateCellsSizeCompactedFromMob(cellsSizeCompactedFromMob); - mobStore.updateCellsSizeCompactedToMob(cellsSizeCompactedToMob); - progress.complete(); - return true; } + + protected static String createKey(TableName tableName, String encodedName, + String columnFamilyName) { + return tableName.getNameAsString()+ "_" + encodedName + "_"+ columnFamilyName; + } + + @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(); + 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 3de7992cb121..70591fb14da9 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.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 @@ protected void performMobFlush(MemStoreSnapshot snapshot, long cacheFlushId, 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 @@ protected void performMobFlush(MemStoreSnapshot snapshot, long cacheFlushId, 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 @@ protected void performMobFlush(MemStoreSnapshot snapshot, long cacheFlushId, status.setStatus("Flushing mob file " + store + ": closing flushed file"); mobFileWriter.close(); mobStore.commitFile(mobFileWriter.getPath(), targetPath); + LOG.debug("Flush store file: {}", writer.getPath()); 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,16 @@ protected void performMobFlush(MemStoreSnapshot snapshot, long cacheFlushId, } } } + + 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(); + } } 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 42e78bbd007d..f82cf1a4cb14 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 void cleanExpiredMobFiles(String tableName, ColumnFamilyDescriptor family 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 4afd75b9c680..fa0944be21db 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 @@ -43,7 +43,7 @@ public final class MobConstants { 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 int DEFAULT_MOB_CLEANER_PERIOD = 24 * 60 * 60; // one day public static final String MOB_CACHE_EVICT_PERIOD = "hbase.mob.cache.evict.period"; @@ -55,33 +55,22 @@ public final class MobConstants { public static final long DEFAULT_MOB_CACHE_EVICT_PERIOD = 3600L; public final static String TEMP_DIR_NAME = ".tmp"; - 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 = ""; - /** - * 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. - */ - public static final String 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. + * 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_DELFILE_MAX_COUNT = "hbase.mob.delfile.max.count"; - public static final int DEFAULT_MOB_DELFILE_MAX_COUNT = 3; + + public static final String MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE = + "hbase.mob.major.compaction.region.batch.size"; + /** - * 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 int DEFAULT_MOB_COMPACTION_BATCH_SIZE = 100; + + 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. @@ -91,12 +80,46 @@ public final class MobConstants { 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", "io_optimized" + * "full" - run full major compaction (during migration) + * "io_optimized" - optimized version for use case with infrequent updates/deletes + */ + public final static String MOB_COMPACTION_TYPE_KEY = "hbase.mob.compaction.type"; + + public final static String DEFAULT_MOB_COMPACTION_TYPE = "full"; + + public final static String IO_OPTIMIZED_MOB_COMPACTION_TYPE = "optimized"; + + public final static String FULL_MOB_COMPACTION_TYPE = "full"; + + + /** + * 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; + /** - * The max number of threads used in MobCompactor. + * Use this configuration option with caution, only during upgrade procedure + * to handle missing MOB cells during compaction. */ - public static final String MOB_COMPACTION_THREADS_MAX = - "hbase.mob.compaction.threads.max"; - public static final int DEFAULT_MOB_COMPACTION_THREADS_MAX = 1; + public static final 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 + private MobConstants() { } 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 3a29274285b6..64cfa4dfcafa 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 @@ /** * 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. @@ -45,10 +45,13 @@ public final class MobFileName { 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 @@ private MobFileName(byte[] startKey, String date, String uuid) { * 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 @@ private MobFileName(String startKey, String date, String uuid) { * @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 static MobFileName create(byte[] startKey, String date, String uuid) { * @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); } /** @@ -119,7 +128,8 @@ public static MobFileName create(String fileName) { 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); } /** @@ -148,6 +158,13 @@ public String getStartKey() { 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 ee1fe7db14fc..6adb4b58ded5 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 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 @@ */ @InterfaceAudience.Private public class MobStoreEngine extends DefaultStoreEngine { - + public final static String DEFAULT_MOB_COMPACTOR_CLASS_KEY = "hbase.mob.default.compactor"; @Override protected void createStoreFlusher(Configuration conf, HStore store) throws IOException { // When using MOB, we use DefaultMobStoreFlusher always @@ -43,6 +44,13 @@ protected void createStoreFlusher(Configuration conf, HStore store) throws IOExc */ @Override protected void createCompactor(Configuration conf, HStore store) throws IOException { - compactor = new DefaultMobStoreCompactor(conf, store); + String className = + conf.get(DEFAULT_MOB_COMPACTOR_CLASS_KEY, DefaultMobStoreCompactor.class.getName()); + try { + compactor = ReflectionUtils.instantiateWithCustomCtor(className, + new Class[] { Configuration.class, HStore.class }, new Object[] { conf, store }); + } catch (Exception 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 1c00e25be3c2..f20435d4d8ea 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.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.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.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.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 @@ 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); - } + @Override + protected SimpleDateFormat initialValue() { + return new SimpleDateFormat("yyyyMMdd"); + } + }; /** * Private constructor to keep this class from being instantiated. @@ -127,45 +110,6 @@ public static Date parseDate(String dateString) throws ParseException { 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 static boolean hasMobReferenceTag(List tags) { } /** - * 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 static boolean isRawMobScan(Scan scan) { } /** - * 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 static boolean isRefOnlyScan(Scan scan) { } /** - * 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 static boolean isCacheMobBlocks(Scan scan) { /** * 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 static void cleanExpiredMobFiles(FileSystem fs, Configuration conf, Table 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 static void cleanExpiredMobFiles(FileSystem fs, Configuration conf, Table } 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 static Path getMobHome(Configuration conf) { } /** - * 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 static Path getQualifiedMobRootDir(Configuration conf) throws IOException } /** - * 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 static Path getMobTableDir(Path rootDir, TableName tableName) { } /** - * 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 static Path getMobRegionPath(Configuration conf, TableName tableName) { } /** - * 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 static Path getMobRegionPath(Path rootDir, TableName tableName) { } /** - * 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 static Path getMobFamilyPath(Configuration conf, TableName tableName, Str } /** - * 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 static Path getMobFamilyPath(Path regionPath, String familyName) { } /** - * 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 static RegionInfo getMobRegionInfo(TableName tableName) { * @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 static boolean isMobRegionName(TableName tableName, byte[] regionName) { 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,18 @@ public static Path getCompactionWorkingPath(Path root, String jobName) { 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 +477,14 @@ public static Cell createMobRefCell(Cell cell, byte[] fileName, byte[] refCellTa 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 +502,10 @@ public static StoreFileWriter createDelFileWriter(Configuration conf, FileSystem 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 +526,9 @@ public static StoreFileWriter createWriter(Configuration conf, FileSystem fs, * @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 +541,10 @@ public static StoreFileWriter createWriter(Configuration conf, FileSystem fs, 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 +554,9 @@ public static StoreFileWriter createWriter(Configuration conf, FileSystem fs, } /** - * 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 +565,9 @@ public static boolean hasValidMobRefCellValue(Cell cell) { } /** - * 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 +576,15 @@ public static int getMobValueLength(Cell cell) { } /** - * 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); } /** @@ -898,6 +602,23 @@ public static boolean hasMobColumns(TableDescriptor htd) { return false; } + /** + * 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. @@ -906,7 +627,7 @@ public static boolean hasMobColumns(TableDescriptor htd) { */ 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 +635,6 @@ public static boolean isReadEmptyValueOnMobCellMiss(Scan scan) { } } - /** - * 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 +667,41 @@ public static boolean isMobFileExpired(ColumnFamilyDescriptor column, long curre } /** - * 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 + * Gets encoded region name from a MOB file name + * @param mobFileName MOB file name + * @return encoded region name or null */ - 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 String getEncodedRegionName(String mobFileName) { + int index = mobFileName.lastIndexOf(MobFileName.REGION_SEP); + if (index < 0) { + return null; } + return mobFileName.substring(index+1); + } - 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; - } + /** + * 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 + */ - /* 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); + public static List getReferencedMobFiles(Collection storeFiles, Path mobDir) { - // 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 5025ed261229..000000000000 --- 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 c5d93ea6e064..000000000000 --- 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 ab917a2d9228..000000000000 --- 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 669bffcb4cbd..000000000000 --- 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 b8ea9609e2bc..8f7b69068aa3 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 StoreFileWriter createWriterInTmp(Date date, long maxKeyCount, 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 StoreFileWriter createWriterInTmp(String date, Path basePath, long maxKey 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,7 +243,7 @@ public void commitFile(final Path sourceFile, Path targetPath) throws IOExceptio } Path dstPath = new Path(targetPath, sourceFile.getName()); validateMobFile(sourceFile); - String msg = "Renaming flushed file from " + sourceFile + " to " + dstPath; + String msg = " FLUSH Renaming flushed file from " + sourceFile + " to " + dstPath; LOG.info(msg); Path parent = dstPath.getParent(); if (!region.getFilesystem().exists(parent)) { @@ -309,6 +287,20 @@ public MobCell resolve(Cell reference, boolean cacheBlocks) throws IOException { 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 +512,6 @@ public long getMobScanCellsSize() { public byte[] getRefCellTags() { return this.refCellTags; } + + } 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 d77a4725951f..affd43fc14b2 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 void cancelRequestedCompaction(CompactionContext compaction) { 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 fa152c5061ae..0f227be472fc 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 15ed359a8822..b51300aafd86 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.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; @@ -133,51 +134,50 @@ public StoreFileInfo(final Configuration conf, final FileSystem fs, final Path i this(conf, fs, null, initialPath, primaryReplica); } - private StoreFileInfo(final Configuration conf, final FileSystem fs, final FileStatus fileStatus, - final Path initialPath, final boolean primaryReplica) throws IOException { - assert fs != null; - assert initialPath != null; - assert conf != null; - - this.fs = fs; - this.conf = conf; - this.initialPath = initialPath; - this.primaryReplica = primaryReplica; - this.noReadahead = this.conf.getBoolean(STORE_FILE_READER_NO_READAHEAD, - DEFAULT_STORE_FILE_READER_NO_READAHEAD); - Path p = initialPath; - if (HFileLink.isHFileLink(p)) { - // HFileLink - this.reference = null; - this.link = HFileLink.buildFromHFileLinkPattern(conf, p); - LOG.trace("{} is a link", p); - } else if (isReference(p)) { - this.reference = Reference.read(fs, p); - Path referencePath = getReferredToFile(p); - if (HFileLink.isHFileLink(referencePath)) { - // HFileLink Reference - this.link = HFileLink.buildFromHFileLinkPattern(conf, referencePath); - } else { - // Reference - this.link = null; - } - LOG.trace("{} is a {} reference to {}", p, reference.getFileRegion(), referencePath); - } else if (isHFile(p)) { - // HFile - if (fileStatus != null) { - this.createdTimestamp = fileStatus.getModificationTime(); - this.size = fileStatus.getLen(); - } else { - FileStatus fStatus = fs.getFileStatus(initialPath); - this.createdTimestamp = fStatus.getModificationTime(); - this.size = fStatus.getLen(); - } - this.reference = null; - this.link = null; - } else { - throw new IOException("path=" + p + " doesn't look like a valid StoreFile"); - } - } + private StoreFileInfo(final Configuration conf, final FileSystem fs, final FileStatus fileStatus, + final Path initialPath, final boolean primaryReplica) throws IOException { + assert fs != null; + assert initialPath != null; + assert conf != null; + + this.fs = fs; + this.conf = conf; + this.initialPath = initialPath; + this.primaryReplica = primaryReplica; + this.noReadahead = this.conf.getBoolean(STORE_FILE_READER_NO_READAHEAD, DEFAULT_STORE_FILE_READER_NO_READAHEAD); + Path p = initialPath; + if (HFileLink.isHFileLink(p)) { + // HFileLink + this.reference = null; + this.link = HFileLink.buildFromHFileLinkPattern(conf, p); + LOG.trace("{} is a link", p); + } else if (isReference(p)) { + this.reference = Reference.read(fs, p); + Path referencePath = getReferredToFile(p); + if (HFileLink.isHFileLink(referencePath)) { + // HFileLink Reference + this.link = HFileLink.buildFromHFileLinkPattern(conf, referencePath); + } else { + // Reference + this.link = null; + } + LOG.trace("{} is a {} reference to {}", p, reference.getFileRegion(), referencePath); + } else if (isHFile(p) || isMobFile(p) || isMobRefFile(p)) { + // HFile + if (fileStatus != null) { + this.createdTimestamp = fileStatus.getModificationTime(); + this.size = fileStatus.getLen(); + } else { + FileStatus fStatus = fs.getFileStatus(initialPath); + this.createdTimestamp = fStatus.getModificationTime(); + this.size = fStatus.getLen(); + } + this.reference = null; + this.link = null; + } else { + throw new IOException("path=" + p + " doesn't look like a valid StoreFile"); + } + } /** * Create a Store File Info @@ -442,6 +442,30 @@ public static boolean isHFile(final String fileName) { return m.matches() && m.groupCount() > 0; } + 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(); + } + + 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 del file. 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 02665a345418..2f6a8fb3c110 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.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; @@ -47,6 +49,7 @@ 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 +168,9 @@ private StoreFileWriter(FileSystem fs, Path path, final Configuration conf, Cach } } + 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 +244,29 @@ public void appendMetadata(final long maxSequenceId, final boolean majorCompacti 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) { + StringBuilder sb = new StringBuilder(2 * mobRefSet.size() - 1); + String[] arr = new String[mobRefSet.size()]; + arr = mobRefSet.toArray(arr); + for (int i = 0; i < arr.length; i++) { + sb.append(arr[i]); + if (i < arr.length - 1) { + sb.append(","); + } + } + 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/regionserver/compactions/Compactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java index 10fac550180b..46e7a2eb3165 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java @@ -115,7 +115,7 @@ public CompactionProgress getProgress() { } /** The sole reason this class exists is that java has no ref/out/pointer parameters. */ - protected static class FileDetails { + public static class FileDetails { /** Maximum key count after compaction (for blooms) */ public long maxKeyCount = 0; /** Earliest put timestamp if major compaction */ 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 81e6fd26d263..00e38bbea428 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 @@ -1140,7 +1140,7 @@ public HFileFilter(FileSystem fs) { @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/mob/FaultyMobStoreCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java new file mode 100644 index 000000000000..057d81fd766d --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java @@ -0,0 +1,373 @@ +/** + * + * 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("injected.fault.probability", 0.1);
    + *   }
    + * }
    + * }
    + * @see org.apache.hadoop.hbase.mob.TestMobCompaction 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("injected.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 = FileSystem.get(conf); + + // 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 { + // 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) { + // 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("\n\n INJECTED FAULT mobCounter=" + mobCounter.get() + "\n\n"); + throw new CorruptHFileException("injected fault"); + } + String fName = MobUtils.getMobFileName(c); + Path pp = new Path(new Path(fs.getUri()), new Path(path, fName)); + + // Added to support migration + try { + mobCell = mobStore.resolve(c, true, false).getCell(); + } catch (FileNotFoundException fnfe) { + if (discardMobMiss) { + LOG.error("Missing MOB cell: file=" + pp + " not found"); + continue; + } else { + throw fnfe; + } + } + + if (discardMobMiss && mobCell.getValueLength() == 0) { + LOG.error("Missing MOB cell value: file=" + pp + " cell=" + 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())); + cellsCountCompactedFromMob++; + cellsSizeCompactedFromMob += mobCell.getValueLength(); + mobCells++; + } else { + // If MOB value is less than threshold, append it directly to a store file + PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId()); + writer.append(mobCell); + } + + } 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++; + } 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 { + // TODO ???? + 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 000000000000..146fb216a08f --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java @@ -0,0 +1,79 @@ +/** + * 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.hbase.thirdparty.org.apache.commons.cli.CommandLine; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.yetus.audience.InterfaceStability; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@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; + } + } + + TestMobCompaction test = new TestMobCompaction(); + test.init(getConf(), numRowsToInsert); + test.testMobCompaction(); + 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/TesMobFileCleanerChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TesMobFileCleanerChore.java new file mode 100644 index 000000000000..deb1cd465518 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TesMobFileCleanerChore.java @@ -0,0 +1,236 @@ +/** + * + * 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.MobFileCleanerChore; +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.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +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 TesMobFileCleanerChore { + private static final Logger LOG = LoggerFactory.getLogger(TesMobFileCleanerChore.class); + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TesMobFileCleanerChore.class); + @Rule + public TestName testName = new TestName(); + + 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 TesMobFileCleanerChore() { + } + + + @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 { + HTU.shutdownMiniCluster(); + } + + @Test + public void testMobFileCleanerChore() throws InterruptedException, IOException { + + try { + + 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); + + } finally { + + admin.disableTable(hdt.getTableName()); + admin.deleteTable(hdt.getTableName()); + } + LOG.info("MOB Stress Test finished OK"); + + } + + 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; + } + + + 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/TestMobCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java new file mode 100644 index 000000000000..2b2cac365b2d --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java @@ -0,0 +1,375 @@ +/** + * + * 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.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.KeepDeletedCells; +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.MobFileCleanerChore; +import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Ignore; +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; +/** + 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. + + */ +@SuppressWarnings("deprecation") +@Category(LargeTests.class) +public class TestMobCompaction { + private static final Logger LOG = LoggerFactory.getLogger(TestMobCompaction.class); + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestMobCompaction.class); + @Rule + public TestName testName = new TestName(); + + 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 TestMobCompaction() { + + } + + public void init(Configuration conf, long numRows) throws IOException { + this.conf = conf; + this.count = numRows; + printConf(); + hdt = createTableDescriptor("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("To run stress test, please change HBase configuration as following:"); + 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("injected.fault.probability=x, where x is between 0. and 1."); + + } + + private HTableDescriptor createTableDescriptor(final String name, final int minVersions, + final int versions, final int ttl, KeepDeletedCells keepDeleted) { + HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name)); + return htd; + } + + private HTableDescriptor createTableDescriptor(final String name) { + return createTableDescriptor(name, HColumnDescriptor.DEFAULT_MIN_VERSIONS, 1, + HConstants.FOREVER, HColumnDescriptor.DEFAULT_KEEP_DELETED); + } + + @Before + public void setUp() throws Exception { + HTU = new HBaseTestingUtility(); + hdt = HTU.createTableDescriptor("testMobCompactTable"); + conf = HTU.getConfiguration(); + + initConf(); + + // HTU.getConfiguration().setInt("hbase.mob.compaction.chore.period", 0); + HTU.startMiniCluster(); + admin = HTU.getAdmin(); + + 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.setDouble("injected.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.IO_OPTIMIZED_MOB_COMPACTION_TYPE); + conf.setLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, 1000000); + + } + + @After + public void tearDown() throws Exception { + HTU.shutdownMiniCluster(); + } + + 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) { + e.printStackTrace(); + } + } + } + } + + 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); + } + } + } + + @Ignore + @Test + public void testMobCompaction() 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++; + } + +// for (int i=0; i < count; i++) { +// byte[] key = Bytes.toBytes(i); +// Get get = new Get(key); +// Result res = table.get(get); +// assertTrue(Arrays.equals(res.getValue(fam, qualifier), +// Bytes.add(key,mobVal))); +// if (i % 1000 == 0) { +// LOG.info("GET=" + i); +// } +// } + 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 000000000000..169c74c15c1e --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java @@ -0,0 +1,242 @@ +/** + * + * 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 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.master.MobFileCleanerChore; +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); + byte[][] splitKeys = generateSplitKeys(); + table = HTU.createTable(hdt, splitKeys); + + } + + private byte[][] generateSplitKeys() { + RegionSplitter.UniformSplit splitAlgo = new RegionSplitter.UniformSplit(); + return splitAlgo.split(numRegions); + } + + + 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); + assertTrue(false); + } + } + + @After + public void tearDown() throws Exception { + HTU.shutdownMiniCluster(); + } + + + public void baseTestMobFileCompaction() throws InterruptedException, IOException { + + try { + + // 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 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 + cleanerChore.cleanupObsoleteMobFiles(conf, table.getName()); + + num = getNumberOfMobFiles(conf, table.getName(), new String(fam)); + assertEquals(numRegions, num); + + long scanned = scanTable(); + assertEquals(3 * rows, scanned); + + } finally { + + admin.disableTable(hdt.getTableName()); + admin.deleteTable(hdt.getTableName()); + } + + } + + 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) { + e.printStackTrace(); + LOG.error("MOB file compaction 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/TestMobCompactionOptMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java new file mode 100644 index 000000000000..4b7bea2184a0 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java @@ -0,0 +1,88 @@ +/** + * + * 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.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +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); + @Rule + public TestName testName = new TestName(); + + + public TestMobCompactionOptMode() { + } + + @Override + protected void initConf() { + super.initConf(); + conf.set(MobConstants.MOB_COMPACTION_TYPE_KEY, + MobConstants.IO_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 000000000000..ba5e47e0409e --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java @@ -0,0 +1,99 @@ +/** + * + * 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.master.MobFileCompactionChore; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.Before; +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; + +/** + * 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); + @Rule + public TestName testName = new TestName(); + + 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.IO_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 000000000000..cec57e8a22a9 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java @@ -0,0 +1,80 @@ +/** + * + * 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.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +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); + @Rule + public TestName testName = new TestName(); + + + 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 000000000000..936173cefae1 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java @@ -0,0 +1,96 @@ +/** + * + * 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.master.MobFileCompactionChore; +import org.apache.hadoop.hbase.testclassification.LargeTests; +import org.junit.Before; +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; + +/** + * 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); + @Rule + public TestName testName = new TestName(); + + 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/TestMobFileName.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java index ae53ff21feba..a6a2ee55009c 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 void setUp() { @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/mob/compactions/TestMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java deleted file mode 100644 index b8e3ce0d4409..000000000000 --- 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 27801533dc5a..000000000000 --- 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 73ee96561e02..000000000000 --- 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 33ab4e3bc349..30e456edffbe 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 @@ -69,7 +69,7 @@ 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.mob.compactions.PartitionedMobCompactor; import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.TestReplicationBase; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -83,6 +83,7 @@ import org.junit.Before; import org.junit.BeforeClass; import org.junit.ClassRule; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -257,20 +258,21 @@ public void testBulkLoadReplicationActiveActive() throws Exception { assertEquals(9, BULK_LOADS_COUNT.get()); } + @Ignore @Test public void testPartionedMOBCompactionBulkLoadDoesntReplicate() throws Exception { - Path path = createMobFiles(UTIL3); +// 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); +// 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); +// 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()); @@ -348,35 +350,36 @@ private String createHFileForFamilies(byte[] row, byte[] value, 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; - } + +// 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 { 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/regionserver/TestMobStoreCompaction.java index 74c8d7934dac..fa44c18d1cb4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java @@ -270,30 +270,6 @@ public void testMajorCompactionAfterDelete() throws Exception { // 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 +400,4 @@ private int countReferencedMobFiles() throws IOException { 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/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java index b6227fc428cf..8c8379080cc2 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 @@ String createMobFileName(String oldFileName) { 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 3a3becdeec04..9870b73f5f9e 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 void setUp() { /** * 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()); From a09e6ffac966839d70b99e8860b0788a283cc7f1 Mon Sep 17 00:00:00 2001 From: Vladimir Rodionov Date: Thu, 12 Dec 2019 16:55:57 -0800 Subject: [PATCH 02/17] HBASE-22749: Distributed MOB compactions. (fixed UTs and code clean up) --- .../hbase/IntegrationTestMobCompaction.java | 7 +- .../hbase/master/MasterRpcServices.java | 38 +-- .../hbase/master/MobFileCleanerChore.java | 55 +++-- .../hbase/master/MobFileCompactionChore.java | 46 ++-- .../hbase/mob/DefaultMobStoreCompactor.java | 108 ++++----- .../org/apache/hadoop/hbase/mob/MobUtils.java | 11 +- .../hbase/regionserver/StoreFileInfo.java | 219 +++++++++--------- .../hbase/client/TestAsyncRegionAdminApi.java | 4 +- .../hbase/mob/FaultyMobStoreCompactor.java | 4 +- .../hadoop/hbase/mob/MobStressTool.java | 4 +- .../regionserver/TestBulkLoadReplication.java | 69 +----- .../regionserver/TestMobStoreCompaction.java | 29 ++- .../regionserver/TestRegionServerMetrics.java | 21 +- 13 files changed, 290 insertions(+), 325 deletions(-) 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 index 3373ce98f6bd..47400a4154cb 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java @@ -44,8 +44,7 @@ import org.apache.hadoop.hbase.testclassification.IntegrationTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.util.ToolRunner; -import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects; -import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; + import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -53,6 +52,8 @@ 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 @@ -189,7 +190,7 @@ protected void addOptions() { "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 + "'"); + + DEFAULT_FAILURE_PROB + "'"); } @Override 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 6d97f5d6a45c..c1541fbfa05a 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 @@ -104,6 +104,27 @@ 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; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +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; +import org.slf4j.LoggerFactory; + +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; @@ -328,21 +349,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription; -import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils; -import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -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.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.yetus.audience.InterfaceAudience; -import org.apache.zookeeper.KeeperException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; + + /** * Implements the master RPC services. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java index 0f1d0e1f156e..ca3df667bb33 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java @@ -51,11 +51,12 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.HFileArchiveUtil; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; 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 ExpiredMobFileCleanerChore for running cleaner regularly to remove the expired * and obsolete (files which have no active references to) mob files. @@ -69,10 +70,12 @@ public class MobFileCleanerChore extends ScheduledChore { private long minAgeToArchive; public MobFileCleanerChore(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); + 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()); @@ -104,8 +107,8 @@ public MobFileCleanerChore() { } @Override - @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION", - justification="Intentional") + @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "REC_CATCH_EXCEPTION", + justification = "Intentional") protected void chore() { try { @@ -127,23 +130,23 @@ protected void chore() { LOG.error("Fail to clean the expired mob files", e); } } + /** * Performs housekeeping file cleaning (called by MOB Cleaner chore) * @param conf configuration * @param table table name - * @throws IOException + * @throws IOException exception */ - public void cleanupObsoleteMobFiles(Configuration conf, TableName table) - throws IOException { + public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws IOException { - this.minAgeToArchive = conf.getLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, - MobConstants.DEFAULT_MIN_AGE_TO_ARCHIVE); + this.minAgeToArchive = + conf.getLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, MobConstants.DEFAULT_MIN_AGE_TO_ARCHIVE); 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); + LOG.info("Skipping non-MOB table [{}]", table); return; } // We check only those MOB files, which creation time is less @@ -191,13 +194,15 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) 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); + + "can not proceed until all old files will be MOB-compacted.", + pp); return; } else if (mobRefData == null && bulkloadMarkerData != null) { LOG.info("Skipping file without MOB references (bulkloaded file):{}", pp); continue; } - if (mobRefData.length > 1) { + // 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(","); @@ -206,7 +211,7 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) } } } catch (FileNotFoundException e) { - LOG.warn("Starting MOB cleaning cycle from the beginning due to error:",e); + LOG.warn("Starting MOB cleaning cycle from the beginning due to error:", e); continue; } succeed = true; @@ -236,9 +241,9 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) // 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) { - /* DEBUG */ LOG.info( - " Archiving MOB file{} creation time=" + (fs.getFileStatus(p).getModificationTime()), p); + if (creationTime < maxCreationTimeToArchive) { + LOG.info("Archiving MOB file {} creation time={}", p, + (fs.getFileStatus(p).getModificationTime())); toArchive.add(p); } else { LOG.info("Skipping fresh file: {}", p); @@ -247,21 +252,20 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) LOG.info("Keepeing active MOB file: {}", p); } } - LOG.info(" MOB Cleaner found {} files for family={}", toArchive.size() , family); + LOG.info(" MOB Cleaner found {} files for family={}", toArchive.size(), family); removeMobFiles(conf, table, family.getBytes(), toArchive); LOG.info(" MOB Cleaner archived {} files", toArchive.size()); } } } - /** * 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 + * @throws IOException exception */ public void removeMobFiles(Configuration conf, TableName tableName, byte[] family, List storeFiles) throws IOException { @@ -273,8 +277,8 @@ public void removeMobFiles(Configuration conf, TableName tableName, byte[] famil } Path mobTableDir = FSUtils.getTableDir(MobUtils.getMobHome(conf), tableName); FileSystem fs = storeFiles.get(0).getFileSystem(conf); - Path storeArchiveDir = HFileArchiveUtil.getStoreArchivePath(conf, MobUtils.getMobRegionInfo(tableName), - mobTableDir, family); + Path storeArchiveDir = HFileArchiveUtil.getStoreArchivePath(conf, + MobUtils.getMobRegionInfo(tableName), mobTableDir, family); for (Path p : storeFiles) { Path archiveFilePath = new Path(storeArchiveDir, p.getName()); @@ -283,7 +287,8 @@ public void removeMobFiles(Configuration conf, TableName tableName, byte[] famil continue; } LOG.info("MOB Cleaner is archiving: {}", p); - HFileArchiver.archiveStoreFile(conf, fs, MobUtils.getMobRegionInfo(tableName), mobTableDir, family, p); + HFileArchiver.archiveStoreFile(conf, fs, MobUtils.getMobRegionInfo(tableName), mobTableDir, + family, p); } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java index 402718ea68de..251abc8d75a8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java @@ -38,11 +38,13 @@ import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.mob.MobConstants; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; 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 @@ -61,11 +63,12 @@ public class MobFileCompactionChore extends ScheduledChore { 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); + 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.conf = master.getConfiguration(); this.regionBatchSize = @@ -86,7 +89,7 @@ protected void chore() { boolean reported = false; try (Connection conn = ConnectionFactory.createConnection(conf); - Admin admin = conn.getAdmin(); ) { + Admin admin = conn.getAdmin();) { TableDescriptors htds = master.getTableDescriptors(); Map map = htds.getAll(); @@ -120,13 +123,13 @@ protected void chore() { hcd.getNameAsString()); } } catch (IOException e) { - LOG.error("Failed to compact table="+ htd.getTableName() +" cf="+ hcd.getNameAsString(), - e); + LOG.error( + "Failed to compact table=" + htd.getTableName() + " cf=" + hcd.getNameAsString(), e); } catch (InterruptedException ee) { Thread.currentThread().interrupt(); master.reportMobCompactionEnd(htd.getTableName()); - LOG.warn("Failed to compact table="+ htd.getTableName() +" cf="+ hcd.getNameAsString(), - ee); + LOG.warn( + "Failed to compact table=" + htd.getTableName() + " cf=" + hcd.getNameAsString(), ee); // Quit the chore return; } @@ -156,25 +159,25 @@ public void performMajorCompactionInBatches(Admin admin, TableDescriptor htd, Collections.shuffle(regions); // Create first batch List toCompact = new ArrayList(); - for (int i=0; i < this.regionBatchSize; i++) { + for (int i = 0; i < this.regionBatchSize; i++) { toCompact.add(regions.remove(0)); } // Start compaction now - for(RegionInfo ri: toCompact) { + for (RegionInfo ri : toCompact) { startCompaction(admin, htd.getTableName(), ri, hcd.getName()); } List compacted = new ArrayList(); int totalCompacted = 0; - while(!toCompact.isEmpty()) { + while (!toCompact.isEmpty()) { // Check status of active compactions - for (RegionInfo ri: toCompact) { + for (RegionInfo ri : toCompact) { try { if (admin.getCompactionStateForRegion(ri.getRegionName()) == CompactionState.NONE) { totalCompacted++; LOG.info("Finished major compaction: table={} region={}, compacted regions={}", - htd.getTableName(),ri.getRegionNameAsString(), totalCompacted); + htd.getTableName(), ri.getRegionNameAsString(), totalCompacted); compacted.add(ri); } } catch (IOException e) { @@ -182,11 +185,11 @@ public void performMajorCompactionInBatches(Admin admin, TableDescriptor htd, } } // Update batch: remove compacted regions and add new ones - for (RegionInfo ri: compacted) { + for (RegionInfo ri : compacted) { toCompact.remove(ri); if (regions.size() > 0) { RegionInfo region = regions.remove(0); - startCompaction(admin, htd.getTableName(),region, hcd.getName()); + startCompaction(admin, htd.getTableName(), region, hcd.getName()); toCompact.add(region); } } @@ -199,17 +202,16 @@ public void performMajorCompactionInBatches(Admin admin, TableDescriptor htd, } - private void startCompaction(Admin admin, TableName table, RegionInfo region, byte[] cf) + private void startCompaction(Admin admin, TableName table, RegionInfo region, byte[] cf) throws IOException, InterruptedException { - LOG.info("Started major compaction: table={} region={}", table, - region.getRegionNameAsString()); + LOG.info("Started major compaction: table={} region={}", table, region.getRegionNameAsString()); admin.majorCompactRegion(region.getRegionName()); // 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) { + while (admin.getCompactionStateForRegion(region.getRegionName()) == CompactionState.NONE) { // Is 1 second too aggressive? Thread.sleep(1000); if (EnvironmentEdgeManager.currentTime() - startTime > waitTime) { 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 d69272384600..aa7caa9b0c03 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 @@ -57,11 +57,12 @@ import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hbase.thirdparty.com.google.common.collect.Lists; 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. */ @@ -73,11 +74,10 @@ public class DefaultMobStoreCompactor extends DefaultCompactor { protected HMobStore mobStore; /* - * 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. + * 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>() { @@ -99,8 +99,7 @@ protected Boolean initialValue() { }; /* - * Map : MOB file name - file length - * Can be expensive for large amount of MOB files? + * Map : MOB file name - file length Can be expensive for large amount of MOB files? */ static ThreadLocal> mobLengthMap = new ThreadLocal>() { @@ -150,7 +149,6 @@ public DefaultMobStoreCompactor(Configuration conf, HStore store) { mobSizeThreshold = store.getColumnFamilyDescriptor().getMobThreshold(); } - @Override public List compact(CompactionRequestImpl request, ThroughputController throughputController, User user) throws IOException { @@ -166,8 +164,8 @@ public List compact(CompactionRequestImpl request, if (conf.get(MobConstants.MOB_COMPACTION_TYPE_KEY, MobConstants.DEFAULT_MOB_COMPACTION_TYPE) .equals(MobConstants.IO_OPTIMIZED_MOB_COMPACTION_TYPE)) { if (request.isMajor() && request.getPriority() == HStore.PRIORITY_USER) { - Path mobDir = MobUtils.getMobFamilyPath(conf, store.getTableName(), - store.getColumnFamilyName()); + Path mobDir = + MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName()); List mobFiles = MobUtils.getReferencedMobFiles(request.getFiles(), mobDir); if (mobFiles.size() > 0) { calculateMobLengthMap(mobFiles); @@ -182,7 +180,7 @@ private void calculateMobLengthMap(List mobFiles) throws IOException { FileSystem fs = mobFiles.get(0).getFileSystem(this.conf); HashMap map = mobLengthMap.get(); map.clear(); - for (Path p: mobFiles) { + for (Path p : mobFiles) { FileStatus st = fs.getFileStatus(p); long size = st.getLen(); LOG.info("Ref MOB file={} size={}", p, size); @@ -190,39 +188,30 @@ private void calculateMobLengthMap(List mobFiles) throws IOException { } } - /** - * 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:
    + * 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. + * 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. *
    * @param fd File details * @param scanner Where to read from. @@ -245,13 +234,12 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel mobRefSet.get().clear(); boolean isUserRequest = userRequest.get(); boolean compactMOBs = major && isUserRequest; - boolean ioOptimizedMode = conf.get(MobConstants.MOB_COMPACTION_TYPE_KEY, - MobConstants.DEFAULT_MOB_COMPACTION_TYPE) - .equals(MobConstants.IO_OPTIMIZED_MOB_COMPACTION_TYPE); + boolean ioOptimizedMode = + conf.get(MobConstants.MOB_COMPACTION_TYPE_KEY, MobConstants.DEFAULT_MOB_COMPACTION_TYPE) + .equals(MobConstants.IO_OPTIMIZED_MOB_COMPACTION_TYPE); - boolean discardMobMiss = - conf.getBoolean(MobConstants.MOB_UNSAFE_DISCARD_MISS_KEY, - MobConstants.DEFAULT_MOB_DISCARD_MISS); + boolean discardMobMiss = conf.getBoolean(MobConstants.MOB_UNSAFE_DISCARD_MISS_KEY, + MobConstants.DEFAULT_MOB_DISCARD_MISS); long maxMobFileSize = conf.getLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, MobConstants.DEFAULT_MOB_COMPACTION_MAX_FILE_SIZE); @@ -321,7 +309,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel LOG.error("Missing MOB cell value: file=" + pp + " cell=" + mobCell); continue; } else if (mobCell.getValueLength() == 0) { - //TODO: what to do here? This is data corruption? + // TODO: what to do here? This is data corruption? LOG.warn("Found 0 length MOB cell in a file={} cell={}", pp, mobCell); } @@ -331,8 +319,8 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel if (!ioOptimizedMode) { mobFileWriter.append(mobCell); mobCells++; - writer.append(MobUtils.createMobRefCell(mobCell, fileName, - this.mobStore.getRefCellTags())); + writer.append( + MobUtils.createMobRefCell(mobCell, fileName, this.mobStore.getRefCellTags())); } else { // I/O optimized mode // Check if MOB cell origin file size is @@ -340,9 +328,8 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel Long size = mobLengthMap.get().get(fName); if (size == null) { // FATAL error, abort compaction - String msg = - String.format("Found unreferenced MOB file during compaction %s, aborting.", - fName); + String msg = String.format( + "Found unreferenced MOB file during compaction %s, aborting.", fName); LOG.error(msg); throw new IOException(msg); } @@ -353,15 +340,15 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel mobFileWriter.append(mobCell); // Update number of mobCells in a current mob writer mobCells++; - writer.append(MobUtils.createMobRefCell(mobCell, fileName, - this.mobStore.getRefCellTags())); + 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 = getLength(mobFileWriter); if (len > maxMobFileSize) { - LOG.debug("Closing output MOB File, length={} file={}", - len, Bytes.toString(fileName)); + LOG.debug("Closing output MOB File, length={} file={}", len, + Bytes.toString(fileName)); commitOrAbortMobWriter(mobFileWriter, fd.maxSeqId, mobCells, major); mobFileWriter = newMobWriter(fd); fileName = Bytes.toBytes(mobFileWriter.getPath().getName()); @@ -532,8 +519,7 @@ private long getLength(StoreFileWriter mobFileWriter) throws IOException { return mobFileWriter.getPos(); } - - private StoreFileWriter newMobWriter(FileDetails fd/*, boolean compactMOBs*/) + private StoreFileWriter newMobWriter(FileDetails fd/* , boolean compactMOBs */) throws IOException { try { StoreFileWriter mobFileWriter = mobStore.createWriterInTmp(new Date(fd.latestPutTs), @@ -549,18 +535,19 @@ private StoreFileWriter newMobWriter(FileDetails fd/*, boolean compactMOBs*/) } } - private void commitOrAbortMobWriter(StoreFileWriter mobFileWriter, long maxSeqId, - long mobCells, boolean major) throws IOException - { + 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 - LOG.debug("Commit or abort size={} mobCells={} major={} file={}", - mobFileWriter.getPos(), mobCells, major, mobFileWriter.getPath().getName()); - Path path = MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName()); + if (mobFileWriter != null) { + LOG.info("Commit or abort size={} mobCells={} major={} file={}", mobFileWriter.getPos(), + mobCells, major, mobFileWriter.getPath().getName()); + 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); @@ -568,18 +555,19 @@ private void commitOrAbortMobWriter(StoreFileWriter mobFileWriter, long maxSeqId 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", - mobFileWriter.getPath()); + LOG.debug("Aborting writer for {} because there are no MOB cells", mobFileWriter.getPath()); // Remove MOB file from reference set mobRefSet.get().remove(mobFileWriter.getPath().getName()); abortWriter(mobFileWriter); } + } else { + LOG.info("Mob file writer is null, skipping commit/abort."); } } protected static String createKey(TableName tableName, String encodedName, String columnFamilyName) { - return tableName.getNameAsString()+ "_" + encodedName + "_"+ columnFamilyName; + return tableName.getNameAsString() + "_" + encodedName + "_" + columnFamilyName; } @Override 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 f20435d4d8ea..0aed8b2e8b0b 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 @@ -428,8 +428,6 @@ public static void removeMobFiles(Configuration conf, FileSystem fs, TableName t storeFiles); } - - /** * Creates a mob reference KeyValue. The value of the mob reference KeyValue is mobCellValueSize + * mobFileName. @@ -620,8 +618,8 @@ public static List getMobColumnFamilies(TableDescriptor } /** - * 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. + * 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. */ @@ -676,12 +674,11 @@ public static String getEncodedRegionName(String mobFileName) { if (index < 0) { return null; } - return mobFileName.substring(index+1); + return mobFileName.substring(index + 1); } /** - * Get list of referenced MOB files from a given collection - * of store files + * 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 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 b51300aafd86..c715d6b91ded 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 @@ -53,36 +53,34 @@ 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. + * A non-capture group, for del files, so that this can be embedded. A del file has (_del) as + * suffix. */ 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 + ")"); + 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. + * 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; @@ -134,50 +132,51 @@ public StoreFileInfo(final Configuration conf, final FileSystem fs, final Path i this(conf, fs, null, initialPath, primaryReplica); } - private StoreFileInfo(final Configuration conf, final FileSystem fs, final FileStatus fileStatus, - final Path initialPath, final boolean primaryReplica) throws IOException { - assert fs != null; - assert initialPath != null; - assert conf != null; - - this.fs = fs; - this.conf = conf; - this.initialPath = initialPath; - this.primaryReplica = primaryReplica; - this.noReadahead = this.conf.getBoolean(STORE_FILE_READER_NO_READAHEAD, DEFAULT_STORE_FILE_READER_NO_READAHEAD); - Path p = initialPath; - if (HFileLink.isHFileLink(p)) { - // HFileLink - this.reference = null; - this.link = HFileLink.buildFromHFileLinkPattern(conf, p); - LOG.trace("{} is a link", p); - } else if (isReference(p)) { - this.reference = Reference.read(fs, p); - Path referencePath = getReferredToFile(p); - if (HFileLink.isHFileLink(referencePath)) { - // HFileLink Reference - this.link = HFileLink.buildFromHFileLinkPattern(conf, referencePath); - } else { - // Reference - this.link = null; - } - LOG.trace("{} is a {} reference to {}", p, reference.getFileRegion(), referencePath); - } else if (isHFile(p) || isMobFile(p) || isMobRefFile(p)) { - // HFile - if (fileStatus != null) { - this.createdTimestamp = fileStatus.getModificationTime(); - this.size = fileStatus.getLen(); - } else { - FileStatus fStatus = fs.getFileStatus(initialPath); - this.createdTimestamp = fStatus.getModificationTime(); - this.size = fStatus.getLen(); - } - this.reference = null; - this.link = null; - } else { - throw new IOException("path=" + p + " doesn't look like a valid StoreFile"); - } - } + private StoreFileInfo(final Configuration conf, final FileSystem fs, final FileStatus fileStatus, + final Path initialPath, final boolean primaryReplica) throws IOException { + assert fs != null; + assert initialPath != null; + assert conf != null; + + this.fs = fs; + this.conf = conf; + this.initialPath = initialPath; + this.primaryReplica = primaryReplica; + this.noReadahead = this.conf.getBoolean(STORE_FILE_READER_NO_READAHEAD, + DEFAULT_STORE_FILE_READER_NO_READAHEAD); + Path p = initialPath; + if (HFileLink.isHFileLink(p)) { + // HFileLink + this.reference = null; + this.link = HFileLink.buildFromHFileLinkPattern(conf, p); + LOG.trace("{} is a link", p); + } else if (isReference(p)) { + this.reference = Reference.read(fs, p); + Path referencePath = getReferredToFile(p); + if (HFileLink.isHFileLink(referencePath)) { + // HFileLink Reference + this.link = HFileLink.buildFromHFileLinkPattern(conf, referencePath); + } else { + // Reference + this.link = null; + } + LOG.trace("{} is a {} reference to {}", p, reference.getFileRegion(), referencePath); + } else if (isHFile(p) || isMobFile(p) || isMobRefFile(p)) { + // HFile + if (fileStatus != null) { + this.createdTimestamp = fileStatus.getModificationTime(); + this.size = fileStatus.getLen(); + } else { + FileStatus fStatus = fs.getFileStatus(initialPath); + this.createdTimestamp = fStatus.getModificationTime(); + this.size = fStatus.getLen(); + } + this.reference = null; + this.link = null; + } else { + throw new IOException("path=" + p + " doesn't look like a valid StoreFile"); + } + } /** * Create a Store File Info @@ -227,11 +226,11 @@ public StoreFileInfo(final Configuration conf, final FileSystem fs, final FileSt 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); } /** @@ -251,8 +250,9 @@ public void setRegionCoprocessorHost(RegionCoprocessorHost coprocessorHost) { } /* - * @return the Reference object associated to this StoreFileInfo. - * null if the StoreFile is not a reference. + * @return the Reference object associated to this StoreFileInfo. + * null if the StoreFile is not a + * reference. */ public Reference getReference() { return this.reference; @@ -278,8 +278,7 @@ public HDFSBlocksDistribution getHDFSBlockDistribution() { 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 +315,9 @@ ReaderContext createReaderContext(boolean doDropBehind, long readahead, ReaderTy 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 +421,8 @@ public long getModificationTime() throws IOException { @Override public String toString() { - return this.getPath() + - (isReference() ? "->" + getReferredToFile(this.getPath()) + "-" + reference : ""); + return this.getPath() + + (isReference() ? "->" + getReferredToFile(this.getPath()) + "-" + reference : ""); } /** @@ -463,9 +459,10 @@ public static boolean isMobRefFile(final Path path) { 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); + 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 del file. @@ -508,8 +505,8 @@ public long getCreatedTimestamp() { } /* - * 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. @@ -518,8 +515,7 @@ public static Path getReferredToFile(final Path p) { 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 @@ -530,9 +526,9 @@ public static Path getReferredToFile(final Path p) { 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); } /** @@ -541,8 +537,7 @@ public static Path getReferredToFile(final Path p) { * @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("-"); } @@ -551,12 +546,10 @@ public static boolean validateStoreFileName(final String fileName) { * @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()) - return false; + 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 @@ -570,21 +563,19 @@ public static boolean isValid(final FileStatus 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 + * 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; } @@ -593,11 +584,11 @@ private static HDFSBlocksDistribution computeRefFileHDFSBlockDistribution( 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); } @@ -609,16 +600,16 @@ public boolean equals(Object that) { if (!(that instanceof StoreFileInfo)) return false; - StoreFileInfo o = (StoreFileInfo)that; + StoreFileInfo o = (StoreFileInfo) that; if (initialPath != null && o.initialPath == null) return false; if (initialPath == null && o.initialPath != null) return false; - if (initialPath != o.initialPath && initialPath != null - && !initialPath.equals(o.initialPath)) 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; - if (reference != o.reference && reference != null - && !reference.equals(o.reference)) return false; + if (reference != o.reference && reference != null && !reference.equals(o.reference)) + return false; if (link != null && o.link == null) return false; if (link == null && o.link != null) return false; @@ -631,9 +622,9 @@ public boolean equals(Object that) { 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; } /** @@ -676,8 +667,7 @@ StoreFileReader preStoreFileReaderOpen(ReaderContext context, CacheConfig cacheC 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; } @@ -687,8 +677,7 @@ StoreFileReader postStoreFileReaderOpen(ReaderContext context, CacheConfig cache 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/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java index aeff96e8964b..fe79a65dbee5 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 void testCompactMob() throws Exception { 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 index 057d81fd766d..2003d6378102 100644 --- 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 @@ -100,8 +100,8 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel 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 discardMobMiss = conf.getBoolean(MobConstants.MOB_UNSAFE_DISCARD_MISS_KEY, + MobConstants.DEFAULT_MOB_DISCARD_MISS); boolean mustFail = false; if (compactMOBs) { 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 index 146fb216a08f..81f5cf9451e1 100644 --- 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 @@ -23,12 +23,14 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.util.AbstractHBaseTool; import org.apache.hadoop.util.ToolRunner; -import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; 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 { 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 30e456edffbe..43e3e40e136d 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,19 @@ 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; @@ -62,21 +60,14 @@ 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; @@ -258,31 +249,6 @@ public void testBulkLoadReplicationActiveActive() throws Exception { assertEquals(9, BULK_LOADS_COUNT.get()); } - @Ignore - @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 { @@ -350,37 +316,6 @@ private String createHFileForFamilies(byte[] row, byte[] value, 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/TestMobStoreCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java index fa44c18d1cb4..6b252c62a573 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java @@ -29,6 +29,7 @@ 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.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; @@ -63,7 +62,11 @@ 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.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; @@ -158,6 +161,9 @@ public void testSmallerValue() throws Exception { /** * During compaction, the mob threshold size is changed. + * The test is no longer valid. Major MOB compaction must be triggered by User + * HRegion does not provide public API to trigger major-compaction by User + * This test will move to mob sub-package. */ @Test public void testLargerValue() throws Exception { @@ -178,8 +184,21 @@ public void testLargerValue() throws Exception { // 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()); assertEquals("After compaction: referenced mob file count", 0, countReferencedMobFiles()); 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 bc44daa5cff2..cfed2564b8ea 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 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.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 void testMobMetrics() throws IOException, InterruptedException { 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); From 0855b4149a7f7cd5c041fe88d3a1e432b19284a9 Mon Sep 17 00:00:00 2001 From: Vladimir Rodionov Date: Mon, 13 Jan 2020 12:14:25 -0800 Subject: [PATCH 03/17] HBASE-22749: Distributed MOB compactions (some comments addressed) --- .../hbase/IntegrationTestMobCompaction.java | 16 ++- .../apache/hadoop/hbase/master/HMaster.java | 14 +-- .../hbase/master/MasterRpcServices.java | 4 +- .../apache/hadoop/hbase/mob/MobConstants.java | 5 + .../{master => mob}/MobFileCleanerChore.java | 116 ++++++++++-------- .../MobFileCompactionChore.java | 4 +- .../hbase/mob/TesMobFileCleanerChore.java | 1 - .../hadoop/hbase/mob/TestMobCompaction.java | 1 - .../hbase/mob/TestMobCompactionBase.java | 1 - .../TestMobCompactionOptRegionBatchMode.java | 1 - ...stMobCompactionRegularRegionBatchMode.java | 1 - 11 files changed, 91 insertions(+), 73 deletions(-) rename hbase-server/src/main/java/org/apache/hadoop/hbase/{master => mob}/MobFileCleanerChore.java (75%) rename hbase-server/src/main/java/org/apache/hadoop/hbase/{master => mob}/MobFileCompactionChore.java (99%) 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 index 47400a4154cb..0ee1e09c3e79 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java @@ -34,10 +34,10 @@ 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.MobFileCleanerChore; 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; @@ -63,6 +63,13 @@ * 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") @@ -142,6 +149,9 @@ public void tearDown() throws IOException { @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 { @@ -400,10 +410,6 @@ private void scanTable() { } } - /** - * - * @param args argument list - */ public static void main(String[] args) throws Exception { Configuration conf = HBaseConfiguration.create(); initConf(conf); 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 d578cb46f8bc..9b7ea23b61ae 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 @@ -148,6 +148,8 @@ 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; @@ -1266,16 +1268,8 @@ public void updateConfigurationForQuotasObserver(Configuration conf) { private void initMobCleaner() { this.mobFileCleanerChore = new MobFileCleanerChore(this); getChoreService().scheduleChore(mobFileCleanerChore); - - int mobCompactionPeriod = conf.getInt(MobConstants.MOB_COMPACTION_CHORE_PERIOD, - MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD); - - if (mobCompactionPeriod > 0) { - this.mobFileCompactionChore = new MobFileCompactionChore(this); - getChoreService().scheduleChore(mobFileCompactionChore); - } else { - LOG.info("The period is " + mobCompactionPeriod + " seconds, MobCompactionChore is disabled"); - } + this.mobFileCompactionChore = new MobFileCompactionChore(this); + getChoreService().scheduleChore(mobFileCompactionChore); } /** 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 c1541fbfa05a..fb19fe17d325 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 @@ -1762,7 +1762,9 @@ public CompactRegionResponse compactRegion(final RpcController controller, if (MobUtils.isMobRegionName(tableName, regionName)) { checkHFileFormatVersionForMob(); //TODO: support CompactType.MOB - LOG.warn("CompactType.MOB is not supported yet, will run regular compaction."); + // 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); 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 fa0944be21db..b2d54fb0a1a8 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 @@ -44,6 +44,11 @@ public final class MobConstants { public static final byte[] MOB_REGION_NAME_BYTES = Bytes.toBytes(MOB_REGION_NAME); public static final String MOB_CLEANER_PERIOD = "hbase.master.mob.cleaner.period"; + + @Deprecated + 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"; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerChore.java similarity index 75% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerChore.java index ca3df667bb33..8594474ed268 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCleanerChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerChore.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.hbase.master; +package org.apache.hadoop.hbase.mob; import java.io.FileNotFoundException; import java.io.IOException; @@ -43,14 +43,11 @@ 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.mob.ExpiredMobFileCleaner; -import org.apache.hadoop.hbase.mob.MobConstants; -import org.apache.hadoop.hbase.mob.MobUtils; +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.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.hbase.util.HFileArchiveUtil; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -58,16 +55,21 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; /** - * The Class ExpiredMobFileCleanerChore for running cleaner regularly to remove the expired + * The class MobFileCleanerChore for running cleaner regularly to remove the expired * and obsolete (files which have no active references to) mob files. */ +@SuppressWarnings("deprecation") @InterfaceAudience.Private public class MobFileCleanerChore extends ScheduledChore { private static final Logger LOG = LoggerFactory.getLogger(MobFileCleanerChore.class); private final HMaster master; private ExpiredMobFileCleaner cleaner; - private long minAgeToArchive; + + static { + Configuration.addDeprecation(MobConstants.DEPRECATED_MOB_CLEANER_PERIOD, + MobConstants.MOB_CLEANER_PERIOD); + } public MobFileCleanerChore(HMaster master) { super(master.getServerName() + "-ExpiredMobFileCleanerChore", master, @@ -84,9 +86,7 @@ public MobFileCleanerChore(HMaster master) { private void checkObsoleteConfigurations() { Configuration conf = master.getConfiguration(); - if (conf.get("hbase.master.mob.ttl.cleaner.period") != null) { - LOG.warn("'hbase.master.mob.ttl.cleaner.period' is obsolete and not used anymore."); - } + if (conf.get("hbase.mob.compaction.mergeable.threshold") != null) { LOG.warn("'hbase.mob.compaction.mergeable.threshold' is obsolete and not used anymore."); } @@ -111,23 +111,34 @@ public MobFileCleanerChore() { justification = "Intentional") protected void chore() { - try { + TableDescriptors htds = master.getTableDescriptors(); - TableDescriptors htds = master.getTableDescriptors(); - Map map = htds.getAll(); - for (TableDescriptor htd : map.values()) { - for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) { - if (hcd.isMobEnabled() && hcd.getMinVersions() == 0) { + 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=" + htd.getTableName() + + " family=" + hcd.getNameAsString(), e); } } + } + try { // Now clean obsolete files for a table - LOG.info("Cleaning obsolete MOB files ..."); + LOG.info("Cleaning obsolete MOB files from table={}", htd.getTableName()); cleanupObsoleteMobFiles(master.getConfiguration(), htd.getTableName()); - LOG.info("Cleaning obsolete MOB files finished"); + 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); } - } catch (Exception e) { - LOG.error("Fail to clean the expired mob files", e); } } @@ -139,8 +150,17 @@ protected void chore() { */ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws IOException { - this.minAgeToArchive = + 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; + LOG.info("Only MOB files whose creation time older than {} will be archived", + maxCreationTimeToArchive); try (final Connection conn = ConnectionFactory.createConnection(conf); final Admin admin = conn.getAdmin();) { TableDescriptor htd = admin.getDescriptor(table); @@ -149,15 +169,7 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws LOG.info("Skipping non-MOB table [{}]", table); return; } - // We check only those MOB files, which creation time is less - // than maxTimeToArchive. 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; - LOG.info("Only MOB files whose creation time less than {} will be archived", - maxCreationTimeToArchive); + Path rootDir = FSUtils.getRootDir(conf); Path tableDir = FSUtils.getTableDir(rootDir, table); // How safe is this call? @@ -172,7 +184,11 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws boolean succeed = false; Set regionMobs = new HashSet(); while (!succeed) { - + if (!fs.exists(storePath)) { + LOG.warn("Directory {} was deleted during cleaner procedure execution," + +" skipping.", storePath); + break; + } RemoteIterator rit = fs.listLocatedStatus(storePath); List storeFiles = new ArrayList(); // Load list of store files first @@ -182,6 +198,7 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws storeFiles.add(p); } } + LOG.info("Found {} store files in: {}", storeFiles.size(), storePath); try { for (Path pp : storeFiles) { LOG.debug("Store file: {}", pp); @@ -198,7 +215,7 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws pp); return; } else if (mobRefData == null && bulkloadMarkerData != null) { - LOG.info("Skipping file without MOB references (bulkloaded file):{}", pp); + LOG.debug("Skipping file without MOB references (bulkloaded file):{}", pp); continue; } // mobRefData will never be null here, but to make FindBugs happy @@ -206,12 +223,16 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws // 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(","); - LOG.debug("Found: {} mob refs: ", mobs.length, Arrays.toString(mobs)); + if (LOG.isTraceEnabled()) { + LOG.debug("Found: {} mob references: {}", mobs.length, Arrays.toString(mobs)); + } else { + LOG.debug("Found: {} mob references", mobs.length); + } regionMobs.addAll(Arrays.asList(mobs)); } } } catch (FileNotFoundException e) { - LOG.warn("Starting MOB cleaning cycle from the beginning due to error:", e); + LOG.warn("Starting MOB cleaning cycle from the beginning due to error", e); continue; } succeed = true; @@ -226,7 +247,7 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws allActiveMobFileName.size()); } LOG.debug("Found: {} active mob refs", allActiveMobFileName.size()); - allActiveMobFileName.stream().forEach(LOG::debug); + allActiveMobFileName.stream().forEach(LOG::trace); // Now scan MOB directories and find MOB files with no references to them for (ColumnFamilyDescriptor hcd : list) { @@ -242,19 +263,21 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws // fresh, skip it in this case long creationTime = fs.getFileStatus(p).getModificationTime(); if (creationTime < maxCreationTimeToArchive) { - LOG.info("Archiving MOB file {} creation time={}", p, + LOG.debug("Archiving MOB file {} creation time={}", p, (fs.getFileStatus(p).getModificationTime())); toArchive.add(p); } else { - LOG.info("Skipping fresh file: {}", p); + LOG.debug("Skipping fresh file: {}", p); } } else { - LOG.info("Keepeing active MOB file: {}", p); + LOG.debug("Keeping active MOB file: {}", p); } } - LOG.info(" MOB Cleaner found {} files for family={}", toArchive.size(), family); + LOG.info(" MOB Cleaner found {} files to archive for table={} family={}", + toArchive.size(), table, family); removeMobFiles(conf, table, family.getBytes(), toArchive); - LOG.info(" MOB Cleaner archived {} files", toArchive.size()); + LOG.info(" MOB Cleaner archived {} files, table={} family={}", + toArchive.size(), table, family); } } } @@ -272,23 +295,16 @@ public void removeMobFiles(Configuration conf, TableName tableName, byte[] famil if (storeFiles.size() == 0) { // nothing to remove - LOG.debug("Skipping archiving old MOB file: collection is empty"); + LOG.debug("Skipping archiving old MOB files - no files found."); return; } Path mobTableDir = FSUtils.getTableDir(MobUtils.getMobHome(conf), tableName); FileSystem fs = storeFiles.get(0).getFileSystem(conf); - Path storeArchiveDir = HFileArchiveUtil.getStoreArchivePath(conf, - MobUtils.getMobRegionInfo(tableName), mobTableDir, family); for (Path p : storeFiles) { - Path archiveFilePath = new Path(storeArchiveDir, p.getName()); - if (fs.exists(archiveFilePath)) { - LOG.warn("MOB Cleaner skip archiving: {} because it has been archived already", p); - continue; - } LOG.info("MOB Cleaner is archiving: {}", p); - HFileArchiver.archiveStoreFile(conf, fs, MobUtils.getMobRegionInfo(tableName), mobTableDir, - family, p); + HFileArchiver.archiveStoreFile(conf, fs, MobUtils.getMobRegionInfo(tableName), + mobTableDir, family, p); } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCompactionChore.java similarity index 99% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCompactionChore.java index 251abc8d75a8..26280133407e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MobFileCompactionChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCompactionChore.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hbase.master; +package org.apache.hadoop.hbase.mob; import java.io.IOException; import java.util.ArrayList; @@ -36,7 +36,7 @@ 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.mob.MobConstants; +import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TesMobFileCleanerChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TesMobFileCleanerChore.java index deb1cd465518..f13b82404d2f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TesMobFileCleanerChore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TesMobFileCleanerChore.java @@ -38,7 +38,6 @@ 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.MobFileCleanerChore; import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java index 2b2cac365b2d..e8e7357d49b1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java @@ -41,7 +41,6 @@ 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.MobFileCleanerChore; import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; 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 index 169c74c15c1e..50637da6c4a1 100644 --- 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 @@ -38,7 +38,6 @@ 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.MobFileCleanerChore; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.RegionSplitter; import org.junit.After; 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 index ba5e47e0409e..7496f8c18e34 100644 --- 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 @@ -23,7 +23,6 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.master.MobFileCompactionChore; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.Before; import org.junit.ClassRule; 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 index 936173cefae1..f6c984af2ebd 100644 --- 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 @@ -23,7 +23,6 @@ import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.master.MobFileCompactionChore; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.junit.Before; import org.junit.ClassRule; From d56413df48f7fb720c1fd3ce5065a32008f5d615 Mon Sep 17 00:00:00 2001 From: Vladimir Rodionov Date: Thu, 23 Jan 2020 17:59:14 -0800 Subject: [PATCH 04/17] HBASE-22749: code review --- .../hbase/IntegrationTestMobCompaction.java | 5 +- .../apache/hadoop/hbase/master/HMaster.java | 1 - .../hbase/mob/DefaultMobStoreCompactor.java | 134 ++++++++++-------- .../hbase/mob/DefaultMobStoreFlusher.java | 8 +- .../apache/hadoop/hbase/mob/MobConstants.java | 77 ++++++---- .../hadoop/hbase/mob/MobFileCleanerChore.java | 73 ++++++---- .../hbase/mob/MobFileCompactionChore.java | 81 +++++++---- .../hadoop/hbase/mob/MobStoreEngine.java | 4 +- .../org/apache/hadoop/hbase/mob/MobUtils.java | 25 +--- .../hadoop/hbase/regionserver/HMobStore.java | 3 +- .../hadoop/hbase/regionserver/HRegion.java | 2 +- .../hbase/regionserver/StoreFileInfo.java | 99 +++++++------ .../hbase/regionserver/StoreFileWriter.java | 12 +- .../regionserver/compactions/Compactor.java | 2 +- .../hbase/mob/FaultyMobStoreCompactor.java | 8 +- .../hadoop/hbase/mob/MobStressTool.java | 6 +- ...mpaction.java => MobStressToolRunner.java} | 75 ++-------- .../hbase/mob/TestMobCompactionBase.java | 77 +++++----- .../hbase/mob/TestMobCompactionOptMode.java | 2 +- .../TestMobCompactionOptRegionBatchMode.java | 2 +- ...hore.java => TestMobFileCleanerChore.java} | 65 ++++----- .../TestMobStoreCompaction.java | 21 +-- .../TestMobStoreScanner.java | 4 +- .../regionserver/TestBulkLoadReplication.java | 4 - .../regionserver/TestRegionServerMetrics.java | 6 +- 25 files changed, 404 insertions(+), 392 deletions(-) rename hbase-server/src/test/java/org/apache/hadoop/hbase/mob/{TestMobCompaction.java => MobStressToolRunner.java} (83%) rename hbase-server/src/test/java/org/apache/hadoop/hbase/mob/{TesMobFileCleanerChore.java => TestMobFileCleanerChore.java} (82%) rename hbase-server/src/test/java/org/apache/hadoop/hbase/{regionserver => mob}/TestMobStoreCompaction.java (97%) rename hbase-server/src/test/java/org/apache/hadoop/hbase/{regionserver => mob}/TestMobStoreScanner.java (99%) 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 index 0ee1e09c3e79..9aa1978745ba 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java @@ -65,7 +65,6 @@ * @see HBASE-22749 *

    * Sample usage: - * *

      * hbase org.apache.hadoop.hbase.IntegrationTestMobCompaction -Dservers=10 -Drows=1000000
      * -Dfailprob=0.2
    @@ -233,8 +232,8 @@ private static void initConf(Configuration conf) {
         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("injected.fault.probability", failureProb);
    -    conf.set(MobStoreEngine.DEFAULT_MOB_COMPACTOR_CLASS_KEY,
    +    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);
    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 9b7ea23b61ae..eaabd53b879b 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,6 @@
     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;
    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 aa7caa9b0c03..871a9d8e187d 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
    @@ -72,6 +72,7 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
       private static final Logger LOG = LoggerFactory.getLogger(DefaultMobStoreCompactor.class);
       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
    @@ -99,15 +100,15 @@ protected Boolean initialValue() {
       };
     
       /*
    -   * Map : MOB file name - file length Can be expensive for large amount of MOB files?
    +   * 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();
    -        }
    -      };
    +    new ThreadLocal>() {
    +      @Override
    +      protected HashMap initialValue() {
    +        return new HashMap();
    +      }
    +    };
     
       private final InternalScannerFactory scannerFactory = new InternalScannerFactory() {
     
    @@ -145,24 +146,33 @@ public DefaultMobStoreCompactor(Configuration conf, HStore store) {
         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 {
    -    LOG.info("Mob compaction: major=" + request.isMajor() + " isAll=" + request.isAllFiles()
    -        + " priority=" + request.getPriority());
    +    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.info("Mob compaction files: " + request.getFiles());
    +    LOG.debug("MOB compaction table={} cf={} region={} files: ", tableName, familyName,
    +      regionName, request.getFiles());
         // Check if I/O optimized MOB compaction
    -    if (conf.get(MobConstants.MOB_COMPACTION_TYPE_KEY, MobConstants.DEFAULT_MOB_COMPACTION_TYPE)
    -        .equals(MobConstants.IO_OPTIMIZED_MOB_COMPACTION_TYPE)) {
    +    if (ioOptimizedMode) {
           if (request.isMajor() && request.getPriority() == HStore.PRIORITY_USER) {
             Path mobDir =
                 MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName());
    @@ -170,9 +180,11 @@ public List compact(CompactionRequestImpl request,
             if (mobFiles.size() > 0) {
               calculateMobLengthMap(mobFiles);
             }
    -        LOG.info("I/O optimized MOB compaction. Total referenced MOB files: {}", mobFiles.size());
    +        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);
       }
     
    @@ -183,7 +195,7 @@ private void calculateMobLengthMap(List mobFiles) throws IOException {
         for (Path p : mobFiles) {
           FileStatus st = fs.getFileStatus(p);
           long size = st.getLen();
    -      LOG.info("Ref MOB file={} size={}", p, size);
    +      LOG.debug("Referenced MOB file={} size={}", p, size);
           map.put(p.getName(), fs.getFileStatus(p).getLen());
         }
       }
    @@ -234,20 +246,18 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
         mobRefSet.get().clear();
         boolean isUserRequest = userRequest.get();
         boolean compactMOBs = major && isUserRequest;
    -    boolean ioOptimizedMode =
    -        conf.get(MobConstants.MOB_COMPACTION_TYPE_KEY, MobConstants.DEFAULT_MOB_COMPACTION_TYPE)
    -            .equals(MobConstants.IO_OPTIMIZED_MOB_COMPACTION_TYPE);
    -
         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"+
    +        " during upgrade process from MOB 1.0 to MOB 2.0 versions.",
    +        MobConstants.MOB_UNSAFE_DISCARD_MISS_KEY);
    +    }
         long maxMobFileSize = conf.getLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY,
           MobConstants.DEFAULT_MOB_COMPACTION_MAX_FILE_SIZE);
    -    LOG.info("Compact MOB={} optimized={} maximum MOB file size={} major={}", compactMOBs,
    -      ioOptimizedMode, maxMobFileSize, major);
    -
    +    LOG.info("Compact MOB={} optimized={} maximum MOB file size={} major={} store={}", compactMOBs,
    +      ioOptimizedMode, maxMobFileSize, major, getStoreInfo());
         FileSystem fs = FileSystem.get(conf);
    -
         // Since scanner.next() can return 'false' but still be delivering data,
         // we have to use a do/while loop.
         List cells = new ArrayList<>();
    @@ -298,7 +308,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
                     mobCell = mobStore.resolve(c, true, false).getCell();
                   } catch (FileNotFoundException fnfe) {
                     if (discardMobMiss) {
    -                  LOG.debug("Missing MOB cell: file={} not found cell={}", pp, c);
    +                  LOG.error("Missing MOB cell: file={} not found cell={}", fName, c);
                       continue;
                     } else {
                       throw fnfe;
    @@ -306,11 +316,12 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
                   }
     
                   if (discardMobMiss && mobCell.getValueLength() == 0) {
    -                LOG.error("Missing MOB cell value: file=" + pp + " cell=" + mobCell);
    +                LOG.error("Missing MOB cell value: file={} cell={}", pp, mobCell);
                     continue;
                   } else if (mobCell.getValueLength() == 0) {
    -                // TODO: what to do here? This is data corruption?
    -                LOG.warn("Found 0 length MOB cell in a file={} cell={}", pp, mobCell);
    +                String errMsg = String.format("Found 0 length MOB cell in a file=%s cell=%s",
    +                  fName, mobCell);
    +                throw new IOException(errMsg);
                   }
     
                   if (mobCell.getValueLength() > mobSizeThreshold) {
    @@ -329,8 +340,8 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
                       if (size == null) {
                         // FATAL error, abort compaction
                         String msg = String.format(
    -                      "Found unreferenced MOB file during compaction %s, aborting.", fName);
    -                    LOG.error(msg);
    +                      "Found unreferenced MOB file during compaction %s, aborting compaction %s",
    +                      fName, getStoreInfo());
                         throw new IOException(msg);
                       }
                       // Can not be null
    @@ -344,11 +355,10 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
                           MobUtils.createMobRefCell(mobCell, fileName, this.mobStore.getRefCellTags()));
                         // Update total size of the output (we do not take into account
                         // file compression yet)
    -                    long len = getLength(mobFileWriter);
    -
    +                    long len = mobFileWriter.getPos();
                         if (len > maxMobFileSize) {
    -                      LOG.debug("Closing output MOB File, length={} file={}", len,
    -                        Bytes.toString(fileName));
    +                      LOG.debug("Closing output MOB File, length={} file={}, store=", len,
    +                        Bytes.toString(fileName), getStoreInfo());
                           commitOrAbortMobWriter(mobFileWriter, fd.maxSeqId, mobCells, major);
                           mobFileWriter = newMobWriter(fd);
                           fileName = Bytes.toBytes(mobFileWriter.getPath().getName());
    @@ -384,7 +394,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
                     if (ioOptimizedMode) {
                       // Update total size of the output (we do not take into account
                       // file compression yet)
    -                  long len = getLength(mobFileWriter);
    +                  long len = mobFileWriter.getPos();
                       if (len > maxMobFileSize) {
                         commitOrAbortMobWriter(mobFileWriter, fd.maxSeqId, mobCells, major);
                         mobFileWriter = newMobWriter(fd);
    @@ -411,9 +421,8 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
                   // Add MOB reference to a MOB reference set
                   mobRefSet.get().add(MobUtils.getMobFileName(c));
                 } else {
    -              // TODO ????
    -              LOG.error("Corrupted MOB reference: " + c);
    -              writer.append(c);
    +              String errMsg = String.format("Corrupted MOB reference: %s", c.toString());
    +              throw new IOException(errMsg);
                 }
               } else if (c.getValueLength() <= mobSizeThreshold) {
                 // If the value size of a cell is not larger than the threshold, directly write it to
    @@ -431,7 +440,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
                 cellsCountCompactedToMob++;
                 cellsSizeCompactedToMob += c.getValueLength();
                 if (ioOptimizedMode) {
    -              long len = getLength(mobFileWriter);
    +              long len = mobFileWriter.getPos();
                   if (len > maxMobFileSize) {
                     commitOrAbortMobWriter(mobFileWriter, fd.maxSeqId, mobCells, major);
                     mobFileWriter = newMobWriter(fd);
    @@ -486,8 +495,9 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
           throw new InterruptedIOException(
               "Interrupted while control throughput of compacting " + compactionName);
         } catch (IOException t) {
    -      LOG.error("Mob compaction failed for region:{} ", store.getRegionInfo().getEncodedName());
    -      throw 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
    @@ -498,15 +508,15 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
             // Remove all MOB references because compaction failed
             mobRefSet.get().clear();
             // Abort writer
    -        LOG.debug("Aborting writer for {} because of a compaction failure",
    -          mobFileWriter.getPath());
    +        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);
    -
    +    clearThreadLocals();
         mobStore.updateCellsCountCompactedFromMob(cellsCountCompactedFromMob);
         mobStore.updateCellsCountCompactedToMob(cellsCountCompactedToMob);
         mobStore.updateCellsSizeCompactedFromMob(cellsSizeCompactedFromMob);
    @@ -515,11 +525,20 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
         return true;
       }
     
    -  private long getLength(StoreFileWriter mobFileWriter) throws IOException {
    -    return mobFileWriter.getPos();
    +  private String getStoreInfo() {
    +    return String.format("[table=%s family=%s region=%s]", store.getTableName().getNameAsString(),
    +      store.getColumnFamilyName(), store.getRegionInfo().getEncodedName()) ;
       }
     
    -  private StoreFileWriter newMobWriter(FileDetails fd/* , boolean compactMOBs */)
    +  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),
    @@ -530,8 +549,8 @@ private StoreFileWriter newMobWriter(FileDetails fd/* , boolean compactMOBs */)
           return mobFileWriter;
         } catch (IOException e) {
           // Bailing out
    -      LOG.error("Failed to create mob writer, ", e);
    -      throw e;
    +      throw new IOException(String.format("Failed to create mob writer, store=%s",
    +        getStoreInfo()), e);
         }
       }
     
    @@ -544,8 +563,9 @@ private void commitOrAbortMobWriter(StoreFileWriter mobFileWriter, long maxSeqId
         // become orphans and will be deleted during next MOB cleaning chore cycle
     
         if (mobFileWriter != null) {
    -      LOG.info("Commit or abort size={} mobCells={} major={} file={}", mobFileWriter.getPos(),
    -        mobCells, major, mobFileWriter.getPath().getName());
    +      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) {
    @@ -555,20 +575,18 @@ private void commitOrAbortMobWriter(StoreFileWriter mobFileWriter, long maxSeqId
             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", mobFileWriter.getPath());
    +        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.info("Mob file writer is null, skipping commit/abort.");
    +      LOG.debug("Mob file writer is null, skipping commit/abort, store=",
    +        getStoreInfo());
         }
       }
     
    -  protected static String createKey(TableName tableName, String encodedName,
    -      String columnFamilyName) {
    -    return tableName.getNameAsString() + "_" + encodedName + "_" + columnFamilyName;
    -  }
     
       @Override
       protected List commitWriter(StoreFileWriter writer, FileDetails fd,
    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 70591fb14da9..5c4c6020ae09 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
    @@ -254,7 +254,7 @@ protected void performMobFlush(MemStoreSnapshot snapshot, long cacheFlushId,
           status.setStatus("Flushing mob file " + store + ": closing flushed file");
           mobFileWriter.close();
           mobStore.commitFile(mobFileWriter.getPath(), targetPath);
    -      LOG.debug("Flush store file: {}", writer.getPath());
    +      LOG.debug("Flush store file: {}, store: {}", writer.getPath(), getStoreInfo());
           mobStore.updateMobFlushCount();
           mobStore.updateMobFlushedCellsCount(mobCount);
           mobStore.updateMobFlushedCellsSize(mobSize);
    @@ -272,6 +272,7 @@ protected void performMobFlush(MemStoreSnapshot snapshot, long cacheFlushId,
         }
       }
     
    +  @Override
       protected void finalizeWriter(StoreFileWriter writer, long cacheFlushSeqNum,
           MonitoredTask status) throws IOException {
         // Write out the log sequence number that corresponds to this output
    @@ -283,4 +284,9 @@ protected void finalizeWriter(StoreFileWriter writer, long cacheFlushSeqNum,
         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/MobConstants.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobConstants.java
    index b2d54fb0a1a8..77f3dc04ff09 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,18 @@ 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.cleaner.period";
    -  
    -  @Deprecated
    -  public static final String DEPRECATED_MOB_CLEANER_PERIOD = 
    -      "hbase.master.mob.ttl.cleaner.period";
    -  
    +  public static final String DEPRECATED_MOB_CLEANER_PERIOD = "hbase.master.mob.ttl.cleaner.period";
       public static final int DEFAULT_MOB_CLEANER_PERIOD = 24 * 60 * 60; // one day
    -
       public static final 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;
    @@ -62,13 +54,13 @@ public final class MobConstants {
       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.
    +   * 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";
    +      "hbase.mob.major.compaction.region.batch.size";
     
       /**
        * Default is 0 - means no limit - all regions of a MOB table will be compacted at once
    @@ -77,28 +69,24 @@ public final class MobConstants {
       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.
    +   * 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_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", "io_optimized"
    -   * "full" - run full major compaction (during migration)
    -   * "io_optimized" - optimized version for use  case with infrequent updates/deletes
    +   * 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 MOB_COMPACTION_TYPE_KEY = "hbase.mob.compaction.type";
    -
    -  public final static String DEFAULT_MOB_COMPACTION_TYPE = "full";
     
    -  public final static String IO_OPTIMIZED_MOB_COMPACTION_TYPE = "optimized";
    +  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
    @@ -111,8 +99,8 @@ public final class MobConstants {
       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.
    +   * 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";
     
    @@ -125,6 +113,35 @@ public final class MobConstants {
     
       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 = "";
    +  /**
    +   * 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.
    +   */
    +  public static final String MOB_COMPACTION_MERGEABLE_THRESHOLD =
    +      "hbase.mob.compaction.mergeable.threshold";
    +  public static final long DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD = 10 * 128 * 1024 * 1024;
    +  public static final 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. 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 int DEFAULT_MOB_COMPACTION_BATCH_SIZE = 100;
    +  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
    index 8594474ed268..c7d713dc2e12 100644
    --- 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
    @@ -46,6 +46,7 @@
     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;
    @@ -58,16 +59,15 @@
      * The class MobFileCleanerChore for running cleaner regularly to remove the expired
      * and obsolete (files which have no active references to) mob files.
      */
    -@SuppressWarnings("deprecation")
     @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, 
    +    Configuration.addDeprecation(MobConstants.DEPRECATED_MOB_CLEANER_PERIOD,
           MobConstants.MOB_CLEANER_PERIOD);
       }
     
    @@ -126,8 +126,9 @@ protected void chore() {
               try {
                 cleaner.cleanExpiredMobFiles(htd.getTableName().getNameAsString(), hcd);
               } catch (IOException e) {
    -            LOG.error("Failed to clean the expired mob files table=" + htd.getTableName()
    -                + " family=" + hcd.getNameAsString(), e);
    +            String errMsg = String.format("Failed to clean the expired mob files table=%s" +
    +                " family=%s", htd.getTableName().getNameAsString(), hcd.getNameAsString());
    +            LOG.error(errMsg, e);
               }
             }
           }
    @@ -137,7 +138,9 @@ protected void chore() {
             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);
    +        String errMsg =
    +            String.format("Failed to clean the obsolete mob files for table=",htd.getTableName());
    +        LOG.error(errMsg, e);
           }
         }
       }
    @@ -159,8 +162,8 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws
         // 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;
    -    LOG.info("Only MOB files whose creation time older than {} will be archived",
    -      maxCreationTimeToArchive);
    +    LOG.info("Only MOB files whose creation time older than {} will be archived, table={}",
    +      maxCreationTimeToArchive, table);
         try (final Connection conn = ConnectionFactory.createConnection(conf);
             final Admin admin = conn.getAdmin();) {
           TableDescriptor htd = admin.getDescriptor(table);
    @@ -178,6 +181,7 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws
           Set allActiveMobFileName = new HashSet();
           FileSystem fs = FileSystem.get(conf);
           for (Path regionPath : regionDirs) {
    +        region:
             for (ColumnFamilyDescriptor hcd : list) {
               String family = hcd.getNameAsString();
               Path storePath = new Path(regionPath, family);
    @@ -185,9 +189,9 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws
               Set regionMobs = new HashSet();
               while (!succeed) {
                 if (!fs.exists(storePath)) {
    -              LOG.warn("Directory {} was deleted during cleaner procedure execution," 
    -                  +" skipping.", storePath);
    -              break;
    +              LOG.warn("Directory {} was deleted during cleaner procedure execution,"
    +                  +" skipping region {}", storePath, regionPath.getName());
    +              continue region;
                 }
                 RemoteIterator rit = fs.listLocatedStatus(storePath);
                 List storeFiles = new ArrayList();
    @@ -199,9 +203,11 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws
                   }
                 }
                 LOG.info("Found {} store files in: {}", storeFiles.size(), storePath);
    +            Path currentPath = null;
                 try {
                   for (Path pp : storeFiles) {
    -                LOG.debug("Store file: {}", pp);
    +                currentPath = pp;
    +                LOG.trace("Store file: {}", pp);
                     HStoreFile sf =
                         new HStoreFile(fs, pp, conf, CacheConfig.DISABLED, BloomType.NONE, true);
                     sf.initReader();
    @@ -224,7 +230,7 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws
                       // in this store file, but the file was created by new MOB code
                       String[] mobs = new String(mobRefData).split(",");
                       if (LOG.isTraceEnabled()) {
    -                    LOG.debug("Found: {} mob references: {}", mobs.length, Arrays.toString(mobs));
    +                    LOG.trace("Found: {} mob references: {}", mobs.length, Arrays.toString(mobs));
                       } else {
                         LOG.debug("Found: {} mob references", mobs.length);
                       }
    @@ -232,21 +238,30 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws
                     }
                   }
                 } catch (FileNotFoundException e) {
    -              LOG.warn("Starting MOB cleaning cycle from the beginning due to error", e);
    +              String warnMsg =
    +                  String.format("Missing file:%s" +
    +                      " Starting MOB cleaning cycle from the beginning due to error",
    +                      currentPath);
    +              LOG.warn(warnMsg, e);
                   continue;
                 }
                 succeed = true;
               }
    -          // Add MOB refs for current region/family
    +          if (!succeed) {
    +            continue; // continue next cf
    +          }
    +          // 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: {}, this may result in high memory pressure.",
    -          allActiveMobFileName.size());
    +        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", allActiveMobFileName.size());
    +      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
    @@ -263,20 +278,21 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws
                 // fresh, skip it in this case
                 long creationTime = fs.getFileStatus(p).getModificationTime();
                 if (creationTime < maxCreationTimeToArchive) {
    -              LOG.debug("Archiving MOB file {} creation time={}", p,
    +              LOG.trace("Archiving MOB file {} creation time={}", p,
                     (fs.getFileStatus(p).getModificationTime()));
                   toArchive.add(p);
                 } else {
    -              LOG.debug("Skipping fresh file: {}", p);
    +              LOG.trace("Skipping fresh file: {}. Creation time={}", p,
    +                fs.getFileStatus(p).getModificationTime());
                 }
               } else {
    -            LOG.debug("Keeping active MOB file: {}", p);
    +            LOG.trace("Keeping active MOB file: {}", p);
               }
             }
    -        LOG.info(" MOB Cleaner found {} files to archive for table={} family={}", 
    +        LOG.info(" MOB Cleaner found {} files to archive for table={} family={}",
               toArchive.size(), table, family);
    -        removeMobFiles(conf, table, family.getBytes(), toArchive);
    -        LOG.info(" MOB Cleaner archived {} files, table={} family={}", 
    +        archiveMobFiles(conf, table, family.getBytes(), toArchive);
    +        LOG.info(" MOB Cleaner archived {} files, table={} family={}",
               toArchive.size(), table, family);
           }
         }
    @@ -290,20 +306,21 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws
        * @param storeFiles The files to be archived.
        * @throws IOException exception
        */
    -  public void removeMobFiles(Configuration conf, TableName tableName, byte[] family,
    +  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.");
    +      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.info("MOB Cleaner is archiving: {}", p);
    -      HFileArchiver.archiveStoreFile(conf, fs, MobUtils.getMobRegionInfo(tableName), 
    +      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
    index 26280133407e..82d4a82b0e8c 100644
    --- 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
    @@ -37,6 +37,7 @@
     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;
    @@ -96,11 +97,12 @@ protected void chore() {
           for (TableDescriptor htd : map.values()) {
             if (!master.getTableStateManager().isTableState(htd.getTableName(),
               TableState.State.ENABLED)) {
    -          LOG.debug("Skipping MOB compaction on table {} because it is not ENABLED",
    +          LOG.info("Skipping MOB compaction on table {} because it is not ENABLED",
                 htd.getTableName());
               continue;
             } else {
    -          LOG.debug("Starting MOB compaction on table {}", htd.getTableName());
    +          LOG.info("Starting MOB compaction on table {}, checking {} column families",
    +            htd.getTableName(), htd.getColumnFamilyCount());
             }
             for (ColumnFamilyDescriptor hcd : htd.getColumnFamilies()) {
               try {
    @@ -109,27 +111,33 @@ protected void chore() {
                     master.reportMobCompactionStart(htd.getTableName());
                     reported = true;
                   }
    -              LOG.info(" Major compacting {} cf={}", htd.getTableName(), hcd.getNameAsString());
    +              LOG.info("Major MOB compacting table={} cf={}", htd.getTableName(),
    +                hcd.getNameAsString());
                   if (regionBatchSize == MobConstants.DEFAULT_MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE) {
    -                LOG.debug("Batch compaction is disabled, {}=0", "hbase.mob.compaction.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.debug("Performing compaction in batches, {}={}",
    -                  "hbase.mob.compaction.batch.size", regionBatchSize);
    +                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 column family {} because it is not MOB-enabled",
    -                hcd.getNameAsString());
    +              LOG.info("Skipping table={} column family={} because it is not MOB-enabled",
    +                htd.getTableName(), hcd.getNameAsString());
                 }
               } catch (IOException e) {
    -            LOG.error(
    -              "Failed to compact table=" + htd.getTableName() + " cf=" + hcd.getNameAsString(), e);
    +            String errMsg = String.format("Failed to compact table=%s cf=%s",
    +              htd.getTableName(), hcd.getNameAsString());
    +            LOG.error(errMsg, e);
               } catch (InterruptedException ee) {
                 Thread.currentThread().interrupt();
                 master.reportMobCompactionEnd(htd.getTableName());
    -            LOG.warn(
    -              "Failed to compact table=" + htd.getTableName() + " cf=" + hcd.getNameAsString(), ee);
    +            String warnMsg = String.format("Failed to compact table=%s cf=%s",
    +              htd.getTableName(), hcd.getNameAsString());
    +            LOG.warn(warnMsg, ee);
                 // Quit the chore
                 return;
               }
    @@ -150,15 +158,17 @@ public void performMajorCompactionInBatches(Admin admin, TableDescriptor htd,
     
         List regions = admin.getRegions(htd.getTableName());
         if (regions.size() <= this.regionBatchSize) {
    -      LOG.debug("Performing compaction in non-batched mode, regions={}, batch size={}",
    -        regions.size(), 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();
    +    List toCompact = new ArrayList(regions.size());
         for (int i = 0; i < this.regionBatchSize; i++) {
           toCompact.add(regions.remove(0));
         }
    @@ -168,7 +178,8 @@ public void performMajorCompactionInBatches(Admin admin, TableDescriptor htd,
           startCompaction(admin, htd.getTableName(), ri, hcd.getName());
         }
     
    -    List compacted = new ArrayList();
    +    List compacted = new ArrayList(toCompact.size());
    +    List failed = new ArrayList();
         int totalCompacted = 0;
         while (!toCompact.isEmpty()) {
           // Check status of active compactions
    @@ -176,14 +187,26 @@ public void performMajorCompactionInBatches(Admin admin, TableDescriptor htd,
             try {
               if (admin.getCompactionStateForRegion(ri.getRegionName()) == CompactionState.NONE) {
                 totalCompacted++;
    -            LOG.info("Finished major compaction: table={} region={}, compacted regions={}",
    -              htd.getTableName(), ri.getRegionNameAsString(), 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.warn("Could not get compaction state for region {}", ri.getEncodedName());
    +          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
    +      for(RegionInfo ri: failed) {
    +        toCompact.remove(ri);
    +      }
    +      failed.clear();
           // Update batch: remove compacted regions and add new ones
           for (RegionInfo ri : compacted) {
             toCompact.remove(ri);
    @@ -194,19 +217,24 @@ public void performMajorCompactionInBatches(Admin admin, TableDescriptor htd,
             }
           }
           compacted.clear();
    -      LOG.debug("Wait for 10 sec, toCompact size={} regions left={} compacted so far={}",
    -        toCompact.size(), regions.size(), totalCompacted);
    +
    +      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 compacting {}. cf={}", htd.getTableName(), hcd.getNameAsString());
    +    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={} region={}", table, region.getRegionNameAsString());
    -    admin.majorCompactRegion(region.getRegionName());
    +    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
    @@ -215,8 +243,9 @@ private void startCompaction(Admin admin, TableName table, RegionInfo region, by
           // Is 1 second too aggressive?
           Thread.sleep(1000);
           if (EnvironmentEdgeManager.currentTime() - startTime > waitTime) {
    -        LOG.warn("Waited for {} ms to start major compaction on table: {} region: {}. Aborted.",
    -          waitTime, table.getNameAsString(), region.getRegionNameAsString());
    +        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/MobStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java
    index 6adb4b58ded5..86fffbf94f0a 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
    @@ -30,7 +30,7 @@
      */
     @InterfaceAudience.Private
     public class MobStoreEngine extends DefaultStoreEngine {
    -  public final static String DEFAULT_MOB_COMPACTOR_CLASS_KEY = "hbase.mob.default.compactor";
    +  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
    @@ -45,7 +45,7 @@ protected void createStoreFlusher(Configuration conf, HStore store) throws IOExc
       @Override
       protected void createCompactor(Configuration conf, HStore store) throws IOException {
         String className =
    -        conf.get(DEFAULT_MOB_COMPACTOR_CLASS_KEY, DefaultMobStoreCompactor.class.getName());
    +        conf.get(MOB_COMPACTOR_CLASS_KEY, DefaultMobStoreCompactor.class.getName());
         try {
           compactor = ReflectionUtils.instantiateWithCustomCtor(className,
             new Class[] { Configuration.class, HStore.class }, new Object[] { conf, store });
    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 0aed8b2e8b0b..3a6a55da9e34 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
    @@ -79,12 +79,12 @@ public final class MobUtils {
       public static final String SEP = "_";
     
       private static final ThreadLocal LOCAL_FORMAT =
    -      new ThreadLocal() {
    -        @Override
    -        protected SimpleDateFormat initialValue() {
    -          return new SimpleDateFormat("yyyyMMdd");
    -        }
    -      };
    +    new ThreadLocal() {
    +      @Override
    +      protected SimpleDateFormat initialValue() {
    +        return new SimpleDateFormat("yyyyMMdd");
    +      }
    +    };
     
       /**
        * Private constructor to keep this class from being instantiated.
    @@ -664,19 +664,6 @@ public static boolean isMobFileExpired(ColumnFamilyDescriptor column, long curre
         return false;
       }
     
    -  /**
    -   * Gets encoded region name from a MOB file name
    -   * @param mobFileName MOB file name
    -   * @return encoded region name or null
    -   */
    -  public static String getEncodedRegionName(String mobFileName) {
    -    int index = mobFileName.lastIndexOf(MobFileName.REGION_SEP);
    -    if (index < 0) {
    -      return null;
    -    }
    -    return mobFileName.substring(index + 1);
    -  }
    -
       /**
        * Get list of referenced MOB files from a given collection of store files
        * @param storeFiles store files
    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 8f7b69068aa3..d7f2ba3999f8 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
    @@ -243,8 +243,7 @@ public void commitFile(final Path sourceFile, Path targetPath) throws IOExceptio
         }
         Path dstPath = new Path(targetPath, sourceFile.getName());
         validateMobFile(sourceFile);
    -    String msg = " FLUSH 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);
    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 6a92be4f0652..e1c55ec8b59f 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 TableDescriptor getTableDescriptor() {
       }
     
       @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/StoreFileInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java
    index c715d6b91ded..04adeb507196 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
    @@ -61,19 +61,10 @@ public class StoreFileInfo {
       /** Regex that will work for hfiles */
       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.
    -   */
    -  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 
    +   * 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 @@ -249,8 +240,8 @@ public void setRegionCoprocessorHost(RegionCoprocessorHost coprocessorHost) { this.coprocessorHost = coprocessorHost; } - /* - * @return the Reference object associated to this StoreFileInfo. + /** + * @return the Reference object associated to this StoreFileInfo. * null if the StoreFile is not a * reference. */ @@ -438,6 +429,11 @@ public static boolean isHFile(final String fileName) { return m.matches() && m.groupCount() > 0; } + /** + * Checks if the file is a MOB file + * @param path path to a file + * @return true, if - yes, false otherwise + */ public static boolean isMobFile(final Path path) { String fileName = path.getName(); String[] parts = fileName.split(MobUtils.SEP); @@ -449,6 +445,12 @@ public static boolean isMobFile(final Path path) { return m.matches() && mm.matches(); } + /** + * 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 isMobRefFile(final Path path) { String fileName = path.getName(); int lastIndex = fileName.lastIndexOf(MobUtils.SEP); @@ -463,22 +465,6 @@ public static boolean isMobRefFile(final Path path) { return m.matches() && m.groupCount() > 1; } - /** - * @param path Path to check. - * @return True if the path has format of a del file. - */ - public static boolean isDelFile(final Path path) { - return isDelFile(path.getName()); - } - - /** - * @param fileName Sting version of path to validate. - * @return True if the file name has format of a del file. - */ - public static boolean isDelFile(final String fileName) { - Matcher m = DELFILE_NAME_PATTERN.matcher(fileName); - return m.matches() && m.groupCount() > 0; - } /** * @param path Path to check. @@ -537,7 +523,9 @@ public static Path getReferredToFile(final Path p) { * @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("-"); } @@ -549,7 +537,9 @@ public static boolean validateStoreFileName(final String fileName) { public static boolean isValid(final FileStatus fileStatus) throws IOException { final Path p = fileStatus.getPath(); - if (fileStatus.isDirectory()) return false; + 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 @@ -595,25 +585,48 @@ private static HDFSBlocksDistribution computeRefFileHDFSBlockDistribution(final @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; - if (initialPath != o.initialPath && initialPath != null && !initialPath.equals(o.initialPath)) + if (initialPath != null && o.initialPath == null) { return false; - - if (reference != null && o.reference == null) return false; - if (reference == null && o.reference != null) return false; - if (reference != o.reference && reference != null && !reference.equals(o.reference)) + } + 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; + } + if (reference != o.reference && reference != null + && !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; } 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 2f6a8fb3c110..3de97e80e195 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 @@ -38,6 +38,8 @@ 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; @@ -251,15 +253,7 @@ public void appendMetadata(final long maxSequenceId, final boolean majorCompacti */ public void appendMobMetadata(Set mobRefSet) throws IOException { if (mobRefSet != null && mobRefSet.size() > 0) { - StringBuilder sb = new StringBuilder(2 * mobRefSet.size() - 1); - String[] arr = new String[mobRefSet.size()]; - arr = mobRefSet.toArray(arr); - for (int i = 0; i < arr.length; i++) { - sb.append(arr[i]); - if (i < arr.length - 1) { - sb.append(","); - } - } + String sb = StringUtils.join(mobRefSet, ","); byte[] bytes = Bytes.toBytes(sb.toString()); writer.appendFileInfo(MOB_FILE_REFS, bytes); } else { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java index 46e7a2eb3165..10fac550180b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java @@ -115,7 +115,7 @@ public CompactionProgress getProgress() { } /** The sole reason this class exists is that java has no ref/out/pointer parameters. */ - public static class FileDetails { + protected static class FileDetails { /** Maximum key count after compaction (for blooms) */ public long maxKeyCount = 0; /** Earliest put timestamp if major compaction */ 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 index 2003d6378102..1dd447cddc2f 100644 --- 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 @@ -58,12 +58,12 @@ * public void initConfiguration(Configuration conf){ * conf.set(MobStoreEngine.DEFAULT_MOB_COMPACTOR_CLASS_KEY, FaultyMobStoreCompactor.class.getName()); - conf.setDouble("injected.fault.probability", 0.1); + conf.setDouble("hbase.mob.compaction.fault.probability", 0.1); * } * } * } - * @see org.apache.hadoop.hbase.mob.TestMobCompaction on how to use and configure - * this class. + * @see org.apache.hadoop.hbase.mob.MobStressToolRunner on how to use and configure + * this class. * */ @InterfaceAudience.Private @@ -81,7 +81,7 @@ public class FaultyMobStoreCompactor extends DefaultMobStoreCompactor { public FaultyMobStoreCompactor(Configuration conf, HStore store) { super(conf, store); - failureProb = conf.getDouble("injected.fault.probability", 0.1); + failureProb = conf.getDouble("hbase.mob.compaction.fault.probability", 0.1); } @Override 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 index 81f5cf9451e1..e9c6969d9697 100644 --- 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 @@ -66,9 +66,9 @@ protected int doWork() throws Exception { } } - TestMobCompaction test = new TestMobCompaction(); - test.init(getConf(), numRowsToInsert); - test.testMobCompaction(); + MobStressToolRunner runner = new MobStressToolRunner(); + runner.init(getConf(), numRowsToInsert); + runner.runStressTest(); return 0; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java similarity index 83% rename from hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java rename to hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java index e8e7357d49b1..5295ec2e1b7d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java @@ -27,7 +27,6 @@ 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.HConstants; @@ -42,16 +41,7 @@ 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.LargeTests; import org.apache.hadoop.hbase.util.Bytes; -import org.junit.After; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Ignore; -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; /** @@ -65,16 +55,13 @@ 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") -@Category(LargeTests.class) -public class TestMobCompaction { - private static final Logger LOG = LoggerFactory.getLogger(TestMobCompaction.class); - @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestMobCompaction.class); - @Rule - public TestName testName = new TestName(); +public class MobStressToolRunner { + private static final Logger LOG = LoggerFactory.getLogger(MobStressToolRunner.class); + private HBaseTestingUtility HTU; @@ -96,13 +83,14 @@ public class TestMobCompaction { private static volatile boolean run = true; - public TestMobCompaction() { + public MobStressToolRunner() { } public void init(Configuration conf, long numRows) throws IOException { this.conf = conf; this.count = numRows; + initConf(); printConf(); hdt = createTableDescriptor("testMobCompactTable"); Connection conn = ConnectionFactory.createConnection(this.conf); @@ -121,7 +109,7 @@ public void init(Configuration conf, long numRows) throws IOException { } private void printConf() { - LOG.info("To run stress test, please change HBase configuration as following:"); + 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"); @@ -132,7 +120,7 @@ private void printConf() { 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("injected.fault.probability=x, where x is between 0. and 1."); + LOG.warn("hbase.mob.compaction.fault.probability=x, where x is between 0. and 1."); } @@ -147,26 +135,6 @@ private HTableDescriptor createTableDescriptor(final String name) { HConstants.FOREVER, HColumnDescriptor.DEFAULT_KEEP_DELETED); } - @Before - public void setUp() throws Exception { - HTU = new HBaseTestingUtility(); - hdt = HTU.createTableDescriptor("testMobCompactTable"); - conf = HTU.getConfiguration(); - - initConf(); - - // HTU.getConfiguration().setInt("hbase.mob.compaction.chore.period", 0); - HTU.startMiniCluster(); - admin = HTU.getAdmin(); - - 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); @@ -177,23 +145,18 @@ private void initConf() { 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("injected.fault.probability", failureProb); + 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.IO_OPTIMIZED_MOB_COMPACTION_TYPE); + MobConstants.OPTIMIZED_MOB_COMPACTION_TYPE); conf.setLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, 1000000); } - @After - public void tearDown() throws Exception { - HTU.shutdownMiniCluster(); - } - class MajorCompaction implements Runnable { @Override @@ -222,7 +185,7 @@ public void run() { Thread.sleep(130000); } catch (Exception e) { - e.printStackTrace(); + LOG.error("CleanMobAndArchive", e); } } } @@ -266,9 +229,7 @@ public void run() { } } - @Ignore - @Test - public void testMobCompaction() throws InterruptedException, IOException { + public void runStressTest() throws InterruptedException, IOException { try { @@ -350,16 +311,6 @@ private void scanTable() { counter++; } -// for (int i=0; i < count; i++) { -// byte[] key = Bytes.toBytes(i); -// Get get = new Get(key); -// Result res = table.get(get); -// assertTrue(Arrays.equals(res.getValue(fam, qualifier), -// Bytes.add(key,mobVal))); -// if (i % 1000 == 0) { -// LOG.info("GET=" + i); -// } -// } assertEquals(count, counter); } catch (Exception e) { e.printStackTrace(); 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 index 50637da6c4a1..2d9268c17ded 100644 --- 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 @@ -19,6 +19,7 @@ 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; @@ -106,17 +107,12 @@ public void setUp() throws Exception { hcd.setMobThreshold(mobLen); hcd.setMaxVersions(1); hdt.addFamily(hcd); - byte[][] splitKeys = generateSplitKeys(); + RegionSplitter.UniformSplit splitAlgo = new RegionSplitter.UniformSplit(); + byte[][] splitKeys = splitAlgo.split(numRegions); table = HTU.createTable(hdt, splitKeys); } - private byte[][] generateSplitKeys() { - RegionSplitter.UniformSplit splitAlgo = new RegionSplitter.UniformSplit(); - return splitAlgo.split(numRegions); - } - - protected void initConf() { conf.setInt("hfile.format.version", 3); @@ -146,55 +142,49 @@ private void loadData(int num) { LOG.info("Finished loading {} rows", num); } catch (Exception e) { LOG.error("MOB file compaction chore test FAILED", e); - assertTrue(false); + 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 { - try { - - // 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 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 - cleanerChore.cleanupObsoleteMobFiles(conf, table.getName()); + // 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, num); + 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); - long scanned = scanTable(); - assertEquals(3 * rows, scanned); + Thread.sleep(minAgeToArchive + 1000); + LOG.info("Cleaning up MOB files"); + // Cleanup again + cleanerChore.cleanupObsoleteMobFiles(conf, table.getName()); - } finally { + num = getNumberOfMobFiles(conf, table.getName(), new String(fam)); + assertEquals(numRegions, num); - admin.disableTable(hdt.getTableName()); - admin.deleteTable(hdt.getTableName()); - } + long scanned = scanTable(); + assertEquals(3 * rows, scanned); } @@ -228,10 +218,9 @@ protected long scanTable() { } return counter; } catch (Exception e) { - e.printStackTrace(); - LOG.error("MOB file compaction test FAILED"); + LOG.error("MOB file compaction test FAILED", e); if (HTU != null) { - assertTrue(false); + fail(e.getMessage()); } else { System.exit(-1); } 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 index 4b7bea2184a0..4508a3887b66 100644 --- 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 @@ -66,7 +66,7 @@ public TestMobCompactionOptMode() { protected void initConf() { super.initConf(); conf.set(MobConstants.MOB_COMPACTION_TYPE_KEY, - MobConstants.IO_OPTIMIZED_MOB_COMPACTION_TYPE); + MobConstants.OPTIMIZED_MOB_COMPACTION_TYPE); conf.setLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, 1000000); } 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 index 7496f8c18e34..d354ab03530e 100644 --- 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 @@ -76,7 +76,7 @@ protected void initConf() { super.initConf(); conf.setInt(MobConstants.MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE, batchSize); conf.set(MobConstants.MOB_COMPACTION_TYPE_KEY, - MobConstants.IO_OPTIMIZED_MOB_COMPACTION_TYPE); + MobConstants.OPTIMIZED_MOB_COMPACTION_TYPE); conf.setLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, 1000000); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TesMobFileCleanerChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCleanerChore.java similarity index 82% rename from hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TesMobFileCleanerChore.java rename to hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCleanerChore.java index f13b82404d2f..21d0a64e9592 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TesMobFileCleanerChore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCleanerChore.java @@ -63,11 +63,11 @@ */ @SuppressWarnings("deprecation") @Category(MediumTests.class) -public class TesMobFileCleanerChore { - private static final Logger LOG = LoggerFactory.getLogger(TesMobFileCleanerChore.class); +public class TestMobFileCleanerChore { + private static final Logger LOG = LoggerFactory.getLogger(TestMobFileCleanerChore.class); @ClassRule public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TesMobFileCleanerChore.class); + HBaseClassTestRule.forClass(TestMobFileCleanerChore.class); @Rule public TestName testName = new TestName(); @@ -88,7 +88,7 @@ public class TesMobFileCleanerChore { private MobFileCleanerChore chore; private long minAgeToArchive = 10000; - public TesMobFileCleanerChore() { + public TestMobFileCleanerChore() { } @@ -150,48 +150,43 @@ private void loadData(int start, int num) { @After public void tearDown() throws Exception { + admin.disableTable(hdt.getTableName()); + admin.deleteTable(hdt.getTableName()); HTU.shutdownMiniCluster(); } @Test public void testMobFileCleanerChore() throws InterruptedException, IOException { - try { - - 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); + 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); + } - 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(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); - num = getNumberOfMobFiles(conf, table.getName(), new String(fam)); - assertEquals(1, num); + Thread.sleep(minAgeToArchive + 1000); + LOG.info("Cleaning up MOB files"); + // Cleanup again + chore.cleanupObsoleteMobFiles(conf, table.getName()); - long scanned = scanTable(); - assertEquals(30, scanned); + num = getNumberOfMobFiles(conf, table.getName(), new String(fam)); + assertEquals(1, num); - } finally { + long scanned = scanTable(); + assertEquals(30, scanned); - admin.disableTable(hdt.getTableName()); - admin.deleteTable(hdt.getTableName()); - } LOG.info("MOB Stress Test finished OK"); } 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 97% 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 6b252c62a573..610ccb6abc8f 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; @@ -62,6 +62,12 @@ 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.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; @@ -161,9 +167,6 @@ public void testSmallerValue() throws Exception { /** * During compaction, the mob threshold size is changed. - * The test is no longer valid. Major MOB compaction must be triggered by User - * HRegion does not provide public API to trigger major-compaction by User - * This test will move to mob sub-package. */ @Test public void testLargerValue() throws Exception { @@ -184,21 +187,21 @@ public void testLargerValue() throws Exception { // Change the threshold larger than the data size setMobThreshold(region, COLUMN_FAMILY, 500); region.initialize(); - + List stores = region.getStores(); for (HStore store: stores) { // Force major compaction store.triggerMajorCompaction(); - Optional context = - store.requestCompaction(HStore.PRIORITY_USER, CompactionLifeCycleTracker.DUMMY, + Optional context = + store.requestCompaction(HStore.PRIORITY_USER, CompactionLifeCycleTracker.DUMMY, User.getCurrent()); if (!context.isPresent()) { continue; } - region.compact(context.get(), store, + region.compact(context.get(), store, NoLimitThroughputController.INSTANCE, User.getCurrent()); } - + assertEquals("After compaction: store files", 1, countStoreFiles()); assertEquals("After compaction: mob file count", compactionThreshold, countMobFiles()); assertEquals("After compaction: referenced mob file count", 0, countReferencedMobFiles()); 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 99% 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 faae36aa74f2..fcafb9868e05 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; @@ -44,8 +44,8 @@ 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; 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 43e3e40e136d..de47d9f9642e 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 @@ -29,8 +29,6 @@ import java.util.Map; import java.util.Optional; 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; @@ -47,7 +45,6 @@ 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; @@ -74,7 +71,6 @@ import org.junit.Before; import org.junit.BeforeClass; import org.junit.ClassRule; -import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; 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 cfed2564b8ea..833e78ba3b0e 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 @@ -505,13 +505,13 @@ public void testMobMetrics() throws IOException, InterruptedException { for (HStore store: stores) { // Force major compaction store.triggerMajorCompaction(); - Optional context = - store.requestCompaction(HStore.PRIORITY_USER, CompactionLifeCycleTracker.DUMMY, + Optional context = + store.requestCompaction(HStore.PRIORITY_USER, CompactionLifeCycleTracker.DUMMY, User.getCurrent()); if (!context.isPresent()) { continue; } - region.compact(context.get(), store, + region.compact(context.get(), store, NoLimitThroughputController.INSTANCE, User.getCurrent()); } metricsRegionServer.getRegionServerWrapper().forceRecompute(); From 1678d72b04795b26c364847686114265c8353eb5 Mon Sep 17 00:00:00 2001 From: Sean Busbey Date: Fri, 31 Jan 2020 14:09:49 -0600 Subject: [PATCH 05/17] HBASE-22749 fix missing import for TestBulkLoadReplication --- .../hadoop/hbase/regionserver/TestBulkLoadReplication.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 de47d9f9642e..9c58c7af965a 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 @@ -58,7 +58,7 @@ import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.RegionObserver; 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.replication.ReplicationPeerConfig; import org.apache.hadoop.hbase.replication.TestReplicationBase; import org.apache.hadoop.hbase.testclassification.MediumTests; From 9365f40193bbcbaf550d3b4ab7d62bd14de7e8b7 Mon Sep 17 00:00:00 2001 From: Vladimir Rodionov Date: Mon, 3 Feb 2020 18:26:22 -0800 Subject: [PATCH 06/17] fixed compilation issue --- .../org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java index 610ccb6abc8f..f191ac294a23 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java @@ -88,7 +88,7 @@ /** * Test mob store compaction */ -@Category(SmallTests.class) +@Category(MediumTests.class) public class TestMobStoreCompaction { @ClassRule From 50df62bfb898fcdfd7e86cae55abd6c3c2d673a7 Mon Sep 17 00:00:00 2001 From: Vladimir Rodionov Date: Tue, 4 Feb 2020 21:24:13 -0800 Subject: [PATCH 07/17] fixed failing mob tests --- .../hadoop/hbase/IntegrationTestMobCompaction.java | 2 +- .../hadoop/hbase/mob/DefaultMobStoreCompactor.java | 12 +++++++----- .../apache/hadoop/hbase/mob/MobFileCleanerChore.java | 2 ++ .../hadoop/hbase/mob/MobFileCompactionChore.java | 4 ++-- .../hadoop/hbase/regionserver/StoreFileInfo.java | 4 ++-- .../hadoop/hbase/mob/TestMobFileCleanerChore.java | 2 +- .../hadoop/hbase/mob/TestMobStoreCompaction.java | 3 --- .../apache/hadoop/hbase/mob/TestMobStoreScanner.java | 2 -- 8 files changed, 15 insertions(+), 16 deletions(-) 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 index 9aa1978745ba..093f9e49816c 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMobCompaction.java @@ -64,7 +64,7 @@ * argument 'failprob'. * @see HBASE-22749 *

    - * Sample usage: + * Sample usage: *

      * hbase org.apache.hadoop.hbase.IntegrationTestMobCompaction -Dservers=10 -Drows=1000000
      * -Dfailprob=0.2
    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 871a9d8e187d..0a0a024ab15f 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
    @@ -37,7 +37,6 @@
     import org.apache.hadoop.hbase.Cell;
     import org.apache.hadoop.hbase.KeyValue;
     import org.apache.hadoop.hbase.PrivateCellUtil;
    -import org.apache.hadoop.hbase.TableName;
     import org.apache.hadoop.hbase.regionserver.CellSink;
     import org.apache.hadoop.hbase.regionserver.HMobStore;
     import org.apache.hadoop.hbase.regionserver.HStore;
    @@ -516,7 +515,6 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
     
         // Commit last MOB writer
         commitOrAbortMobWriter(mobFileWriter, fd.maxSeqId, mobCells, major);
    -    clearThreadLocals();
         mobStore.updateCellsCountCompactedFromMob(cellsCountCompactedFromMob);
         mobStore.updateCellsCountCompactedToMob(cellsCountCompactedToMob);
         mobStore.updateCellsSizeCompactedFromMob(cellsSizeCompactedFromMob);
    @@ -532,12 +530,15 @@ private String getStoreInfo() {
     
       private void clearThreadLocals() {
         Set set = mobRefSet.get();
    -    if (set != null) set.clear();
    +    if (set != null) {
    +      set.clear();
    +    }
         HashMap map = mobLengthMap.get();
    -    if (map != null) map.clear();
    +    if (map != null) {
    +      map.clear();
    +    }
       }
     
    -
       private StoreFileWriter newMobWriter(FileDetails fd)
           throws IOException {
         try {
    @@ -597,6 +598,7 @@ protected List commitWriter(StoreFileWriter writer, FileDetails fd,
         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/MobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerChore.java
    index c7d713dc2e12..955d54937adc 100644
    --- 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
    @@ -235,6 +235,8 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws
                         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) {
    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
    index 82d4a82b0e8c..6f32be4bb259 100644
    --- 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
    @@ -115,8 +115,8 @@ protected void chore() {
                     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");
    +                  " 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 "+
    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 04adeb507196..5eaab23fc6cf 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
    @@ -242,8 +242,8 @@ public void setRegionCoprocessorHost(RegionCoprocessorHost 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;
    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
    index 21d0a64e9592..4db01ded2548 100644
    --- 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
    @@ -197,7 +197,7 @@ private  long getNumberOfMobFiles(Configuration conf, TableName tableName, Strin
         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("DDDD MOB Directory content: {} size={}", st.getPath(), st.getLen());
         }
         LOG.debug("MOB Directory content total files: {}", stat.length);
     
    diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java
    index f191ac294a23..99dd65312d24 100644
    --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java
    +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java
    @@ -59,9 +59,6 @@
     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.regionserver.BloomType;
     import org.apache.hadoop.hbase.regionserver.HRegion;
     import org.apache.hadoop.hbase.regionserver.HStore;
    diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java
    index fcafb9868e05..3fd6daeb931a 100644
    --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java
    +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java
    @@ -43,8 +43,6 @@
     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.MobUtils;
     import org.apache.hadoop.hbase.regionserver.HRegion;
     import org.apache.hadoop.hbase.testclassification.MediumTests;
     import org.apache.hadoop.hbase.util.Bytes;
    
    From ea939c508e73dc8c4d438bcb74c190030c5838a5 Mon Sep 17 00:00:00 2001
    From: Vladimir Rodionov 
    Date: Wed, 5 Feb 2020 13:40:42 -0800
    Subject: [PATCH 08/17] fixed checkstyle, findbugs, whitespaces
    
    ---
     .../src/main/resources/hbase/checkstyle-suppressions.xml  | 1 +
     .../org/apache/hadoop/hbase/mob/MobFileCleanerChore.java  | 4 ++++
     .../org/apache/hadoop/hbase/mob/TestMobStoreScanner.java  | 8 ++++----
     3 files changed, 9 insertions(+), 4 deletions(-)
    
    diff --git a/hbase-checkstyle/src/main/resources/hbase/checkstyle-suppressions.xml b/hbase-checkstyle/src/main/resources/hbase/checkstyle-suppressions.xml
    index 9351ecbfe6a0..2d8e880ddf4a 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-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCleanerChore.java
    index 955d54937adc..80f699efedf1 100644
    --- 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
    @@ -53,6 +53,8 @@
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
    +import edu.umd.cs.findbugs.annotations.SuppressWarnings;
    +
     import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
     
     /**
    @@ -151,6 +153,8 @@ protected void chore() {
        * @param table table name
        * @throws IOException exception
        */
    +  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "UC_USELESS_CONDITION",
    +  justification = "Intentional")
       public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws IOException {
     
         long minAgeToArchive =
    diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java
    index 3fd6daeb931a..1beeffe76de3 100644
    --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java
    +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java
    @@ -139,8 +139,8 @@ public void setScan(Scan scan, boolean reversed, boolean mobScanRaw) {
     
       @Test
       public void testMobStoreScanner() throws Exception {
    -	  testGetFromFiles(false);
    -	  testGetFromMemStore(false);
    +    testGetFromFiles(false);
    +    testGetFromMemStore(false);
         testGetReferences(false);
         testMobThreshold(false);
         testGetFromArchive(false);
    @@ -148,8 +148,8 @@ public void testMobStoreScanner() throws Exception {
     
       @Test
       public void testReversedMobStoreScanner() throws Exception {
    -	  testGetFromFiles(true);
    -	  testGetFromMemStore(true);
    +    testGetFromFiles(true);
    +    testGetFromMemStore(true);
         testGetReferences(true);
         testMobThreshold(true);
         testGetFromArchive(true);
    
    From 2ad64c076dd3844d65507231bb41912f252f23c2 Mon Sep 17 00:00:00 2001
    From: Vladimir Rodionov 
    Date: Wed, 5 Feb 2020 13:57:40 -0800
    Subject: [PATCH 09/17] small findbug fix
    
    ---
     .../hadoop/hbase/mob/MobFileCleanerChore.java    | 16 ++++++++--------
     1 file changed, 8 insertions(+), 8 deletions(-)
    
    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
    index 80f699efedf1..168d8cf92f2c 100644
    --- 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
    @@ -191,10 +191,12 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws
               Path storePath = new Path(regionPath, family);
               boolean succeed = false;
               Set regionMobs = new HashSet();
    +          
               while (!succeed) {
                 if (!fs.exists(storePath)) {
                   LOG.warn("Directory {} was deleted during cleaner procedure execution,"
    -                  +" skipping region {}", storePath, regionPath.getName());
    +                  + " skipping region {}",
    +                storePath, regionPath.getName());
                   continue region;
                 }
                 RemoteIterator rit = fs.listLocatedStatus(storePath);
    @@ -244,18 +246,16 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws
                     }
                   }
                 } catch (FileNotFoundException e) {
    -              String warnMsg =
    -                  String.format("Missing file:%s" +
    -                      " Starting MOB cleaning cycle from the beginning due to error",
    -                      currentPath);
    +              String warnMsg = String.format(
    +                "Missing file:%s" + " Starting MOB cleaning cycle from the beginning due to error",
    +                currentPath);
                   LOG.warn(warnMsg, e);
    +              regionMobs.clear();
                   continue;
                 }
                 succeed = true;
               }
    -          if (!succeed) {
    -            continue; // continue next cf
    -          }
    +          
               // Add MOB references for current region/family
               allActiveMobFileName.addAll(regionMobs);
             } // END column families
    
    From 72dafd14d1130637e591974bcca32b1c660c8db2 Mon Sep 17 00:00:00 2001
    From: Vladimir Rodionov 
    Date: Wed, 5 Feb 2020 16:16:46 -0800
    Subject: [PATCH 10/17] small findbug fix again
    
    ---
     .../java/org/apache/hadoop/hbase/mob/MobFileCleanerChore.java | 4 +---
     1 file changed, 1 insertion(+), 3 deletions(-)
    
    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
    index 168d8cf92f2c..0688847a9020 100644
    --- 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
    @@ -53,8 +53,6 @@
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
    -import edu.umd.cs.findbugs.annotations.SuppressWarnings;
    -
     import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
     
     /**
    @@ -154,7 +152,7 @@ protected void chore() {
        * @throws IOException exception
        */
       @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "UC_USELESS_CONDITION",
    -  justification = "Intentional")
    +      justification = "Intentional")
       public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws IOException {
     
         long minAgeToArchive =
    
    From 29fcdc6b6961f317b4c0bc91509d0a8e977dbd6f Mon Sep 17 00:00:00 2001
    From: Vladimir Rodionov 
    Date: Wed, 5 Feb 2020 19:30:21 -0800
    Subject: [PATCH 11/17] small findbug fix again
    
    ---
     .../java/org/apache/hadoop/hbase/mob/MobFileCleanerChore.java | 4 ++--
     1 file changed, 2 insertions(+), 2 deletions(-)
    
    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
    index 0688847a9020..37ece18aa3a1 100644
    --- 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
    @@ -189,7 +189,7 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws
               Path storePath = new Path(regionPath, family);
               boolean succeed = false;
               Set regionMobs = new HashSet();
    -          
    +
               while (!succeed) {
                 if (!fs.exists(storePath)) {
                   LOG.warn("Directory {} was deleted during cleaner procedure execution,"
    @@ -253,7 +253,7 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws
                 }
                 succeed = true;
               }
    -          
    +
               // Add MOB references for current region/family
               allActiveMobFileName.addAll(regionMobs);
             } // END column families
    
    From 064aa05a26bbc78fca5a7f65513c7cb91468b5c3 Mon Sep 17 00:00:00 2001
    From: Vladimir Rodionov 
    Date: Wed, 12 Feb 2020 12:30:00 -0800
    Subject: [PATCH 12/17] addressed review comments
    
    ---
     .../src/main/resources/hbase-default.xml      | 49 +++----------------
     .../hbase/mob/DefaultMobStoreCompactor.java   | 35 +++++++------
     .../hadoop/hbase/mob/MobFileCleanerChore.java | 25 +++-------
     .../hbase/mob/MobFileCompactionChore.java     | 28 ++++-------
     .../hadoop/hbase/mob/MobStoreEngine.java      |  2 +-
     .../hbase/mob/FaultyMobStoreCompactor.java    | 21 ++++----
     .../hadoop/hbase/mob/MobStressToolRunner.java | 13 +----
     .../hbase/mob/TestMobCompactionOptMode.java   |  3 --
     .../TestMobCompactionOptRegionBatchMode.java  |  2 -
     .../mob/TestMobCompactionRegularMode.java     |  3 --
     ...stMobCompactionRegularRegionBatchMode.java |  2 -
     .../hbase/mob/TestMobFileCleanerChore.java    |  5 --
     12 files changed, 56 insertions(+), 132 deletions(-)
    
    diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml
    index cafb12c0e40f..6c2d3f4472ed 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
    -    
    -      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
    +    hbase.mob.major.compaction.region.batch.size
    +    0
         
    -      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-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 0a0a024ab15f..06ea6fd2894b 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
    @@ -168,7 +168,7 @@ public List compact(CompactionRequestImpl request,
         } else {
           userRequest.set(Boolean.FALSE);
         }
    -    LOG.debug("MOB compaction table={} cf={} region={} files: ", tableName, familyName,
    +    LOG.debug("MOB compaction table={} cf={} region={} files: {}", tableName, familyName,
           regionName, request.getFiles());
         // Check if I/O optimized MOB compaction
         if (ioOptimizedMode) {
    @@ -188,7 +188,7 @@ public List compact(CompactionRequestImpl request,
       }
     
       private void calculateMobLengthMap(List mobFiles) throws IOException {
    -    FileSystem fs = mobFiles.get(0).getFileSystem(this.conf);
    +    FileSystem fs = store.getFileSystem();
         HashMap map = mobLengthMap.get();
         map.clear();
         for (Path p : mobFiles) {
    @@ -248,15 +248,15 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
         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"+
    -        " during upgrade process from MOB 1.0 to MOB 2.0 versions.",
    -        MobConstants.MOB_UNSAFE_DISCARD_MISS_KEY);
    +      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);
         LOG.info("Compact MOB={} optimized={} maximum MOB file size={} major={} store={}", compactMOBs,
           ioOptimizedMode, maxMobFileSize, major, getStoreInfo());
    -    FileSystem fs = FileSystem.get(conf);
    +    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<>();
    @@ -300,8 +300,6 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
               if (compactMOBs) {
                 if (MobUtils.isMobReferenceCell(c)) {
                   String fName = MobUtils.getMobFileName(c);
    -              Path pp = new Path(new Path(fs.getUri()), new Path(path, fName));
    -
                   // Added to support migration
                   try {
                     mobCell = mobStore.resolve(c, true, false).getCell();
    @@ -315,11 +313,13 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
                   }
     
                   if (discardMobMiss && mobCell.getValueLength() == 0) {
    -                LOG.error("Missing MOB cell value: file={} cell={}", pp, mobCell);
    +                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 cell=%s",
    -                  fName, mobCell);
    +                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);
                   }
     
    @@ -337,9 +337,11 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
                       // greater than threshold
                       Long size = mobLengthMap.get().get(fName);
                       if (size == null) {
    -                    // FATAL error, abort compaction
    +                    // 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 unreferenced MOB file during compaction %s, aborting compaction %s",
    +                      "Found an unexpected MOB file during compaction %s, aborting compaction %s",
                           fName, getStoreInfo());
                         throw new IOException(msg);
                       }
    @@ -356,8 +358,8 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
                         // file compression yet)
                         long len = mobFileWriter.getPos();
                         if (len > maxMobFileSize) {
    -                      LOG.debug("Closing output MOB File, length={} file={}, store=", len,
    -                        Bytes.toString(fileName), getStoreInfo());
    +                      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());
    @@ -544,7 +546,8 @@ private StoreFileWriter newMobWriter(FileDetails fd)
         try {
           StoreFileWriter mobFileWriter = mobStore.createWriterInTmp(new Date(fd.latestPutTs),
             fd.maxKeyCount, compactionCompression, store.getRegionInfo().getStartKey(), true);
    -      LOG.debug("New MOB writer created={}", mobFileWriter.getPath().getName());
    +      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;
    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
    index 37ece18aa3a1..76b637471728 100644
    --- 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
    @@ -107,9 +107,6 @@ public MobFileCleanerChore() {
       }
     
       @Override
    -  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "REC_CATCH_EXCEPTION",
    -      justification = "Intentional")
    -
       protected void chore() {
         TableDescriptors htds = master.getTableDescriptors();
     
    @@ -126,9 +123,8 @@ protected void chore() {
               try {
                 cleaner.cleanExpiredMobFiles(htd.getTableName().getNameAsString(), hcd);
               } catch (IOException e) {
    -            String errMsg = String.format("Failed to clean the expired mob files table=%s" +
    -                " family=%s", htd.getTableName().getNameAsString(), hcd.getNameAsString());
    -            LOG.error(errMsg, e);
    +            LOG.error("Failed to clean the expired mob files table={} family={}",
    +              htd.getTableName().getNameAsString(), hcd.getNameAsString(), e);
               }
             }
           }
    @@ -138,9 +134,7 @@ protected void chore() {
             cleanupObsoleteMobFiles(master.getConfiguration(), htd.getTableName());
             LOG.info("Cleaning obsolete MOB files finished for table={}", htd.getTableName());
           } catch (IOException e) {
    -        String errMsg =
    -            String.format("Failed to clean the obsolete mob files for table=",htd.getTableName());
    -        LOG.error(errMsg, e);
    +        LOG.error("Failed to clean the obsolete mob files for table={}",htd.getTableName(), e);
           }
         }
       }
    @@ -151,8 +145,6 @@ protected void chore() {
        * @param table table name
        * @throws IOException exception
        */
    -  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "UC_USELESS_CONDITION",
    -      justification = "Intentional")
       public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws IOException {
     
         long minAgeToArchive =
    @@ -164,8 +156,6 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws
         // 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;
    -    LOG.info("Only MOB files whose creation time older than {} will be archived, table={}",
    -      maxCreationTimeToArchive, table);
         try (final Connection conn = ConnectionFactory.createConnection(conf);
             final Admin admin = conn.getAdmin();) {
           TableDescriptor htd = admin.getDescriptor(table);
    @@ -173,6 +163,9 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws
           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);
    @@ -244,10 +237,8 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws
                     }
                   }
                 } catch (FileNotFoundException e) {
    -              String warnMsg = String.format(
    -                "Missing file:%s" + " Starting MOB cleaning cycle from the beginning due to error",
    -                currentPath);
    -              LOG.warn(warnMsg, e);
    +              LOG.warn("Missing file:{} Starting MOB cleaning cycle from the beginning"+
    +                " due to error", currentPath, e);
                   regionMobs.clear();
                   continue;
                 }
    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
    index 6f32be4bb259..314729833959 100644
    --- 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
    @@ -32,7 +32,6 @@
     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.ConnectionFactory;
     import org.apache.hadoop.hbase.client.RegionInfo;
     import org.apache.hadoop.hbase.client.TableDescriptor;
     import org.apache.hadoop.hbase.client.TableState;
    @@ -59,7 +58,6 @@
     public class MobFileCompactionChore extends ScheduledChore {
     
       private static final Logger LOG = LoggerFactory.getLogger(MobFileCompactionChore.class);
    -  private Configuration conf;
       private HMaster master;
       private int regionBatchSize = 0;// not set - compact all
     
    @@ -71,7 +69,6 @@ public MobFileCompactionChore(HMaster master) {
               MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD),
             TimeUnit.SECONDS);
         this.master = master;
    -    this.conf = master.getConfiguration();
         this.regionBatchSize =
             master.getConfiguration().getInt(MobConstants.MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE,
               MobConstants.DEFAULT_MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE);
    @@ -80,7 +77,6 @@ public MobFileCompactionChore(HMaster master) {
     
       @VisibleForTesting
       public MobFileCompactionChore(Configuration conf, int batchSize) {
    -    this.conf = conf;
         this.regionBatchSize = batchSize;
       }
     
    @@ -89,7 +85,7 @@ protected void chore() {
     
         boolean reported = false;
     
    -    try (Connection conn = ConnectionFactory.createConnection(conf);
    +    try (Connection conn = master.getConnection();
             Admin admin = conn.getAdmin();) {
     
           TableDescriptors htds = master.getTableDescriptors();
    @@ -125,19 +121,17 @@ protected void chore() {
                     performMajorCompactionInBatches(admin, htd, hcd);
                   }
                 } else {
    -              LOG.info("Skipping table={} column family={} because it is not MOB-enabled",
    +              LOG.debug("Skipping table={} column family={} because it is not MOB-enabled",
                     htd.getTableName(), hcd.getNameAsString());
                 }
               } catch (IOException e) {
    -            String errMsg = String.format("Failed to compact table=%s cf=%s",
    -              htd.getTableName(), hcd.getNameAsString());
    -            LOG.error(errMsg, e);
    +            LOG.error("Failed to compact table={} cf={}",
    +              htd.getTableName(), hcd.getNameAsString(), e);
               } catch (InterruptedException ee) {
                 Thread.currentThread().interrupt();
                 master.reportMobCompactionEnd(htd.getTableName());
    -            String warnMsg = String.format("Failed to compact table=%s cf=%s",
    -              htd.getTableName(), hcd.getNameAsString());
    -            LOG.warn(warnMsg, ee);
    +            LOG.warn("Failed to compact table={} cf={}",
    +              htd.getTableName(), hcd.getNameAsString(), ee);
                 // Quit the chore
                 return;
               }
    @@ -168,7 +162,7 @@ public void performMajorCompactionInBatches(Admin admin, TableDescriptor htd,
         // Shuffle list of regions in case if they come ordered by region server
         Collections.shuffle(regions);
         // Create first batch
    -    List toCompact = new ArrayList(regions.size());
    +    List toCompact = new ArrayList(this.regionBatchSize);
         for (int i = 0; i < this.regionBatchSize; i++) {
           toCompact.add(regions.remove(0));
         }
    @@ -188,7 +182,7 @@ public void performMajorCompactionInBatches(Admin admin, TableDescriptor htd,
               if (admin.getCompactionStateForRegion(ri.getRegionName()) == CompactionState.NONE) {
                 totalCompacted++;
                 LOG.info(
    -              "Finished major MOB compaction: table={} cf={} region={}," + " compacted regions={}",
    +              "Finished major MOB compaction: table={} cf={} region={} compacted regions={}",
                   htd.getTableName(), hcd.getNameAsString(), ri.getRegionNameAsString(),
                   totalCompacted);
                 compacted.add(ri);
    @@ -203,17 +197,15 @@ public void performMajorCompactionInBatches(Admin admin, TableDescriptor htd,
           }
           // Remove failed regions to avoid
           // endless compaction loop
    -      for(RegionInfo ri: failed) {
    -        toCompact.remove(ri);
    -      }
    +      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);
    -          startCompaction(admin, htd.getTableName(), region, hcd.getName());
               toCompact.add(region);
    +          startCompaction(admin, htd.getTableName(), region, hcd.getName());
             }
           }
           compacted.clear();
    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 86fffbf94f0a..331ecdb1b176 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
    @@ -49,7 +49,7 @@ protected void createCompactor(Configuration conf, HStore store) throws IOExcept
         try {
           compactor = ReflectionUtils.instantiateWithCustomCtor(className,
             new Class[] { Configuration.class, HStore.class }, new Object[] { conf, store });
    -    } catch (Exception e) {
    +    } catch (RuntimeException e) {
           throw new IOException("Unable to load configured compactor '" + className + "'", e);
         }
       }
    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
    index 1dd447cddc2f..02ace93d6679 100644
    --- 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
    @@ -113,7 +113,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
           }
         }
     
    -    FileSystem fs = FileSystem.get(conf);
    +    FileSystem fs = store.getFileSystem();
     
         // Since scanner.next() can return 'false' but still be delivering data,
         // we have to use a do/while loop.
    @@ -152,7 +152,6 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
     
         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());
    @@ -177,18 +176,16 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
               if (compactMOBs) {
                 if (MobUtils.isMobReferenceCell(c)) {
                   if (counter == countFailAt) {
    -                LOG.warn("\n\n INJECTED FAULT mobCounter=" + mobCounter.get() + "\n\n");
    +                LOG.warn("INJECTED FAULT mobCounter={}", mobCounter.get());
                     throw new CorruptHFileException("injected fault");
                   }
                   String fName = MobUtils.getMobFileName(c);
    -              Path pp = new Path(new Path(fs.getUri()), new Path(path, fName));
    -
                   // Added to support migration
                   try {
                     mobCell = mobStore.resolve(c, true, false).getCell();
                   } catch (FileNotFoundException fnfe) {
                     if (discardMobMiss) {
    -                  LOG.error("Missing MOB cell: file=" + pp + " not found");
    +                  LOG.error("Missing MOB cell: file={} not found", fName);
                       continue;
                     } else {
                       throw fnfe;
    @@ -196,7 +193,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
                   }
     
                   if (discardMobMiss && mobCell.getValueLength() == 0) {
    -                LOG.error("Missing MOB cell value: file=" + pp + " cell=" + mobCell);
    +                LOG.error("Missing MOB cell value: file={} cell={}", fName, mobCell);
                     continue;
                   }
     
    @@ -206,15 +203,14 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
                     mobFileWriter.append(mobCell);
                     writer.append(
                       MobUtils.createMobRefCell(mobCell, fileName, this.mobStore.getRefCellTags()));
    -                cellsCountCompactedFromMob++;
    -                cellsSizeCompactedFromMob += mobCell.getValueLength();
                     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();
    @@ -223,6 +219,8 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
                     writer
                         .append(MobUtils.createMobRefCell(c, fileName, this.mobStore.getRefCellTags()));
                     mobCells++;
    +                cellsCountCompactedToMob++;
    +                cellsSizeCompactedToMob += c.getValueLength();
                   } else {
                     writer.append(c);
                   }
    @@ -263,8 +261,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
                     }
                   }
                 } else {
    -              // TODO ????
    -              LOG.error("Corrupted MOB reference: " + c);
    +              LOG.error("Corrupted MOB reference: {}", c);
                   writer.append(c);
                 }
               } else if (c.getValueLength() <= mobSizeThreshold) {
    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
    index 5295ec2e1b7d..a38e83235642 100644
    --- 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
    @@ -92,7 +92,7 @@ public void init(Configuration conf, long numRows) throws IOException {
         this.count = numRows;
         initConf();
         printConf();
    -    hdt = createTableDescriptor("testMobCompactTable");
    +    hdt = new HTableDescriptor(TableName.valueOf("testMobCompactTable"));
         Connection conn = ConnectionFactory.createConnection(this.conf);
         this.admin = conn.getAdmin();
         this.hcd = new HColumnDescriptor(fam);
    @@ -124,17 +124,6 @@ private void printConf() {
     
       }
     
    -  private HTableDescriptor createTableDescriptor(final String name, final int minVersions,
    -      final int versions, final int ttl, KeepDeletedCells keepDeleted) {
    -    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(name));
    -    return htd;
    -  }
    -
    -  private HTableDescriptor createTableDescriptor(final String name) {
    -    return createTableDescriptor(name, HColumnDescriptor.DEFAULT_MIN_VERSIONS, 1,
    -      HConstants.FOREVER, HColumnDescriptor.DEFAULT_KEEP_DELETED);
    -  }
    -
       private void initConf() {
     
         conf.setInt("hfile.format.version", 3);
    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
    index 4508a3887b66..4a7da05a63ec 100644
    --- 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
    @@ -55,9 +55,6 @@ public class TestMobCompactionOptMode extends TestMobCompactionBase{
       @ClassRule
       public static final HBaseClassTestRule CLASS_RULE =
           HBaseClassTestRule.forClass(TestMobCompactionOptMode.class);
    -  @Rule
    -  public TestName testName = new TestName();
    -
     
       public TestMobCompactionOptMode() {
       }
    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
    index d354ab03530e..0133bf2bcca8 100644
    --- 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
    @@ -57,8 +57,6 @@ public class TestMobCompactionOptRegionBatchMode extends TestMobCompactionBase{
       @ClassRule
       public static final HBaseClassTestRule CLASS_RULE =
           HBaseClassTestRule.forClass(TestMobCompactionOptRegionBatchMode.class);
    -  @Rule
    -  public TestName testName = new TestName();
     
       private int batchSize = 7;
       private MobFileCompactionChore compactionChore;
    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
    index cec57e8a22a9..c08249c6be29 100644
    --- 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
    @@ -55,9 +55,6 @@ public class TestMobCompactionRegularMode extends TestMobCompactionBase{
       @ClassRule
       public static final HBaseClassTestRule CLASS_RULE =
           HBaseClassTestRule.forClass(TestMobCompactionRegularMode.class);
    -  @Rule
    -  public TestName testName = new TestName();
    -
     
       public TestMobCompactionRegularMode() {
       }
    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
    index f6c984af2ebd..d97245d549b1 100644
    --- 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
    @@ -57,8 +57,6 @@ public class TestMobCompactionRegularRegionBatchMode extends TestMobCompactionBa
       @ClassRule
       public static final HBaseClassTestRule CLASS_RULE =
           HBaseClassTestRule.forClass(TestMobCompactionRegularRegionBatchMode.class);
    -  @Rule
    -  public TestName testName = new TestName();
     
       private int batchSize = 7;
       private MobFileCompactionChore compactionChore;
    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
    index 4db01ded2548..48858265f5fc 100644
    --- 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
    @@ -68,8 +68,6 @@ public class TestMobFileCleanerChore {
       @ClassRule
       public static final HBaseClassTestRule CLASS_RULE =
           HBaseClassTestRule.forClass(TestMobFileCleanerChore.class);
    -  @Rule
    -  public TestName testName = new TestName();
     
       private HBaseTestingUtility HTU;
     
    @@ -186,9 +184,6 @@ public void testMobFileCleanerChore() throws InterruptedException, IOException {
     
         long scanned = scanTable();
         assertEquals(30, scanned);
    -
    -    LOG.info("MOB Stress Test finished OK");
    -
       }
     
       private  long getNumberOfMobFiles(Configuration conf, TableName tableName, String family)
    
    From 54cce910d7ea6ce372a3fe512a572d6b29cbb4c7 Mon Sep 17 00:00:00 2001
    From: Vladimir Rodionov 
    Date: Thu, 13 Feb 2020 10:40:38 -0800
    Subject: [PATCH 13/17] fixed checkstyle and findbugs
    
    ---
     .../org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java   | 2 --
     .../java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java   | 2 --
     .../org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java   | 2 --
     .../hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java   | 2 --
     .../apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java   | 2 --
     .../hbase/mob/TestMobCompactionRegularRegionBatchMode.java      | 2 --
     .../org/apache/hadoop/hbase/mob/TestMobFileCleanerChore.java    | 2 --
     7 files changed, 14 deletions(-)
    
    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 06ea6fd2894b..661c57f70c1d 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
    @@ -256,7 +256,6 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
           MobConstants.DEFAULT_MOB_COMPACTION_MAX_FILE_SIZE);
         LOG.info("Compact MOB={} optimized={} maximum MOB file size={} major={} store={}", compactMOBs,
           ioOptimizedMode, maxMobFileSize, major, getStoreInfo());
    -    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<>();
    @@ -269,7 +268,6 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
         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;
         /*
    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
    index a38e83235642..011fcadbf862 100644
    --- 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
    @@ -29,9 +29,7 @@
     import org.apache.hadoop.fs.Path;
     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.KeepDeletedCells;
     import org.apache.hadoop.hbase.TableName;
     import org.apache.hadoop.hbase.client.Admin;
     import org.apache.hadoop.hbase.client.Connection;
    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
    index 4a7da05a63ec..d4f13f6cb75d 100644
    --- 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
    @@ -25,10 +25,8 @@
     import org.apache.hadoop.hbase.client.Admin;
     import org.apache.hadoop.hbase.testclassification.LargeTests;
     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;
     
    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
    index 0133bf2bcca8..9871b9667115 100644
    --- 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
    @@ -26,10 +26,8 @@
     import org.apache.hadoop.hbase.testclassification.LargeTests;
     import org.junit.Before;
     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;
     
    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
    index c08249c6be29..9145655a23b3 100644
    --- 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
    @@ -25,10 +25,8 @@
     import org.apache.hadoop.hbase.client.Admin;
     import org.apache.hadoop.hbase.testclassification.LargeTests;
     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;
     
    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
    index d97245d549b1..de1fecd46897 100644
    --- 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
    @@ -26,10 +26,8 @@
     import org.apache.hadoop.hbase.testclassification.LargeTests;
     import org.junit.Before;
     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;
     
    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
    index 48858265f5fc..9e9970dfb19e 100644
    --- 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
    @@ -44,10 +44,8 @@
     import org.junit.After;
     import org.junit.Before;
     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;
     
    
    From 42bd6832ddfef1b59bf3b8a132557c83ec003682 Mon Sep 17 00:00:00 2001
    From: Vladimir Rodionov 
    Date: Fri, 14 Feb 2020 11:18:31 -0800
    Subject: [PATCH 14/17] HBASE-23840: revert IO optiized mode back to general
     during migration
    
    ---
     .../hbase/mob/DefaultMobStoreCompactor.java   | 28 +++++++++++++++++--
     .../apache/hadoop/hbase/mob/MobFileName.java  |  8 +++++-
     2 files changed, 33 insertions(+), 3 deletions(-)
    
    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 661c57f70c1d..53d9479df377 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
    @@ -98,6 +98,23 @@ protected Boolean initialValue() {
         }
       };
     
    +  
    +  /*
    +   * 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.
        */
    @@ -176,6 +193,8 @@ public List compact(CompactionRequestImpl request,
             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);
             }
    @@ -192,6 +211,9 @@ private void calculateMobLengthMap(List mobFiles) throws IOException {
         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);
    @@ -254,8 +276,10 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
         }
         long maxMobFileSize = conf.getLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY,
           MobConstants.DEFAULT_MOB_COMPACTION_MAX_FILE_SIZE);
    -    LOG.info("Compact MOB={} optimized={} maximum MOB file size={} major={} store={}", compactMOBs,
    -      ioOptimizedMode, maxMobFileSize, major, getStoreInfo());
    +    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<>();
    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 64cfa4dfcafa..00668d56c478 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
    @@ -40,7 +40,7 @@
      * 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;
    @@ -124,6 +124,7 @@ public static MobFileName create(String startKey, String date, String uuid, Stri
        */
       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);
    @@ -132,6 +133,11 @@ public static MobFileName create(String fileName) {
         return new MobFileName(startKey, date, uuid, regionName);
       }
     
    +  
    +  public static boolean isOldMobFileName(String name) {
    +    return name.indexOf(REGION_SEP) < 0;
    +  }
    +  
       /**
        * get startKey from MobFileName.
        * @param fileName file name.
    
    From 8ee22c2eea3a0b423532be4aaffec7a7814870d6 Mon Sep 17 00:00:00 2001
    From: Vladimir Rodionov 
    Date: Fri, 14 Feb 2020 11:31:40 -0800
    Subject: [PATCH 15/17] Abort MOB file cleaner chore in case of CF failure
    
    ---
     .../apache/hadoop/hbase/mob/MobFileCleanerChore.java | 12 ++++++------
     1 file changed, 6 insertions(+), 6 deletions(-)
    
    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
    index 76b637471728..65883817cb04 100644
    --- 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
    @@ -72,7 +72,7 @@ public class MobFileCleanerChore extends ScheduledChore {
       }
     
       public MobFileCleanerChore(HMaster master) {
    -    super(master.getServerName() + "-ExpiredMobFileCleanerChore", 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,
    @@ -176,7 +176,6 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws
           Set allActiveMobFileName = new HashSet();
           FileSystem fs = FileSystem.get(conf);
           for (Path regionPath : regionDirs) {
    -        region:
             for (ColumnFamilyDescriptor hcd : list) {
               String family = hcd.getNameAsString();
               Path storePath = new Path(regionPath, family);
    @@ -185,10 +184,11 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws
     
               while (!succeed) {
                 if (!fs.exists(storePath)) {
    -              LOG.warn("Directory {} was deleted during cleaner procedure execution,"
    -                  + " skipping region {}",
    -                storePath, regionPath.getName());
    -              continue region;
    +              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();
    
    From e11ec68e057b955d7aa9c4bf12c3e7bee25e45e4 Mon Sep 17 00:00:00 2001
    From: Vladimir Rodionov 
    Date: Fri, 14 Feb 2020 16:09:15 -0800
    Subject: [PATCH 16/17] whitespace checkstyle
    
    ---
     .../hadoop/hbase/mob/DefaultMobStoreCompactor.java | 14 +++++++-------
     .../org/apache/hadoop/hbase/mob/MobFileName.java   |  3 +--
     2 files changed, 8 insertions(+), 9 deletions(-)
    
    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 53d9479df377..391522f003a6 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
    @@ -98,15 +98,15 @@ protected Boolean initialValue() {
         }
       };
     
    -  
    +
       /*
        * Disable IO mode. IO mode can be forcefully disabled if compactor finds
    -   * old MOB file (pre-distributed compaction). This means that migration has not 
    +   * 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
    @@ -114,7 +114,7 @@ protected Boolean initialValue() {
           return Boolean.FALSE;
         }    
       };
    -  
    +
       /*
        * Map : MOB file name - file length Can be expensive for large amount of MOB files.
        */
    @@ -277,8 +277,8 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
         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,
    +    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.
    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 00668d56c478..73bb07a90d49 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
    @@ -133,11 +133,10 @@ public static MobFileName create(String fileName) {
         return new MobFileName(startKey, date, uuid, regionName);
       }
     
    -  
       public static boolean isOldMobFileName(String name) {
         return name.indexOf(REGION_SEP) < 0;
       }
    -  
    +
       /**
        * get startKey from MobFileName.
        * @param fileName file name.
    
    From e02bc4017ad213e22ad14cc20711fb4e34915fc5 Mon Sep 17 00:00:00 2001
    From: Vladimir Rodionov 
    Date: Fri, 14 Feb 2020 20:32:43 -0800
    Subject: [PATCH 17/17] whitespace checkstyle
    
    ---
     .../org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java   | 2 +-
     .../java/org/apache/hadoop/hbase/mob/MobFileCleanerChore.java   | 2 +-
     2 files changed, 2 insertions(+), 2 deletions(-)
    
    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 391522f003a6..c5ef6a6b992b 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
    @@ -112,7 +112,7 @@ protected Boolean initialValue() {
         @Override
         protected Boolean initialValue() {
           return Boolean.FALSE;
    -    }    
    +    }
       };
     
       /*
    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
    index 65883817cb04..7f7d90c5f64b 100644
    --- 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
    @@ -184,7 +184,7 @@ public void cleanupObsoleteMobFiles(Configuration conf, TableName table) throws
     
               while (!succeed) {
                 if (!fs.exists(storePath)) {
    -              String errMsg = 
    +              String errMsg =
                       String.format("Directory %s was deleted during MOB file cleaner chore"
                       + " execution, aborting MOB file cleaner chore.",
                     storePath);