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..b4812b997b11 --- /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 93cca8bd3623..5e48205c173e 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 @@ -227,6 +227,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 98841f95c838..9a1ae8e93911 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 @@ -395,9 +395,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. @@ -1300,14 +1299,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"); + } } /** @@ -1495,9 +1498,7 @@ protected void stopServiceThreads() { } } stopChores(); - if (this.mobCompactThread != null) { - this.mobCompactThread.close(); - } + super.stopServiceThreads(); if (cleanerPool != null) { cleanerPool.shutdownNow(); @@ -1618,8 +1619,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); @@ -3444,17 +3445,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, fs, tableName, columns, allFiles); - } - /** * Queries the state of the {@link LoadBalancerTracker}. If the balancer is not initialized, * false is returned. 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 06a99fa5432d..e53afa1aea24 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.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; @@ -103,23 +103,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; @@ -342,6 +325,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. @@ -1744,10 +1742,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); } @@ -1790,57 +1791,6 @@ public GetRegionInfoResponse getRegionInfo(final RpcController controller, } } - /** - * 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..718bc0f21b03 --- /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..02559da75fa0 --- /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..cc28a8cd9124 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,91 @@ 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..08fb2c69a6a8 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); @@ -203,8 +214,7 @@ protected void performMobFlush(MemStoreSnapshot snapshot, long cacheFlushId, // append the original keyValue in the mob file. mobFileWriter.append(c); mobSize += c.getValueLength(); - mobCount++; - + 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..dc0c0808b2c1 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; + + /** + * Use this configuration option with caution, only during upgrade procedure + * to handle missing MOB cells during compaction. + */ + public static final String MOB_UNSAFE_DISCARD_MISS_KEY = "hbase.unsafe.mob.discard.miss"; + + public static final boolean DEFAULT_MOB_DISCARD_MISS = false; + /** - * The max number of threads used in MobCompactor. + * Minimum MOB file age to archive, default (3600000 - 1h) */ - 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 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 304a62e8901b..2356b927fa12 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,13 @@ 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; @@ -50,7 +47,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; @@ -62,10 +58,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; @@ -74,8 +66,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; @@ -87,23 +77,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. @@ -130,45 +112,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. @@ -216,11 +159,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. */ @@ -234,9 +176,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. */ @@ -250,8 +192,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. */ @@ -266,22 +208,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. @@ -334,11 +271,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)); @@ -349,19 +286,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. */ @@ -371,8 +306,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. */ @@ -394,8 +328,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. @@ -405,8 +339,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. @@ -416,8 +350,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. @@ -429,8 +363,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. @@ -441,8 +375,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. @@ -452,19 +386,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(); } /** @@ -473,8 +402,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()); } /** @@ -487,16 +417,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. @@ -510,16 +430,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) { @@ -561,92 +483,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. - * @throws IOException - */ - 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. - * @throws IOException - */ - 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. - * @throws IOException - */ - 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. @@ -663,14 +507,12 @@ public static StoreFileWriter createDelFileWriter(Configuration conf, FileSystem * @throws IOException */ public static StoreFileWriter createWriter(Configuration conf, FileSystem fs, - ColumnFamilyDescriptor family, MobFileName mobFileName, Path basePath, long maxKeyCount, + ColumnFamilyDescriptor family, MobFileName mobFileName, Path basePath, long maxKeyCount, Compression.Algorithm compression, CacheConfig cacheConfig, Encryption.Context cryptoContext, - 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); } /** @@ -692,10 +534,9 @@ public static StoreFileWriter createWriter(Configuration conf, FileSystem fs, * @throws IOException */ 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; @@ -708,80 +549,23 @@ 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) - .withFilePath(path) + StoreFileWriter w = new StoreFileWriter.Builder(conf, writerCacheConf, fs).withFilePath(path) .withComparator(CellComparator.getInstance()).withBloomType(bloomType) .withMaxKeyCount(maxKeyCount).withFileContext(hFileContext).build(); return w; } - /** - * 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. - * @throws IOException - */ - 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. */ @@ -790,11 +574,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. */ @@ -803,93 +585,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(); - 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 { - 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); } /** @@ -907,6 +611,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. @@ -915,7 +636,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) { @@ -923,15 +644,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. @@ -939,7 +651,8 @@ public static Cell createMobRefDeleteMarker(Cell cell) { * @param fileDate The date string parsed from the mob file name. * @return True if the mob file is expired. */ - public static boolean isMobFileExpired(ColumnFamilyDescriptor column, long current, String fileDate) { + public static boolean isMobFileExpired(ColumnFamilyDescriptor column, long current, + String fileDate) { if (column.getMinVersions() > 0) { return false; } @@ -961,87 +674,43 @@ public static boolean isMobFileExpired(ColumnFamilyDescriptor column, long curre } return false; } - + /** - * 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; - } - - 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; + public static String getEncodedRegionName(String mobFileName) { + int index = mobFileName.lastIndexOf(MobFileName.REGION_SEP); + if (index < 0) { + return null; } + return mobFileName.substring(index+1); + } - /* 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); + /** + * Get list of referenced MOB files from a given collection + * of store files + * @param storeFiles store files + * @param mobDir MOB file directory + * @return list of MOB file paths + */ + + public static 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; } -} +} \ No newline at end of file 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 a5823ec7d7bb..000000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java +++ /dev/null @@ -1,929 +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 " + request.getCompactionType() + ", the request has " - + totalDelFiles + " del files, " + selectedFileCount + " selected files, and " - + irrelevantFileCount + " irrelevant files"); - 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 " + totalDelFileCount + " del files"); - // compact the mob files by partitions. - paths = compactMobFiles(request); - LOG.info("After compaction, there are " + paths.size() + " mob files"); - } 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 (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 " + delPartition.getStoreFiles(), 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"); - 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 " + partition.getPartitionId()); - 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 " + result.getKey(), 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); - } - } 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 " + files.size() - + " to " + newFiles.size()); - 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 - bulkloadRefFile(table.getName(), bulkloadPathOfPartition, filePath.getName()); - cleanupCommittedMobFile = false; - 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) { - deletePath(new Path(mobFamilyDir, filePath.getName())); - } - } - } - - /** - * 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.create(conf).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) { - 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 b5e64673146f..5138e05ea1a1 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 @@ -1905,7 +1905,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 c14e3a8a3b7e..413c43a1e0fb 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 @@ -110,6 +110,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"); 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 578f80688560..7af748ce38fe 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 @@ -30,15 +30,16 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HDFSBlocksDistribution; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.io.HalfStoreFileReader; import org.apache.hadoop.hbase.io.Reference; import org.apache.hadoop.hbase.io.hfile.CacheConfig; +import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Describe a StoreFile (hfile, reference, link) @@ -141,7 +142,7 @@ private StoreFileInfo(final Configuration conf, final FileSystem fs, } if (LOG.isTraceEnabled()) LOG.trace(p + " is a " + reference.getFileRegion() + " reference to " + referencePath); - } else if (isHFile(p)) { + } else if (isHFile(p) || isMobFile(p) || isMobRefFile(p)) { // HFile if (fileStatus != null) { this.createdTimestamp = fileStatus.getModificationTime(); @@ -436,6 +437,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 7b4fb4a0c06f..cc010ebcbb74 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; @@ -50,6 +52,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; @@ -169,6 +172,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. @@ -242,6 +248,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 503325a1a374..3eab002dba5a 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 b30d3499f212..a9be1bae730d 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 @@ -1126,7 +1126,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..4c754e4d57e8 --- /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..2b83310011a5 --- /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..6b7302819e22 --- /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..773a5e182b83 --- /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..fdb7ed9f6cc3 --- /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..fd451daee933 --- /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..017eb545cc41 --- /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..0903a635d998 --- /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 130a6d31cba6..000000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java +++ /dev/null @@ -1,1202 +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.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.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 HTableDescriptor desc; - private HColumnDescriptor hcd1; - private HColumnDescriptor hcd2; - 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); - hcd1 = new HColumnDescriptor(family1); - hcd1.setMobEnabled(true); - hcd1.setMobThreshold(5); - hcd2 = new HColumnDescriptor(family2); - hcd2.setMobEnabled(true); - hcd2.setMobThreshold(5); - desc = new HTableDescriptor(tableName); - desc.addFamily(hcd1); - desc.addFamily(hcd2); - admin.createTable(desc, 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); - hcd1 = new HColumnDescriptor(family1); - hcd1.setMobEnabled(true); - hcd1.setMobThreshold(10); - hcd1.setMobCompactPartitionPolicy(type); - desc = new HTableDescriptor(tableName); - desc.addFamily(hcd1); - admin.createTable(desc); - table = conn.getTable(tableName); - bufMut = conn.getBufferedMutator(tableName); - } - - // alter mob compaction policy - private void alterForPolicyTest(final MobCompactPartitionPolicy type) - throws Exception { - - hcd1.setMobCompactPartitionPolicy(type); - desc.modifyFamily(hcd1); - admin.modifyTable(desc); - 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, hcd1, 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, hcd1, 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()); - HTableDescriptor desc = new HTableDescriptor(tableName); - HColumnDescriptor hcd1 = new HColumnDescriptor(family1); - hcd1.setMobEnabled(true); - hcd1.setMobThreshold(0); - hcd1.setEncryptionType(algorithm); - hcd1.setEncryptionKey(encryptionKey); - HColumnDescriptor hcd2 = new HColumnDescriptor(family2); - hcd2.setMobEnabled(true); - hcd2.setMobThreshold(0); - desc.addFamily(hcd1); - desc.addFamily(hcd2); - admin.createTable(desc, 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, hcd1.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); - Cell cell = result.getColumnLatestCell(hcd1.getName(), Bytes.toBytes(qf1)); - assertEquals("Before compaction: mob value of k0", newValue0, - Bytes.toString(CellUtil.cloneValue(cell))); - admin.majorCompact(tableName, hcd1.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(hcd1.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(hcd1.getName(), Bytes.toBytes(qf1)); - // the ref name is the new file - Path mobFamilyPath = - MobUtils.getMobFamilyPath(TEST_UTIL.getConfiguration(), tableName, hcd1.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, hcd1.getName(), CompactType.MOB); - } else { - admin.compact(tableName, hcd1.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 4e6461aaf800..000000000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java +++ /dev/null @@ -1,958 +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.HColumnDescriptor; -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.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 HColumnDescriptor hcd = new HColumnDescriptor(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.hcd.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.hcd.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, hcd, 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), hcd, 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), hcd, 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), hcd, 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), hcd, 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), hcd, 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(hcd); - ScanInfo scanInfo = new ScanInfo(conf, hcd, 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/TestMobStoreCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java index cccc97005398..822a85651350 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 e8096fb95106..5bbca546389d 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 @@ -402,7 +402,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 09ae96fc0a2b..4412dc85c79f 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());