diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
index 54b82b2598d..010fd37850e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALSplitter.java
@@ -515,13 +515,14 @@ public class WALSplitter {
* @param fs
* @param logEntry
* @param rootDir HBase root dir.
- * @param fileBeingSplit the file being split currently. Used to generate tmp file name.
+ * @param fileNameBeingSplit the file being split currently. Used to generate tmp file name.
* @return Path to file into which to dump split log edits.
* @throws IOException
*/
@SuppressWarnings("deprecation")
- private static Path getRegionSplitEditsPath(final FileSystem fs,
- final Entry logEntry, final Path rootDir, FileStatus fileBeingSplit)
+ @VisibleForTesting
+ static Path getRegionSplitEditsPath(final FileSystem fs,
+ final Entry logEntry, final Path rootDir, String fileNameBeingSplit)
throws IOException {
Path tableDir = FSUtils.getTableDir(rootDir, logEntry.getKey().getTablename());
String encodedRegionName = Bytes.toString(logEntry.getKey().getEncodedRegionName());
@@ -556,7 +557,7 @@ public class WALSplitter {
// Append file name ends with RECOVERED_LOG_TMPFILE_SUFFIX to ensure
// region's replayRecoveredEdits will not delete it
String fileName = formatRecoveredEditsFileName(logEntry.getKey().getSequenceId());
- fileName = getTmpRecoveredEditsFileName(fileName + "-" + fileBeingSplit.getPath().getName());
+ fileName = getTmpRecoveredEditsFileName(fileName + "-" + fileNameBeingSplit);
return new Path(dir, fileName);
}
@@ -1518,7 +1519,7 @@ public class WALSplitter {
* @return a path with a write for that path. caller should close.
*/
private WriterAndPath createWAP(byte[] region, Entry entry, Path rootdir) throws IOException {
- Path regionedits = getRegionSplitEditsPath(fs, entry, rootdir, fileBeingSplit);
+ Path regionedits = getRegionSplitEditsPath(fs, entry, rootdir, fileBeingSplit.getPath().getName());
if (regionedits == null) {
return null;
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
new file mode 100644
index 00000000000..a6f8373eb0d
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestPartialResultsFromClientSide.java
@@ -0,0 +1,832 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.client.ClientScanner;
+import org.apache.hadoop.hbase.client.Delete;
+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.filter.ColumnPrefixFilter;
+import org.apache.hadoop.hbase.filter.ColumnRangeFilter;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
+import org.apache.hadoop.hbase.filter.FirstKeyValueMatchingQualifiersFilter;
+import org.apache.hadoop.hbase.filter.RandomRowFilter;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * These tests are focused on testing how partial results appear to a client. Partial results are
+ * {@link Result}s that contain only a portion of a row's complete list of cells. Partial results
+ * are formed when the server breaches its maximum result size when trying to service a client's RPC
+ * request. It is the responsibility of the scanner on the client side to recognize when partial
+ * results have been returned and to take action to form the complete results.
+ *
+ * Unless the flag {@link Scan#setAllowPartialResults(boolean)} has been set to true, the caller of
+ * {@link ResultScanner#next()} should never see partial results.
+ */
+@Category(MediumTests.class)
+public class TestPartialResultsFromClientSide {
+ private static final Log LOG = LogFactory.getLog(TestPartialResultsFromClientSide.class);
+
+ private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+ private static Table TABLE = null;
+
+ /**
+ * Table configuration
+ */
+ private static TableName TABLE_NAME = TableName.valueOf("testTable");
+
+ private static int NUM_ROWS = 5;
+ private static byte[] ROW = Bytes.toBytes("testRow");
+ private static byte[][] ROWS = HTestConst.makeNAscii(ROW, NUM_ROWS);
+
+ // Should keep this value below 10 to keep generation of expected kv's simple. If above 10 then
+ // table/row/cf1/... will be followed by table/row/cf10/... instead of table/row/cf2/... which
+ // breaks the simple generation of expected kv's
+ private static int NUM_FAMILIES = 10;
+ private static byte[] FAMILY = Bytes.toBytes("testFamily");
+ private static byte[][] FAMILIES = HTestConst.makeNAscii(FAMILY, NUM_FAMILIES);
+
+ private static int NUM_QUALIFIERS = 10;
+ private static byte[] QUALIFIER = Bytes.toBytes("testQualifier");
+ private static byte[][] QUALIFIERS = HTestConst.makeNAscii(QUALIFIER, NUM_QUALIFIERS);
+
+ private static int VALUE_SIZE = 1024;
+ private static byte[] VALUE = Bytes.createMaxByteArray(VALUE_SIZE);
+
+ private static int NUM_COLS = NUM_FAMILIES * NUM_QUALIFIERS;
+
+ // Approximation of how large the heap size of cells in our table. Should be accessed through
+ // getCellHeapSize().
+ private static long CELL_HEAP_SIZE = -1;
+
+ @BeforeClass
+ public static void setUpBeforeClass() throws Exception {
+ TEST_UTIL.startMiniCluster(3);
+ TABLE = createTestTable(TABLE_NAME, ROWS, FAMILIES, QUALIFIERS, VALUE);
+ }
+
+ static Table createTestTable(TableName name, byte[][] rows, byte[][] families,
+ byte[][] qualifiers, byte[] cellValue) throws IOException {
+ Table ht = TEST_UTIL.createTable(name, families);
+ List puts = createPuts(rows, families, qualifiers, cellValue);
+ ht.put(puts);
+
+ return ht;
+ }
+
+ @AfterClass
+ public static void tearDownAfterClass() throws Exception {
+ TEST_UTIL.shutdownMiniCluster();
+ }
+
+ /**
+ * Ensure that the expected key values appear in a result returned from a scanner that is
+ * combining partial results into complete results
+ * @throws Exception
+ */
+ @Test
+ public void testExpectedValuesOfPartialResults() throws Exception {
+ testExpectedValuesOfPartialResults(false);
+ testExpectedValuesOfPartialResults(true);
+ }
+
+ public void testExpectedValuesOfPartialResults(boolean reversed) throws Exception {
+ Scan partialScan = new Scan();
+ partialScan.setMaxVersions();
+ // Max result size of 1 ensures that each RPC request will return a single cell. The scanner
+ // will need to reconstruct the results into a complete result before returning to the caller
+ partialScan.setMaxResultSize(1);
+ partialScan.setReversed(reversed);
+ ResultScanner partialScanner = TABLE.getScanner(partialScan);
+
+ final int startRow = reversed ? ROWS.length - 1 : 0;
+ final int endRow = reversed ? -1 : ROWS.length;
+ final int loopDelta = reversed ? -1 : 1;
+ String message;
+
+ for (int row = startRow; row != endRow; row = row + loopDelta) {
+ message = "Ensuring the expected keyValues are present for row " + row;
+ List expectedKeyValues = createKeyValuesForRow(ROWS[row], FAMILIES, QUALIFIERS, VALUE);
+ Result result = partialScanner.next();
+ assertFalse(result.isPartial());
+ verifyResult(result, expectedKeyValues, message);
+ }
+
+ partialScanner.close();
+ }
+
+ /**
+ * Ensure that we only see Results marked as partial when the allowPartial flag is set
+ * @throws Exception
+ */
+ @Test
+ public void testAllowPartialResults() throws Exception {
+ Scan scan = new Scan();
+ scan.setAllowPartialResults(true);
+ scan.setMaxResultSize(1);
+ ResultScanner scanner = TABLE.getScanner(scan);
+ Result result = scanner.next();
+
+ assertTrue(result != null);
+ assertTrue(result.isPartial());
+ assertTrue(result.rawCells() != null);
+ assertTrue(result.rawCells().length == 1);
+
+ scanner.close();
+
+ scan.setAllowPartialResults(false);
+ scanner = TABLE.getScanner(scan);
+ result = scanner.next();
+
+ assertTrue(result != null);
+ assertTrue(!result.isPartial());
+ assertTrue(result.rawCells() != null);
+ assertTrue(result.rawCells().length == NUM_COLS);
+
+ scanner.close();
+ }
+
+ /**
+ * Ensure that the results returned from a scanner that retrieves all results in a single RPC call
+ * matches the results that are returned from a scanner that must incrementally combine partial
+ * results into complete results. A variety of scan configurations can be tested
+ * @throws Exception
+ */
+ @Test
+ public void testEquivalenceOfScanResults() throws Exception {
+ Scan oneShotScan = new Scan();
+ oneShotScan.setMaxResultSize(Long.MAX_VALUE);
+
+ Scan partialScan = new Scan(oneShotScan);
+ partialScan.setMaxResultSize(1);
+
+ testEquivalenceOfScanResults(TABLE, oneShotScan, partialScan);
+ }
+
+ public void testEquivalenceOfScanResults(Table table, Scan scan1, Scan scan2) throws Exception {
+ ResultScanner scanner1 = table.getScanner(scan1);
+ ResultScanner scanner2 = table.getScanner(scan2);
+
+ Result r1 = null;
+ Result r2 = null;
+ int count = 0;
+
+ while ((r1 = scanner1.next()) != null) {
+ r2 = scanner2.next();
+
+ assertTrue(r2 != null);
+ compareResults(r1, r2, "Comparing result #" + count);
+ count++;
+ }
+
+ r2 = scanner2.next();
+ assertTrue("r2: " + r2 + " Should be null", r2 == null);
+
+ scanner1.close();
+ scanner2.close();
+ }
+
+ /**
+ * Order of cells in partial results matches the ordering of cells from complete results
+ * @throws Exception
+ */
+ @Test
+ public void testOrderingOfCellsInPartialResults() throws Exception {
+ Scan scan = new Scan();
+
+ for (int col = 1; col <= NUM_COLS; col++) {
+ scan.setMaxResultSize(getResultSizeForNumberOfCells(col));
+ testOrderingOfCellsInPartialResults(scan);
+
+ // Test again with a reversed scanner
+ scan.setReversed(true);
+ testOrderingOfCellsInPartialResults(scan);
+ }
+ }
+
+ public void testOrderingOfCellsInPartialResults(final Scan basePartialScan) throws Exception {
+ // Scan that retrieves results in pieces (partials). By setting allowPartialResults to be true
+ // the results will NOT be reconstructed and instead the caller will see the partial results
+ // returned by the server
+ Scan partialScan = new Scan(basePartialScan);
+ partialScan.setAllowPartialResults(true);
+ ResultScanner partialScanner = TABLE.getScanner(partialScan);
+
+ // Scan that retrieves all table results in single RPC request
+ Scan oneShotScan = new Scan(basePartialScan);
+ oneShotScan.setMaxResultSize(Long.MAX_VALUE);
+ oneShotScan.setCaching(ROWS.length);
+ ResultScanner oneShotScanner = TABLE.getScanner(oneShotScan);
+
+ Result oneShotResult = oneShotScanner.next();
+ Result partialResult = null;
+ int iterationCount = 0;
+
+ while (oneShotResult != null && oneShotResult.rawCells() != null) {
+ List aggregatePartialCells = new ArrayList();
+ do {
+ partialResult = partialScanner.next();
+ assertTrue("Partial Result is null. iteration: " + iterationCount, partialResult != null);
+ assertTrue("Partial cells are null. iteration: " + iterationCount,
+ partialResult.rawCells() != null);
+
+ for (Cell c : partialResult.rawCells()) {
+ aggregatePartialCells.add(c);
+ }
+ } while (partialResult.isPartial());
+
+ assertTrue("Number of cells differs. iteration: " + iterationCount,
+ oneShotResult.rawCells().length == aggregatePartialCells.size());
+ final Cell[] oneShotCells = oneShotResult.rawCells();
+ for (int cell = 0; cell < oneShotCells.length; cell++) {
+ Cell oneShotCell = oneShotCells[cell];
+ Cell partialCell = aggregatePartialCells.get(cell);
+
+ assertTrue("One shot cell was null", oneShotCell != null);
+ assertTrue("Partial cell was null", partialCell != null);
+ assertTrue("Cell differs. oneShotCell:" + oneShotCell + " partialCell:" + partialCell,
+ oneShotCell.equals(partialCell));
+ }
+
+ oneShotResult = oneShotScanner.next();
+ iterationCount++;
+ }
+
+ assertTrue(partialScanner.next() == null);
+
+ partialScanner.close();
+ oneShotScanner.close();
+ }
+
+ /**
+ * Setting the max result size allows us to control how many cells we expect to see on each call
+ * to next on the scanner. Test a variety of different sizes for correctness
+ * @throws Exception
+ */
+ @Test
+ public void testExpectedNumberOfCellsPerPartialResult() throws Exception {
+ Scan scan = new Scan();
+ testExpectedNumberOfCellsPerPartialResult(scan);
+
+ scan.setReversed(true);
+ testExpectedNumberOfCellsPerPartialResult(scan);
+ }
+
+ public void testExpectedNumberOfCellsPerPartialResult(Scan baseScan) throws Exception {
+ for (int expectedCells = 1; expectedCells <= NUM_COLS; expectedCells++) {
+ testExpectedNumberOfCellsPerPartialResult(baseScan, expectedCells);
+ }
+ }
+
+ public void testExpectedNumberOfCellsPerPartialResult(Scan baseScan, int expectedNumberOfCells)
+ throws Exception {
+
+ if (LOG.isInfoEnabled()) LOG.info("groupSize:" + expectedNumberOfCells);
+
+ // Use the cellHeapSize to set maxResultSize such that we know how many cells to expect back
+ // from the call. The returned results should NOT exceed expectedNumberOfCells but may be less
+ // than it in cases where expectedNumberOfCells is not an exact multiple of the number of
+ // columns in the table.
+ Scan scan = new Scan(baseScan);
+ scan.setAllowPartialResults(true);
+ scan.setMaxResultSize(getResultSizeForNumberOfCells(expectedNumberOfCells));
+
+ ResultScanner scanner = TABLE.getScanner(scan);
+ Result result = null;
+ byte[] prevRow = null;
+ while ((result = scanner.next()) != null) {
+ assertTrue(result.rawCells() != null);
+
+ // Cases when cell count won't equal expectedNumberOfCells:
+ // 1. Returned result is the final result needed to form the complete result for that row
+ // 2. It is the first result we have seen for that row and thus may have been fetched as
+ // the last group of cells that fit inside the maxResultSize
+ assertTrue(
+ "Result's cell count differed from expected number. result: " + result,
+ result.rawCells().length == expectedNumberOfCells || !result.isPartial()
+ || !Bytes.equals(prevRow, result.getRow()));
+ prevRow = result.getRow();
+ }
+
+ scanner.close();
+ }
+
+ /**
+ * @return The approximate heap size of a cell in the test table. All cells should have
+ * approximately the same heap size, so the value is cached to avoid repeating the
+ * calculation
+ * @throws Exception
+ */
+ private long getCellHeapSize() throws Exception {
+ if (CELL_HEAP_SIZE == -1) {
+ // Do a partial scan that will return a single result with a single cell
+ Scan scan = new Scan();
+ scan.setMaxResultSize(1);
+ scan.setAllowPartialResults(true);
+ ResultScanner scanner = TABLE.getScanner(scan);
+
+ Result result = scanner.next();
+
+ assertTrue(result != null);
+ assertTrue(result.rawCells() != null);
+ assertTrue(result.rawCells().length == 1);
+
+ CELL_HEAP_SIZE = CellUtil.estimatedHeapSizeOf(result.rawCells()[0]);
+ if (LOG.isInfoEnabled()) LOG.info("Cell heap size: " + CELL_HEAP_SIZE);
+ scanner.close();
+ }
+
+ return CELL_HEAP_SIZE;
+ }
+
+ /**
+ * @param numberOfCells
+ * @return the result size that should be used in {@link Scan#setMaxResultSize(long)} if you want
+ * the server to return exactly numberOfCells cells
+ * @throws Exception
+ */
+ private long getResultSizeForNumberOfCells(int numberOfCells) throws Exception {
+ return getCellHeapSize() * numberOfCells;
+ }
+
+ /**
+ * Test various combinations of batching and partial results for correctness
+ */
+ @Test
+ public void testPartialResultsAndBatch() throws Exception {
+ for (int batch = 1; batch <= NUM_COLS / 4; batch++) {
+ for (int cellsPerPartial = 1; cellsPerPartial <= NUM_COLS / 4; cellsPerPartial++) {
+ testPartialResultsAndBatch(batch, cellsPerPartial);
+ }
+ }
+ }
+
+ public void testPartialResultsAndBatch(final int batch, final int cellsPerPartialResult)
+ throws Exception {
+ if (LOG.isInfoEnabled()) {
+ LOG.info("batch: " + batch + " cellsPerPartialResult: " + cellsPerPartialResult);
+ }
+
+ Scan scan = new Scan();
+ scan.setMaxResultSize(getResultSizeForNumberOfCells(cellsPerPartialResult));
+ scan.setBatch(batch);
+ ResultScanner scanner = TABLE.getScanner(scan);
+ Result result = scanner.next();
+ int repCount = 0;
+
+ while ((result = scanner.next()) != null) {
+ assertTrue(result.rawCells() != null);
+
+ if (result.isPartial()) {
+ final String error =
+ "Cells:" + result.rawCells().length + " Batch size:" + batch
+ + " cellsPerPartialResult:" + cellsPerPartialResult + " rep:" + repCount;
+ assertTrue(error, result.rawCells().length <= Math.min(batch, cellsPerPartialResult));
+ } else {
+ assertTrue(result.rawCells().length <= batch);
+ }
+ repCount++;
+ }
+
+ scanner.close();
+ }
+
+ /**
+ * Test the method {@link Result#createCompleteResult(List, Result)}
+ * @throws Exception
+ */
+ @Test
+ public void testPartialResultsReassembly() throws Exception {
+ Scan scan = new Scan();
+ testPartialResultsReassembly(scan);
+ scan.setReversed(true);
+ testPartialResultsReassembly(scan);
+ }
+
+ public void testPartialResultsReassembly(Scan scanBase) throws Exception {
+ Scan partialScan = new Scan(scanBase);
+ partialScan.setMaxResultSize(1);
+ partialScan.setAllowPartialResults(true);
+ ResultScanner partialScanner = TABLE.getScanner(partialScan);
+
+ Scan oneShotScan = new Scan(scanBase);
+ oneShotScan.setMaxResultSize(Long.MAX_VALUE);
+ ResultScanner oneShotScanner = TABLE.getScanner(oneShotScan);
+
+ ArrayList partials = new ArrayList<>();
+ for (int i = 0; i < NUM_ROWS; i++) {
+ Result partialResult = null;
+ Result completeResult = null;
+ Result oneShotResult = null;
+ partials.clear();
+
+ do {
+ partialResult = partialScanner.next();
+ partials.add(partialResult);
+ } while (partialResult != null && partialResult.isPartial());
+
+ completeResult = Result.createCompleteResult(partials);
+ oneShotResult = oneShotScanner.next();
+
+ compareResults(completeResult, oneShotResult, null);
+ }
+
+ assertTrue(oneShotScanner.next() == null);
+ assertTrue(partialScanner.next() == null);
+
+ oneShotScanner.close();
+ partialScanner.close();
+ }
+
+ /**
+ * When reconstructing the complete result from its partials we ensure that the row of each
+ * partial result is the same. If one of the rows differs, an exception is thrown.
+ */
+ @Test
+ public void testExceptionThrownOnMismatchedPartialResults() throws IOException {
+ assertTrue(NUM_ROWS >= 2);
+
+ ArrayList partials = new ArrayList<>();
+ Scan scan = new Scan();
+ scan.setMaxResultSize(Long.MAX_VALUE);
+ ResultScanner scanner = TABLE.getScanner(scan);
+ Result r1 = scanner.next();
+ partials.add(r1);
+ Result r2 = scanner.next();
+ partials.add(r2);
+
+ assertFalse(Bytes.equals(r1.getRow(), r2.getRow()));
+
+ try {
+ Result.createCompleteResult(partials);
+ fail("r1 and r2 are from different rows. It should not be possible to combine them into"
+ + " a single result");
+ } catch (IOException e) {
+ }
+
+ scanner.close();
+ }
+
+ /**
+ * When a scan has a filter where {@link org.apache.hadoop.hbase.filter.Filter#hasFilterRow()} is
+ * true, the scanner should not return partial results. The scanner cannot return partial results
+ * because the entire row needs to be read for the include/exclude decision to be made
+ */
+ @Test
+ public void testNoPartialResultsWhenRowFilterPresent() throws Exception {
+ Scan scan = new Scan();
+ scan.setMaxResultSize(1);
+ scan.setAllowPartialResults(true);
+ // If a filter hasFilter() is true then partial results should not be returned else filter
+ // application server side would break.
+ scan.setFilter(new RandomRowFilter(1.0f));
+ ResultScanner scanner = TABLE.getScanner(scan);
+
+ Result r = null;
+ while ((r = scanner.next()) != null) {
+ assertFalse(r.isPartial());
+ }
+
+ scanner.close();
+ }
+
+ /**
+ * Examine the interaction between the maxResultSize and caching. If the caching limit is reached
+ * before the maxResultSize limit, we should not see partial results. On the other hand, if the
+ * maxResultSize limit is reached before the caching limit, it is likely that partial results will
+ * be seen.
+ * @throws Exception
+ */
+ @Test
+ public void testPartialResultsAndCaching() throws Exception {
+ for (int caching = 1; caching <= NUM_ROWS; caching++) {
+ for (int maxResultRows = 0; maxResultRows <= NUM_ROWS; maxResultRows++) {
+ testPartialResultsAndCaching(maxResultRows, caching);
+ }
+ }
+ }
+
+ /**
+ * @param resultSizeRowLimit The row limit that will be enforced through maxResultSize
+ * @param cachingRowLimit The row limit that will be enforced through caching
+ * @throws Exception
+ */
+ public void testPartialResultsAndCaching(int resultSizeRowLimit, int cachingRowLimit)
+ throws Exception {
+ Scan scan = new Scan();
+ scan.setAllowPartialResults(true);
+
+ // The number of cells specified in the call to getResultSizeForNumberOfCells is offset to
+ // ensure that the result size we specify is not an exact multiple of the number of cells
+ // in a row. This ensures that partial results will be returned when the result size limit
+ // is reached before the caching limit.
+ int cellOffset = NUM_COLS / 3;
+ long maxResultSize = getResultSizeForNumberOfCells(resultSizeRowLimit * NUM_COLS + cellOffset);
+ scan.setMaxResultSize(maxResultSize);
+ scan.setCaching(cachingRowLimit);
+
+ ResultScanner scanner = TABLE.getScanner(scan);
+ ClientScanner clientScanner = (ClientScanner) scanner;
+ Result r = null;
+
+ // Approximate the number of rows we expect will fit into the specified max rsult size. If this
+ // approximation is less than caching, then we expect that the max result size limit will be
+ // hit before the caching limit and thus partial results may be seen
+ boolean expectToSeePartialResults = resultSizeRowLimit < cachingRowLimit;
+ while ((r = clientScanner.next()) != null) {
+ assertTrue(!r.isPartial() || expectToSeePartialResults);
+ }
+
+ scanner.close();
+ }
+
+ /**
+ * Small scans should not return partial results because it would prevent small scans from
+ * retrieving all of the necessary results in a single RPC request which is what makese small
+ * scans useful. Thus, ensure that even when {@link Scan#getAllowPartialResults()} is true, small
+ * scans do not return partial results
+ * @throws Exception
+ */
+ @Test
+ public void testSmallScansDoNotAllowPartials() throws Exception {
+ Scan scan = new Scan();
+ testSmallScansDoNotAllowPartials(scan);
+ scan.setReversed(true);
+ testSmallScansDoNotAllowPartials(scan);
+ }
+
+ public void testSmallScansDoNotAllowPartials(Scan baseScan) throws Exception {
+ Scan scan = new Scan(baseScan);
+ scan.setAllowPartialResults(true);
+ scan.setSmall(true);
+ scan.setMaxResultSize(1);
+
+ ResultScanner scanner = TABLE.getScanner(scan);
+ Result r = null;
+
+ while ((r = scanner.next()) != null) {
+ assertFalse(r.isPartial());
+ }
+
+ scanner.close();
+ }
+
+ /**
+ * Make puts to put the input value into each combination of row, family, and qualifier
+ * @param rows
+ * @param families
+ * @param qualifiers
+ * @param value
+ * @return
+ * @throws IOException
+ */
+ static ArrayList createPuts(byte[][] rows, byte[][] families, byte[][] qualifiers,
+ byte[] value) throws IOException {
+ Put put;
+ ArrayList puts = new ArrayList<>();
+
+ for (int row = 0; row < rows.length; row++) {
+ put = new Put(rows[row]);
+ for (int fam = 0; fam < families.length; fam++) {
+ for (int qual = 0; qual < qualifiers.length; qual++) {
+ KeyValue kv = new KeyValue(rows[row], families[fam], qualifiers[qual], qual, value);
+ put.add(kv);
+ }
+ }
+ puts.add(put);
+ }
+
+ return puts;
+ }
+
+ /**
+ * Make key values to represent each possible combination of family and qualifier in the specified
+ * row.
+ * @param row
+ * @param families
+ * @param qualifiers
+ * @param value
+ * @return
+ */
+ static ArrayList createKeyValuesForRow(byte[] row, byte[][] families, byte[][] qualifiers,
+ byte[] value) {
+ ArrayList outList = new ArrayList<>();
+ for (int fam = 0; fam < families.length; fam++) {
+ for (int qual = 0; qual < qualifiers.length; qual++) {
+ outList.add(new KeyValue(row, families[fam], qualifiers[qual], qual, value));
+ }
+ }
+ return outList;
+ }
+
+ /**
+ * Verifies that result contains all the key values within expKvList. Fails the test otherwise
+ * @param result
+ * @param expKvList
+ * @param msg
+ */
+ static void verifyResult(Result result, List expKvList, String msg) {
+ if (LOG.isInfoEnabled()) {
+ LOG.info(msg);
+ LOG.info("Expected count: " + expKvList.size());
+ LOG.info("Actual count: " + result.size());
+ }
+
+ if (expKvList.size() == 0) return;
+
+ int i = 0;
+ for (Cell kv : result.rawCells()) {
+ if (i >= expKvList.size()) {
+ break; // we will check the size later
+ }
+
+ Cell kvExp = expKvList.get(i++);
+ assertTrue("Not equal. get kv: " + kv.toString() + " exp kv: " + kvExp.toString(),
+ kvExp.equals(kv));
+ }
+
+ assertEquals(expKvList.size(), result.size());
+ }
+
+ /**
+ * Compares two results and fails the test if the results are different
+ * @param r1
+ * @param r2
+ * @param message
+ */
+ static void compareResults(Result r1, Result r2, final String message) {
+ if (LOG.isInfoEnabled()) {
+ if (message != null) LOG.info(message);
+ LOG.info("r1: " + r1);
+ LOG.info("r2: " + r2);
+ }
+
+ final String failureMessage = "Results r1:" + r1 + " \nr2:" + r2 + " are not equivalent";
+ if (r1 == null && r2 == null) fail(failureMessage);
+ else if (r1 == null || r2 == null) fail(failureMessage);
+
+ try {
+ Result.compareResults(r1, r2);
+ } catch (Exception e) {
+ fail(failureMessage);
+ }
+ }
+
+ @Test
+ public void testReadPointAndPartialResults() throws Exception {
+ TableName testName = TableName.valueOf("testReadPointAndPartialResults");
+ int numRows = 5;
+ int numFamilies = 5;
+ int numQualifiers = 5;
+ byte[][] rows = HTestConst.makeNAscii(Bytes.toBytes("testRow"), numRows);
+ byte[][] families = HTestConst.makeNAscii(Bytes.toBytes("testFamily"), numFamilies);
+ byte[][] qualifiers = HTestConst.makeNAscii(Bytes.toBytes("testQualifier"), numQualifiers);
+ byte[] value = Bytes.createMaxByteArray(100);
+
+ Table tmpTable = createTestTable(testName, rows, families, qualifiers, value);
+
+ Scan scan = new Scan();
+ scan.setMaxResultSize(1);
+ scan.setAllowPartialResults(true);
+
+ // Open scanner before deletes
+ ResultScanner scanner = tmpTable.getScanner(scan);
+
+ Delete delete1 = new Delete(rows[0]);
+ delete1.addColumn(families[0], qualifiers[0], 0);
+ tmpTable.delete(delete1);
+
+ Delete delete2 = new Delete(rows[1]);
+ delete2.addColumn(families[1], qualifiers[1], 1);
+ tmpTable.delete(delete2);
+
+ // Should see all cells because scanner was opened prior to deletes
+ int scannerCount = countCellsFromScanner(scanner);
+ int expectedCount = numRows * numFamilies * numQualifiers;
+ assertTrue("scannerCount: " + scannerCount + " expectedCount: " + expectedCount,
+ scannerCount == expectedCount);
+
+ // Minus 2 for the two cells that were deleted
+ scanner = tmpTable.getScanner(scan);
+ scannerCount = countCellsFromScanner(scanner);
+ expectedCount = numRows * numFamilies * numQualifiers - 2;
+ assertTrue("scannerCount: " + scannerCount + " expectedCount: " + expectedCount,
+ scannerCount == expectedCount);
+
+ scanner = tmpTable.getScanner(scan);
+ // Put in 2 new rows. The timestamps differ from the deleted rows
+ Put put1 = new Put(rows[0]);
+ put1.add(new KeyValue(rows[0], families[0], qualifiers[0], 1, value));
+ tmpTable.put(put1);
+
+ Put put2 = new Put(rows[1]);
+ put2.add(new KeyValue(rows[1], families[1], qualifiers[1], 2, value));
+ tmpTable.put(put2);
+
+ // Scanner opened prior to puts. Cell count shouldn't have changed
+ scannerCount = countCellsFromScanner(scanner);
+ expectedCount = numRows * numFamilies * numQualifiers - 2;
+ assertTrue("scannerCount: " + scannerCount + " expectedCount: " + expectedCount,
+ scannerCount == expectedCount);
+
+ // Now the scanner should see the cells that were added by puts
+ scanner = tmpTable.getScanner(scan);
+ scannerCount = countCellsFromScanner(scanner);
+ expectedCount = numRows * numFamilies * numQualifiers;
+ assertTrue("scannerCount: " + scannerCount + " expectedCount: " + expectedCount,
+ scannerCount == expectedCount);
+
+ TEST_UTIL.deleteTable(testName);
+ }
+
+ /**
+ * Exhausts the scanner by calling next repetitively. Once completely exhausted, close scanner and
+ * return total cell count
+ * @param scanner
+ * @return
+ * @throws Exception
+ */
+ private int countCellsFromScanner(ResultScanner scanner) throws Exception {
+ Result result = null;
+ int numCells = 0;
+ while ((result = scanner.next()) != null) {
+ numCells += result.rawCells().length;
+ }
+
+ scanner.close();
+ return numCells;
+ }
+
+ /**
+ * Test partial Result re-assembly in the presence of different filters. The Results from the
+ * partial scanner should match the Results returned from a scanner that receives all of the
+ * results in one RPC to the server. The partial scanner is tested with a variety of different
+ * result sizes (all of which are less than the size necessary to fetch an entire row)
+ * @throws Exception
+ */
+ @Test
+ public void testPartialResultsWithColumnFilter() throws Exception {
+ testPartialResultsWithColumnFilter(new FirstKeyOnlyFilter());
+ testPartialResultsWithColumnFilter(new ColumnPrefixFilter(Bytes.toBytes("testQualifier5")));
+ testPartialResultsWithColumnFilter(new ColumnRangeFilter(Bytes.toBytes("testQualifer1"), true,
+ Bytes.toBytes("testQualifier7"), true));
+
+ Set qualifiers = new LinkedHashSet<>();
+ qualifiers.add(Bytes.toBytes("testQualifier5"));
+ testPartialResultsWithColumnFilter(new FirstKeyValueMatchingQualifiersFilter(qualifiers));
+ }
+
+ public void testPartialResultsWithColumnFilter(Filter filter) throws Exception {
+ assertTrue(!filter.hasFilterRow());
+
+ Scan partialScan = new Scan();
+ partialScan.setFilter(filter);
+
+ Scan oneshotScan = new Scan();
+ oneshotScan.setFilter(filter);
+ oneshotScan.setMaxResultSize(Long.MAX_VALUE);
+
+ for (int i = 1; i <= NUM_COLS; i++) {
+ partialScan.setMaxResultSize(getResultSizeForNumberOfCells(i));
+ testEquivalenceOfScanResults(TABLE, partialScan, oneshotScan);
+ }
+ }
+}
\ No newline at end of file
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotCloneIndependence.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotCloneIndependence.java
new file mode 100644
index 00000000000..dcf20e5f985
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMobSnapshotCloneIndependence.java
@@ -0,0 +1,69 @@
+/**
+ * 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.client;
+
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.mob.MobConstants;
+import org.apache.hadoop.hbase.snapshot.MobSnapshotTestingUtils;
+import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test to verify that the cloned table is independent of the table from which it was cloned
+ */
+@Category(LargeTests.class)
+public class TestMobSnapshotCloneIndependence extends TestSnapshotCloneIndependence {
+ private static final Log LOG = LogFactory.getLog(TestMobSnapshotCloneIndependence.class);
+
+ /**
+ * Setup the config for the cluster and start it
+ * @throws Exception on failure
+ */
+ @BeforeClass
+ public static void setupCluster() throws Exception {
+ setupConf(UTIL.getConfiguration());
+ UTIL.startMiniCluster(NUM_RS);
+ }
+
+ protected static void setupConf(Configuration conf) {
+ TestSnapshotCloneIndependence.setupConf(conf);
+ conf.setInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY, 0);
+ }
+
+ @Override
+ protected Table createTable(final TableName table, byte[] family) throws Exception {
+ return MobSnapshotTestingUtils.createMobTable(UTIL, table, family);
+ }
+
+ @Override
+ public void loadData(final Table table, byte[]... families) throws Exception {
+ SnapshotTestingUtils.loadData(UTIL, table.getName(), 1000, families);
+ }
+
+ @Override
+ protected int countRows(final Table table, final byte[]... families) throws Exception {
+ return MobSnapshotTestingUtils.countMobRows(table, families);
+ }
+}
\ No newline at end of file
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
new file mode 100644
index 00000000000..0a3093b0c06
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotCloneIndependence.java
@@ -0,0 +1,481 @@
+/**
+ * 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.client;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+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.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
+import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
+import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.Ignore;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test to verify that the cloned table is independent of the table from which it was cloned
+ */
+@Category({LargeTests.class, ClientTests.class})
+public class TestSnapshotCloneIndependence {
+ private static final Log LOG = LogFactory.getLog(TestSnapshotCloneIndependence.class);
+
+ protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+ protected static final int NUM_RS = 2;
+ private static final String STRING_TABLE_NAME = "test";
+ private static final String TEST_FAM_STR = "fam";
+ protected static final byte[] TEST_FAM = Bytes.toBytes(TEST_FAM_STR);
+ protected static final TableName TABLE_NAME = TableName.valueOf(STRING_TABLE_NAME);
+ private static final int CLEANER_INTERVAL = 100;
+
+ /**
+ * Setup the config for the cluster and start it
+ * @throws Exception on fOailure
+ */
+ @BeforeClass
+ public static void setupCluster() throws Exception {
+ setupConf(UTIL.getConfiguration());
+ UTIL.startMiniCluster(NUM_RS);
+ }
+
+ static void setupConf(Configuration conf) {
+ // Up the handlers; this test needs more than usual.
+ conf.setInt(HConstants.REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT, 15);
+ // enable snapshot support
+ conf.setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, true);
+ // disable the ui
+ conf.setInt("hbase.regionsever.info.port", -1);
+ conf.setInt("hbase.master.info.port", -1);
+ // change the flush size to a small amount, regulating number of store files
+ conf.setInt("hbase.hregion.memstore.flush.size", 25000);
+ // so make sure we get a compaction when doing a load, but keep around
+ // some files in the store
+ conf.setInt("hbase.hstore.compaction.min", 10);
+ conf.setInt("hbase.hstore.compactionThreshold", 10);
+ // block writes if we get to 12 store files
+ conf.setInt("hbase.hstore.blockingStoreFiles", 12);
+ conf.setInt("hbase.regionserver.msginterval", 100);
+ conf.setBoolean("hbase.master.enabletable.roundrobin", true);
+ // Avoid potentially aggressive splitting which would cause snapshot to fail
+ conf.set(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
+ ConstantSizeRegionSplitPolicy.class.getName());
+ // Execute cleaner frequently to induce failures
+ conf.setInt("hbase.master.cleaner.interval", CLEANER_INTERVAL);
+ conf.setInt("hbase.master.hfilecleaner.plugins.snapshot.period", CLEANER_INTERVAL);
+ // Effectively disable TimeToLiveHFileCleaner. Don't want to fully disable it because that
+ // will even trigger races between creating the directory containing back references and
+ // the back reference itself.
+ conf.setInt("hbase.master.hfilecleaner.ttl", CLEANER_INTERVAL);
+ }
+
+ @Before
+ public void setup() throws Exception {
+ createTable(TABLE_NAME, TEST_FAM);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ UTIL.deleteTable(TABLE_NAME);
+ SnapshotTestingUtils.deleteAllSnapshots(UTIL.getHBaseAdmin());
+ SnapshotTestingUtils.deleteArchiveDirectory(UTIL);
+ }
+
+ @AfterClass
+ public static void cleanupTest() throws Exception {
+ try {
+ UTIL.shutdownMiniCluster();
+ } catch (Exception e) {
+ LOG.warn("failure shutting down cluster", e);
+ }
+ }
+
+ /**
+ * Verify that adding data to the cloned table will not affect the original, and vice-versa when
+ * it is taken as an online snapshot.
+ */
+ @Ignore ("Flakey. Fix") @Test (timeout=300000)
+ public void testOnlineSnapshotAppendIndependent() throws Exception {
+ runTestSnapshotAppendIndependent(true);
+ }
+
+ /**
+ * Verify that adding data to the cloned table will not affect the original, and vice-versa when
+ * it is taken as an offline snapshot.
+ */
+ @Test (timeout=300000)
+ @Ignore
+ public void testOfflineSnapshotAppendIndependent() throws Exception {
+ runTestSnapshotAppendIndependent(false);
+ }
+
+ /**
+ * Verify that adding metadata to the cloned table will not affect the original, and vice-versa
+ * when it is taken as an online snapshot.
+ */
+ @Test (timeout=300000)
+ public void testOnlineSnapshotMetadataChangesIndependent() throws Exception {
+ runTestSnapshotMetadataChangesIndependent(true);
+ }
+
+ /**
+ * Verify that adding netadata to the cloned table will not affect the original, and vice-versa
+ * when is taken as an online snapshot.
+ */
+ @Test (timeout=300000)
+ @Ignore
+ public void testOfflineSnapshotMetadataChangesIndependent() throws Exception {
+ runTestSnapshotMetadataChangesIndependent(false);
+ }
+
+ /**
+ * Verify that region operations, in this case splitting a region, are independent between the
+ * cloned table and the original.
+ */
+ @Test (timeout=300000)
+ @Ignore
+ public void testOfflineSnapshotRegionOperationsIndependent() throws Exception {
+ runTestRegionOperationsIndependent(false);
+ }
+
+ /**
+ * Verify that region operations, in this case splitting a region, are independent between the
+ * cloned table and the original.
+ */
+ @Test (timeout=300000)
+ public void testOnlineSnapshotRegionOperationsIndependent() throws Exception {
+ runTestRegionOperationsIndependent(true);
+ }
+
+ @Test (timeout=300000)
+ @Ignore
+ public void testOfflineSnapshotDeleteIndependent() throws Exception {
+ runTestSnapshotDeleteIndependent(false);
+ }
+
+ @Ignore ("Flakey test") @Test (timeout=300000)
+ public void testOnlineSnapshotDeleteIndependent() throws Exception {
+ runTestSnapshotDeleteIndependent(true);
+ }
+
+ private static void waitOnSplit(Connection c, final Table t, int originalCount) throws Exception {
+ for (int i = 0; i < 200; i++) {
+ try {
+ Thread.sleep(500);
+ } catch (InterruptedException e) {
+ // Restore the interrupted status
+ Thread.currentThread().interrupt();
+ }
+ try (RegionLocator locator = c.getRegionLocator(t.getName())) {
+ if (locator.getAllRegionLocations().size() > originalCount) {
+ return;
+ }
+ }
+ }
+ throw new Exception("Split did not increase the number of regions");
+ }
+
+ /*
+ * Take a snapshot of a table, add data, and verify that this only
+ * affects one table
+ * @param online - Whether the table is online or not during the snapshot
+ */
+ private void runTestSnapshotAppendIndependent(boolean online) throws Exception {
+ FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
+ Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
+
+ Admin admin = UTIL.getHBaseAdmin();
+ final long startTime = System.currentTimeMillis();
+ final TableName localTableName =
+ TableName.valueOf(STRING_TABLE_NAME + startTime);
+
+ try (Table original = createTable(localTableName, TEST_FAM)) {
+ loadData(original, TEST_FAM);
+ final int origTableRowCount = countRows(original);
+
+ // Take a snapshot
+ final String snapshotNameAsString = "snapshot_" + localTableName;
+ byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
+
+ SnapshotTestingUtils.createSnapshotAndValidate(admin, localTableName, TEST_FAM_STR,
+ snapshotNameAsString, rootDir, fs, online);
+
+ if (!online) {
+ tryDisable(admin, localTableName);
+ }
+ TableName cloneTableName = TableName.valueOf("test-clone-" + localTableName);
+ admin.cloneSnapshot(snapshotName, cloneTableName);
+
+ try (Table clonedTable = UTIL.getConnection().getTable(cloneTableName)) {
+
+ // Make sure that all the regions are available before starting
+ UTIL.waitUntilAllRegionsAssigned(cloneTableName);
+
+ final int clonedTableRowCount = countRows(clonedTable);
+
+ Assert.assertEquals(
+ "The line counts of original and cloned tables do not match after clone. ",
+ origTableRowCount, clonedTableRowCount);
+
+ // Attempt to add data to the test
+ final String rowKey = "new-row-" + System.currentTimeMillis();
+
+ Put p = new Put(Bytes.toBytes(rowKey));
+ p.addColumn(TEST_FAM, Bytes.toBytes("someQualifier"), Bytes.toBytes("someString"));
+ original.put(p);
+
+ // Verify that it is not present in the original table
+ Assert.assertEquals("The row count of the original table was not modified by the put",
+ origTableRowCount + 1, countRows(original));
+ Assert.assertEquals(
+ "The row count of the cloned table changed as a result of addition to the original",
+ clonedTableRowCount, countRows(clonedTable));
+
+ p = new Put(Bytes.toBytes(rowKey));
+ p.addColumn(TEST_FAM, Bytes.toBytes("someQualifier"), Bytes.toBytes("someString"));
+ clonedTable.put(p);
+
+ // Verify that the new family is not in the restored table's description
+ Assert.assertEquals(
+ "The row count of the original table was modified by the put to the clone",
+ origTableRowCount + 1, countRows(original));
+ Assert.assertEquals("The row count of the cloned table was not modified by the put",
+ clonedTableRowCount + 1, countRows(clonedTable));
+ }
+ }
+ }
+
+ /*
+ * Take a snapshot of a table, do a split, and verify that this only affects one table
+ * @param online - Whether the table is online or not during the snapshot
+ */
+ private void runTestRegionOperationsIndependent(boolean online) throws Exception {
+ FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
+ Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
+
+ // Create a table
+ Admin admin = UTIL.getHBaseAdmin();
+ final long startTime = System.currentTimeMillis();
+ final TableName localTableName =
+ TableName.valueOf(STRING_TABLE_NAME + startTime);
+ Table original = createTable(localTableName, TEST_FAM);
+ loadData(original, TEST_FAM);
+ final int loadedTableCount = countRows(original);
+ System.out.println("Original table has: " + loadedTableCount + " rows");
+
+ final String snapshotNameAsString = "snapshot_" + localTableName;
+
+ // Create a snapshot
+ SnapshotTestingUtils.createSnapshotAndValidate(admin, localTableName, TEST_FAM_STR,
+ snapshotNameAsString, rootDir, fs, online);
+
+ if (!online) {
+ tryDisable(admin, localTableName);
+ }
+
+ TableName cloneTableName = TableName.valueOf("test-clone-" + localTableName);
+
+ // Clone the snapshot
+ byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
+ admin.cloneSnapshot(snapshotName, cloneTableName);
+
+ // Verify that region information is the same pre-split
+ ((ClusterConnection) UTIL.getConnection()).clearRegionCache();
+ List originalTableHRegions = admin.getTableRegions(localTableName);
+
+ final int originalRegionCount = originalTableHRegions.size();
+ final int cloneTableRegionCount = admin.getTableRegions(cloneTableName).size();
+ Assert.assertEquals(
+ "The number of regions in the cloned table is different than in the original table.",
+ originalRegionCount, cloneTableRegionCount);
+
+ // Split a region on the parent table
+ admin.splitRegion(originalTableHRegions.get(0).getRegionName());
+ waitOnSplit(UTIL.getConnection(), original, originalRegionCount);
+
+ // Verify that the cloned table region is not split
+ final int cloneTableRegionCount2 = admin.getTableRegions(cloneTableName).size();
+ Assert.assertEquals(
+ "The number of regions in the cloned table changed though none of its regions were split.",
+ cloneTableRegionCount, cloneTableRegionCount2);
+ }
+
+ /*
+ * Take a snapshot of a table, add metadata, and verify that this only
+ * affects one table
+ * @param online - Whether the table is online or not during the snapshot
+ */
+ private void runTestSnapshotMetadataChangesIndependent(boolean online) throws Exception {
+ FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
+ Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
+
+ // Create a table
+ Admin admin = UTIL.getHBaseAdmin();
+ final long startTime = System.currentTimeMillis();
+ final TableName localTableName =
+ TableName.valueOf(STRING_TABLE_NAME + startTime);
+ Table original = createTable(localTableName, TEST_FAM);
+ loadData(original, TEST_FAM);
+
+ final String snapshotNameAsString = "snapshot_" + localTableName;
+
+ // Create a snapshot
+ SnapshotTestingUtils.createSnapshotAndValidate(admin, localTableName, TEST_FAM_STR,
+ snapshotNameAsString, rootDir, fs, online);
+
+ if (!online) {
+ tryDisable(admin, localTableName);
+ }
+
+ TableName cloneTableName = TableName.valueOf("test-clone-" + localTableName);
+
+ // Clone the snapshot
+ byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
+ admin.cloneSnapshot(snapshotName, cloneTableName);
+
+ // Add a new column family to the original table
+ byte[] TEST_FAM_2 = Bytes.toBytes("fam2");
+ HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAM_2);
+
+ tryDisable(admin, localTableName);
+ admin.addColumnFamily(localTableName, hcd);
+
+ // Verify that it is not in the snapshot
+ admin.enableTable(localTableName);
+ UTIL.waitTableAvailable(localTableName);
+
+ // get a description of the cloned table
+ // get a list of its families
+ // assert that the family is there
+ HTableDescriptor originalTableDescriptor = original.getTableDescriptor();
+ HTableDescriptor clonedTableDescriptor = admin.getTableDescriptor(cloneTableName);
+
+ Assert.assertTrue("The original family was not found. There is something wrong. ",
+ originalTableDescriptor.hasFamily(TEST_FAM));
+ Assert.assertTrue("The original family was not found in the clone. There is something wrong. ",
+ clonedTableDescriptor.hasFamily(TEST_FAM));
+
+ Assert.assertTrue("The new family was not found. ",
+ originalTableDescriptor.hasFamily(TEST_FAM_2));
+ Assert.assertTrue("The new family was not found. ",
+ !clonedTableDescriptor.hasFamily(TEST_FAM_2));
+ }
+
+ private void tryDisable(Admin admin, TableName localTableName) throws IOException {
+ int offlineRetry = 0;
+ while ( offlineRetry < 5 && admin.isTableEnabled(localTableName)) {
+ try {
+ admin.disableTable(localTableName);
+ } catch (IOException ioe) {
+ LOG.warn("Error disabling the table", ioe);
+ }
+ offlineRetry ++;
+ }
+ }
+
+ /*
+ * Take a snapshot of a table, add data, and verify that deleting the snapshot does not affect
+ * either table.
+ * @param online - Whether the table is online or not during the snapshot
+ */
+ private void runTestSnapshotDeleteIndependent(boolean online) throws Exception {
+ FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
+ Path rootDir = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
+
+ final Admin admin = UTIL.getHBaseAdmin();
+ final long startTime = System.currentTimeMillis();
+ final TableName localTableName =
+ TableName.valueOf(STRING_TABLE_NAME + startTime);
+
+ try (Table original = createTable(localTableName, TEST_FAM)) {
+ loadData(original, TEST_FAM);
+ }
+
+ // Take a snapshot
+ final String snapshotNameAsString = "snapshot_" + localTableName;
+ byte[] snapshotName = Bytes.toBytes(snapshotNameAsString);
+
+ SnapshotTestingUtils.createSnapshotAndValidate(admin, localTableName, TEST_FAM_STR,
+ snapshotNameAsString, rootDir, fs, online);
+
+ if (!online) {
+ tryDisable(admin, localTableName);
+ }
+
+ TableName cloneTableName = TableName.valueOf("test-clone-" + localTableName);
+ admin.cloneSnapshot(snapshotName, cloneTableName);
+
+ UTIL.waitUntilAllRegionsAssigned(cloneTableName);
+
+ // Ensure the original table does not reference the HFiles anymore
+ admin.majorCompact(localTableName);
+
+ // Deleting the snapshot used to break the cloned table by deleting in-use HFiles
+ admin.deleteSnapshot(snapshotName);
+
+ // Wait for cleaner run and DFS heartbeats so that anything that is deletable is fully deleted
+ do {
+ Thread.sleep(5000);
+ } while (!admin.listSnapshots(snapshotNameAsString).isEmpty());
+
+ try (Table original = UTIL.getConnection().getTable(localTableName)) {
+ try (Table clonedTable = UTIL.getConnection().getTable(cloneTableName)) {
+ // Verify that all regions of both tables are readable
+ final int origTableRowCount = countRows(original);
+ final int clonedTableRowCount = countRows(clonedTable);
+ Assert.assertEquals(origTableRowCount, clonedTableRowCount);
+ }
+ }
+ }
+
+ protected Table createTable(final TableName table, byte[] family) throws Exception {
+ Table t = UTIL.createTable(table, family);
+ // Wait for everything to be ready with the table
+ UTIL.waitUntilAllRegionsAssigned(table);
+
+ // At this point the table should be good to go.
+ return t;
+ }
+
+ protected void loadData(final Table table, byte[]... families) throws Exception {
+ UTIL.loadTable(table, families);
+ }
+
+ protected int countRows(final Table table, final byte[]... families) throws Exception {
+ return UTIL.countRows(table, families);
+ }
+}
\ No newline at end of file
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
new file mode 100644
index 00000000000..c5728cfa8e4
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
@@ -0,0 +1,1799 @@
+/**
+ *
+
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master;
+
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_wait_for_zk_delete;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_final_transition_failed;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_preempt_task;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_acquired;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_done;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_err;
+import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_resigned;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.NavigableSet;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.SplitLogCounters;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.ConnectionUtils;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Increment;
+import org.apache.hadoop.hbase.client.NonceGenerator;
+import org.apache.hadoop.hbase.client.PerClientRandomNonceGenerator;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
+import org.apache.hadoop.hbase.coordination.ZKSplitLogManagerCoordination;
+import org.apache.hadoop.hbase.exceptions.OperationConflictException;
+import org.apache.hadoop.hbase.exceptions.RegionInRecoveryException;
+import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
+import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
+import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.wal.DefaultWALProvider;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.apache.hadoop.hbase.wal.WALFactory;
+import org.apache.hadoop.hbase.wal.WALSplitter;
+import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MasterTests.class, LargeTests.class})
+@SuppressWarnings("deprecation")
+public class TestDistributedLogSplitting {
+ private static final Log LOG = LogFactory.getLog(TestSplitLogManager.class);
+ static {
+ // Uncomment the following line if more verbosity is needed for
+ // debugging (see HBASE-12285 for details).
+ //Logger.getLogger("org.apache.hadoop.hbase").setLevel(Level.DEBUG);
+
+ // test ThreeRSAbort fails under hadoop2 (2.0.2-alpha) if shortcircuit-read (scr) is on. this
+ // turns it off for this test. TODO: Figure out why scr breaks recovery.
+ System.setProperty("hbase.tests.use.shortcircuit.reads", "false");
+
+ }
+
+ // Start a cluster with 2 masters and 6 regionservers
+ static final int NUM_MASTERS = 2;
+ static final int NUM_RS = 5;
+
+ MiniHBaseCluster cluster;
+ HMaster master;
+ Configuration conf;
+ static Configuration originalConf;
+ static HBaseTestingUtility TEST_UTIL;
+ static MiniDFSCluster dfsCluster;
+ static MiniZooKeeperCluster zkCluster;
+
+ @BeforeClass
+ public static void setup() throws Exception {
+ TEST_UTIL = new HBaseTestingUtility(HBaseConfiguration.create());
+ dfsCluster = TEST_UTIL.startMiniDFSCluster(1);
+ zkCluster = TEST_UTIL.startMiniZKCluster();
+ originalConf = TEST_UTIL.getConfiguration();
+ }
+
+ @AfterClass
+ public static void tearDown() throws IOException {
+ TEST_UTIL.shutdownMiniZKCluster();
+ TEST_UTIL.shutdownMiniDFSCluster();
+ TEST_UTIL.shutdownMiniHBaseCluster();
+ }
+
+ private void startCluster(int num_rs) throws Exception {
+ SplitLogCounters.resetCounters();
+ LOG.info("Starting cluster");
+ conf.getLong("hbase.splitlog.max.resubmit", 0);
+ // Make the failure test faster
+ conf.setInt("zookeeper.recovery.retry", 0);
+ conf.setInt(HConstants.REGIONSERVER_INFO_PORT, -1);
+ conf.setFloat(HConstants.LOAD_BALANCER_SLOP_KEY, (float) 100.0); // no load balancing
+ conf.setInt("hbase.regionserver.wal.max.splitters", 3);
+ conf.setInt(HConstants.REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT, 10);
+ TEST_UTIL.shutdownMiniHBaseCluster();
+ TEST_UTIL = new HBaseTestingUtility(conf);
+ TEST_UTIL.setDFSCluster(dfsCluster);
+ TEST_UTIL.setZkCluster(zkCluster);
+ TEST_UTIL.startMiniHBaseCluster(NUM_MASTERS, num_rs);
+ cluster = TEST_UTIL.getHBaseCluster();
+ LOG.info("Waiting for active/ready master");
+ cluster.waitForActiveAndReadyMaster();
+ master = cluster.getMaster();
+ while (cluster.getLiveRegionServerThreads().size() < num_rs) {
+ Threads.sleep(10);
+ }
+ }
+
+ @Before
+ public void before() throws Exception {
+ // refresh configuration
+ conf = HBaseConfiguration.create(originalConf);
+ }
+
+ @After
+ public void after() throws Exception {
+ try {
+ if (TEST_UTIL.getHBaseCluster() != null) {
+ for (MasterThread mt : TEST_UTIL.getHBaseCluster().getLiveMasterThreads()) {
+ mt.getMaster().abort("closing...", null);
+ }
+ }
+ TEST_UTIL.shutdownMiniHBaseCluster();
+ } finally {
+ TEST_UTIL.getTestFileSystem().delete(FSUtils.getRootDir(TEST_UTIL.getConfiguration()), true);
+ ZKUtil.deleteNodeRecursively(TEST_UTIL.getZooKeeperWatcher(), "/hbase");
+ }
+ }
+
+ @Ignore("DLR is broken by HBASE-12751") @Test (timeout=300000)
+ public void testRecoveredEdits() throws Exception {
+ LOG.info("testRecoveredEdits");
+ conf.setLong("hbase.regionserver.hlog.blocksize", 30 * 1024); // create more than one wal
+ conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false);
+ startCluster(NUM_RS);
+
+ final int NUM_LOG_LINES = 1000;
+ final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
+ // turn off load balancing to prevent regions from moving around otherwise
+ // they will consume recovered.edits
+ master.balanceSwitch(false);
+ FileSystem fs = master.getMasterFileSystem().getFileSystem();
+
+ List rsts = cluster.getLiveRegionServerThreads();
+
+ Path rootdir = FSUtils.getRootDir(conf);
+
+ Table t = installTable(new ZooKeeperWatcher(conf, "table-creation", null),
+ "table", "family", 40);
+ try {
+ TableName table = t.getName();
+ List regions = null;
+ HRegionServer hrs = null;
+ for (int i = 0; i < NUM_RS; i++) {
+ boolean foundRs = false;
+ hrs = rsts.get(i).getRegionServer();
+ regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ for (HRegionInfo region : regions) {
+ if (region.getTable().getNameAsString().equalsIgnoreCase("table")) {
+ foundRs = true;
+ break;
+ }
+ }
+ if (foundRs) break;
+ }
+ final Path logDir = new Path(rootdir, DefaultWALProvider.getWALDirectoryName(hrs
+ .getServerName().toString()));
+
+ LOG.info("#regions = " + regions.size());
+ Iterator it = regions.iterator();
+ while (it.hasNext()) {
+ HRegionInfo region = it.next();
+ if (region.getTable().getNamespaceAsString()
+ .equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) {
+ it.remove();
+ }
+ }
+
+ makeWAL(hrs, regions, "table", "family", NUM_LOG_LINES, 100);
+
+ slm.splitLogDistributed(logDir);
+
+ int count = 0;
+ for (HRegionInfo hri : regions) {
+
+ Path tdir = FSUtils.getTableDir(rootdir, table);
+ Path editsdir =
+ WALSplitter.getRegionDirRecoveredEditsDir(
+ HRegion.getRegionDir(tdir, hri.getEncodedName()));
+ LOG.debug("checking edits dir " + editsdir);
+ FileStatus[] files = fs.listStatus(editsdir, new PathFilter() {
+ @Override
+ public boolean accept(Path p) {
+ if (WALSplitter.isSequenceIdFile(p)) {
+ return false;
+ }
+ return true;
+ }
+ });
+ assertTrue(
+ "edits dir should have more than a single file in it. instead has " + files.length,
+ files.length > 1);
+ for (int i = 0; i < files.length; i++) {
+ int c = countWAL(files[i].getPath(), fs, conf);
+ count += c;
+ }
+ LOG.info(count + " edits in " + files.length + " recovered edits files.");
+ }
+
+ // check that the log file is moved
+ assertFalse(fs.exists(logDir));
+
+ assertEquals(NUM_LOG_LINES, count);
+ } finally {
+ if (t != null) t.close();
+ }
+ }
+
+ @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+ public void testLogReplayWithNonMetaRSDown() throws Exception {
+ LOG.info("testLogReplayWithNonMetaRSDown");
+ conf.setLong("hbase.regionserver.hlog.blocksize", 30 * 1024); // create more than one wal
+ conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
+ startCluster(NUM_RS);
+ final int NUM_REGIONS_TO_CREATE = 40;
+ final int NUM_LOG_LINES = 1000;
+ // turn off load balancing to prevent regions from moving around otherwise
+ // they will consume recovered.edits
+ master.balanceSwitch(false);
+
+ final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+ Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
+ try {
+ HRegionServer hrs = findRSToKill(false, "table");
+ List regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ makeWAL(hrs, regions, "table", "family", NUM_LOG_LINES, 100);
+
+ // wait for abort completes
+ this.abortRSAndVerifyRecovery(hrs, ht, zkw, NUM_REGIONS_TO_CREATE, NUM_LOG_LINES);
+ } finally {
+ if (ht != null) ht.close();
+ if (zkw != null) zkw.close();
+ }
+ }
+
+ private static class NonceGeneratorWithDups extends PerClientRandomNonceGenerator {
+ private boolean isDups = false;
+ private LinkedList nonces = new LinkedList();
+
+ public void startDups() {
+ isDups = true;
+ }
+
+ @Override
+ public long newNonce() {
+ long nonce = isDups ? nonces.removeFirst() : super.newNonce();
+ if (!isDups) {
+ nonces.add(nonce);
+ }
+ return nonce;
+ }
+ }
+
+ @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+ public void testNonceRecovery() throws Exception {
+ LOG.info("testNonceRecovery");
+ final String TABLE_NAME = "table";
+ final String FAMILY_NAME = "family";
+ final int NUM_REGIONS_TO_CREATE = 40;
+
+ conf.setLong("hbase.regionserver.hlog.blocksize", 100*1024);
+ conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
+ startCluster(NUM_RS);
+ master.balanceSwitch(false);
+
+ final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+ Table ht = installTable(zkw, TABLE_NAME, FAMILY_NAME, NUM_REGIONS_TO_CREATE);
+ NonceGeneratorWithDups ng = new NonceGeneratorWithDups();
+ NonceGenerator oldNg =
+ ConnectionUtils.injectNonceGeneratorForTesting(
+ (ClusterConnection)TEST_UTIL.getConnection(), ng);
+
+ try {
+ List reqs = new ArrayList();
+ for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
+ HRegionServer hrs = rst.getRegionServer();
+ List hris = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ for (HRegionInfo hri : hris) {
+ if (TABLE_NAME.equalsIgnoreCase(hri.getTable().getNameAsString())) {
+ byte[] key = hri.getStartKey();
+ if (key == null || key.length == 0) {
+ key = Bytes.copy(hri.getEndKey());
+ --(key[key.length - 1]);
+ }
+ Increment incr = new Increment(key);
+ incr.addColumn(Bytes.toBytes(FAMILY_NAME), Bytes.toBytes("q"), 1);
+ ht.increment(incr);
+ reqs.add(incr);
+ }
+ }
+ }
+
+ HRegionServer hrs = findRSToKill(false, "table");
+ abortRSAndWaitForRecovery(hrs, zkw, NUM_REGIONS_TO_CREATE);
+ ng.startDups();
+ for (Increment incr : reqs) {
+ try {
+ ht.increment(incr);
+ fail("should have thrown");
+ } catch (OperationConflictException ope) {
+ LOG.debug("Caught as expected: " + ope.getMessage());
+ }
+ }
+ } finally {
+ ConnectionUtils.injectNonceGeneratorForTesting((ClusterConnection)
+ TEST_UTIL.getConnection(), oldNg);
+ if (ht != null) ht.close();
+ if (zkw != null) zkw.close();
+ }
+ }
+
+ @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+ public void testLogReplayWithMetaRSDown() throws Exception {
+ LOG.info("testRecoveredEditsReplayWithMetaRSDown");
+ conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
+ startCluster(NUM_RS);
+ final int NUM_REGIONS_TO_CREATE = 40;
+ final int NUM_LOG_LINES = 1000;
+ // turn off load balancing to prevent regions from moving around otherwise
+ // they will consume recovered.edits
+ master.balanceSwitch(false);
+
+ final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+ Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
+ try {
+ HRegionServer hrs = findRSToKill(true, "table");
+ List regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ makeWAL(hrs, regions, "table", "family", NUM_LOG_LINES, 100);
+
+ this.abortRSAndVerifyRecovery(hrs, ht, zkw, NUM_REGIONS_TO_CREATE, NUM_LOG_LINES);
+ } finally {
+ if (ht != null) ht.close();
+ if (zkw != null) zkw.close();
+ }
+ }
+
+ private void abortRSAndVerifyRecovery(HRegionServer hrs, Table ht, final ZooKeeperWatcher zkw,
+ final int numRegions, final int numofLines) throws Exception {
+
+ abortRSAndWaitForRecovery(hrs, zkw, numRegions);
+ assertEquals(numofLines, TEST_UTIL.countRows(ht));
+ }
+
+ private void abortRSAndWaitForRecovery(HRegionServer hrs, final ZooKeeperWatcher zkw,
+ final int numRegions) throws Exception {
+ final MiniHBaseCluster tmpCluster = this.cluster;
+
+ // abort RS
+ LOG.info("Aborting region server: " + hrs.getServerName());
+ hrs.abort("testing");
+
+ // wait for abort completes
+ TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate() {
+ @Override
+ public boolean evaluate() throws Exception {
+ return (tmpCluster.getLiveRegionServerThreads().size() <= (NUM_RS - 1));
+ }
+ });
+
+ // wait for regions come online
+ TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate() {
+ @Override
+ public boolean evaluate() throws Exception {
+ return (HBaseTestingUtility.getAllOnlineRegions(tmpCluster).size()
+ >= (numRegions + 1));
+ }
+ });
+
+ // wait for all regions are fully recovered
+ TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate() {
+ @Override
+ public boolean evaluate() throws Exception {
+ List recoveringRegions = zkw.getRecoverableZooKeeper().getChildren(
+ zkw.recoveringRegionsZNode, false);
+ return (recoveringRegions != null && recoveringRegions.size() == 0);
+ }
+ });
+ }
+
+ @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+ public void testMasterStartsUpWithLogSplittingWork() throws Exception {
+ LOG.info("testMasterStartsUpWithLogSplittingWork");
+ conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false);
+ conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, NUM_RS - 1);
+ startCluster(NUM_RS);
+
+ final int NUM_REGIONS_TO_CREATE = 40;
+ final int NUM_LOG_LINES = 1000;
+ // turn off load balancing to prevent regions from moving around otherwise
+ // they will consume recovered.edits
+ master.balanceSwitch(false);
+
+ final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+ Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
+ try {
+ HRegionServer hrs = findRSToKill(false, "table");
+ List regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ makeWAL(hrs, regions, "table", "family", NUM_LOG_LINES, 100);
+
+ // abort master
+ abortMaster(cluster);
+
+ // abort RS
+ LOG.info("Aborting region server: " + hrs.getServerName());
+ hrs.abort("testing");
+
+ // wait for abort completes
+ TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate() {
+ @Override
+ public boolean evaluate() throws Exception {
+ return (cluster.getLiveRegionServerThreads().size() <= (NUM_RS - 1));
+ }
+ });
+
+ Thread.sleep(2000);
+ LOG.info("Current Open Regions:"
+ + HBaseTestingUtility.getAllOnlineRegions(cluster).size());
+
+ // wait for abort completes
+ TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate() {
+ @Override
+ public boolean evaluate() throws Exception {
+ return (HBaseTestingUtility.getAllOnlineRegions(cluster).size()
+ >= (NUM_REGIONS_TO_CREATE + 1));
+ }
+ });
+
+ LOG.info("Current Open Regions After Master Node Starts Up:"
+ + HBaseTestingUtility.getAllOnlineRegions(cluster).size());
+
+ assertEquals(NUM_LOG_LINES, TEST_UTIL.countRows(ht));
+ } finally {
+ if (ht != null) ht.close();
+ if (zkw != null) zkw.close();
+ }
+ }
+
+ @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+ public void testMasterStartsUpWithLogReplayWork() throws Exception {
+ LOG.info("testMasterStartsUpWithLogReplayWork");
+ conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
+ conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, NUM_RS - 1);
+ startCluster(NUM_RS);
+
+ final int NUM_REGIONS_TO_CREATE = 40;
+ final int NUM_LOG_LINES = 1000;
+ // turn off load balancing to prevent regions from moving around otherwise
+ // they will consume recovered.edits
+ master.balanceSwitch(false);
+
+ final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+ Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
+ try {
+ HRegionServer hrs = findRSToKill(false, "table");
+ List regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ makeWAL(hrs, regions, "table", "family", NUM_LOG_LINES, 100);
+
+ // abort master
+ abortMaster(cluster);
+
+ // abort RS
+ LOG.info("Aborting region server: " + hrs.getServerName());
+ hrs.abort("testing");
+
+ // wait for the RS dies
+ TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate() {
+ @Override
+ public boolean evaluate() throws Exception {
+ return (cluster.getLiveRegionServerThreads().size() <= (NUM_RS - 1));
+ }
+ });
+
+ Thread.sleep(2000);
+ LOG.info("Current Open Regions:" + HBaseTestingUtility.getAllOnlineRegions(cluster).size());
+
+ // wait for all regions are fully recovered
+ TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate() {
+ @Override
+ public boolean evaluate() throws Exception {
+ List recoveringRegions = zkw.getRecoverableZooKeeper().getChildren(
+ zkw.recoveringRegionsZNode, false);
+ boolean done = recoveringRegions != null && recoveringRegions.size() == 0;
+ if (!done) {
+ LOG.info("Recovering regions: " + recoveringRegions);
+ }
+ return done;
+ }
+ });
+
+ LOG.info("Current Open Regions After Master Node Starts Up:"
+ + HBaseTestingUtility.getAllOnlineRegions(cluster).size());
+
+ assertEquals(NUM_LOG_LINES, TEST_UTIL.countRows(ht));
+ } finally {
+ if (ht != null) ht.close();
+ if (zkw != null) zkw.close();
+ }
+ }
+
+
+ @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+ public void testLogReplayTwoSequentialRSDown() throws Exception {
+ LOG.info("testRecoveredEditsReplayTwoSequentialRSDown");
+ conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
+ startCluster(NUM_RS);
+ final int NUM_REGIONS_TO_CREATE = 40;
+ final int NUM_LOG_LINES = 1000;
+ // turn off load balancing to prevent regions from moving around otherwise
+ // they will consume recovered.edits
+ master.balanceSwitch(false);
+
+ List rsts = cluster.getLiveRegionServerThreads();
+ final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+ Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
+ try {
+ List regions = null;
+ HRegionServer hrs1 = findRSToKill(false, "table");
+ regions = ProtobufUtil.getOnlineRegions(hrs1.getRSRpcServices());
+
+ makeWAL(hrs1, regions, "table", "family", NUM_LOG_LINES, 100);
+
+ // abort RS1
+ LOG.info("Aborting region server: " + hrs1.getServerName());
+ hrs1.abort("testing");
+
+ // wait for abort completes
+ TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate() {
+ @Override
+ public boolean evaluate() throws Exception {
+ return (cluster.getLiveRegionServerThreads().size() <= (NUM_RS - 1));
+ }
+ });
+
+ // wait for regions come online
+ TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate() {
+ @Override
+ public boolean evaluate() throws Exception {
+ return (HBaseTestingUtility.getAllOnlineRegions(cluster).size()
+ >= (NUM_REGIONS_TO_CREATE + 1));
+ }
+ });
+
+ // sleep a little bit in order to interrupt recovering in the middle
+ Thread.sleep(300);
+ // abort second region server
+ rsts = cluster.getLiveRegionServerThreads();
+ HRegionServer hrs2 = rsts.get(0).getRegionServer();
+ LOG.info("Aborting one more region server: " + hrs2.getServerName());
+ hrs2.abort("testing");
+
+ // wait for abort completes
+ TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate() {
+ @Override
+ public boolean evaluate() throws Exception {
+ return (cluster.getLiveRegionServerThreads().size() <= (NUM_RS - 2));
+ }
+ });
+
+ // wait for regions come online
+ TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate() {
+ @Override
+ public boolean evaluate() throws Exception {
+ return (HBaseTestingUtility.getAllOnlineRegions(cluster).size()
+ >= (NUM_REGIONS_TO_CREATE + 1));
+ }
+ });
+
+ // wait for all regions are fully recovered
+ TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate() {
+ @Override
+ public boolean evaluate() throws Exception {
+ List recoveringRegions = zkw.getRecoverableZooKeeper().getChildren(
+ zkw.recoveringRegionsZNode, false);
+ return (recoveringRegions != null && recoveringRegions.size() == 0);
+ }
+ });
+
+ assertEquals(NUM_LOG_LINES, TEST_UTIL.countRows(ht));
+ } finally {
+ if (ht != null) ht.close();
+ if (zkw != null) zkw.close();
+ }
+ }
+
+ @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+ public void testMarkRegionsRecoveringInZK() throws Exception {
+ LOG.info("testMarkRegionsRecoveringInZK");
+ conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
+ startCluster(NUM_RS);
+ master.balanceSwitch(false);
+ List rsts = cluster.getLiveRegionServerThreads();
+ final ZooKeeperWatcher zkw = master.getZooKeeper();
+ Table ht = installTable(zkw, "table", "family", 40);
+ try {
+ final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
+
+ Set regionSet = new HashSet();
+ HRegionInfo region = null;
+ HRegionServer hrs = null;
+ ServerName firstFailedServer = null;
+ ServerName secondFailedServer = null;
+ for (int i = 0; i < NUM_RS; i++) {
+ hrs = rsts.get(i).getRegionServer();
+ List regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ if (regions.isEmpty()) continue;
+ region = regions.get(0);
+ regionSet.add(region);
+ firstFailedServer = hrs.getServerName();
+ secondFailedServer = rsts.get((i + 1) % NUM_RS).getRegionServer().getServerName();
+ break;
+ }
+
+ slm.markRegionsRecovering(firstFailedServer, regionSet);
+ slm.markRegionsRecovering(secondFailedServer, regionSet);
+
+ List recoveringRegions = ZKUtil.listChildrenNoWatch(zkw,
+ ZKUtil.joinZNode(zkw.recoveringRegionsZNode, region.getEncodedName()));
+
+ assertEquals(recoveringRegions.size(), 2);
+
+ // wait for splitLogWorker to mark them up because there is no WAL files recorded in ZK
+ final HRegionServer tmphrs = hrs;
+ TEST_UTIL.waitFor(60000, 1000, new Waiter.Predicate() {
+ @Override
+ public boolean evaluate() throws Exception {
+ return (tmphrs.getRecoveringRegions().size() == 0);
+ }
+ });
+ } finally {
+ if (ht != null) ht.close();
+ if (zkw != null) zkw.close();
+ }
+ }
+
+ @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+ public void testReplayCmd() throws Exception {
+ LOG.info("testReplayCmd");
+ conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
+ startCluster(NUM_RS);
+ final int NUM_REGIONS_TO_CREATE = 40;
+ // turn off load balancing to prevent regions from moving around otherwise
+ // they will consume recovered.edits
+ master.balanceSwitch(false);
+
+ List rsts = cluster.getLiveRegionServerThreads();
+ final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+ Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
+ try {
+ List regions = null;
+ HRegionServer hrs = null;
+ for (int i = 0; i < NUM_RS; i++) {
+ boolean isCarryingMeta = false;
+ hrs = rsts.get(i).getRegionServer();
+ regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ for (HRegionInfo region : regions) {
+ if (region.isMetaRegion()) {
+ isCarryingMeta = true;
+ break;
+ }
+ }
+ if (isCarryingMeta) {
+ continue;
+ }
+ if (regions.size() > 0) break;
+ }
+
+ this.prepareData(ht, Bytes.toBytes("family"), Bytes.toBytes("c1"));
+ String originalCheckSum = TEST_UTIL.checksumRows(ht);
+
+ // abort RA and trigger replay
+ abortRSAndWaitForRecovery(hrs, zkw, NUM_REGIONS_TO_CREATE);
+
+ assertEquals("Data should remain after reopening of regions", originalCheckSum,
+ TEST_UTIL.checksumRows(ht));
+ } finally {
+ if (ht != null) ht.close();
+ if (zkw != null) zkw.close();
+ }
+ }
+
+ @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+ public void testLogReplayForDisablingTable() throws Exception {
+ LOG.info("testLogReplayForDisablingTable");
+ conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
+ startCluster(NUM_RS);
+ final int NUM_REGIONS_TO_CREATE = 40;
+ final int NUM_LOG_LINES = 1000;
+
+ List rsts = cluster.getLiveRegionServerThreads();
+ final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+ Table disablingHT = installTable(zkw, "disableTable", "family", NUM_REGIONS_TO_CREATE);
+ Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE, NUM_REGIONS_TO_CREATE);
+ try {
+ // turn off load balancing to prevent regions from moving around otherwise
+ // they will consume recovered.edits
+ master.balanceSwitch(false);
+
+ List regions = null;
+ HRegionServer hrs = null;
+ boolean hasRegionsForBothTables = false;
+ String tableName = null;
+ for (int i = 0; i < NUM_RS; i++) {
+ tableName = null;
+ hasRegionsForBothTables = false;
+ boolean isCarryingSystem = false;
+ hrs = rsts.get(i).getRegionServer();
+ regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ for (HRegionInfo region : regions) {
+ if (region.getTable().isSystemTable()) {
+ isCarryingSystem = true;
+ break;
+ }
+ if (tableName != null &&
+ !tableName.equalsIgnoreCase(region.getTable().getNameAsString())) {
+ // make sure that we find a RS has online regions for both "table" and "disableTable"
+ hasRegionsForBothTables = true;
+ break;
+ } else if (tableName == null) {
+ tableName = region.getTable().getNameAsString();
+ }
+ }
+ if (isCarryingSystem) {
+ continue;
+ }
+ if (hasRegionsForBothTables) {
+ break;
+ }
+ }
+
+ // make sure we found a good RS
+ Assert.assertTrue(hasRegionsForBothTables);
+
+ LOG.info("#regions = " + regions.size());
+ Iterator it = regions.iterator();
+ while (it.hasNext()) {
+ HRegionInfo region = it.next();
+ if (region.isMetaTable()) {
+ it.remove();
+ }
+ }
+ makeWAL(hrs, regions, "disableTable", "family", NUM_LOG_LINES, 100, false);
+ makeWAL(hrs, regions, "table", "family", NUM_LOG_LINES, 100);
+
+ LOG.info("Disabling table\n");
+ TEST_UTIL.getHBaseAdmin().disableTable(TableName.valueOf("disableTable"));
+ TEST_UTIL.waitTableDisabled(TableName.valueOf("disableTable").getName());
+
+ // abort RS
+ LOG.info("Aborting region server: " + hrs.getServerName());
+ hrs.abort("testing");
+
+ // wait for abort completes
+ TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate() {
+ @Override
+ public boolean evaluate() throws Exception {
+ return (cluster.getLiveRegionServerThreads().size() <= (NUM_RS - 1));
+ }
+ });
+
+ // wait for regions come online
+ TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate() {
+ @Override
+ public boolean evaluate() throws Exception {
+ return (HBaseTestingUtility.getAllOnlineRegions(cluster).size()
+ >= (NUM_REGIONS_TO_CREATE + 1));
+ }
+ });
+
+ // wait for all regions are fully recovered
+ TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate() {
+ @Override
+ public boolean evaluate() throws Exception {
+ List recoveringRegions = zkw.getRecoverableZooKeeper().getChildren(
+ zkw.recoveringRegionsZNode, false);
+ ServerManager serverManager = master.getServerManager();
+ return (!serverManager.areDeadServersInProgress() &&
+ recoveringRegions != null && recoveringRegions.size() == 0);
+ }
+ });
+
+ int count = 0;
+ FileSystem fs = master.getMasterFileSystem().getFileSystem();
+ Path rootdir = FSUtils.getRootDir(conf);
+ Path tdir = FSUtils.getTableDir(rootdir, TableName.valueOf("disableTable"));
+ for (HRegionInfo hri : regions) {
+ Path editsdir =
+ WALSplitter.getRegionDirRecoveredEditsDir(
+ HRegion.getRegionDir(tdir, hri.getEncodedName()));
+ LOG.debug("checking edits dir " + editsdir);
+ if(!fs.exists(editsdir)) continue;
+ FileStatus[] files = fs.listStatus(editsdir, new PathFilter() {
+ @Override
+ public boolean accept(Path p) {
+ if (WALSplitter.isSequenceIdFile(p)) {
+ return false;
+ }
+ return true;
+ }
+ });
+ if(files != null) {
+ for(FileStatus file : files) {
+ int c = countWAL(file.getPath(), fs, conf);
+ count += c;
+ LOG.info(c + " edits in " + file.getPath());
+ }
+ }
+ }
+
+ LOG.info("Verify edits in recovered.edits files");
+ assertEquals(NUM_LOG_LINES, count);
+ LOG.info("Verify replayed edits");
+ assertEquals(NUM_LOG_LINES, TEST_UTIL.countRows(ht));
+
+ // clean up
+ for (HRegionInfo hri : regions) {
+ Path editsdir =
+ WALSplitter.getRegionDirRecoveredEditsDir(
+ HRegion.getRegionDir(tdir, hri.getEncodedName()));
+ fs.delete(editsdir, true);
+ }
+ disablingHT.close();
+ } finally {
+ if (ht != null) ht.close();
+ if (zkw != null) zkw.close();
+ }
+ }
+
+ @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+ public void testDisallowWritesInRecovering() throws Exception {
+ LOG.info("testDisallowWritesInRecovering");
+ conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
+ conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3);
+ conf.setBoolean(HConstants.DISALLOW_WRITES_IN_RECOVERING, true);
+ startCluster(NUM_RS);
+ final int NUM_REGIONS_TO_CREATE = 40;
+ // turn off load balancing to prevent regions from moving around otherwise
+ // they will consume recovered.edits
+ master.balanceSwitch(false);
+
+ List rsts = cluster.getLiveRegionServerThreads();
+ final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+ Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
+ try {
+ final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
+
+ Set regionSet = new HashSet();
+ HRegionInfo region = null;
+ HRegionServer hrs = null;
+ HRegionServer dstRS = null;
+ for (int i = 0; i < NUM_RS; i++) {
+ hrs = rsts.get(i).getRegionServer();
+ List regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ if (regions.isEmpty()) continue;
+ region = regions.get(0);
+ regionSet.add(region);
+ dstRS = rsts.get((i+1) % NUM_RS).getRegionServer();
+ break;
+ }
+
+ slm.markRegionsRecovering(hrs.getServerName(), regionSet);
+ // move region in order for the region opened in recovering state
+ final HRegionInfo hri = region;
+ final HRegionServer tmpRS = dstRS;
+ TEST_UTIL.getHBaseAdmin().move(region.getEncodedNameAsBytes(),
+ Bytes.toBytes(dstRS.getServerName().getServerName()));
+ // wait for region move completes
+ final RegionStates regionStates =
+ TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStates();
+ TEST_UTIL.waitFor(45000, 200, new Waiter.Predicate() {
+ @Override
+ public boolean evaluate() throws Exception {
+ ServerName sn = regionStates.getRegionServerOfRegion(hri);
+ return (sn != null && sn.equals(tmpRS.getServerName()));
+ }
+ });
+
+ try {
+ byte[] key = region.getStartKey();
+ if (key == null || key.length == 0) {
+ key = new byte[] { 0, 0, 0, 0, 1 };
+ }
+ Put put = new Put(key);
+ put.addColumn(Bytes.toBytes("family"), Bytes.toBytes("c1"), new byte[]{'b'});
+ ht.put(put);
+ } catch (IOException ioe) {
+ Assert.assertTrue(ioe instanceof RetriesExhaustedWithDetailsException);
+ RetriesExhaustedWithDetailsException re = (RetriesExhaustedWithDetailsException) ioe;
+ boolean foundRegionInRecoveryException = false;
+ for (Throwable t : re.getCauses()) {
+ if (t instanceof RegionInRecoveryException) {
+ foundRegionInRecoveryException = true;
+ break;
+ }
+ }
+ Assert.assertTrue(
+ "No RegionInRecoveryException. Following exceptions returned=" + re.getCauses(),
+ foundRegionInRecoveryException);
+ }
+ } finally {
+ if (ht != null) ht.close();
+ if (ht != null) zkw.close();
+ }
+ }
+
+ /**
+ * The original intention of this test was to force an abort of a region
+ * server and to make sure that the failure path in the region servers is
+ * properly evaluated. But it is difficult to ensure that the region server
+ * doesn't finish the log splitting before it aborts. Also now, there is
+ * this code path where the master will preempt the region server when master
+ * detects that the region server has aborted.
+ * @throws Exception
+ */
+ @Ignore ("Disabled because flakey") @Test (timeout=300000)
+ public void testWorkerAbort() throws Exception {
+ LOG.info("testWorkerAbort");
+ startCluster(3);
+ final int NUM_LOG_LINES = 10000;
+ final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
+ FileSystem fs = master.getMasterFileSystem().getFileSystem();
+
+ final List rsts = cluster.getLiveRegionServerThreads();
+ HRegionServer hrs = findRSToKill(false, "table");
+ Path rootdir = FSUtils.getRootDir(conf);
+ final Path logDir = new Path(rootdir,
+ DefaultWALProvider.getWALDirectoryName(hrs.getServerName().toString()));
+
+ Table t = installTable(new ZooKeeperWatcher(conf, "table-creation", null),
+ "table", "family", 40);
+ try {
+ makeWAL(hrs, ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices()),
+ "table", "family", NUM_LOG_LINES, 100);
+
+ new Thread() {
+ @Override
+ public void run() {
+ waitForCounter(tot_wkr_task_acquired, 0, 1, 1000);
+ for (RegionServerThread rst : rsts) {
+ rst.getRegionServer().abort("testing");
+ break;
+ }
+ }
+ }.start();
+ // slm.splitLogDistributed(logDir);
+ FileStatus[] logfiles = fs.listStatus(logDir);
+ TaskBatch batch = new TaskBatch();
+ slm.enqueueSplitTask(logfiles[0].getPath().toString(), batch);
+ //waitForCounter but for one of the 2 counters
+ long curt = System.currentTimeMillis();
+ long waitTime = 80000;
+ long endt = curt + waitTime;
+ while (curt < endt) {
+ if ((tot_wkr_task_resigned.get() + tot_wkr_task_err.get() +
+ tot_wkr_final_transition_failed.get() + tot_wkr_task_done.get() +
+ tot_wkr_preempt_task.get()) == 0) {
+ Thread.yield();
+ curt = System.currentTimeMillis();
+ } else {
+ assertTrue(1 <= (tot_wkr_task_resigned.get() + tot_wkr_task_err.get() +
+ tot_wkr_final_transition_failed.get() + tot_wkr_task_done.get() +
+ tot_wkr_preempt_task.get()));
+ return;
+ }
+ }
+ fail("none of the following counters went up in " + waitTime +
+ " milliseconds - " +
+ "tot_wkr_task_resigned, tot_wkr_task_err, " +
+ "tot_wkr_final_transition_failed, tot_wkr_task_done, " +
+ "tot_wkr_preempt_task");
+ } finally {
+ if (t != null) t.close();
+ }
+ }
+
+ @Test (timeout=300000)
+ public void testThreeRSAbort() throws Exception {
+ LOG.info("testThreeRSAbort");
+ final int NUM_REGIONS_TO_CREATE = 40;
+ final int NUM_ROWS_PER_REGION = 100;
+
+ startCluster(NUM_RS); // NUM_RS=6.
+
+ final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
+ "distributed log splitting test", null);
+
+ Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
+ try {
+ populateDataInTable(NUM_ROWS_PER_REGION, "family");
+
+
+ List rsts = cluster.getLiveRegionServerThreads();
+ assertEquals(NUM_RS, rsts.size());
+ rsts.get(0).getRegionServer().abort("testing");
+ rsts.get(1).getRegionServer().abort("testing");
+ rsts.get(2).getRegionServer().abort("testing");
+
+ long start = EnvironmentEdgeManager.currentTime();
+ while (cluster.getLiveRegionServerThreads().size() > (NUM_RS - 3)) {
+ if (EnvironmentEdgeManager.currentTime() - start > 60000) {
+ assertTrue(false);
+ }
+ Thread.sleep(200);
+ }
+
+ start = EnvironmentEdgeManager.currentTime();
+ while (HBaseTestingUtility.getAllOnlineRegions(cluster).size()
+ < (NUM_REGIONS_TO_CREATE + 1)) {
+ if (EnvironmentEdgeManager.currentTime() - start > 60000) {
+ assertTrue("Timedout", false);
+ }
+ Thread.sleep(200);
+ }
+
+ // wait for all regions are fully recovered
+ TEST_UTIL.waitFor(180000, 200, new Waiter.Predicate() {
+ @Override
+ public boolean evaluate() throws Exception {
+ List recoveringRegions = zkw.getRecoverableZooKeeper().getChildren(
+ zkw.recoveringRegionsZNode, false);
+ return (recoveringRegions != null && recoveringRegions.size() == 0);
+ }
+ });
+
+ assertEquals(NUM_REGIONS_TO_CREATE * NUM_ROWS_PER_REGION,
+ TEST_UTIL.countRows(ht));
+ } finally {
+ if (ht != null) ht.close();
+ if (zkw != null) zkw.close();
+ }
+ }
+
+
+
+ @Test(timeout=30000)
+ public void testDelayedDeleteOnFailure() throws Exception {
+ LOG.info("testDelayedDeleteOnFailure");
+ startCluster(1);
+ final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
+ final FileSystem fs = master.getMasterFileSystem().getFileSystem();
+ final Path logDir = new Path(FSUtils.getRootDir(conf), "x");
+ fs.mkdirs(logDir);
+ ExecutorService executor = null;
+ try {
+ final Path corruptedLogFile = new Path(logDir, "x");
+ FSDataOutputStream out;
+ out = fs.create(corruptedLogFile);
+ out.write(0);
+ out.write(Bytes.toBytes("corrupted bytes"));
+ out.close();
+ ZKSplitLogManagerCoordination coordination =
+ (ZKSplitLogManagerCoordination) ((BaseCoordinatedStateManager) master
+ .getCoordinatedStateManager()).getSplitLogManagerCoordination();
+ coordination.setIgnoreDeleteForTesting(true);
+ executor = Executors.newSingleThreadExecutor();
+ Runnable runnable = new Runnable() {
+ @Override
+ public void run() {
+ try {
+ // since the logDir is a fake, corrupted one, so the split log worker
+ // will finish it quickly with error, and this call will fail and throw
+ // an IOException.
+ slm.splitLogDistributed(logDir);
+ } catch (IOException ioe) {
+ try {
+ assertTrue(fs.exists(corruptedLogFile));
+ // this call will block waiting for the task to be removed from the
+ // tasks map which is not going to happen since ignoreZKDeleteForTesting
+ // is set to true, until it is interrupted.
+ slm.splitLogDistributed(logDir);
+ } catch (IOException e) {
+ assertTrue(Thread.currentThread().isInterrupted());
+ return;
+ }
+ fail("did not get the expected IOException from the 2nd call");
+ }
+ fail("did not get the expected IOException from the 1st call");
+ }
+ };
+ Future> result = executor.submit(runnable);
+ try {
+ result.get(2000, TimeUnit.MILLISECONDS);
+ } catch (TimeoutException te) {
+ // it is ok, expected.
+ }
+ waitForCounter(tot_mgr_wait_for_zk_delete, 0, 1, 10000);
+ executor.shutdownNow();
+ executor = null;
+
+ // make sure the runnable is finished with no exception thrown.
+ result.get();
+ } finally {
+ if (executor != null) {
+ // interrupt the thread in case the test fails in the middle.
+ // it has no effect if the thread is already terminated.
+ executor.shutdownNow();
+ }
+ fs.delete(logDir, true);
+ }
+ }
+
+ @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+ public void testMetaRecoveryInZK() throws Exception {
+ LOG.info("testMetaRecoveryInZK");
+ conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
+ startCluster(NUM_RS);
+
+ // turn off load balancing to prevent regions from moving around otherwise
+ // they will consume recovered.edits
+ master.balanceSwitch(false);
+ final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+
+ // only testing meta recovery in ZK operation
+ HRegionServer hrs = findRSToKill(true, null);
+ List regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+
+ LOG.info("#regions = " + regions.size());
+ Set tmpRegions = new HashSet();
+ tmpRegions.add(HRegionInfo.FIRST_META_REGIONINFO);
+ master.getMasterFileSystem().prepareLogReplay(hrs.getServerName(), tmpRegions);
+ Set userRegionSet = new HashSet();
+ userRegionSet.addAll(regions);
+ master.getMasterFileSystem().prepareLogReplay(hrs.getServerName(), userRegionSet);
+ boolean isMetaRegionInRecovery = false;
+ List recoveringRegions =
+ zkw.getRecoverableZooKeeper().getChildren(zkw.recoveringRegionsZNode, false);
+ for (String curEncodedRegionName : recoveringRegions) {
+ if (curEncodedRegionName.equals(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName())) {
+ isMetaRegionInRecovery = true;
+ break;
+ }
+ }
+ assertTrue(isMetaRegionInRecovery);
+
+ master.getMasterFileSystem().splitMetaLog(hrs.getServerName());
+
+ isMetaRegionInRecovery = false;
+ recoveringRegions =
+ zkw.getRecoverableZooKeeper().getChildren(zkw.recoveringRegionsZNode, false);
+ for (String curEncodedRegionName : recoveringRegions) {
+ if (curEncodedRegionName.equals(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName())) {
+ isMetaRegionInRecovery = true;
+ break;
+ }
+ }
+ // meta region should be recovered
+ assertFalse(isMetaRegionInRecovery);
+ zkw.close();
+ }
+
+ @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+ public void testSameVersionUpdatesRecovery() throws Exception {
+ LOG.info("testSameVersionUpdatesRecovery");
+ conf.setLong("hbase.regionserver.hlog.blocksize", 15 * 1024);
+ conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
+ startCluster(NUM_RS);
+ final AtomicLong sequenceId = new AtomicLong(100);
+ final int NUM_REGIONS_TO_CREATE = 40;
+ final int NUM_LOG_LINES = 1000;
+ // turn off load balancing to prevent regions from moving around otherwise
+ // they will consume recovered.edits
+ master.balanceSwitch(false);
+
+ List rsts = cluster.getLiveRegionServerThreads();
+ final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+ Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
+ try {
+ List regions = null;
+ HRegionServer hrs = null;
+ for (int i = 0; i < NUM_RS; i++) {
+ boolean isCarryingMeta = false;
+ hrs = rsts.get(i).getRegionServer();
+ regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ for (HRegionInfo region : regions) {
+ if (region.isMetaRegion()) {
+ isCarryingMeta = true;
+ break;
+ }
+ }
+ if (isCarryingMeta) {
+ continue;
+ }
+ break;
+ }
+
+ LOG.info("#regions = " + regions.size());
+ Iterator it = regions.iterator();
+ while (it.hasNext()) {
+ HRegionInfo region = it.next();
+ if (region.isMetaTable()
+ || region.getEncodedName().equals(
+ HRegionInfo.FIRST_META_REGIONINFO.getEncodedName())) {
+ it.remove();
+ }
+ }
+ if (regions.size() == 0) return;
+ HRegionInfo curRegionInfo = regions.get(0);
+ byte[] startRow = curRegionInfo.getStartKey();
+ if (startRow == null || startRow.length == 0) {
+ startRow = new byte[] { 0, 0, 0, 0, 1 };
+ }
+ byte[] row = Bytes.incrementBytes(startRow, 1);
+ // use last 5 bytes because HBaseTestingUtility.createMultiRegions use 5 bytes key
+ row = Arrays.copyOfRange(row, 3, 8);
+ long value = 0;
+ TableName tableName = TableName.valueOf("table");
+ byte[] family = Bytes.toBytes("family");
+ byte[] qualifier = Bytes.toBytes("c1");
+ long timeStamp = System.currentTimeMillis();
+ HTableDescriptor htd = new HTableDescriptor(tableName);
+ htd.addFamily(new HColumnDescriptor(family));
+ final WAL wal = hrs.getWAL(curRegionInfo);
+ for (int i = 0; i < NUM_LOG_LINES; i += 1) {
+ WALEdit e = new WALEdit();
+ value++;
+ e.add(new KeyValue(row, family, qualifier, timeStamp, Bytes.toBytes(value)));
+ wal.append(htd, curRegionInfo,
+ new HLogKey(curRegionInfo.getEncodedNameAsBytes(), tableName, System.currentTimeMillis()),
+ e, true);
+ }
+ wal.sync();
+ wal.shutdown();
+
+ // wait for abort completes
+ this.abortRSAndWaitForRecovery(hrs, zkw, NUM_REGIONS_TO_CREATE);
+
+ // verify we got the last value
+ LOG.info("Verification Starts...");
+ Get g = new Get(row);
+ Result r = ht.get(g);
+ long theStoredVal = Bytes.toLong(r.getValue(family, qualifier));
+ assertEquals(value, theStoredVal);
+
+ // after flush
+ LOG.info("Verification after flush...");
+ TEST_UTIL.getHBaseAdmin().flush(tableName);
+ r = ht.get(g);
+ theStoredVal = Bytes.toLong(r.getValue(family, qualifier));
+ assertEquals(value, theStoredVal);
+ } finally {
+ if (ht != null) ht.close();
+ if (zkw != null) zkw.close();
+ }
+ }
+
+ @Ignore("DLR is broken by HBASE-12751") @Test(timeout = 300000)
+ public void testSameVersionUpdatesRecoveryWithCompaction() throws Exception {
+ LOG.info("testSameVersionUpdatesRecoveryWithWrites");
+ conf.setLong("hbase.regionserver.hlog.blocksize", 15 * 1024);
+ conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
+ conf.setInt(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 30 * 1024);
+ conf.setInt("hbase.hstore.compactionThreshold", 3);
+ startCluster(NUM_RS);
+ final AtomicLong sequenceId = new AtomicLong(100);
+ final int NUM_REGIONS_TO_CREATE = 40;
+ final int NUM_LOG_LINES = 2000;
+ // turn off load balancing to prevent regions from moving around otherwise
+ // they will consume recovered.edits
+ master.balanceSwitch(false);
+
+ List rsts = cluster.getLiveRegionServerThreads();
+ final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+ Table ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
+ try {
+ List regions = null;
+ HRegionServer hrs = null;
+ for (int i = 0; i < NUM_RS; i++) {
+ boolean isCarryingMeta = false;
+ hrs = rsts.get(i).getRegionServer();
+ regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ for (HRegionInfo region : regions) {
+ if (region.isMetaRegion()) {
+ isCarryingMeta = true;
+ break;
+ }
+ }
+ if (isCarryingMeta) {
+ continue;
+ }
+ break;
+ }
+
+ LOG.info("#regions = " + regions.size());
+ Iterator it = regions.iterator();
+ while (it.hasNext()) {
+ HRegionInfo region = it.next();
+ if (region.isMetaTable()
+ || region.getEncodedName().equals(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName())) {
+ it.remove();
+ }
+ }
+ if (regions.size() == 0) return;
+ HRegionInfo curRegionInfo = regions.get(0);
+ byte[] startRow = curRegionInfo.getStartKey();
+ if (startRow == null || startRow.length == 0) {
+ startRow = new byte[] { 0, 0, 0, 0, 1 };
+ }
+ byte[] row = Bytes.incrementBytes(startRow, 1);
+ // use last 5 bytes because HBaseTestingUtility.createMultiRegions use 5 bytes key
+ row = Arrays.copyOfRange(row, 3, 8);
+ long value = 0;
+ final TableName tableName = TableName.valueOf("table");
+ byte[] family = Bytes.toBytes("family");
+ byte[] qualifier = Bytes.toBytes("c1");
+ long timeStamp = System.currentTimeMillis();
+ HTableDescriptor htd = new HTableDescriptor(tableName);
+ htd.addFamily(new HColumnDescriptor(family));
+ final WAL wal = hrs.getWAL(curRegionInfo);
+ for (int i = 0; i < NUM_LOG_LINES; i += 1) {
+ WALEdit e = new WALEdit();
+ value++;
+ e.add(new KeyValue(row, family, qualifier, timeStamp, Bytes.toBytes(value)));
+ wal.append(htd, curRegionInfo, new HLogKey(curRegionInfo.getEncodedNameAsBytes(),
+ tableName, System.currentTimeMillis()), e, true);
+ }
+ wal.sync();
+ wal.shutdown();
+
+ // wait for abort completes
+ this.abortRSAndWaitForRecovery(hrs, zkw, NUM_REGIONS_TO_CREATE);
+
+ // verify we got the last value
+ LOG.info("Verification Starts...");
+ Get g = new Get(row);
+ Result r = ht.get(g);
+ long theStoredVal = Bytes.toLong(r.getValue(family, qualifier));
+ assertEquals(value, theStoredVal);
+
+ // after flush & compaction
+ LOG.info("Verification after flush...");
+ TEST_UTIL.getHBaseAdmin().flush(tableName);
+ TEST_UTIL.getHBaseAdmin().compact(tableName);
+
+ // wait for compaction completes
+ TEST_UTIL.waitFor(30000, 200, new Waiter.Predicate() {
+ @Override
+ public boolean evaluate() throws Exception {
+ return (TEST_UTIL.getHBaseAdmin().getCompactionState(tableName) == CompactionState.NONE);
+ }
+ });
+
+ r = ht.get(g);
+ theStoredVal = Bytes.toLong(r.getValue(family, qualifier));
+ assertEquals(value, theStoredVal);
+ } finally {
+ if (ht != null) ht.close();
+ if (zkw != null) zkw.close();
+ }
+ }
+
+ @Test(timeout = 300000)
+ public void testReadWriteSeqIdFiles() throws Exception {
+ LOG.info("testReadWriteSeqIdFiles");
+ startCluster(2);
+ final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
+ Table ht = installTable(zkw, "table", "family", 10);
+ try {
+ FileSystem fs = master.getMasterFileSystem().getFileSystem();
+ Path tableDir = FSUtils.getTableDir(FSUtils.getRootDir(conf), TableName.valueOf("table"));
+ List regionDirs = FSUtils.getRegionDirs(fs, tableDir);
+ long newSeqId = WALSplitter.writeRegionSequenceIdFile(fs, regionDirs.get(0), 1L, 1000L);
+ WALSplitter.writeRegionSequenceIdFile(fs, regionDirs.get(0) , 1L, 1000L);
+ assertEquals(newSeqId + 2000,
+ WALSplitter.writeRegionSequenceIdFile(fs, regionDirs.get(0), 3L, 1000L));
+
+ Path editsdir = WALSplitter.getRegionDirRecoveredEditsDir(regionDirs.get(0));
+ FileStatus[] files = FSUtils.listStatus(fs, editsdir, new PathFilter() {
+ @Override
+ public boolean accept(Path p) {
+ return WALSplitter.isSequenceIdFile(p);
+ }
+ });
+ // only one seqid file should exist
+ assertEquals(1, files.length);
+
+ // verify all seqId files aren't treated as recovered.edits files
+ NavigableSet recoveredEdits =
+ WALSplitter.getSplitEditFilesSorted(fs, regionDirs.get(0));
+ assertEquals(0, recoveredEdits.size());
+ } finally {
+ if (ht != null) ht.close();
+ if (zkw != null) zkw.close();
+ }
+ }
+
+ Table installTable(ZooKeeperWatcher zkw, String tname, String fname, int nrs) throws Exception {
+ return installTable(zkw, tname, fname, nrs, 0);
+ }
+
+ Table installTable(ZooKeeperWatcher zkw, String tname, String fname, int nrs,
+ int existingRegions) throws Exception {
+ // Create a table with regions
+ TableName table = TableName.valueOf(tname);
+ byte [] family = Bytes.toBytes(fname);
+ LOG.info("Creating table with " + nrs + " regions");
+ Table ht = TEST_UTIL.createMultiRegionTable(table, family, nrs);
+ int numRegions = -1;
+ try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(table)) {
+ numRegions = r.getStartKeys().length;
+ }
+ assertEquals(nrs, numRegions);
+ LOG.info("Waiting for no more RIT\n");
+ blockUntilNoRIT(zkw, master);
+ // disable-enable cycle to get rid of table's dead regions left behind
+ // by createMultiRegions
+ LOG.debug("Disabling table\n");
+ TEST_UTIL.getHBaseAdmin().disableTable(table);
+ LOG.debug("Waiting for no more RIT\n");
+ blockUntilNoRIT(zkw, master);
+ NavigableSet regions = HBaseTestingUtility.getAllOnlineRegions(cluster);
+ LOG.debug("Verifying only catalog and namespace regions are assigned\n");
+ if (regions.size() != 2) {
+ for (String oregion : regions)
+ LOG.debug("Region still online: " + oregion);
+ }
+ assertEquals(2 + existingRegions, regions.size());
+ LOG.debug("Enabling table\n");
+ TEST_UTIL.getHBaseAdmin().enableTable(table);
+ LOG.debug("Waiting for no more RIT\n");
+ blockUntilNoRIT(zkw, master);
+ LOG.debug("Verifying there are " + numRegions + " assigned on cluster\n");
+ regions = HBaseTestingUtility.getAllOnlineRegions(cluster);
+ assertEquals(numRegions + 2 + existingRegions, regions.size());
+ return ht;
+ }
+
+ void populateDataInTable(int nrows, String fname) throws Exception {
+ byte [] family = Bytes.toBytes(fname);
+
+ List rsts = cluster.getLiveRegionServerThreads();
+ assertEquals(NUM_RS, rsts.size());
+
+ for (RegionServerThread rst : rsts) {
+ HRegionServer hrs = rst.getRegionServer();
+ List hris = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ for (HRegionInfo hri : hris) {
+ if (hri.getTable().isSystemTable()) {
+ continue;
+ }
+ LOG.debug("adding data to rs = " + rst.getName() +
+ " region = "+ hri.getRegionNameAsString());
+ Region region = hrs.getOnlineRegion(hri.getRegionName());
+ assertTrue(region != null);
+ putData(region, hri.getStartKey(), nrows, Bytes.toBytes("q"), family);
+ }
+ }
+
+ for (MasterThread mt : cluster.getLiveMasterThreads()) {
+ HRegionServer hrs = mt.getMaster();
+ List hris;
+ try {
+ hris = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ } catch (ServerNotRunningYetException e) {
+ // It's ok: this master may be a backup. Ignored.
+ continue;
+ }
+ for (HRegionInfo hri : hris) {
+ if (hri.getTable().isSystemTable()) {
+ continue;
+ }
+ LOG.debug("adding data to rs = " + mt.getName() +
+ " region = "+ hri.getRegionNameAsString());
+ Region region = hrs.getOnlineRegion(hri.getRegionName());
+ assertTrue(region != null);
+ putData(region, hri.getStartKey(), nrows, Bytes.toBytes("q"), family);
+ }
+ }
+ }
+
+ public void makeWAL(HRegionServer hrs, List regions, String tname, String fname,
+ int num_edits, int edit_size) throws IOException {
+ makeWAL(hrs, regions, tname, fname, num_edits, edit_size, true);
+ }
+
+ public void makeWAL(HRegionServer hrs, List regions, String tname, String fname,
+ int num_edits, int edit_size, boolean cleanShutdown) throws IOException {
+ TableName fullTName = TableName.valueOf(tname);
+ // remove root and meta region
+ regions.remove(HRegionInfo.FIRST_META_REGIONINFO);
+ // using one sequenceId for edits across all regions is ok.
+ final AtomicLong sequenceId = new AtomicLong(10);
+
+
+ for(Iterator iter = regions.iterator(); iter.hasNext(); ) {
+ HRegionInfo regionInfo = iter.next();
+ if(regionInfo.getTable().isSystemTable()) {
+ iter.remove();
+ }
+ }
+ HTableDescriptor htd = new HTableDescriptor(fullTName);
+ byte[] family = Bytes.toBytes(fname);
+ htd.addFamily(new HColumnDescriptor(family));
+ byte[] value = new byte[edit_size];
+
+ List hris = new ArrayList();
+ for (HRegionInfo region : regions) {
+ if (!region.getTable().getNameAsString().equalsIgnoreCase(tname)) {
+ continue;
+ }
+ hris.add(region);
+ }
+ LOG.info("Creating wal edits across " + hris.size() + " regions.");
+ for (int i = 0; i < edit_size; i++) {
+ value[i] = (byte) ('a' + (i % 26));
+ }
+ int n = hris.size();
+ int[] counts = new int[n];
+ // sync every ~30k to line up with desired wal rolls
+ final int syncEvery = 30 * 1024 / edit_size;
+ if (n > 0) {
+ for (int i = 0; i < num_edits; i += 1) {
+ WALEdit e = new WALEdit();
+ HRegionInfo curRegionInfo = hris.get(i % n);
+ final WAL log = hrs.getWAL(curRegionInfo);
+ byte[] startRow = curRegionInfo.getStartKey();
+ if (startRow == null || startRow.length == 0) {
+ startRow = new byte[] { 0, 0, 0, 0, 1 };
+ }
+ byte[] row = Bytes.incrementBytes(startRow, counts[i % n]);
+ row = Arrays.copyOfRange(row, 3, 8); // use last 5 bytes because
+ // HBaseTestingUtility.createMultiRegions use 5 bytes
+ // key
+ byte[] qualifier = Bytes.toBytes("c" + Integer.toString(i));
+ e.add(new KeyValue(row, family, qualifier, System.currentTimeMillis(), value));
+ log.append(htd, curRegionInfo,
+ new HLogKey(curRegionInfo.getEncodedNameAsBytes(), fullTName,
+ System.currentTimeMillis()), e, true);
+ if (0 == i % syncEvery) {
+ log.sync();
+ }
+ counts[i % n] += 1;
+ }
+ }
+ // done as two passes because the regions might share logs. shutdown is idempotent, but sync
+ // will cause errors if done after.
+ for (HRegionInfo info : hris) {
+ final WAL log = hrs.getWAL(info);
+ log.sync();
+ }
+ if (cleanShutdown) {
+ for (HRegionInfo info : hris) {
+ final WAL log = hrs.getWAL(info);
+ log.shutdown();
+ }
+ }
+ for (int i = 0; i < n; i++) {
+ LOG.info("region " + hris.get(i).getRegionNameAsString() + " has " + counts[i] + " edits");
+ }
+ return;
+ }
+
+ private int countWAL(Path log, FileSystem fs, Configuration conf)
+ throws IOException {
+ int count = 0;
+ WAL.Reader in = WALFactory.createReader(fs, log, conf);
+ try {
+ WAL.Entry e;
+ while ((e = in.next()) != null) {
+ if (!WALEdit.isMetaEditFamily(e.getEdit().getCells().get(0))) {
+ count++;
+ }
+ }
+ } finally {
+ try {
+ in.close();
+ } catch (IOException exception) {
+ LOG.warn("Problem closing wal: " + exception.getMessage());
+ LOG.debug("exception details.", exception);
+ }
+ }
+ return count;
+ }
+
+ private void blockUntilNoRIT(ZooKeeperWatcher zkw, HMaster master) throws Exception {
+ TEST_UTIL.waitUntilNoRegionsInTransition(60000);
+ }
+
+ private void putData(Region region, byte[] startRow, int numRows, byte [] qf,
+ byte [] ...families)
+ throws IOException {
+ for(int i = 0; i < numRows; i++) {
+ Put put = new Put(Bytes.add(startRow, Bytes.toBytes(i)));
+ for(byte [] family : families) {
+ put.addColumn(family, qf, null);
+ }
+ region.put(put);
+ }
+ }
+
+ /**
+ * Load table with puts and deletes with expected values so that we can verify later
+ */
+ private void prepareData(final Table t, final byte[] f, final byte[] column) throws IOException {
+ byte[] k = new byte[3];
+
+ // add puts
+ List puts = new ArrayList<>();
+ for (byte b1 = 'a'; b1 <= 'z'; b1++) {
+ for (byte b2 = 'a'; b2 <= 'z'; b2++) {
+ for (byte b3 = 'a'; b3 <= 'z'; b3++) {
+ k[0] = b1;
+ k[1] = b2;
+ k[2] = b3;
+ Put put = new Put(k);
+ put.addColumn(f, column, k);
+ puts.add(put);
+ }
+ }
+ }
+ t.put(puts);
+ // add deletes
+ for (byte b3 = 'a'; b3 <= 'z'; b3++) {
+ k[0] = 'a';
+ k[1] = 'a';
+ k[2] = b3;
+ Delete del = new Delete(k);
+ t.delete(del);
+ }
+ }
+
+ private void waitForCounter(AtomicLong ctr, long oldval, long newval,
+ long timems) {
+ long curt = System.currentTimeMillis();
+ long endt = curt + timems;
+ while (curt < endt) {
+ if (ctr.get() == oldval) {
+ Thread.yield();
+ curt = System.currentTimeMillis();
+ } else {
+ assertEquals(newval, ctr.get());
+ return;
+ }
+ }
+ assertTrue(false);
+ }
+
+ private void abortMaster(MiniHBaseCluster cluster) throws InterruptedException {
+ for (MasterThread mt : cluster.getLiveMasterThreads()) {
+ if (mt.getMaster().isActiveMaster()) {
+ mt.getMaster().abort("Aborting for tests", new Exception("Trace info"));
+ mt.join();
+ break;
+ }
+ }
+ LOG.debug("Master is aborted");
+ }
+
+ /**
+ * Find a RS that has regions of a table.
+ * @param hasMetaRegion when true, the returned RS has hbase:meta region as well
+ * @param tableName
+ * @return
+ * @throws Exception
+ */
+ private HRegionServer findRSToKill(boolean hasMetaRegion, String tableName) throws Exception {
+ List rsts = cluster.getLiveRegionServerThreads();
+ List regions = null;
+ HRegionServer hrs = null;
+
+ for (RegionServerThread rst: rsts) {
+ hrs = rst.getRegionServer();
+ while (rst.isAlive() && !hrs.isOnline()) {
+ Thread.sleep(100);
+ }
+ if (!rst.isAlive()) {
+ continue;
+ }
+ boolean isCarryingMeta = false;
+ boolean foundTableRegion = false;
+ regions = ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices());
+ for (HRegionInfo region : regions) {
+ if (region.isMetaRegion()) {
+ isCarryingMeta = true;
+ }
+ if (tableName == null || region.getTable().getNameAsString().equals(tableName)) {
+ foundTableRegion = true;
+ }
+ if (foundTableRegion && (isCarryingMeta || !hasMetaRegion)) {
+ break;
+ }
+ }
+ if (isCarryingMeta && hasMetaRegion) {
+ // clients ask for a RS with META
+ if (!foundTableRegion) {
+ final HRegionServer destRS = hrs;
+ // the RS doesn't have regions of the specified table so we need move one to this RS
+ List tableRegions =
+ TEST_UTIL.getHBaseAdmin().getTableRegions(TableName.valueOf(tableName));
+ final HRegionInfo hri = tableRegions.get(0);
+ TEST_UTIL.getHBaseAdmin().move(hri.getEncodedNameAsBytes(),
+ Bytes.toBytes(destRS.getServerName().getServerName()));
+ // wait for region move completes
+ final RegionStates regionStates =
+ TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStates();
+ TEST_UTIL.waitFor(45000, 200, new Waiter.Predicate() {
+ @Override
+ public boolean evaluate() throws Exception {
+ ServerName sn = regionStates.getRegionServerOfRegion(hri);
+ return (sn != null && sn.equals(destRS.getServerName()));
+ }
+ });
+ }
+ return hrs;
+ } else if (hasMetaRegion || isCarryingMeta) {
+ continue;
+ }
+ if (foundTableRegion) break;
+ }
+
+ return hrs;
+ }
+}
\ No newline at end of file
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer2.java
new file mode 100644
index 00000000000..395eef226a8
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer2.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.balancer;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.testclassification.FlakeyTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({FlakeyTests.class, MediumTests.class})
+public class TestStochasticLoadBalancer2 extends BalancerTestBase {
+ private static final Log LOG = LogFactory.getLog(TestStochasticLoadBalancer2.class);
+
+ @Test (timeout = 800000)
+ public void testRegionReplicasOnMidCluster() {
+ conf.setFloat("hbase.master.balancer.stochastic.maxMovePercent", 1.0f);
+ conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 2000000L);
+ conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 90 * 1000); // 90 sec
+ conf.setFloat("hbase.master.balancer.stochastic.localityCost", 0);
+ TestStochasticLoadBalancer.loadBalancer.setConf(conf);
+ int numNodes = 200;
+ int numRegions = 40 * 200;
+ int replication = 3; // 3 replicas per region
+ int numRegionsPerServer = 30; //all regions are mostly balanced
+ int numTables = 10;
+ testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
+ }
+
+ @Test (timeout = 800000)
+ public void testRegionReplicasOnLargeCluster() {
+ conf.setFloat("hbase.master.balancer.stochastic.maxMovePercent", 1.0f);
+ conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 2000000L);
+ conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 90 * 1000); // 90 sec
+ conf.setFloat("hbase.master.balancer.stochastic.localityCost", 0);
+ loadBalancer.setConf(conf);
+ int numNodes = 1000;
+ int numRegions = 20 * numNodes; // 20 * replication regions per RS
+ int numRegionsPerServer = 19; // all servers except one
+ int numTables = 100;
+ int replication = 3;
+ testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, true);
+ }
+
+ @Test (timeout = 800000)
+ public void testRegionReplicasOnMidClusterHighReplication() {
+ conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 4000000L);
+ conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 120 * 1000); // 120 sec
+ conf.setFloat("hbase.master.balancer.stochastic.localityCost", 0);
+ conf.setFloat("hbase.master.balancer.stochastic.maxMovePercent", 1.0f);
+ loadBalancer.setConf(conf);
+ int numNodes = 80;
+ int numRegions = 6 * numNodes;
+ int replication = 80; // 80 replicas per region, one for each server
+ int numRegionsPerServer = 5;
+ int numTables = 10;
+ testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, false, true);
+ }
+
+ @Test (timeout = 800000)
+ public void testRegionReplicationOnMidClusterReplicationGreaterThanNumNodes() {
+ conf.setLong(StochasticLoadBalancer.MAX_STEPS_KEY, 2000000L);
+ conf.setLong("hbase.master.balancer.stochastic.maxRunningTime", 120 * 1000); // 120 sec
+ conf.setFloat("hbase.master.balancer.stochastic.localityCost", 0);
+ conf.setFloat("hbase.master.balancer.stochastic.maxMovePercent", 1.0f);
+ loadBalancer.setConf(conf);
+ int numNodes = 40;
+ int numRegions = 6 * 50;
+ int replication = 50; // 50 replicas per region, more than numNodes
+ int numRegionsPerServer = 6;
+ int numTables = 10;
+ testWithCluster(numNodes, numRegions, numRegionsPerServer, replication, numTables, true, false);
+ }
+}
\ No newline at end of file
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
new file mode 100644
index 00000000000..125f5a136f5
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestMasterFailoverWithProcedures.java
@@ -0,0 +1,514 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.master.procedure;
+
+import java.io.IOException;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.procedure2.Procedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
+import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility.TestProcedure;
+import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
+import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DeleteTableState;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableState;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.EnableTableState;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableState;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.ModifyRegionUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+@Category({MasterTests.class, LargeTests.class})
+public class TestMasterFailoverWithProcedures {
+ private static final Log LOG = LogFactory.getLog(TestMasterFailoverWithProcedures.class);
+
+ protected static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+ private static void setupConf(Configuration conf) {
+ // don't waste time retrying with the roll, the test is already slow enough.
+ conf.setInt("hbase.procedure.store.wal.max.retries.before.roll", 1);
+ conf.setInt("hbase.procedure.store.wal.wait.before.roll", 0);
+ conf.setInt("hbase.procedure.store.wal.max.roll.retries", 1);
+ conf.setInt("hbase.procedure.store.wal.sync.failure.roll.max", 1);
+ }
+
+ @Before
+ public void setup() throws Exception {
+ setupConf(UTIL.getConfiguration());
+ UTIL.startMiniCluster(2, 1);
+
+ final ProcedureExecutor procExec = getMasterProcedureExecutor();
+ ProcedureTestingUtility.setToggleKillBeforeStoreUpdate(procExec, false);
+ ProcedureTestingUtility.setKillBeforeStoreUpdate(procExec, false);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ try {
+ UTIL.shutdownMiniCluster();
+ } catch (Exception e) {
+ LOG.warn("failure shutting down cluster", e);
+ }
+ }
+
+ @Test(timeout=60000)
+ public void testWalRecoverLease() throws Exception {
+ final ProcedureStore masterStore = getMasterProcedureExecutor().getStore();
+ assertTrue("expected WALStore for this test", masterStore instanceof WALProcedureStore);
+
+ HMaster firstMaster = UTIL.getHBaseCluster().getMaster();
+ // Abort Latch for the master store
+ final CountDownLatch masterStoreAbort = new CountDownLatch(1);
+ masterStore.registerListener(new ProcedureStore.ProcedureStoreListener() {
+ @Override
+ public void postSync() {}
+
+ @Override
+ public void abortProcess() {
+ LOG.debug("Abort store of Master");
+ masterStoreAbort.countDown();
+ }
+ });
+
+ // startup a fake master the new WAL store will take the lease
+ // and the active master should abort.
+ HMaster backupMaster3 = Mockito.mock(HMaster.class);
+ Mockito.doReturn(firstMaster.getConfiguration()).when(backupMaster3).getConfiguration();
+ Mockito.doReturn(true).when(backupMaster3).isActiveMaster();
+ final WALProcedureStore backupStore3 = new WALProcedureStore(firstMaster.getConfiguration(),
+ firstMaster.getMasterFileSystem().getFileSystem(),
+ ((WALProcedureStore)masterStore).getLogDir(),
+ new MasterProcedureEnv.WALStoreLeaseRecovery(backupMaster3));
+ // Abort Latch for the test store
+ final CountDownLatch backupStore3Abort = new CountDownLatch(1);
+ backupStore3.registerListener(new ProcedureStore.ProcedureStoreListener() {
+ @Override
+ public void postSync() {}
+
+ @Override
+ public void abortProcess() {
+ LOG.debug("Abort store of backupMaster3");
+ backupStore3Abort.countDown();
+ backupStore3.stop(true);
+ }
+ });
+ backupStore3.start(1);
+ backupStore3.recoverLease();
+
+ // Try to trigger a command on the master (WAL lease expired on the active one)
+ HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(TableName.valueOf("mtb"), "f");
+ HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, null);
+ LOG.debug("submit proc");
+ try {
+ getMasterProcedureExecutor().submitProcedure(
+ new CreateTableProcedure(getMasterProcedureExecutor().getEnvironment(), htd, regions));
+ fail("expected RuntimeException 'sync aborted'");
+ } catch (RuntimeException e) {
+ LOG.info("got " + e.getMessage());
+ }
+ LOG.debug("wait master store abort");
+ masterStoreAbort.await();
+
+ // Now the real backup master should start up
+ LOG.debug("wait backup master to startup");
+ waitBackupMaster(UTIL, firstMaster);
+ assertEquals(true, firstMaster.isStopped());
+
+ // wait the store in here to abort (the test will fail due to timeout if it doesn't)
+ LOG.debug("wait the store to abort");
+ backupStore3.getStoreTracker().setDeleted(1, false);
+ try {
+ backupStore3.delete(1);
+ fail("expected RuntimeException 'sync aborted'");
+ } catch (RuntimeException e) {
+ LOG.info("got " + e.getMessage());
+ }
+ backupStore3Abort.await();
+ }
+
+ /**
+ * Tests proper fencing in case the current WAL store is fenced
+ */
+ @Test
+ public void testWALfencingWithoutWALRolling() throws IOException {
+ testWALfencing(false);
+ }
+
+ /**
+ * Tests proper fencing in case the current WAL store does not receive writes until after the
+ * new WAL does a couple of WAL rolls.
+ */
+ @Test
+ public void testWALfencingWithWALRolling() throws IOException {
+ testWALfencing(true);
+ }
+
+ public void testWALfencing(boolean walRolls) throws IOException {
+ final ProcedureStore procStore = getMasterProcedureExecutor().getStore();
+ assertTrue("expected WALStore for this test", procStore instanceof WALProcedureStore);
+
+ HMaster firstMaster = UTIL.getHBaseCluster().getMaster();
+
+ // cause WAL rolling after a delete in WAL:
+ firstMaster.getConfiguration().setLong("hbase.procedure.store.wal.roll.threshold", 1);
+
+ HMaster backupMaster3 = Mockito.mock(HMaster.class);
+ Mockito.doReturn(firstMaster.getConfiguration()).when(backupMaster3).getConfiguration();
+ Mockito.doReturn(true).when(backupMaster3).isActiveMaster();
+ final WALProcedureStore procStore2 = new WALProcedureStore(firstMaster.getConfiguration(),
+ firstMaster.getMasterFileSystem().getFileSystem(),
+ ((WALProcedureStore)procStore).getLogDir(),
+ new MasterProcedureEnv.WALStoreLeaseRecovery(backupMaster3));
+
+ // start a second store which should fence the first one out
+ LOG.info("Starting new WALProcedureStore");
+ procStore2.start(1);
+ procStore2.recoverLease();
+
+ // before writing back to the WAL store, optionally do a couple of WAL rolls (which causes
+ // to delete the old WAL files).
+ if (walRolls) {
+ LOG.info("Inserting into second WALProcedureStore, causing WAL rolls");
+ for (int i = 0; i < 512; i++) {
+ // insert something to the second store then delete it, causing a WAL roll(s)
+ Procedure proc2 = new TestProcedure(i);
+ procStore2.insert(proc2, null);
+ procStore2.delete(proc2.getProcId()); // delete the procedure so that the WAL is removed later
+ }
+ }
+
+ // Now, insert something to the first store, should fail.
+ // If the store does a WAL roll and continue with another logId without checking higher logIds
+ // it will incorrectly succeed.
+ LOG.info("Inserting into first WALProcedureStore");
+ try {
+ procStore.insert(new TestProcedure(11), null);
+ fail("Inserting into Procedure Store should have failed");
+ } catch (Exception ex) {
+ LOG.info("Received expected exception", ex);
+ }
+ }
+
+ // ==========================================================================
+ // Test Create Table
+ // ==========================================================================
+ @Test(timeout=60000)
+ public void testCreateWithFailover() throws Exception {
+ // TODO: Should we try every step? (master failover takes long time)
+ // It is already covered by TestCreateTableProcedure
+ // but without the master restart, only the executor/store is restarted.
+ // Without Master restart we may not find bug in the procedure code
+ // like missing "wait" for resources to be available (e.g. RS)
+ testCreateWithFailoverAtStep(CreateTableState.CREATE_TABLE_ASSIGN_REGIONS.ordinal());
+ }
+
+ private void testCreateWithFailoverAtStep(final int step) throws Exception {
+ final TableName tableName = TableName.valueOf("testCreateWithFailoverAtStep" + step);
+
+ // create the table
+ ProcedureExecutor procExec = getMasterProcedureExecutor();
+ ProcedureTestingUtility.setKillBeforeStoreUpdate(procExec, true);
+ ProcedureTestingUtility.setToggleKillBeforeStoreUpdate(procExec, true);
+
+ // Start the Create procedure && kill the executor
+ byte[][] splitKeys = null;
+ HTableDescriptor htd = MasterProcedureTestingUtility.createHTD(tableName, "f1", "f2");
+ HRegionInfo[] regions = ModifyRegionUtils.createHRegionInfos(htd, splitKeys);
+ long procId = procExec.submitProcedure(
+ new CreateTableProcedure(procExec.getEnvironment(), htd, regions));
+ testRecoveryAndDoubleExecution(UTIL, procId, step, CreateTableState.values());
+
+ MasterProcedureTestingUtility.validateTableCreation(
+ UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
+ }
+
+ // ==========================================================================
+ // Test Delete Table
+ // ==========================================================================
+ @Test(timeout=60000)
+ public void testDeleteWithFailover() throws Exception {
+ // TODO: Should we try every step? (master failover takes long time)
+ // It is already covered by TestDeleteTableProcedure
+ // but without the master restart, only the executor/store is restarted.
+ // Without Master restart we may not find bug in the procedure code
+ // like missing "wait" for resources to be available (e.g. RS)
+ testDeleteWithFailoverAtStep(DeleteTableState.DELETE_TABLE_UNASSIGN_REGIONS.ordinal());
+ }
+
+ private void testDeleteWithFailoverAtStep(final int step) throws Exception {
+ final TableName tableName = TableName.valueOf("testDeleteWithFailoverAtStep" + step);
+
+ // create the table
+ byte[][] splitKeys = null;
+ HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+ getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
+ Path tableDir = FSUtils.getTableDir(getRootDir(), tableName);
+ MasterProcedureTestingUtility.validateTableCreation(
+ UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
+ UTIL.getHBaseAdmin().disableTable(tableName);
+
+ ProcedureExecutor procExec = getMasterProcedureExecutor();
+ ProcedureTestingUtility.setKillBeforeStoreUpdate(procExec, true);
+ ProcedureTestingUtility.setToggleKillBeforeStoreUpdate(procExec, true);
+
+ // Start the Delete procedure && kill the executor
+ long procId = procExec.submitProcedure(
+ new DeleteTableProcedure(procExec.getEnvironment(), tableName));
+ testRecoveryAndDoubleExecution(UTIL, procId, step, DeleteTableState.values());
+
+ MasterProcedureTestingUtility.validateTableDeletion(
+ UTIL.getHBaseCluster().getMaster(), tableName, regions, "f1", "f2");
+ }
+
+ // ==========================================================================
+ // Test Truncate Table
+ // ==========================================================================
+ @Test(timeout=90000)
+ public void testTruncateWithFailover() throws Exception {
+ // TODO: Should we try every step? (master failover takes long time)
+ // It is already covered by TestTruncateTableProcedure
+ // but without the master restart, only the executor/store is restarted.
+ // Without Master restart we may not find bug in the procedure code
+ // like missing "wait" for resources to be available (e.g. RS)
+ testTruncateWithFailoverAtStep(true, TruncateTableState.TRUNCATE_TABLE_ADD_TO_META.ordinal());
+ }
+
+ private void testTruncateWithFailoverAtStep(final boolean preserveSplits, final int step)
+ throws Exception {
+ final TableName tableName = TableName.valueOf("testTruncateWithFailoverAtStep" + step);
+
+ // create the table
+ final String[] families = new String[] { "f1", "f2" };
+ final byte[][] splitKeys = new byte[][] {
+ Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
+ };
+ HRegionInfo[] regions = MasterProcedureTestingUtility.createTable(
+ getMasterProcedureExecutor(), tableName, splitKeys, families);
+ // load and verify that there are rows in the table
+ MasterProcedureTestingUtility.loadData(
+ UTIL.getConnection(), tableName, 100, splitKeys, families);
+ assertEquals(100, UTIL.countRows(tableName));
+ // disable the table
+ UTIL.getHBaseAdmin().disableTable(tableName);
+
+ ProcedureExecutor procExec = getMasterProcedureExecutor();
+ ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+ // Start the Truncate procedure && kill the executor
+ long procId = procExec.submitProcedure(
+ new TruncateTableProcedure(procExec.getEnvironment(), tableName, preserveSplits));
+ testRecoveryAndDoubleExecution(UTIL, procId, step, TruncateTableState.values());
+
+ ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, false);
+ UTIL.waitUntilAllRegionsAssigned(tableName);
+
+ // validate the table regions and layout
+ if (preserveSplits) {
+ assertEquals(1 + splitKeys.length, UTIL.getHBaseAdmin().getTableRegions(tableName).size());
+ } else {
+ regions = UTIL.getHBaseAdmin().getTableRegions(tableName).toArray(new HRegionInfo[1]);
+ assertEquals(1, regions.length);
+ }
+ MasterProcedureTestingUtility.validateTableCreation(
+ UTIL.getHBaseCluster().getMaster(), tableName, regions, families);
+
+ // verify that there are no rows in the table
+ assertEquals(0, UTIL.countRows(tableName));
+
+ // verify that the table is read/writable
+ MasterProcedureTestingUtility.loadData(
+ UTIL.getConnection(), tableName, 50, splitKeys, families);
+ assertEquals(50, UTIL.countRows(tableName));
+ }
+
+ // ==========================================================================
+ // Test Disable Table
+ // ==========================================================================
+ @Test(timeout=60000)
+ public void testDisableTableWithFailover() throws Exception {
+ // TODO: Should we try every step? (master failover takes long time)
+ // It is already covered by TestDisableTableProcedure
+ // but without the master restart, only the executor/store is restarted.
+ // Without Master restart we may not find bug in the procedure code
+ // like missing "wait" for resources to be available (e.g. RS)
+ testDisableTableWithFailoverAtStep(
+ DisableTableState.DISABLE_TABLE_MARK_REGIONS_OFFLINE.ordinal());
+ }
+
+ private void testDisableTableWithFailoverAtStep(final int step) throws Exception {
+ final TableName tableName = TableName.valueOf("testDisableTableWithFailoverAtStep" + step);
+
+ // create the table
+ final byte[][] splitKeys = new byte[][] {
+ Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
+ };
+ MasterProcedureTestingUtility.createTable(
+ getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
+
+ ProcedureExecutor procExec = getMasterProcedureExecutor();
+ ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+ // Start the Delete procedure && kill the executor
+ long procId = procExec.submitProcedure(
+ new DisableTableProcedure(procExec.getEnvironment(), tableName, false));
+ testRecoveryAndDoubleExecution(UTIL, procId, step, DisableTableState.values());
+
+ MasterProcedureTestingUtility.validateTableIsDisabled(
+ UTIL.getHBaseCluster().getMaster(), tableName);
+ }
+
+ // ==========================================================================
+ // Test Enable Table
+ // ==========================================================================
+ @Test(timeout=60000)
+ public void testEnableTableWithFailover() throws Exception {
+ // TODO: Should we try every step? (master failover takes long time)
+ // It is already covered by TestEnableTableProcedure
+ // but without the master restart, only the executor/store is restarted.
+ // Without Master restart we may not find bug in the procedure code
+ // like missing "wait" for resources to be available (e.g. RS)
+ testEnableTableWithFailoverAtStep(
+ EnableTableState.ENABLE_TABLE_MARK_REGIONS_ONLINE.ordinal());
+ }
+
+ private void testEnableTableWithFailoverAtStep(final int step) throws Exception {
+ final TableName tableName = TableName.valueOf("testEnableTableWithFailoverAtStep" + step);
+
+ // create the table
+ final byte[][] splitKeys = new byte[][] {
+ Bytes.toBytes("a"), Bytes.toBytes("b"), Bytes.toBytes("c")
+ };
+ MasterProcedureTestingUtility.createTable(
+ getMasterProcedureExecutor(), tableName, splitKeys, "f1", "f2");
+ UTIL.getHBaseAdmin().disableTable(tableName);
+
+ ProcedureExecutor procExec = getMasterProcedureExecutor();
+ ProcedureTestingUtility.setKillAndToggleBeforeStoreUpdate(procExec, true);
+
+ // Start the Delete procedure && kill the executor
+ long procId = procExec.submitProcedure(
+ new EnableTableProcedure(procExec.getEnvironment(), tableName, false));
+ testRecoveryAndDoubleExecution(UTIL, procId, step, EnableTableState.values());
+
+ MasterProcedureTestingUtility.validateTableIsEnabled(
+ UTIL.getHBaseCluster().getMaster(), tableName);
+ }
+
+ // ==========================================================================
+ // Test Helpers
+ // ==========================================================================
+ public static void testRecoveryAndDoubleExecution(final HBaseTestingUtility testUtil,
+ final long procId, final int lastStepBeforeFailover, TState[] states) throws Exception {
+ ProcedureExecutor procExec =
+ testUtil.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+ ProcedureTestingUtility.waitProcedure(procExec, procId);
+
+ for (int i = 0; i < lastStepBeforeFailover; ++i) {
+ LOG.info("Restart "+ i +" exec state: " + states[i]);
+ ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
+ ProcedureTestingUtility.restart(procExec);
+ ProcedureTestingUtility.waitProcedure(procExec, procId);
+ }
+ ProcedureTestingUtility.assertProcNotYetCompleted(procExec, procId);
+
+ LOG.info("Trigger master failover");
+ masterFailover(testUtil);
+
+ procExec = testUtil.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+ ProcedureTestingUtility.waitProcedure(procExec, procId);
+ ProcedureTestingUtility.assertProcNotFailed(procExec, procId);
+ }
+
+ // ==========================================================================
+ // Master failover utils
+ // ==========================================================================
+ public static void masterFailover(final HBaseTestingUtility testUtil)
+ throws Exception {
+ MiniHBaseCluster cluster = testUtil.getMiniHBaseCluster();
+
+ // Kill the master
+ HMaster oldMaster = cluster.getMaster();
+ cluster.killMaster(cluster.getMaster().getServerName());
+
+ // Wait the secondary
+ waitBackupMaster(testUtil, oldMaster);
+ }
+
+ public static void waitBackupMaster(final HBaseTestingUtility testUtil,
+ final HMaster oldMaster) throws Exception {
+ MiniHBaseCluster cluster = testUtil.getMiniHBaseCluster();
+
+ HMaster newMaster = cluster.getMaster();
+ while (newMaster == null || newMaster == oldMaster) {
+ Thread.sleep(250);
+ newMaster = cluster.getMaster();
+ }
+
+ while (!(newMaster.isActiveMaster() && newMaster.isInitialized())) {
+ Thread.sleep(250);
+ }
+ }
+
+ // ==========================================================================
+ // Helpers
+ // ==========================================================================
+ private MasterProcedureEnv getMasterProcedureEnv() {
+ return getMasterProcedureExecutor().getEnvironment();
+ }
+
+ private ProcedureExecutor getMasterProcedureExecutor() {
+ return UTIL.getHBaseCluster().getMaster().getMasterProcedureExecutor();
+ }
+
+ private FileSystem getFileSystem() {
+ return UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
+ }
+
+ private Path getRootDir() {
+ return UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getRootDir();
+ }
+
+ private Path getTempDir() {
+ return UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getTempDir();
+ }
+}
\ No newline at end of file
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobFlushSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobFlushSnapshotFromClient.java
new file mode 100644
index 00000000000..fe297edcf13
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestMobFlushSnapshotFromClient.java
@@ -0,0 +1,72 @@
+/**
+ * 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.snapshot;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.mob.MobConstants;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.junit.BeforeClass;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test creating/using/deleting snapshots from the client
+ *
+ * This is an end-to-end test for the snapshot utility
+ *
+ * TODO This is essentially a clone of TestSnapshotFromClient. This is worth refactoring this
+ * because there will be a few more flavors of snapshots that need to run these tests.
+ */
+@Category({ClientTests.class, LargeTests.class})
+public class TestMobFlushSnapshotFromClient extends TestFlushSnapshotFromClient {
+ private static final Log LOG = LogFactory.getLog(TestFlushSnapshotFromClient.class);
+
+ @BeforeClass
+ public static void setupCluster() throws Exception {
+ setupConf(UTIL.getConfiguration());
+ UTIL.startMiniCluster(3);
+ }
+
+ protected static void setupConf(Configuration conf) {
+ TestFlushSnapshotFromClient.setupConf(conf);
+ UTIL.getConfiguration().setInt(MobConstants.MOB_FILE_CACHE_SIZE_KEY, 0);
+ }
+
+ @Override
+ protected void createTable() throws Exception {
+ MobSnapshotTestingUtils.createMobTable(UTIL, TABLE_NAME, 1, TEST_FAM);
+ }
+
+ @Override
+ protected void verifyRowCount(final HBaseTestingUtility util, final TableName tableName,
+ long expectedRows) throws IOException {
+ MobSnapshotTestingUtils.verifyMobRowCount(util, tableName, expectedRows);
+ }
+
+ @Override
+ protected int countRows(final Table table, final byte[]... families) throws IOException {
+ return MobSnapshotTestingUtils.countMobRows(table, families);
+ }
+}
\ No newline at end of file
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
new file mode 100644
index 00000000000..67fc60a59a8
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
@@ -0,0 +1,1320 @@
+/**
+ *
+ * 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.wal;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.lang.reflect.Method;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.wal.FaultySequenceFileLogReader;
+import org.apache.hadoop.hbase.regionserver.wal.InstrumentedLogWriter;
+import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CancelableProgressable;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.wal.WAL.Entry;
+import org.apache.hadoop.hbase.wal.WAL.Reader;
+import org.apache.hadoop.hbase.wal.WALProvider.Writer;
+import org.apache.hadoop.hbase.wal.WALSplitter.CorruptedLogFileException;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
+import org.apache.hadoop.ipc.RemoteException;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Testing {@link WAL} splitting code.
+ */
+@Category({RegionServerTests.class, LargeTests.class})
+public class TestWALSplit {
+ {
+ // Uncomment the following lines if more verbosity is needed for
+ // debugging (see HBASE-12285 for details).
+ //((Log4JLogger)DataNode.LOG).getLogger().setLevel(Level.ALL);
+ //((Log4JLogger)LeaseManager.LOG).getLogger().setLevel(Level.ALL);
+ //((Log4JLogger)FSNamesystem.LOG).getLogger().setLevel(Level.ALL);
+ }
+ private final static Log LOG = LogFactory.getLog(TestWALSplit.class);
+
+ private static Configuration conf;
+ private FileSystem fs;
+
+ protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+ private Path HBASEDIR;
+ private Path WALDIR;
+ private Path OLDLOGDIR;
+ private Path CORRUPTDIR;
+ private Path TABLEDIR;
+
+ private static final int NUM_WRITERS = 10;
+ private static final int ENTRIES = 10; // entries per writer per region
+
+ private static final String FILENAME_BEING_SPLIT = "testfile";
+ private static final TableName TABLE_NAME =
+ TableName.valueOf("t1");
+ private static final byte[] FAMILY = "f1".getBytes();
+ private static final byte[] QUALIFIER = "q1".getBytes();
+ private static final byte[] VALUE = "v1".getBytes();
+ private static final String WAL_FILE_PREFIX = "wal.dat.";
+ private static List REGIONS = new ArrayList();
+ private static final String HBASE_SKIP_ERRORS = "hbase.hlog.split.skip.errors";
+ private static String ROBBER;
+ private static String ZOMBIE;
+ private static String [] GROUP = new String [] {"supergroup"};
+ private RecoveryMode mode;
+
+ static enum Corruptions {
+ INSERT_GARBAGE_ON_FIRST_LINE,
+ INSERT_GARBAGE_IN_THE_MIDDLE,
+ APPEND_GARBAGE,
+ TRUNCATE,
+ TRUNCATE_TRAILER
+ }
+
+ @BeforeClass
+ public static void setUpBeforeClass() throws Exception {
+ conf = TEST_UTIL.getConfiguration();
+ conf.setClass("hbase.regionserver.hlog.writer.impl",
+ InstrumentedLogWriter.class, Writer.class);
+ conf.setBoolean("dfs.support.broken.append", true);
+ conf.setBoolean("dfs.support.append", true);
+ // This is how you turn off shortcircuit read currently. TODO: Fix. Should read config.
+ System.setProperty("hbase.tests.use.shortcircuit.reads", "false");
+ // Create fake maping user to group and set it to the conf.
+ Map u2g_map = new HashMap(2);
+ ROBBER = User.getCurrent().getName() + "-robber";
+ ZOMBIE = User.getCurrent().getName() + "-zombie";
+ u2g_map.put(ROBBER, GROUP);
+ u2g_map.put(ZOMBIE, GROUP);
+ DFSTestUtil.updateConfWithFakeGroupMapping(conf, u2g_map);
+ conf.setInt("dfs.heartbeat.interval", 1);
+ TEST_UTIL.startMiniDFSCluster(2);
+ }
+
+ @AfterClass
+ public static void tearDownAfterClass() throws Exception {
+ TEST_UTIL.shutdownMiniDFSCluster();
+ }
+
+ @Rule
+ public TestName name = new TestName();
+ private WALFactory wals = null;
+
+ @Before
+ public void setUp() throws Exception {
+ LOG.info("Cleaning up cluster for new test.");
+ fs = TEST_UTIL.getDFSCluster().getFileSystem();
+ HBASEDIR = TEST_UTIL.createRootDir();
+ OLDLOGDIR = new Path(HBASEDIR, HConstants.HREGION_OLDLOGDIR_NAME);
+ CORRUPTDIR = new Path(HBASEDIR, HConstants.CORRUPT_DIR_NAME);
+ TABLEDIR = FSUtils.getTableDir(HBASEDIR, TABLE_NAME);
+ REGIONS.clear();
+ Collections.addAll(REGIONS, "bbb", "ccc");
+ InstrumentedLogWriter.activateFailure = false;
+ this.mode = (conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false) ?
+ RecoveryMode.LOG_REPLAY : RecoveryMode.LOG_SPLITTING);
+ wals = new WALFactory(conf, null, name.getMethodName());
+ WALDIR = new Path(HBASEDIR, DefaultWALProvider.getWALDirectoryName(name.getMethodName()));
+ //fs.mkdirs(WALDIR);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ try {
+ wals.close();
+ } catch(IOException exception) {
+ // Some tests will move WALs out from under us. In those cases, we'll get an error on close.
+ LOG.info("Ignoring an error while closing down our WALFactory. Fine for some tests, but if" +
+ " you see a failure look here.");
+ LOG.debug("exception details", exception);
+ } finally {
+ wals = null;
+ fs.delete(HBASEDIR, true);
+ }
+ }
+
+ /**
+ * Simulates splitting a WAL out from under a regionserver that is still trying to write it.
+ * Ensures we do not lose edits.
+ * @throws IOException
+ * @throws InterruptedException
+ */
+ @Test (timeout=300000)
+ public void testLogCannotBeWrittenOnceParsed() throws IOException, InterruptedException {
+ final AtomicLong counter = new AtomicLong(0);
+ AtomicBoolean stop = new AtomicBoolean(false);
+ // Region we'll write edits too and then later examine to make sure they all made it in.
+ final String region = REGIONS.get(0);
+ final int numWriters = 3;
+ Thread zombie = new ZombieLastLogWriterRegionServer(counter, stop, region, numWriters);
+ try {
+ long startCount = counter.get();
+ zombie.start();
+ // Wait till writer starts going.
+ while (startCount == counter.get()) Threads.sleep(1);
+ // Give it a second to write a few appends.
+ Threads.sleep(1000);
+ final Configuration conf2 = HBaseConfiguration.create(this.conf);
+ final User robber = User.createUserForTesting(conf2, ROBBER, GROUP);
+ int count = robber.runAs(new PrivilegedExceptionAction() {
+ @Override
+ public Integer run() throws Exception {
+ StringBuilder ls = new StringBuilder("Contents of WALDIR (").append(WALDIR)
+ .append("):\n");
+ for (FileStatus status : fs.listStatus(WALDIR)) {
+ ls.append("\t").append(status.toString()).append("\n");
+ }
+ LOG.debug(ls);
+ LOG.info("Splitting WALs out from under zombie. Expecting " + numWriters + " files.");
+ WALSplitter.split(HBASEDIR, WALDIR, OLDLOGDIR, fs, conf2, wals);
+ LOG.info("Finished splitting out from under zombie.");
+ Path[] logfiles = getLogForRegion(HBASEDIR, TABLE_NAME, region);
+ assertEquals("wrong number of split files for region", numWriters, logfiles.length);
+ int count = 0;
+ for (Path logfile: logfiles) {
+ count += countWAL(logfile);
+ }
+ return count;
+ }
+ });
+ LOG.info("zombie=" + counter.get() + ", robber=" + count);
+ assertTrue("The log file could have at most 1 extra log entry, but can't have less. " +
+ "Zombie could write " + counter.get() + " and logfile had only " + count,
+ counter.get() == count || counter.get() + 1 == count);
+ } finally {
+ stop.set(true);
+ zombie.interrupt();
+ Threads.threadDumpingIsAlive(zombie);
+ }
+ }
+
+ /**
+ * This thread will keep writing to a 'wal' file even after the split process has started.
+ * It simulates a region server that was considered dead but woke up and wrote some more to the
+ * last log entry. Does its writing as an alternate user in another filesystem instance to
+ * simulate better it being a regionserver.
+ */
+ class ZombieLastLogWriterRegionServer extends Thread {
+ final AtomicLong editsCount;
+ final AtomicBoolean stop;
+ final int numOfWriters;
+ /**
+ * Region to write edits for.
+ */
+ final String region;
+ final User user;
+
+ public ZombieLastLogWriterRegionServer(AtomicLong counter, AtomicBoolean stop,
+ final String region, final int writers)
+ throws IOException, InterruptedException {
+ super("ZombieLastLogWriterRegionServer");
+ setDaemon(true);
+ this.stop = stop;
+ this.editsCount = counter;
+ this.region = region;
+ this.user = User.createUserForTesting(conf, ZOMBIE, GROUP);
+ numOfWriters = writers;
+ }
+
+ @Override
+ public void run() {
+ try {
+ doWriting();
+ } catch (IOException e) {
+ LOG.warn(getName() + " Writer exiting " + e);
+ } catch (InterruptedException e) {
+ LOG.warn(getName() + " Writer exiting " + e);
+ }
+ }
+
+ private void doWriting() throws IOException, InterruptedException {
+ this.user.runAs(new PrivilegedExceptionAction