mobRefSet) throws IOException {
+ if (mobRefSet != null && mobRefSet.size() > 0) {
+ String sb = StringUtils.join(mobRefSet, ",");
+ byte[] bytes = Bytes.toBytes(sb.toString());
+ writer.appendFileInfo(MOB_FILE_REFS, bytes);
+ } else {
+ writer.appendFileInfo(MOB_FILE_REFS, NULL_VALUE);
+ }
+ }
+
/**
* Add TimestampRange and earliest put timestamp to Metadata
*/
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
index e03de07dc75..04ffb549cb3 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
@@ -1102,7 +1102,7 @@ public final class FSUtils {
@Override
protected boolean accept(Path p, @CheckForNull Boolean isDir) {
- if (!StoreFileInfo.isHFile(p)) {
+ if (!StoreFileInfo.isHFile(p) && !StoreFileInfo.isMobFile(p)) {
return false;
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
index db9ea060825..2455590ec25 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java
@@ -260,9 +260,9 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
byte[][] families = { Bytes.toBytes("mob") };
loadData(tableName, families, 3000, 8);
- admin.majorCompact(tableName, CompactType.MOB).get();
+ admin.majorCompact(tableName).get();
- CompactionState state = admin.getCompactionState(tableName, CompactType.MOB).get();
+ CompactionState state = admin.getCompactionState(tableName).get();
assertNotEquals(CompactionState.NONE, state);
waitUntilMobCompactionFinished(tableName);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.java
new file mode 100644
index 00000000000..3a4c18dd7ea
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/FaultyMobStoreCompactor.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 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.compactions.CloseChecker;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputControlUtil;
+import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class is used for testing only. The main purpose is to emulate random failures during MOB
+ * compaction process. Example of usage:
+ *
+ *
+ * {
+ * @code
+ * public class SomeTest {
+ *
+ * public void initConfiguration(Configuration conf) {
+ * conf.set(MobStoreEngine.DEFAULT_MOB_COMPACTOR_CLASS_KEY,
+ * FaultyMobStoreCompactor.class.getName());
+ * conf.setDouble("hbase.mob.compaction.fault.probability", 0.1);
+ * }
+ * }
+ * }
+ *
+ *
+ * @see org.apache.hadoop.hbase.mob.MobStressToolRunner on how to use and configure this class.
+ */
+@InterfaceAudience.Private
+public class FaultyMobStoreCompactor extends DefaultMobStoreCompactor {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FaultyMobStoreCompactor.class);
+
+ public static AtomicLong mobCounter = new AtomicLong();
+ public static AtomicLong totalFailures = new AtomicLong();
+ public static AtomicLong totalCompactions = new AtomicLong();
+ public static AtomicLong totalMajorCompactions = new AtomicLong();
+
+ static double failureProb = 0.1d;
+ static Random rnd = new Random();
+
+ public FaultyMobStoreCompactor(Configuration conf, HStore store) {
+ super(conf, store);
+ failureProb = conf.getDouble("hbase.mob.compaction.fault.probability", 0.1);
+ }
+
+ @Override
+ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, CellSink writer,
+ long smallestReadPoint, boolean cleanSeqId, ThroughputController throughputController,
+ boolean major, int numofFilesToCompact, CompactionProgress progress) throws IOException {
+
+ totalCompactions.incrementAndGet();
+ if (major) {
+ totalMajorCompactions.incrementAndGet();
+ }
+ long bytesWrittenProgressForCloseCheck = 0;
+ long bytesWrittenProgressForLog = 0;
+ long bytesWrittenProgressForShippedCall = 0;
+ // Clear old mob references
+ mobRefSet.get().clear();
+ boolean isUserRequest = userRequest.get();
+ boolean compactMOBs = major && isUserRequest;
+ boolean discardMobMiss = conf.getBoolean(MobConstants.MOB_UNSAFE_DISCARD_MISS_KEY,
+ MobConstants.DEFAULT_MOB_DISCARD_MISS);
+
+ boolean mustFail = false;
+ if (compactMOBs) {
+ mobCounter.incrementAndGet();
+ double dv = rnd.nextDouble();
+ if (dv < failureProb) {
+ mustFail = true;
+ totalFailures.incrementAndGet();
+ }
+ }
+
+ FileSystem fs = store.getFileSystem();
+
+ // Since scanner.next() can return 'false' but still be delivering data,
+ // we have to use a do/while loop.
+ List cells = new ArrayList<>();
+ // Limit to "hbase.hstore.compaction.kv.max" (default 10) to avoid OOME
+ int closeCheckSizeLimit =
+ conf.getInt(CloseChecker.SIZE_LIMIT_KEY, 10 * 1000 * 1000 /* 10 MB */);
+ long lastMillis = 0;
+ if (LOG.isDebugEnabled()) {
+ lastMillis = EnvironmentEdgeManager.currentTime();
+ }
+ String compactionName = ThroughputControlUtil.getNameForThrottling(store, "compaction");
+ long now = 0;
+ boolean hasMore;
+ Path path = MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName());
+ byte[] fileName = null;
+ StoreFileWriter mobFileWriter = null;
+ long mobCells = 0;
+ long cellsCountCompactedToMob = 0, cellsCountCompactedFromMob = 0;
+ long cellsSizeCompactedToMob = 0, cellsSizeCompactedFromMob = 0;
+ boolean finished = false;
+
+ ScannerContext scannerContext =
+ ScannerContext.newBuilder().setBatchLimit(compactionKVMax).build();
+ throughputController.start(compactionName);
+ KeyValueScanner kvs = (scanner instanceof KeyValueScanner) ? (KeyValueScanner) scanner : null;
+ long shippedCallSizeLimit =
+ (long) numofFilesToCompact * this.store.getColumnFamilyDescriptor().getBlocksize();
+
+ Cell mobCell = null;
+
+ long counter = 0;
+ long countFailAt = -1;
+ if (mustFail) {
+ countFailAt = rnd.nextInt(100); // randomly fail fast
+ }
+
+ try {
+ try {
+ mobFileWriter = mobStore.createWriterInTmp(new Date(fd.latestPutTs), fd.maxKeyCount,
+ major ? majorCompactionCompression : minorCompactionCompression,
+ store.getRegionInfo().getStartKey(), true);
+ fileName = Bytes.toBytes(mobFileWriter.getPath().getName());
+ } catch (IOException e) {
+ // Bailing out
+ LOG.error("Failed to create mob writer, ", e);
+ throw e;
+ }
+ if (compactMOBs) {
+ // Add the only reference we get for compact MOB case
+ // because new store file will have only one MOB reference
+ // in this case - of newly compacted MOB file
+ mobRefSet.get().add(mobFileWriter.getPath().getName());
+ }
+ do {
+ hasMore = scanner.next(cells, scannerContext);
+ if (LOG.isDebugEnabled()) {
+ now = EnvironmentEdgeManager.currentTime();
+ }
+ for (Cell c : cells) {
+ counter++;
+ if (compactMOBs) {
+ if (MobUtils.isMobReferenceCell(c)) {
+ if (counter == countFailAt) {
+ LOG.warn("INJECTED FAULT mobCounter={}", mobCounter.get());
+ throw new CorruptHFileException("injected fault");
+ }
+ String fName = MobUtils.getMobFileName(c);
+ // Added to support migration
+ try {
+ mobCell = mobStore.resolve(c, true, false).getCell();
+ } catch (FileNotFoundException fnfe) {
+ if (discardMobMiss) {
+ LOG.error("Missing MOB cell: file={} not found", fName);
+ continue;
+ } else {
+ throw fnfe;
+ }
+ }
+
+ if (discardMobMiss && mobCell.getValueLength() == 0) {
+ LOG.error("Missing MOB cell value: file={} cell={}", fName, mobCell);
+ continue;
+ }
+
+ if (mobCell.getValueLength() > mobSizeThreshold) {
+ // put the mob data back to the store file
+ PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId());
+ mobFileWriter.append(mobCell);
+ writer.append(
+ MobUtils.createMobRefCell(mobCell, fileName, this.mobStore.getRefCellTags()));
+ mobCells++;
+ } else {
+ // If MOB value is less than threshold, append it directly to a store file
+ PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId());
+ writer.append(mobCell);
+ cellsCountCompactedFromMob++;
+ cellsSizeCompactedFromMob += mobCell.getValueLength();
+ }
+ } else {
+ // Not a MOB reference cell
+ int size = c.getValueLength();
+ if (size > mobSizeThreshold) {
+ mobFileWriter.append(c);
+ writer
+ .append(MobUtils.createMobRefCell(c, fileName, this.mobStore.getRefCellTags()));
+ mobCells++;
+ cellsCountCompactedToMob++;
+ cellsSizeCompactedToMob += c.getValueLength();
+ } else {
+ writer.append(c);
+ }
+ }
+ } else if (c.getTypeByte() != KeyValue.Type.Put.getCode()) {
+ // Not a major compaction or major with MOB disabled
+ // If the kv type is not put, directly write the cell
+ // to the store file.
+ writer.append(c);
+ } else if (MobUtils.isMobReferenceCell(c)) {
+ // Not a major MOB compaction, Put MOB reference
+ if (MobUtils.hasValidMobRefCellValue(c)) {
+ int size = MobUtils.getMobValueLength(c);
+ if (size > mobSizeThreshold) {
+ // If the value size is larger than the threshold, it's regarded as a mob. Since
+ // its value is already in the mob file, directly write this cell to the store file
+ writer.append(c);
+ // Add MOB reference to a set
+ mobRefSet.get().add(MobUtils.getMobFileName(c));
+ } else {
+ // If the value is not larger than the threshold, it's not regarded a mob. Retrieve
+ // the mob cell from the mob file, and write it back to the store file.
+ mobCell = mobStore.resolve(c, true, false).getCell();
+ if (mobCell.getValueLength() != 0) {
+ // put the mob data back to the store file
+ PrivateCellUtil.setSequenceId(mobCell, c.getSequenceId());
+ writer.append(mobCell);
+ cellsCountCompactedFromMob++;
+ cellsSizeCompactedFromMob += mobCell.getValueLength();
+ } else {
+ // If the value of a file is empty, there might be issues when retrieving,
+ // directly write the cell to the store file, and leave it to be handled by the
+ // next compaction.
+ LOG.error("Empty value for: " + c);
+ writer.append(c);
+ // Add MOB reference to a set
+ mobRefSet.get().add(MobUtils.getMobFileName(c));
+ }
+ }
+ } else {
+ LOG.error("Corrupted MOB reference: {}", c);
+ writer.append(c);
+ }
+ } else if (c.getValueLength() <= mobSizeThreshold) {
+ // If the value size of a cell is not larger than the threshold, directly write it to
+ // the store file.
+ writer.append(c);
+ } else {
+ // If the value size of a cell is larger than the threshold, it's regarded as a mob,
+ // write this cell to a mob file, and write the path to the store file.
+ mobCells++;
+ // append the original keyValue in the mob file.
+ mobFileWriter.append(c);
+ Cell reference = MobUtils.createMobRefCell(c, fileName, this.mobStore.getRefCellTags());
+ // write the cell whose value is the path of a mob file to the store file.
+ writer.append(reference);
+ cellsCountCompactedToMob++;
+ cellsSizeCompactedToMob += c.getValueLength();
+ // Add ref we get for compact MOB case
+ mobRefSet.get().add(mobFileWriter.getPath().getName());
+ }
+
+ int len = c.getSerializedSize();
+ ++progress.currentCompactedKVs;
+ progress.totalCompactedSize += len;
+ bytesWrittenProgressForShippedCall += len;
+ if (LOG.isDebugEnabled()) {
+ bytesWrittenProgressForLog += len;
+ }
+ throughputController.control(compactionName, len);
+ // check periodically to see if a system stop is requested
+ if (closeCheckSizeLimit > 0) {
+ bytesWrittenProgressForCloseCheck += len;
+ if (bytesWrittenProgressForCloseCheck > closeCheckSizeLimit) {
+ bytesWrittenProgressForCloseCheck = 0;
+ if (!store.areWritesEnabled()) {
+ progress.cancel();
+ return false;
+ }
+ }
+ }
+ if (kvs != null && bytesWrittenProgressForShippedCall > shippedCallSizeLimit) {
+ ((ShipperListener) writer).beforeShipped();
+ kvs.shipped();
+ bytesWrittenProgressForShippedCall = 0;
+ }
+ }
+ // Log the progress of long running compactions every minute if
+ // logging at DEBUG level
+ if (LOG.isDebugEnabled()) {
+ if ((now - lastMillis) >= COMPACTION_PROGRESS_LOG_INTERVAL) {
+ String rate = String.format("%.2f",
+ (bytesWrittenProgressForLog / 1024.0) / ((now - lastMillis) / 1000.0));
+ LOG.debug("Compaction progress: {} {}, rate={} KB/sec, throughputController is {}",
+ compactionName, progress, rate, throughputController);
+ lastMillis = now;
+ bytesWrittenProgressForLog = 0;
+ }
+ }
+ cells.clear();
+ } while (hasMore);
+ finished = true;
+ } catch (InterruptedException e) {
+ progress.cancel();
+ throw new InterruptedIOException(
+ "Interrupted while control throughput of compacting " + compactionName);
+ } catch (FileNotFoundException e) {
+ LOG.error("MOB Stress Test FAILED, region: " + store.getRegionInfo().getEncodedName(), e);
+ System.exit(-1);
+ } catch (IOException t) {
+ LOG.error("Mob compaction failed for region: " + store.getRegionInfo().getEncodedName());
+ throw t;
+ } finally {
+ // Clone last cell in the final because writer will append last cell when committing. If
+ // don't clone here and once the scanner get closed, then the memory of last cell will be
+ // released. (HBASE-22582)
+ ((ShipperListener) writer).beforeShipped();
+ throughputController.finish(compactionName);
+ if (!finished && mobFileWriter != null) {
+ // Remove all MOB references because compaction failed
+ mobRefSet.get().clear();
+ // Abort writer
+ abortWriter(mobFileWriter);
+ }
+ }
+
+ if (mobFileWriter != null) {
+ if (mobCells > 0) {
+ // If the mob file is not empty, commit it.
+ mobFileWriter.appendMetadata(fd.maxSeqId, major, mobCells);
+ mobFileWriter.close();
+ mobStore.commitFile(mobFileWriter.getPath(), path);
+ } else {
+ // If the mob file is empty, delete it instead of committing.
+ abortWriter(mobFileWriter);
+ }
+ }
+ mobStore.updateCellsCountCompactedFromMob(cellsCountCompactedFromMob);
+ mobStore.updateCellsCountCompactedToMob(cellsCountCompactedToMob);
+ mobStore.updateCellsSizeCompactedFromMob(cellsSizeCompactedFromMob);
+ mobStore.updateCellsSizeCompactedToMob(cellsSizeCompactedToMob);
+ progress.complete();
+ return true;
+
+ }
+
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressTool.java
new file mode 100644
index 00000000000..060ae24c1a4
--- /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.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class MobStressTool extends AbstractHBaseTool {
+ private static final Logger LOG = LoggerFactory.getLogger(MobStressTool.class);
+ private CommandLine cmd;
+
+ public MobStressTool() throws IOException {
+ init();
+ }
+
+ protected void init() throws IOException {
+ // define supported options
+ addOptWithArg("n", "Number of MOB key-values to insert, default - 10000000");
+ }
+
+ @Override
+ protected void addOptions() {
+ }
+
+ @Override
+ protected void processOptions(CommandLine cmd) {
+ this.cmd = cmd;
+ }
+
+ @Override
+ protected int doWork() throws Exception {
+ long numRowsToInsert = 10000000;
+ if (cmd.hasOption("n")) {
+ numRowsToInsert = Long.parseLong(cmd.getOptionValue("n"));
+ if (numRowsToInsert < 0) {
+ LOG.warn("Ignore wrong option '-n'");
+ numRowsToInsert = 10000000;
+ }
+ }
+
+ MobStressToolRunner runner = new MobStressToolRunner();
+ runner.init(getConf(), numRowsToInsert);
+ runner.runStressTest();
+ return 0;
+ }
+
+ public static void main(String[] args) throws Exception {
+ Configuration conf = HBaseConfiguration.create();
+ int ret = ToolRunner.run(conf, new MobStressTool(), args);
+ System.exit(ret);
+ }
+
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java
new file mode 100644
index 00000000000..8dd545f0015
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/MobStressToolRunner.java
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mob;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Arrays;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Reproduction for MOB data loss 1. Settings: Region Size 200 MB, Flush threshold 800 KB. 2. Insert
+ * 10 Million records 3. MOB Compaction and Archiver a) Trigger MOB Compaction (every 2 minutes) b)
+ * Trigger major compaction (every 2 minutes) c) Trigger archive cleaner (every 3 minutes) 4.
+ * Validate MOB data after complete data load. This class is used by MobStressTool only. This is not
+ * a unit test
+ */
+@SuppressWarnings("deprecation")
+public class MobStressToolRunner {
+ private static final Logger LOG = LoggerFactory.getLogger(MobStressToolRunner.class);
+
+ private HBaseTestingUtility HTU;
+
+ private final static String famStr = "f1";
+ private final static byte[] fam = Bytes.toBytes(famStr);
+ private final static byte[] qualifier = Bytes.toBytes("q1");
+ private final static long mobLen = 10;
+ private final static byte[] mobVal = Bytes
+ .toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789");
+
+ private Configuration conf;
+ private HTableDescriptor hdt;
+ private HColumnDescriptor hcd;
+ private Admin admin;
+ private long count = 500000;
+ private double failureProb = 0.1;
+ private Table table = null;
+ private MobFileCleanerChore chore = new MobFileCleanerChore();
+
+ private static volatile boolean run = true;
+
+ public MobStressToolRunner() {
+
+ }
+
+ public void init(Configuration conf, long numRows) throws IOException {
+ this.conf = conf;
+ this.count = numRows;
+ initConf();
+ printConf();
+ hdt = new HTableDescriptor(TableName.valueOf("testMobCompactTable"));
+ Connection conn = ConnectionFactory.createConnection(this.conf);
+ this.admin = conn.getAdmin();
+ this.hcd = new HColumnDescriptor(fam);
+ this.hcd.setMobEnabled(true);
+ this.hcd.setMobThreshold(mobLen);
+ this.hcd.setMaxVersions(1);
+ this.hdt.addFamily(hcd);
+ if (admin.tableExists(hdt.getTableName())) {
+ admin.disableTable(hdt.getTableName());
+ admin.deleteTable(hdt.getTableName());
+ }
+ admin.createTable(hdt);
+ table = conn.getTable(hdt.getTableName());
+ }
+
+ private void printConf() {
+ LOG.info("Please ensure the following HBase configuration is set:");
+ LOG.info("hfile.format.version=3");
+ LOG.info("hbase.master.hfilecleaner.ttl=0");
+ LOG.info("hbase.hregion.max.filesize=200000000");
+ LOG.info("hbase.client.retries.number=100");
+ LOG.info("hbase.hregion.memstore.flush.size=800000");
+ LOG.info("hbase.hstore.blockingStoreFiles=150");
+ LOG.info("hbase.hstore.compaction.throughput.lower.bound=50000000");
+ LOG.info("hbase.hstore.compaction.throughput.higher.bound=100000000");
+ LOG.info("hbase.master.mob.cleaner.period=0");
+ LOG.info("hbase.mob.default.compactor=org.apache.hadoop.hbase.mob.FaultyMobStoreCompactor");
+ LOG.warn("hbase.mob.compaction.fault.probability=x, where x is between 0. and 1.");
+
+ }
+
+ private void initConf() {
+
+ conf.setInt("hfile.format.version", 3);
+ conf.setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0);
+ conf.setInt("hbase.client.retries.number", 100);
+ conf.setInt("hbase.hregion.max.filesize", 200000000);
+ conf.setInt("hbase.hregion.memstore.flush.size", 800000);
+ conf.setInt("hbase.hstore.blockingStoreFiles", 150);
+ conf.setInt("hbase.hstore.compaction.throughput.lower.bound", 52428800);
+ conf.setInt("hbase.hstore.compaction.throughput.higher.bound", 2 * 52428800);
+ conf.setDouble("hbase.mob.compaction.fault.probability", failureProb);
+ // conf.set(MobStoreEngine.DEFAULT_MOB_COMPACTOR_CLASS_KEY,
+ // FaultyMobStoreCompactor.class.getName());
+ conf.setLong(MobConstants.MOB_COMPACTION_CHORE_PERIOD, 0);
+ conf.setLong(MobConstants.MOB_CLEANER_PERIOD, 0);
+ conf.setLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, 120000);
+ conf.set(MobConstants.MOB_COMPACTION_TYPE_KEY, MobConstants.OPTIMIZED_MOB_COMPACTION_TYPE);
+ conf.setLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, 1000000);
+
+ }
+
+ class MajorCompaction implements Runnable {
+
+ @Override
+ public void run() {
+ while (run) {
+ try {
+ admin.majorCompact(hdt.getTableName(), fam);
+ Thread.sleep(120000);
+ } catch (Exception e) {
+ LOG.error("MOB Stress Test FAILED", e);
+ System.exit(-1);
+ }
+ }
+ }
+ }
+
+ class CleanMobAndArchive implements Runnable {
+
+ @Override
+ public void run() {
+ while (run) {
+ try {
+ LOG.info("MOB cleanup chore started ...");
+ chore.cleanupObsoleteMobFiles(conf, table.getName());
+ LOG.info("MOB cleanup chore finished");
+
+ Thread.sleep(130000);
+ } catch (Exception e) {
+ LOG.error("CleanMobAndArchive", e);
+ }
+ }
+ }
+ }
+
+ class WriteData implements Runnable {
+
+ private long rows = -1;
+
+ public WriteData(long rows) {
+ this.rows = rows;
+ }
+
+ @Override
+ public void run() {
+ try {
+
+ // Put Operation
+ for (int i = 0; i < rows; i++) {
+ byte[] key = Bytes.toBytes(i);
+ Put p = new Put(key);
+ p.addColumn(fam, qualifier, Bytes.add(key, mobVal));
+ table.put(p);
+ if (i % 10000 == 0) {
+ LOG.info("LOADED=" + i);
+ try {
+ Thread.sleep(500);
+ } catch (InterruptedException ee) {
+ }
+ }
+ if (i % 100000 == 0) {
+ printStats(i);
+ }
+ }
+ admin.flush(table.getName());
+ run = false;
+ } catch (Exception e) {
+ LOG.error("MOB Stress Test FAILED", e);
+ System.exit(-1);
+ }
+ }
+ }
+
+ public void runStressTest() throws InterruptedException, IOException {
+
+ try {
+
+ Thread writeData = new Thread(new WriteData(count));
+ writeData.start();
+
+ Thread majorcompact = new Thread(new MajorCompaction());
+ majorcompact.start();
+
+ Thread cleaner = new Thread(new CleanMobAndArchive());
+ cleaner.start();
+
+ while (run) {
+ Thread.sleep(1000);
+ }
+
+ getNumberOfMobFiles(conf, table.getName(), new String(fam));
+ LOG.info("Waiting for write thread to finish ...");
+ writeData.join();
+ // Cleanup again
+ chore.cleanupObsoleteMobFiles(conf, table.getName());
+ getNumberOfMobFiles(conf, table.getName(), new String(fam));
+
+ if (HTU != null) {
+ LOG.info("Archive cleaner started ...");
+ // Call archive cleaner again
+ HTU.getMiniHBaseCluster().getMaster().getHFileCleaner().choreForTesting();
+ LOG.info("Archive cleaner finished");
+ }
+
+ scanTable();
+
+ } finally {
+
+ admin.disableTable(hdt.getTableName());
+ admin.deleteTable(hdt.getTableName());
+ }
+ LOG.info("MOB Stress Test finished OK");
+ printStats(count);
+
+ }
+
+ private long getNumberOfMobFiles(Configuration conf, TableName tableName, String family)
+ throws IOException {
+ FileSystem fs = FileSystem.get(conf);
+ Path dir = MobUtils.getMobFamilyPath(conf, tableName, family);
+ FileStatus[] stat = fs.listStatus(dir);
+ long size = 0;
+ for (FileStatus st : stat) {
+ LOG.debug("MOB Directory content: {} len={}", st.getPath(), st.getLen());
+ size += st.getLen();
+ }
+ LOG.debug("MOB Directory content total files: {}, total size={}", stat.length, size);
+
+ return stat.length;
+ }
+
+ public void printStats(long loaded) {
+ LOG.info("MOB Stress Test: loaded=" + loaded + " compactions="
+ + FaultyMobStoreCompactor.totalCompactions.get() + " major="
+ + FaultyMobStoreCompactor.totalMajorCompactions.get() + " mob="
+ + FaultyMobStoreCompactor.mobCounter.get() + " injected failures="
+ + FaultyMobStoreCompactor.totalFailures.get());
+ }
+
+ private void scanTable() {
+ try {
+
+ Result result;
+ ResultScanner scanner = table.getScanner(fam);
+ int counter = 0;
+ while ((result = scanner.next()) != null) {
+ byte[] key = result.getRow();
+ assertTrue(Arrays.equals(result.getValue(fam, qualifier), Bytes.add(key, mobVal)));
+ if (counter % 10000 == 0) {
+ LOG.info("GET=" + counter + " key=" + Bytes.toInt(key));
+ }
+ counter++;
+ }
+
+ assertEquals(count, counter);
+ } catch (Exception e) {
+ e.printStackTrace();
+ LOG.error("MOB Stress Test FAILED");
+ if (HTU != null) {
+ assertTrue(false);
+ } else {
+ System.exit(-1);
+ }
+ }
+ }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java
new file mode 100644
index 00000000000..62ca370197b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionBase.java
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mob;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Random;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.CompactionState;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.RegionSplitter;
+import org.junit.After;
+import org.junit.Before;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Mob file compaction base test. 1. Enables batch mode for regular MOB compaction, Sets batch size
+ * to 7 regions. (Optional) 2. Disables periodic MOB compactions, sets minimum age to archive to 10
+ * sec 3. Creates MOB table with 20 regions 4. Loads MOB data (randomized keys, 1000 rows), flushes
+ * data. 5. Repeats 4. two more times 6. Verifies that we have 20 *3 = 60 mob files (equals to
+ * number of regions x 3) 7. Runs major MOB compaction. 8. Verifies that number of MOB files in a
+ * mob directory is 20 x4 = 80 9. Waits for a period of time larger than minimum age to archive 10.
+ * Runs Mob cleaner chore 11 Verifies that number of MOB files in a mob directory is 20. 12 Runs
+ * scanner and checks all 3 * 1000 rows.
+ */
+@SuppressWarnings("deprecation")
+public abstract class TestMobCompactionBase {
+ private static final Logger LOG = LoggerFactory.getLogger(TestMobCompactionBase.class);
+
+ protected HBaseTestingUtility HTU;
+
+ protected final static String famStr = "f1";
+ protected final static byte[] fam = Bytes.toBytes(famStr);
+ protected final static byte[] qualifier = Bytes.toBytes("q1");
+ protected final static long mobLen = 10;
+ protected final static byte[] mobVal = Bytes
+ .toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789");
+
+ protected Configuration conf;
+ protected HTableDescriptor hdt;
+ private HColumnDescriptor hcd;
+ protected Admin admin;
+ protected Table table = null;
+ protected long minAgeToArchive = 10000;
+ protected int numRegions = 20;
+ protected int rows = 1000;
+
+ protected MobFileCleanerChore cleanerChore;
+
+ public TestMobCompactionBase() {
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ HTU = new HBaseTestingUtility();
+ hdt = HTU.createTableDescriptor(getClass().getName());
+ conf = HTU.getConfiguration();
+
+ initConf();
+
+ HTU.startMiniCluster();
+ admin = HTU.getAdmin();
+ cleanerChore = new MobFileCleanerChore();
+ hcd = new HColumnDescriptor(fam);
+ hcd.setMobEnabled(true);
+ hcd.setMobThreshold(mobLen);
+ hcd.setMaxVersions(1);
+ hdt.addFamily(hcd);
+ RegionSplitter.UniformSplit splitAlgo = new RegionSplitter.UniformSplit();
+ byte[][] splitKeys = splitAlgo.split(numRegions);
+ table = HTU.createTable(hdt, splitKeys);
+
+ }
+
+ protected void initConf() {
+
+ conf.setInt("hfile.format.version", 3);
+ // Disable automatic MOB compaction
+ conf.setLong(MobConstants.MOB_COMPACTION_CHORE_PERIOD, 0);
+ // Disable automatic MOB file cleaner chore
+ conf.setLong(MobConstants.MOB_CLEANER_PERIOD, 0);
+ // Set minimum age to archive to 10 sec
+ conf.setLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, minAgeToArchive);
+ // Set compacted file discharger interval to a half minAgeToArchive
+ conf.setLong("hbase.hfile.compaction.discharger.interval", minAgeToArchive / 2);
+ }
+
+ private void loadData(int num) {
+
+ Random r = new Random();
+ try {
+ LOG.info("Started loading {} rows", num);
+ for (int i = 0; i < num; i++) {
+ byte[] key = new byte[32];
+ r.nextBytes(key);
+ Put p = new Put(key);
+ p.addColumn(fam, qualifier, mobVal);
+ table.put(p);
+ }
+ admin.flush(table.getName());
+ LOG.info("Finished loading {} rows", num);
+ } catch (Exception e) {
+ LOG.error("MOB file compaction chore test FAILED", e);
+ fail("MOB file compaction chore test FAILED");
+ }
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ admin.disableTable(hdt.getTableName());
+ admin.deleteTable(hdt.getTableName());
+ HTU.shutdownMiniCluster();
+ }
+
+ public void baseTestMobFileCompaction() throws InterruptedException, IOException {
+
+ // Load and flush data 3 times
+ loadData(rows);
+ loadData(rows);
+ loadData(rows);
+ long num = getNumberOfMobFiles(conf, table.getName(), new String(fam));
+ assertEquals(numRegions * 3, num);
+ // Major MOB compact
+ mobCompact(admin, hdt, hcd);
+ // wait until compaction is complete
+ while (admin.getCompactionState(hdt.getTableName()) != CompactionState.NONE) {
+ Thread.sleep(100);
+ }
+
+ num = getNumberOfMobFiles(conf, table.getName(), new String(fam));
+ assertEquals(numRegions * 4, num);
+ // We have guarantee, that compacted file discharger will run during this pause
+ // because it has interval less than this wait time
+ LOG.info("Waiting for {}ms", minAgeToArchive + 1000);
+
+ Thread.sleep(minAgeToArchive + 1000);
+ LOG.info("Cleaning up MOB files");
+ // Cleanup again
+ cleanerChore.cleanupObsoleteMobFiles(conf, table.getName());
+
+ num = getNumberOfMobFiles(conf, table.getName(), new String(fam));
+ assertEquals(numRegions, num);
+
+ long scanned = scanTable();
+ assertEquals(3 * rows, scanned);
+
+ }
+
+ protected abstract void mobCompact(Admin admin2, HTableDescriptor hdt2, HColumnDescriptor hcd2)
+ throws IOException, InterruptedException;
+
+ protected long getNumberOfMobFiles(Configuration conf, TableName tableName, String family)
+ throws IOException {
+ FileSystem fs = FileSystem.get(conf);
+ Path dir = MobUtils.getMobFamilyPath(conf, tableName, family);
+ FileStatus[] stat = fs.listStatus(dir);
+ for (FileStatus st : stat) {
+ LOG.debug("MOB Directory content: {}", st.getPath());
+ }
+ LOG.debug("MOB Directory content total files: {}", stat.length);
+
+ return stat.length;
+ }
+
+ protected long scanTable() {
+ try {
+
+ Result result;
+ ResultScanner scanner = table.getScanner(fam);
+ long counter = 0;
+ while ((result = scanner.next()) != null) {
+ assertTrue(Arrays.equals(result.getValue(fam, qualifier), mobVal));
+ counter++;
+ }
+ return counter;
+ } catch (Exception e) {
+ LOG.error("MOB file compaction test FAILED", e);
+ if (HTU != null) {
+ fail(e.getMessage());
+ } else {
+ System.exit(-1);
+ }
+ }
+ return 0;
+ }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java
new file mode 100644
index 00000000000..cfe2db13ca3
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptMode.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mob;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Mob file compaction chore in a generational non-batch mode test. 1. Uses default (non-batch) mode
+ * for regular MOB compaction, sets generational mode ON 2. Disables periodic MOB compactions, sets
+ * minimum age to archive to 10 sec 3. Creates MOB table with 20 regions 4. Loads MOB data
+ * (randomized keys, 1000 rows), flushes data. 5. Repeats 4. two more times 6. Verifies that we have
+ * 20 *3 = 60 mob files (equals to number of regions x 3) 7. Runs major MOB compaction. 8. Verifies
+ * that number of MOB files in a mob directory is 20 x4 = 80 9. Waits for a period of time larger
+ * than minimum age to archive 10. Runs Mob cleaner chore 11 Verifies that number of MOB files in a
+ * mob directory is 20. 12 Runs scanner and checks all 3 * 1000 rows.
+ */
+@SuppressWarnings("deprecation")
+@Category(LargeTests.class)
+public class TestMobCompactionOptMode extends TestMobCompactionBase {
+ private static final Logger LOG = LoggerFactory.getLogger(TestMobCompactionOptMode.class);
+ @ClassRule
+ public static final HBaseClassTestRule CLASS_RULE =
+ HBaseClassTestRule.forClass(TestMobCompactionOptMode.class);
+
+ public TestMobCompactionOptMode() {
+ }
+
+ @Override
+ protected void initConf() {
+ super.initConf();
+ conf.set(MobConstants.MOB_COMPACTION_TYPE_KEY, MobConstants.OPTIMIZED_MOB_COMPACTION_TYPE);
+ conf.setLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, 1000000);
+ }
+
+ @Test
+ public void testMobFileCompactionBatchMode() throws InterruptedException, IOException {
+ LOG.info("MOB compaction generational (non-batch) mode started");
+ baseTestMobFileCompaction();
+ LOG.info("MOB compaction generational (non-batch) mode finished OK");
+
+ }
+
+ @Override
+ protected void mobCompact(Admin admin, HTableDescriptor hdt, HColumnDescriptor hcd)
+ throws IOException, InterruptedException {
+ // Major compact MOB table
+ admin.majorCompact(hdt.getTableName(), hcd.getName());
+ }
+
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java
new file mode 100644
index 00000000000..7ea74a1617a
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionOptRegionBatchMode.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mob;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Mob file compaction chore in a generational batch mode test. 1. Enables batch mode for regular
+ * MOB compaction, Sets batch size to 7 regions. Enables generational mode. 2. Disables periodic MOB
+ * compactions, sets minimum age to archive to 10 sec 3. Creates MOB table with 20 regions 4. Loads
+ * MOB data (randomized keys, 1000 rows), flushes data. 5. Repeats 4. two more times 6. Verifies
+ * that we have 20 *3 = 60 mob files (equals to number of regions x 3) 7. Runs major MOB compaction.
+ * 8. Verifies that number of MOB files in a mob directory is 20 x4 = 80 9. Waits for a period of
+ * time larger than minimum age to archive 10. Runs Mob cleaner chore 11 Verifies that number of MOB
+ * files in a mob directory is 20. 12 Runs scanner and checks all 3 * 1000 rows.
+ */
+@SuppressWarnings("deprecation")
+@Category(LargeTests.class)
+public class TestMobCompactionOptRegionBatchMode extends TestMobCompactionBase {
+ private static final Logger LOG =
+ LoggerFactory.getLogger(TestMobCompactionOptRegionBatchMode.class);
+ @ClassRule
+ public static final HBaseClassTestRule CLASS_RULE =
+ HBaseClassTestRule.forClass(TestMobCompactionOptRegionBatchMode.class);
+
+ private int batchSize = 7;
+ private MobFileCompactionChore compactionChore;
+
+ public TestMobCompactionOptRegionBatchMode() {
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ super.setUp();
+ compactionChore = new MobFileCompactionChore(conf, batchSize);
+ }
+
+ protected void initConf() {
+ super.initConf();
+ conf.setInt(MobConstants.MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE, batchSize);
+ conf.set(MobConstants.MOB_COMPACTION_TYPE_KEY, MobConstants.OPTIMIZED_MOB_COMPACTION_TYPE);
+ conf.setLong(MobConstants.MOB_COMPACTION_MAX_FILE_SIZE_KEY, 1000000);
+ }
+
+ @Test
+ public void testMobFileCompactionBatchMode() throws InterruptedException, IOException {
+ LOG.info("MOB compaction chore generational batch mode started");
+ baseTestMobFileCompaction();
+ LOG.info("MOB compaction chore generational batch mode finished OK");
+
+ }
+
+ @Override
+ protected void mobCompact(Admin admin, HTableDescriptor hdt, HColumnDescriptor hcd)
+ throws IOException, InterruptedException {
+ // Major compact with batch mode enabled
+ compactionChore.performMajorCompactionInBatches(admin, hdt, hcd);
+ }
+
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java
new file mode 100644
index 00000000000..92809ea5511
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularMode.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mob;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Mob file compaction chore in a regular non-batch mode test. 1. Uses default (non-batch) mode for
+ * regular MOB compaction, 2. Disables periodic MOB compactions, sets minimum age to archive to 10
+ * sec 3. Creates MOB table with 20 regions 4. Loads MOB data (randomized keys, 1000 rows), flushes
+ * data. 5. Repeats 4. two more times 6. Verifies that we have 20 *3 = 60 mob files (equals to
+ * number of regions x 3) 7. Runs major MOB compaction. 8. Verifies that number of MOB files in a
+ * mob directory is 20 x4 = 80 9. Waits for a period of time larger than minimum age to archive 10.
+ * Runs Mob cleaner chore 11 Verifies that number of MOB files in a mob directory is 20. 12 Runs
+ * scanner and checks all 3 * 1000 rows.
+ */
+@SuppressWarnings("deprecation")
+@Category(LargeTests.class)
+public class TestMobCompactionRegularMode extends TestMobCompactionBase {
+ private static final Logger LOG = LoggerFactory.getLogger(TestMobCompactionRegularMode.class);
+ @ClassRule
+ public static final HBaseClassTestRule CLASS_RULE =
+ HBaseClassTestRule.forClass(TestMobCompactionRegularMode.class);
+
+ public TestMobCompactionRegularMode() {
+ }
+
+ @Test
+ public void testMobFileCompactionBatchMode() throws InterruptedException, IOException {
+ LOG.info("MOB compaction regular mode started");
+ baseTestMobFileCompaction();
+ LOG.info("MOB compaction regular mode finished OK");
+
+ }
+
+ @Override
+ protected void mobCompact(Admin admin, HTableDescriptor hdt, HColumnDescriptor hcd)
+ throws IOException, InterruptedException {
+ // Major compact MOB table
+ admin.majorCompact(hdt.getTableName(), hcd.getName());
+ }
+
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java
new file mode 100644
index 00000000000..9b8006d1be5
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobCompactionRegularRegionBatchMode.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mob;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Mob file compaction chore in a regular batch mode test. 1. Enables batch mode for regular MOB
+ * compaction, Sets batch size to 7 regions. 2. Disables periodic MOB compactions, sets minimum age
+ * to archive to 10 sec 3. Creates MOB table with 20 regions 4. Loads MOB data (randomized keys,
+ * 1000 rows), flushes data. 5. Repeats 4. two more times 6. Verifies that we have 20 *3 = 60 mob
+ * files (equals to number of regions x 3) 7. Runs major MOB compaction. 8. Verifies that number of
+ * MOB files in a mob directory is 20 x4 = 80 9. Waits for a period of time larger than minimum age
+ * to archive 10. Runs Mob cleaner chore 11 Verifies that number of MOB files in a mob directory is
+ * 20. 12 Runs scanner and checks all 3 * 1000 rows.
+ */
+@SuppressWarnings("deprecation")
+@Category(LargeTests.class)
+public class TestMobCompactionRegularRegionBatchMode extends TestMobCompactionBase {
+ private static final Logger LOG =
+ LoggerFactory.getLogger(TestMobCompactionRegularRegionBatchMode.class);
+ @ClassRule
+ public static final HBaseClassTestRule CLASS_RULE =
+ HBaseClassTestRule.forClass(TestMobCompactionRegularRegionBatchMode.class);
+
+ private int batchSize = 7;
+ private MobFileCompactionChore compactionChore;
+
+ public TestMobCompactionRegularRegionBatchMode() {
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ super.setUp();
+ compactionChore = new MobFileCompactionChore(conf, batchSize);
+ }
+
+ protected void initConf() {
+ super.initConf();
+ conf.setInt(MobConstants.MOB_MAJOR_COMPACTION_REGION_BATCH_SIZE, batchSize);
+ }
+
+ @Test
+ public void testMobFileCompactionBatchMode() throws InterruptedException, IOException {
+ LOG.info("MOB compaction chore regular batch mode started");
+ baseTestMobFileCompaction();
+ LOG.info("MOB compaction chore regular batch mode finished OK");
+
+ }
+
+ @Override
+ protected void mobCompact(Admin admin, HTableDescriptor hdt, HColumnDescriptor hcd)
+ throws IOException, InterruptedException {
+ // Major compact with batch mode enabled
+ compactionChore.performMajorCompactionInBatches(admin, hdt, hcd);
+ }
+
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCleanerChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCleanerChore.java
new file mode 100644
index 00000000000..bac0408eb4a
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCleanerChore.java
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mob;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Arrays;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.CompactionState;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Mob file cleaner chore test. 1. Creates MOB table 2. Load MOB data and flushes it N times 3. Runs
+ * major MOB compaction (N MOB files go to archive) 4. Verifies that number of MOB files in a mob
+ * directory is N+1 5. Waits for a period of time larger than minimum age to archive 6. Runs Mob
+ * cleaner chore 7 Verifies that number of MOB files in a mob directory is 1.
+ */
+@SuppressWarnings("deprecation")
+@Category(MediumTests.class)
+public class TestMobFileCleanerChore {
+ private static final Logger LOG = LoggerFactory.getLogger(TestMobFileCleanerChore.class);
+ @ClassRule
+ public static final HBaseClassTestRule CLASS_RULE =
+ HBaseClassTestRule.forClass(TestMobFileCleanerChore.class);
+
+ private HBaseTestingUtility HTU;
+
+ private final static String famStr = "f1";
+ private final static byte[] fam = Bytes.toBytes(famStr);
+ private final static byte[] qualifier = Bytes.toBytes("q1");
+ private final static long mobLen = 10;
+ private final static byte[] mobVal = Bytes
+ .toBytes("01234567890123456789012345678901234567890123456789012345678901234567890123456789");
+
+ private Configuration conf;
+ private HTableDescriptor hdt;
+ private HColumnDescriptor hcd;
+ private Admin admin;
+ private Table table = null;
+ private MobFileCleanerChore chore;
+ private long minAgeToArchive = 10000;
+
+ public TestMobFileCleanerChore() {
+ }
+
+ @Before
+ public void setUp() throws Exception {
+ HTU = new HBaseTestingUtility();
+ hdt = HTU.createTableDescriptor("testMobCompactTable");
+ conf = HTU.getConfiguration();
+
+ initConf();
+
+ HTU.startMiniCluster();
+ admin = HTU.getAdmin();
+ chore = new MobFileCleanerChore();
+ hcd = new HColumnDescriptor(fam);
+ hcd.setMobEnabled(true);
+ hcd.setMobThreshold(mobLen);
+ hcd.setMaxVersions(1);
+ hdt.addFamily(hcd);
+ table = HTU.createTable(hdt, null);
+ }
+
+ private void initConf() {
+
+ conf.setInt("hfile.format.version", 3);
+ conf.setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, 0);
+ conf.setInt("hbase.client.retries.number", 100);
+ conf.setInt("hbase.hregion.max.filesize", 200000000);
+ conf.setInt("hbase.hregion.memstore.flush.size", 800000);
+ conf.setInt("hbase.hstore.blockingStoreFiles", 150);
+ conf.setInt("hbase.hstore.compaction.throughput.lower.bound", 52428800);
+ conf.setInt("hbase.hstore.compaction.throughput.higher.bound", 2 * 52428800);
+ // conf.set(MobStoreEngine.DEFAULT_MOB_COMPACTOR_CLASS_KEY,
+ // FaultyMobStoreCompactor.class.getName());
+ // Disable automatic MOB compaction
+ conf.setLong(MobConstants.MOB_COMPACTION_CHORE_PERIOD, 0);
+ // Disable automatic MOB file cleaner chore
+ conf.setLong(MobConstants.MOB_CLEANER_PERIOD, 0);
+ // Set minimum age to archive to 10 sec
+ conf.setLong(MobConstants.MIN_AGE_TO_ARCHIVE_KEY, minAgeToArchive);
+ // Set compacted file discharger interval to a half minAgeToArchive
+ conf.setLong("hbase.hfile.compaction.discharger.interval", minAgeToArchive / 2);
+ }
+
+ private void loadData(int start, int num) {
+ try {
+
+ for (int i = 0; i < num; i++) {
+ Put p = new Put(Bytes.toBytes(start + i));
+ p.addColumn(fam, qualifier, mobVal);
+ table.put(p);
+ }
+ admin.flush(table.getName());
+ } catch (Exception e) {
+ LOG.error("MOB file cleaner chore test FAILED", e);
+ assertTrue(false);
+ }
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ admin.disableTable(hdt.getTableName());
+ admin.deleteTable(hdt.getTableName());
+ HTU.shutdownMiniCluster();
+ }
+
+ @Test
+ public void testMobFileCleanerChore() throws InterruptedException, IOException {
+
+ loadData(0, 10);
+ loadData(10, 10);
+ loadData(20, 10);
+ long num = getNumberOfMobFiles(conf, table.getName(), new String(fam));
+ assertEquals(3, num);
+ // Major compact
+ admin.majorCompact(hdt.getTableName(), fam);
+ // wait until compaction is complete
+ while (admin.getCompactionState(hdt.getTableName()) != CompactionState.NONE) {
+ Thread.sleep(100);
+ }
+
+ num = getNumberOfMobFiles(conf, table.getName(), new String(fam));
+ assertEquals(4, num);
+ // We have guarantee, that compcated file discharger will run during this pause
+ // because it has interval less than this wait time
+ LOG.info("Waiting for {}ms", minAgeToArchive + 1000);
+
+ Thread.sleep(minAgeToArchive + 1000);
+ LOG.info("Cleaning up MOB files");
+ // Cleanup again
+ chore.cleanupObsoleteMobFiles(conf, table.getName());
+
+ num = getNumberOfMobFiles(conf, table.getName(), new String(fam));
+ assertEquals(1, num);
+
+ long scanned = scanTable();
+ assertEquals(30, scanned);
+ }
+
+ private long getNumberOfMobFiles(Configuration conf, TableName tableName, String family)
+ throws IOException {
+ FileSystem fs = FileSystem.get(conf);
+ Path dir = MobUtils.getMobFamilyPath(conf, tableName, family);
+ FileStatus[] stat = fs.listStatus(dir);
+ for (FileStatus st : stat) {
+ LOG.debug("DDDD MOB Directory content: {} size={}", st.getPath(), st.getLen());
+ }
+ LOG.debug("MOB Directory content total files: {}", stat.length);
+
+ return stat.length;
+ }
+
+ private long scanTable() {
+ try {
+
+ Result result;
+ ResultScanner scanner = table.getScanner(fam);
+ long counter = 0;
+ while ((result = scanner.next()) != null) {
+ assertTrue(Arrays.equals(result.getValue(fam, qualifier), mobVal));
+ counter++;
+ }
+ return counter;
+ } catch (Exception e) {
+ e.printStackTrace();
+ LOG.error("MOB file cleaner chore test FAILED");
+ if (HTU != null) {
+ assertTrue(false);
+ } else {
+ System.exit(-1);
+ }
+ }
+ return 0;
+ }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileName.java
index c3348756d8b..fecb724bced 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
@@ -45,6 +45,7 @@ public class TestMobFileName {
private Date date;
private String dateStr;
private byte[] startKey;
+ private String regionName = "region";
@Before
public void setUp() {
@@ -56,32 +57,32 @@ public class TestMobFileName {
@Test
public void testHashCode() {
- assertEquals(MobFileName.create(startKey, dateStr, uuid).hashCode(),
- MobFileName.create(startKey, dateStr, uuid).hashCode());
- assertNotSame(MobFileName.create(startKey, dateStr, uuid),
- MobFileName.create(startKey, dateStr, uuid));
+ assertEquals(MobFileName.create(startKey, dateStr, uuid, regionName).hashCode(),
+ MobFileName.create(startKey, dateStr, uuid, regionName).hashCode());
+ assertNotSame(MobFileName.create(startKey, dateStr, uuid, regionName),
+ MobFileName.create(startKey, dateStr, uuid, regionName));
}
@Test
public void testCreate() {
- MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid);
+ MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid, regionName);
assertEquals(mobFileName, MobFileName.create(mobFileName.getFileName()));
}
@Test
public void testGet() {
- MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid);
+ MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid, regionName);
assertEquals(MD5Hash.getMD5AsHex(startKey, 0, startKey.length), mobFileName.getStartKey());
assertEquals(dateStr, mobFileName.getDate());
assertEquals(mobFileName.getFileName(),
- MD5Hash.getMD5AsHex(startKey, 0, startKey.length) + dateStr + uuid);
+ MD5Hash.getMD5AsHex(startKey, 0, startKey.length) + dateStr + uuid + "_" + regionName);
}
@Test
public void testEquals() {
- MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid);
+ MobFileName mobFileName = MobFileName.create(startKey, dateStr, uuid, regionName);
assertTrue(mobFileName.equals(mobFileName));
assertFalse(mobFileName.equals(this));
- assertTrue(mobFileName.equals(MobFileName.create(startKey, dateStr, uuid)));
+ assertTrue(mobFileName.equals(MobFileName.create(startKey, dateStr, uuid, regionName)));
}
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java
similarity index 84%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java
index 3b1fdb78f52..90b3d4085bf 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreCompaction.java
@@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.hadoop.hbase.regionserver;
+package org.apache.hadoop.hbase.mob;
import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY;
import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
@@ -29,14 +29,13 @@ import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
+import java.util.Optional;
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.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;
@@ -58,10 +57,17 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
-import org.apache.hadoop.hbase.mob.MobConstants;
-import org.apache.hadoop.hbase.mob.MobFileCache;
-import org.apache.hadoop.hbase.mob.MobUtils;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
+import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.regionserver.RegionAsTable;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
+import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -78,7 +84,7 @@ import org.slf4j.LoggerFactory;
/**
* Test mob store compaction
*/
-@Category(SmallTests.class)
+@Category(MediumTests.class)
public class TestMobStoreCompaction {
@ClassRule
@@ -177,7 +183,18 @@ public class TestMobStoreCompaction {
// Change the threshold larger than the data size
setMobThreshold(region, COLUMN_FAMILY, 500);
region.initialize();
- region.compactStores();
+
+ List stores = region.getStores();
+ for (HStore store : stores) {
+ // Force major compaction
+ store.triggerMajorCompaction();
+ Optional context = store.requestCompaction(HStore.PRIORITY_USER,
+ CompactionLifeCycleTracker.DUMMY, User.getCurrent());
+ if (!context.isPresent()) {
+ continue;
+ }
+ region.compact(context.get(), store, NoLimitThroughputController.INSTANCE, User.getCurrent());
+ }
assertEquals("After compaction: store files", 1, countStoreFiles());
assertEquals("After compaction: mob file count", compactionThreshold, countMobFiles());
@@ -265,30 +282,6 @@ public class TestMobStoreCompaction {
// region.compactStores();
region.compact(true);
assertEquals("After compaction: store files", 1, countStoreFiles());
- // still have original mob hfiles and now added a mob del file
- assertEquals("After compaction: mob files", numHfiles + 1, countMobFiles());
-
- Scan scan = new Scan();
- scan.setRaw(true);
- InternalScanner scanner = region.getScanner(scan);
- List results = new ArrayList<>();
- scanner.next(results);
- int deleteCount = 0;
- while (!results.isEmpty()) {
- for (Cell c : results) {
- if (c.getTypeByte() == KeyValue.Type.DeleteFamily.getCode()) {
- deleteCount++;
- assertTrue(Bytes.equals(CellUtil.cloneRow(c), deleteRow));
- }
- }
- results.clear();
- scanner.next(results);
- }
- // assert the delete mark is retained after the major compaction
- assertEquals(1, deleteCount);
- scanner.close();
- // assert the deleted cell is not counted
- assertEquals("The cells in mob files", numHfiles - 1, countMobCellsInMobFiles(1));
}
private int countStoreFiles() throws IOException {
@@ -418,39 +411,4 @@ public class TestMobStoreCompaction {
return files.size();
}
-
- private int countMobCellsInMobFiles(int expectedNumDelfiles) throws IOException {
- Configuration copyOfConf = new Configuration(conf);
- copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f);
- CacheConfig cacheConfig = new CacheConfig(copyOfConf);
- Path mobDirPath = MobUtils.getMobFamilyPath(conf, htd.getTableName(), hcd.getNameAsString());
- List sfs = new ArrayList<>();
- int numDelfiles = 0;
- int size = 0;
- if (fs.exists(mobDirPath)) {
- for (FileStatus f : fs.listStatus(mobDirPath)) {
- HStoreFile sf = new HStoreFile(fs, f.getPath(), conf, cacheConfig, BloomType.NONE, true);
- sfs.add(sf);
- if (StoreFileInfo.isDelFile(sf.getPath())) {
- numDelfiles++;
- }
- }
-
- List scanners = StoreFileScanner.getScannersForStoreFiles(sfs, false, true,
- false, false, HConstants.LATEST_TIMESTAMP);
- long timeToPurgeDeletes = Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0);
- long ttl = HStore.determineTTLFromFamily(hcd);
- ScanInfo scanInfo =
- new ScanInfo(copyOfConf, hcd, ttl, timeToPurgeDeletes, CellComparatorImpl.COMPARATOR);
- StoreScanner scanner = new StoreScanner(scanInfo, ScanType.COMPACT_DROP_DELETES, scanners);
- try {
- size += UTIL.countRows(scanner);
- } finally {
- scanner.close();
- }
- }
- // assert the number of the existing del files
- assertEquals(expectedNumDelfiles, numDelfiles);
- return size;
- }
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java
similarity index 98%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java
index 57ceecff058..a5f4191df8d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreScanner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobStoreScanner.java
@@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.hadoop.hbase.regionserver;
+package org.apache.hadoop.hbase.mob;
import java.io.IOException;
import java.util.List;
@@ -42,9 +42,7 @@ import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.io.hfile.CorruptHFileException;
import org.apache.hadoop.hbase.io.hfile.TestHFile;
-import org.apache.hadoop.hbase.mob.MobConstants;
-import org.apache.hadoop.hbase.mob.MobTestUtil;
-import org.apache.hadoop.hbase.mob.MobUtils;
+import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CommonFSUtils;
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 cce5b042250..00000000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java
+++ /dev/null
@@ -1,1214 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.mob.compactions;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.security.Key;
-import java.security.SecureRandom;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Calendar;
-import java.util.Collections;
-import java.util.List;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.Random;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.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.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.Pair;
-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;
-
-import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-@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.getConfiguration().setBoolean("hbase.online.schema.update.enable", true);
- 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(tableName, desc);
- Pair st;
-
- while (null != (st = admin.getAlterStatus(tableName)) && st.getFirst() > 0) {
- LOG.debug(st.getFirst() + " regions left to update");
- Thread.sleep(40);
- }
- 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,
- 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),
- 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),
- 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 = EnvironmentEdgeManager.currentTime();
- byte[] snapshotName1 = Bytes.toBytes("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),
- 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), 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,
- 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,
- 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,
- 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),
- 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),
- 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 extends StoreFile> 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 rows in the given table.
- * @param table to get the scanner
- * @return the number of rows
- */
- private int countMobRows(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));
- return TEST_UTIL.countRows(table, scan);
- }
-
- /**
- * 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,
- new ThreadFactoryBuilder().setNameFormat("MobFileCompactionChore-pool-%d")
- .setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build(),
- (r, 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 00419aa9180..00000000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactionRequest.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.mob.compactions;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartition;
-import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.Assert;
-import org.junit.ClassRule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category(SmallTests.class)
-public class TestPartitionedMobCompactionRequest {
-
- @ClassRule
- public static final HBaseClassTestRule CLASS_RULE =
- HBaseClassTestRule.forClass(TestPartitionedMobCompactionRequest.class);
-
- @Test
- public void testCompactedPartitionId() {
- String startKey1 = "startKey1";
- String startKey2 = "startKey2";
- String date1 = "date1";
- String date2 = "date2";
- CompactionPartitionId partitionId1 = new CompactionPartitionId(startKey1, date1);
- CompactionPartitionId partitionId2 = new CompactionPartitionId(startKey2, date2);
- CompactionPartitionId partitionId3 = new CompactionPartitionId(startKey1, date2);
-
- Assert.assertTrue(partitionId1.equals(partitionId1));
- Assert.assertFalse(partitionId1.equals(partitionId2));
- Assert.assertFalse(partitionId1.equals(partitionId3));
- Assert.assertFalse(partitionId2.equals(partitionId3));
-
- Assert.assertEquals(startKey1, partitionId1.getStartKey());
- Assert.assertEquals(date1, partitionId1.getDate());
- }
-
- @Test
- public void testCompactedPartition() {
- CompactionPartitionId partitionId = new CompactionPartitionId("startKey1", "date1");
- CompactionPartition partition = new CompactionPartition(partitionId);
- FileStatus file = new FileStatus(1, false, 1, 1024, 1, new Path("/test"));
- partition.addFile(file);
- Assert.assertEquals(file, partition.listFiles().get(0));
- }
-}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java
deleted file mode 100644
index cdea37d23f3..00000000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java
+++ /dev/null
@@ -1,959 +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.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.CommonFSUtils;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-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;
-
-import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-@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 = CommonFSUtils.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(EnvironmentEdgeManager.currentTime() - (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(EnvironmentEdgeManager.currentTime() - (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(EnvironmentEdgeManager.currentTime() - (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(EnvironmentEdgeManager.currentTime() - (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(EnvironmentEdgeManager.currentTime() - (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(EnvironmentEdgeManager.currentTime() - (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(EnvironmentEdgeManager.currentTime() - (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 = CommonFSUtils.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 = EnvironmentEdgeManager.currentTime();
- 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 = EnvironmentEdgeManager.currentTime();
- 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
- * @family the family name
- * @qualifier the column qualifier
- * @count the store file number
- * @type the key type
- */
- 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 = EnvironmentEdgeManager.currentTime();
- 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,
- new ThreadFactoryBuilder().setNameFormat("MobFileCompactionChore-pool-%d")
- .setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build(),
- (r, 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;
- }
-
- /**
- * Resets the configuration.
- */
- private void resetConf() {
- conf.setLong(MobConstants.MOB_COMPACTION_MERGEABLE_THRESHOLD,
- MobConstants.DEFAULT_MOB_COMPACTION_MERGEABLE_THRESHOLD);
- conf.setInt(MobConstants.MOB_DELFILE_MAX_COUNT, MobConstants.DEFAULT_MOB_DELFILE_MAX_COUNT);
- conf.setInt(MobConstants.MOB_COMPACTION_BATCH_SIZE,
- MobConstants.DEFAULT_MOB_COMPACTION_BATCH_SIZE);
- }
-
- /**
- * The customized Distributed File System Implementation
- */
- static class FaultyDistributedFileSystem extends DistributedFileSystem {
- private volatile boolean throwException = false;
-
- public FaultyDistributedFileSystem() {
- super();
- }
-
- public void setThrowException(boolean throwException) {
- this.throwException = throwException;
- }
-
- @Override
- public boolean rename(Path src, Path dst) throws IOException {
- if (throwException) {
- throw new IOException("No more files allowed");
- }
- return super.rename(src, dst);
- }
- }
-}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoadReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoadReplication.java
index b591da5d1c5..cd8c046767f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoadReplication.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBulkLoadReplication.java
@@ -25,16 +25,10 @@ import static org.junit.Assert.assertTrue;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
-import java.util.Arrays;
-import java.util.Date;
import java.util.List;
import java.util.Map;
import java.util.Optional;
-import java.util.Random;
-import java.util.UUID;
import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.conf.Configuration;
@@ -50,7 +44,6 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -63,22 +56,14 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.coprocessor.RegionObserver;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile;
-import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
-import org.apache.hadoop.hbase.mob.MobConstants;
-import org.apache.hadoop.hbase.mob.MobFileName;
-import org.apache.hadoop.hbase.mob.MobUtils;
-import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactor;
import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
import org.apache.hadoop.hbase.replication.TestReplicationBase;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.ReplicationTests;
import org.apache.hadoop.hbase.tool.BulkLoadHFilesTool;
import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.CommonFSUtils;
-import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.junit.After;
@@ -249,29 +234,6 @@ public class TestBulkLoadReplication extends TestReplicationBase {
assertEquals(9, BULK_LOADS_COUNT.get());
}
- @Test
- public void testPartionedMOBCompactionBulkLoadDoesntReplicate() throws Exception {
- Path path = createMobFiles(UTIL3);
- ColumnFamilyDescriptor descriptor =
- new ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor(famName);
- ExecutorService pool = null;
- try {
- pool = Executors.newFixedThreadPool(1);
- PartitionedMobCompactor compactor = new PartitionedMobCompactor(UTIL3.getConfiguration(),
- UTIL3.getTestFileSystem(), tableName, descriptor, pool);
- BULK_LOAD_LATCH = new CountDownLatch(1);
- BULK_LOADS_COUNT.set(0);
- compactor.compact(Arrays.asList(UTIL3.getTestFileSystem().listStatus(path)), true);
- assertTrue(BULK_LOAD_LATCH.await(1, TimeUnit.SECONDS));
- Thread.sleep(400);
- assertEquals(1, BULK_LOADS_COUNT.get());
- } finally {
- if (pool != null && !pool.isTerminated()) {
- pool.shutdownNow();
- }
- }
- }
-
protected void assertBulkLoadConditions(TableName tableName, byte[] row, byte[] value,
HBaseTestingUtility utility, Table... tables) throws Exception {
BULK_LOAD_LATCH = new CountDownLatch(3);
@@ -334,35 +296,6 @@ public class TestBulkLoadReplication extends TestReplicationBase {
return hFileLocation.getAbsoluteFile().getAbsolutePath();
}
- private Path createMobFiles(HBaseTestingUtility util) throws IOException {
- Path testDir = CommonFSUtils.getRootDir(util.getConfiguration());
- Path mobTestDir = new Path(testDir, MobConstants.MOB_DIR_NAME);
- Path basePath = new Path(new Path(mobTestDir, tableName.getNameAsString()), "f");
- HFileContext meta = new HFileContextBuilder().withBlockSize(8 * 1024).build();
- MobFileName mobFileName = null;
- byte[] mobFileStartRow = new byte[32];
- for (byte rowKey : Bytes.toBytes("01234")) {
- mobFileName = MobFileName.create(mobFileStartRow, MobUtils.formatDate(new Date()),
- UUID.randomUUID().toString().replaceAll("-", ""));
- StoreFileWriter mobFileWriter = new StoreFileWriter.Builder(util.getConfiguration(),
- new CacheConfig(util.getConfiguration()), util.getTestFileSystem()).withFileContext(meta)
- .withFilePath(new Path(basePath, mobFileName.getFileName())).build();
- long now = EnvironmentEdgeManager.currentTime();
- try {
- for (int i = 0; i < 10; i++) {
- byte[] key = Bytes.add(Bytes.toBytes(rowKey), Bytes.toBytes(i));
- byte[] dummyData = new byte[5000];
- new Random().nextBytes(dummyData);
- mobFileWriter.append(
- new KeyValue(key, famName, Bytes.toBytes("1"), now, KeyValue.Type.Put, dummyData));
- }
- } finally {
- mobFileWriter.close();
- }
- }
- return basePath;
- }
-
public static class BulkReplicationTestObserver implements RegionCoprocessor {
String clusterName;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
index 9ef24302ad9..dba0c141952 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
+import java.util.Optional;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CompatibilityFactory;
import org.apache.hadoop.hbase.HBaseClassTestRule;
@@ -53,6 +54,10 @@ import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.master.LoadBalancer;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
+import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
+import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.test.MetricsAssertHelper;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@@ -507,7 +512,19 @@ public class TestRegionServerMetrics {
setMobThreshold(region, cf, 100);
// metrics are reset by the region initialization
region.initialize();
- region.compact(true);
+ // This is how we MOB compact region
+ List stores = region.getStores();
+ for (HStore store : stores) {
+ // Force major compaction
+ store.triggerMajorCompaction();
+ Optional context = store.requestCompaction(HStore.PRIORITY_USER,
+ CompactionLifeCycleTracker.DUMMY, User.getCurrent());
+ if (!context.isPresent()) {
+ continue;
+ }
+ region.compact(context.get(), store, NoLimitThroughputController.INSTANCE,
+ User.getCurrent());
+ }
metricsRegionServer.getRegionServerWrapper().forceRecompute();
assertCounter("cellsCountCompactedFromMob", numHfiles);
assertCounter("cellsCountCompactedToMob", 0);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java
index 9a9d92f2dd5..dcdc51800d4 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
@@ -389,7 +389,7 @@ public class BaseTestHBaseFsck {
String startKey = mobFileName.getStartKey();
String date = mobFileName.getDate();
return MobFileName
- .create(startKey, date, TEST_UTIL.getRandomUUID().toString().replaceAll("-", ""))
+ .create(startKey, date, 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 a32021572a7..15a8ce6e3d3 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
@@ -102,6 +102,7 @@ public class TestHBaseFsckMOB extends BaseTestHBaseFsck {
/**
* This creates a table and then corrupts a mob file. Hbck should quarantine the file.
*/
+ @SuppressWarnings("deprecation")
@Test
public void testQuarantineCorruptMobFile() throws Exception {
TableName table = TableName.valueOf(name.getMethodName());
| | |