HBASE-3872 Hole in split transaction rollback; edits to .META. need to be rolled back even if it seems like they didn't make it

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1146526 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2011-07-14 00:27:14 +00:00
parent 53b5b9005e
commit 7f10f29425
10 changed files with 206 additions and 94 deletions

View File

@ -409,6 +409,8 @@ Release 0.90.4 - Unreleased
HBASE-4033 The shutdown RegionServer could be added to HBASE-4033 The shutdown RegionServer could be added to
AssignmentManager.servers again (Jieshan Bean) AssignmentManager.servers again (Jieshan Bean)
HBASE-4088 npes in server shutdown HBASE-4088 npes in server shutdown
HBASE-3872 Hole in split transaction rollback; edits to .META. need
to be rolled back even if it seems like they didn't make it
IMPROVEMENT IMPROVEMENT
HBASE-3882 hbase-config.sh needs to be updated so it can auto-detects the HBASE-3882 hbase-config.sh needs to be updated so it can auto-detects the

View File

@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.hbase.util.Writables;
@ -161,16 +162,15 @@ class CatalogJanitor extends Chore {
* the filesystem. * the filesystem.
* @throws IOException * @throws IOException
*/ */
boolean cleanParent(final HRegionInfo parent, boolean cleanParent(final HRegionInfo parent, Result rowContent)
Result rowContent)
throws IOException { throws IOException {
boolean result = false; boolean result = false;
// Run checks on each daughter split. // Run checks on each daughter split.
boolean hasReferencesA = Pair<Boolean, Boolean> a =
checkDaughter(parent, rowContent, HConstants.SPLITA_QUALIFIER); checkDaughter(parent, rowContent, HConstants.SPLITA_QUALIFIER);
boolean hasReferencesB = Pair<Boolean, Boolean> b =
checkDaughter(parent, rowContent, HConstants.SPLITB_QUALIFIER); checkDaughter(parent, rowContent, HConstants.SPLITB_QUALIFIER);
if (!hasReferencesA && !hasReferencesB) { if ((a.getFirst() && !a.getSecond()) && (b.getFirst() && !b.getSecond())) {
LOG.debug("Deleting region " + parent.getRegionNameAsString() + LOG.debug("Deleting region " + parent.getRegionNameAsString() +
" because daughter splits no longer hold references"); " because daughter splits no longer hold references");
// This latter regionOffline should not be necessary but is done for now // This latter regionOffline should not be necessary but is done for now
@ -197,14 +197,26 @@ class CatalogJanitor extends Chore {
* @param parent * @param parent
* @param rowContent * @param rowContent
* @param qualifier * @param qualifier
* @return True if this daughter still has references to the parent. * @return A pair where the first boolean says whether or not the daughter
* region directory exists in the filesystem and then the second boolean says
* whether the daughter has references to the parent.
* @throws IOException * @throws IOException
*/ */
boolean checkDaughter(final HRegionInfo parent, Pair<Boolean, Boolean> checkDaughter(final HRegionInfo parent,
final Result rowContent, final byte [] qualifier) final Result rowContent, final byte [] qualifier)
throws IOException { throws IOException {
HRegionInfo hri = getDaughterRegionInfo(rowContent, qualifier); HRegionInfo hri = getDaughterRegionInfo(rowContent, qualifier);
return hasReferences(parent, rowContent, hri, qualifier); Pair<Boolean, Boolean> result =
checkDaughterInFs(parent, rowContent, hri, qualifier);
if (result.getFirst() && !result.getSecond()) {
// Remove daughter from the parent IFF the daughter region exists in FS.
// If there is no daughter region in the filesystem, must be because of
// a failed split. The ServerShutdownHandler will do the fixup. Don't
// do any deletes in here that could intefere with ServerShutdownHandler
// fixup
removeDaughterFromParent(parent, hri, qualifier);
}
return result;
} }
/** /**
@ -242,23 +254,35 @@ class CatalogJanitor extends Chore {
/** /**
* Checks if a daughter region -- either splitA or splitB -- still holds * Checks if a daughter region -- either splitA or splitB -- still holds
* references to parent. If not, removes reference to the split from * references to parent. If not, removes reference to the split from
* the parent meta region row so we don't check it any more. * the parent meta region row so we don't check it any more. Also checks
* daughter region exists in the filesytem.
* @param parent Parent region name. * @param parent Parent region name.
* @param rowContent Keyed content of the parent row in meta region. * @param rowContent Keyed content of the parent row in meta region.
* @param split Which column family. * @param split Which column family.
* @param qualifier Which of the daughters to look at, splitA or splitB. * @param qualifier Which of the daughters to look at, splitA or splitB.
* @return True if still has references to parent. * @return A pair where the first boolean says whether or not the daughter
* region directory exists in the filesystem and then the second boolean says
* whether the daughter has references to the parent.
* @throws IOException * @throws IOException
*/ */
boolean hasReferences(final HRegionInfo parent, Pair<Boolean, Boolean> checkDaughterInFs(final HRegionInfo parent,
final Result rowContent, final HRegionInfo split, final Result rowContent, final HRegionInfo split,
final byte [] qualifier) final byte [] qualifier)
throws IOException { throws IOException {
boolean result = false; boolean references = false;
if (split == null) return result; boolean exists = false;
if (split == null) {
return new Pair<Boolean, Boolean>(Boolean.FALSE, Boolean.FALSE);
}
FileSystem fs = this.services.getMasterFileSystem().getFileSystem(); FileSystem fs = this.services.getMasterFileSystem().getFileSystem();
Path rootdir = this.services.getMasterFileSystem().getRootDir(); Path rootdir = this.services.getMasterFileSystem().getRootDir();
Path tabledir = new Path(rootdir, split.getTableNameAsString()); Path tabledir = new Path(rootdir, split.getTableNameAsString());
Path regiondir = new Path(tabledir, split.getEncodedName());
exists = fs.exists(regiondir);
if (!exists) {
LOG.warn("Daughter regiondir does not exist: " + regiondir.toString());
return new Pair<Boolean, Boolean>(exists, Boolean.FALSE);
}
HTableDescriptor parentDescriptor = getTableDescriptor(parent.getTableName()); HTableDescriptor parentDescriptor = getTableDescriptor(parent.getTableName());
for (HColumnDescriptor family: parentDescriptor.getFamilies()) { for (HColumnDescriptor family: parentDescriptor.getFamilies()) {
@ -275,18 +299,16 @@ class CatalogJanitor extends Chore {
); );
if (ps != null && ps.length > 0) { if (ps != null && ps.length > 0) {
result = true; references = true;
break; break;
} }
} }
if (!result) { return new Pair<Boolean, Boolean>(Boolean.valueOf(exists),
removeDaughterFromParent(parent, split, qualifier); Boolean.valueOf(references));
}
return result;
} }
private HTableDescriptor getTableDescriptor(byte[] tableName) private HTableDescriptor getTableDescriptor(byte[] tableName)
throws TableExistsException, FileNotFoundException, IOException { throws TableExistsException, FileNotFoundException, IOException {
return this.services.getTableDescriptors().get(Bytes.toString(tableName)); return this.services.getTableDescriptors().get(Bytes.toString(tableName));
} }
} }

View File

@ -237,11 +237,12 @@ public class ServerShutdownHandler extends EventHandler {
*/ */
static void fixupDaughters(final Result result, static void fixupDaughters(final Result result,
final AssignmentManager assignmentManager, final AssignmentManager assignmentManager,
final CatalogTracker catalogTracker) throws IOException { final CatalogTracker catalogTracker)
throws IOException {
fixupDaughter(result, HConstants.SPLITA_QUALIFIER, assignmentManager, fixupDaughter(result, HConstants.SPLITA_QUALIFIER, assignmentManager,
catalogTracker); catalogTracker);
fixupDaughter(result, HConstants.SPLITB_QUALIFIER, assignmentManager, fixupDaughter(result, HConstants.SPLITB_QUALIFIER, assignmentManager,
catalogTracker); catalogTracker);
} }
/** /**
@ -282,8 +283,8 @@ public class ServerShutdownHandler extends EventHandler {
} }
/** /**
* Look for presence of the daughter OR of a split of the daughter. Daughter * Look for presence of the daughter OR of a split of the daughter in .META.
* could have been split over on regionserver before a run of the * Daughter could have been split over on regionserver before a run of the
* catalogJanitor had chance to clear reference from parent. * catalogJanitor had chance to clear reference from parent.
* @param daughter Daughter region to search for. * @param daughter Daughter region to search for.
* @throws IOException * @throws IOException
@ -350,4 +351,4 @@ public class ServerShutdownHandler extends EventHandler {
return false; return false;
} }
} }
} }

View File

@ -1380,7 +1380,8 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
public void postOpenDeployTasks(final HRegion r, final CatalogTracker ct, public void postOpenDeployTasks(final HRegion r, final CatalogTracker ct,
final boolean daughter) final boolean daughter)
throws KeeperException, IOException { throws KeeperException, IOException {
LOG.info("HRS.PostOpenDeployTasks"); LOG.info("Post open deploy tasks for region=" + r.getRegionNameAsString() +
", daughter=" + daughter);
// Do checks to see if we need to compact (references or too many files) // Do checks to see if we need to compact (references or too many files)
for (Store s : r.getStores().values()) { for (Store s : r.getStores().values()) {
if (s.hasReferences() || s.needsCompaction()) { if (s.hasReferences() || s.needsCompaction()) {
@ -1393,32 +1394,23 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
LOG.info("addToOnlineRegions is done" + r.getRegionInfo()); LOG.info("addToOnlineRegions is done" + r.getRegionInfo());
// Update ZK, ROOT or META // Update ZK, ROOT or META
if (r.getRegionInfo().isRootRegion()) { if (r.getRegionInfo().isRootRegion()) {
LOG.info("setRootLocation");
RootLocationEditor.setRootLocation(getZooKeeper(), RootLocationEditor.setRootLocation(getZooKeeper(),
this.serverNameFromMasterPOV); this.serverNameFromMasterPOV);
} else if (r.getRegionInfo().isMetaRegion()) { } else if (r.getRegionInfo().isMetaRegion()) {
LOG.info("updateMetaLocation");
MetaEditor.updateMetaLocation(ct, r.getRegionInfo(), MetaEditor.updateMetaLocation(ct, r.getRegionInfo(),
this.serverNameFromMasterPOV); this.serverNameFromMasterPOV);
} else { } else {
LOG.info("updateMetaLocation 111");
if (daughter) { if (daughter) {
LOG.info("updateMetaLocation 22");
// If daughter of a split, update whole row, not just location. // If daughter of a split, update whole row, not just location.
MetaEditor.addDaughter(ct, r.getRegionInfo(), MetaEditor.addDaughter(ct, r.getRegionInfo(),
this.serverNameFromMasterPOV); this.serverNameFromMasterPOV);
} else { } else {
LOG.info("updateMetaLocation 33");
MetaEditor.updateRegionLocation(ct, r.getRegionInfo(), MetaEditor.updateRegionLocation(ct, r.getRegionInfo(),
this.serverNameFromMasterPOV); this.serverNameFromMasterPOV);
} }
} }
LOG.info("END HRS.PostOpenDeployTasks"); LOG.info("Done with post open deploy taks for region=" +
r.getRegionNameAsString() + ", daughter=" + daughter);
} }

View File

@ -24,7 +24,6 @@ import java.io.IOException;
import org.apache.hadoop.hbase.catalog.CatalogTracker; import org.apache.hadoop.hbase.catalog.CatalogTracker;
import org.apache.hadoop.hbase.ipc.HBaseRpcMetrics; import org.apache.hadoop.hbase.ipc.HBaseRpcMetrics;
import org.apache.hadoop.hbase.regionserver.wal.HLog; import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
import java.util.Set; import java.util.Set;

View File

@ -62,15 +62,20 @@ class SplitRequest implements Runnable {
st.execute(this.server, this.server); st.execute(this.server, this.server);
} catch (Exception e) { } catch (Exception e) {
try { try {
LOG.info("Running rollback of failed split of " + parent + "; " LOG.info("Running rollback/cleanup of failed split of " +
+ e.getMessage()); parent.getRegionNameAsString() + "; " + e.getMessage());
st.rollback(this.server, this.server); if (st.rollback(this.server, this.server)) {
LOG.info("Successful rollback of failed split of " + parent); LOG.info("Successful rollback of failed split of " +
parent.getRegionNameAsString());
} else {
this.server.abort("Abort; we got an error after point-of-no-return");
}
} catch (RuntimeException ee) { } catch (RuntimeException ee) {
// If failed rollback, kill server to avoid having a hole in table. String msg = "Failed rollback of failed split of " +
LOG.info("Failed rollback of failed split of " parent.getRegionNameAsString() + " -- aborting server";
+ parent.getRegionNameAsString() + " -- aborting server", ee); // If failed rollback, kill this server to avoid having a hole in table.
this.server.abort("Failed split"); LOG.info(msg, ee);
this.server.abort(msg);
} }
return; return;
} }

View File

@ -60,18 +60,18 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
/** /**
* Executes region split as a "transaction". Call {@link #prepare()} to setup * Executes region split as a "transaction". Call {@link #prepare()} to setup
* the transaction, {@link #execute(OnlineRegions)} to run the transaction and * the transaction, {@link #execute(Server, RegionServerServices)} to run the
* {@link #rollback(OnlineRegions)} to cleanup if execute fails. * transaction and {@link #rollback(OnlineRegions)} to cleanup if execute fails.
* *
* <p>Here is an example of how you would use this class: * <p>Here is an example of how you would use this class:
* <pre> * <pre>
* SplitTransaction st = new SplitTransaction(this.conf, parent, midKey) * SplitTransaction st = new SplitTransaction(this.conf, parent, midKey)
* if (!st.prepare()) return; * if (!st.prepare()) return;
* try { * try {
* st.execute(myOnlineRegions); * st.execute(server, services);
* } catch (IOException ioe) { * } catch (IOException ioe) {
* try { * try {
* st.rollback(myOnlineRegions); * st.rollback(server, services);
* return; * return;
* } catch (RuntimeException e) { * } catch (RuntimeException e) {
* myAbortable.abort("Failed split, abort"); * myAbortable.abort("Failed split, abort");
@ -101,7 +101,9 @@ public class SplitTransaction {
private final byte [] splitrow; private final byte [] splitrow;
/** /**
* Types to add to the transaction journal * Types to add to the transaction journal.
* Each enum is a step in the split transaction. Used to figure how much
* we need to rollback.
*/ */
enum JournalEntry { enum JournalEntry {
/** /**
@ -127,7 +129,13 @@ public class SplitTransaction {
/** /**
* Started in on the creation of the second daughter region. * Started in on the creation of the second daughter region.
*/ */
STARTED_REGION_B_CREATION STARTED_REGION_B_CREATION,
/**
* Point of no return.
* If we got here, then transaction is not recoverable other than by
* crashing out the regionserver.
*/
PONR
} }
/* /*
@ -137,7 +145,7 @@ public class SplitTransaction {
/** /**
* Constructor * Constructor
* @param services So we can online new servces. If null, we'll skip onlining * @param services So we can online new regions. If null, we'll skip onlining
* (Useful testing). * (Useful testing).
* @param c Configuration to use running split * @param c Configuration to use running split
* @param r Region to split * @param r Region to split
@ -156,7 +164,7 @@ public class SplitTransaction {
*/ */
public boolean prepare() { public boolean prepare() {
if (this.parent.isClosed() || this.parent.isClosing()) return false; if (this.parent.isClosed() || this.parent.isClosing()) return false;
// Split key can be false if this region is unsplittable; i.e. has refs. // Split key can be null if this region is unsplittable; i.e. has refs.
if (this.splitrow == null) return false; if (this.splitrow == null) return false;
HRegionInfo hri = this.parent.getRegionInfo(); HRegionInfo hri = this.parent.getRegionInfo();
parent.prepareToSplit(); parent.prepareToSplit();
@ -196,13 +204,14 @@ public class SplitTransaction {
/** /**
* Run the transaction. * Run the transaction.
* @param server Hosting server instance. * @param server Hosting server instance. Can be null when testing (won't try
* and update in zk if a null server)
* @param services Used to online/offline regions. * @param services Used to online/offline regions.
* @throws IOException If thrown, transaction failed. Call {@link #rollback(OnlineRegions)} * @throws IOException If thrown, transaction failed. Call {@link #rollback(Server, RegionServerServices)}
* @return Regions created * @return Regions created
* @throws KeeperException * @throws KeeperException
* @throws NodeExistsException * @throws NodeExistsException
* @see #rollback(OnlineRegions) * @see #rollback(Server, RegionServerServices)
*/ */
public PairOfSameType<HRegion> execute(final Server server, public PairOfSameType<HRegion> execute(final Server server,
final RegionServerServices services) final RegionServerServices services)
@ -227,7 +236,7 @@ public class SplitTransaction {
this.fileSplitTimeout); this.fileSplitTimeout);
// Set ephemeral SPLITTING znode up in zk. Mocked servers sometimes don't // Set ephemeral SPLITTING znode up in zk. Mocked servers sometimes don't
// have zookeeper so don't do zk stuff if zookeeper is null // have zookeeper so don't do zk stuff if server or zookeeper is null
if (server != null && server.getZooKeeper() != null) { if (server != null && server.getZooKeeper() != null) {
try { try {
this.znodeVersion = createNodeSplitting(server.getZooKeeper(), this.znodeVersion = createNodeSplitting(server.getZooKeeper(),
@ -259,13 +268,13 @@ public class SplitTransaction {
} }
this.journal.add(JournalEntry.OFFLINED_PARENT); this.journal.add(JournalEntry.OFFLINED_PARENT);
// TODO: If the below were multithreaded would we complete steps in less // TODO: If splitStoreFiles were multithreaded would we complete steps in
// elapsed time? St.Ack 20100920 // less elapsed time? St.Ack 20100920
//
splitStoreFiles(this.splitdir, hstoreFilesToSplit);
// splitStoreFiles creates daughter region dirs under the parent splits dir // splitStoreFiles creates daughter region dirs under the parent splits dir
// Nothing to unroll here if failure -- clean up of CREATE_SPLIT_DIR will // Nothing to unroll here if failure -- clean up of CREATE_SPLIT_DIR will
// clean this up. // clean this up.
splitStoreFiles(this.splitdir, hstoreFilesToSplit);
// Log to the journal that we are creating region A, the first daughter // Log to the journal that we are creating region A, the first daughter
// region. We could fail halfway through. If we do, we could have left // region. We could fail halfway through. If we do, we could have left
@ -278,27 +287,38 @@ public class SplitTransaction {
this.journal.add(JournalEntry.STARTED_REGION_B_CREATION); this.journal.add(JournalEntry.STARTED_REGION_B_CREATION);
HRegion b = createDaughterRegion(this.hri_b, this.parent.rsServices); HRegion b = createDaughterRegion(this.hri_b, this.parent.rsServices);
// Edit parent in meta // Edit parent in meta. Offlines parent region and adds splita and splitb.
if (!testing) { if (!testing) {
MetaEditor.offlineParentInMeta(server.getCatalogTracker(), MetaEditor.offlineParentInMeta(server.getCatalogTracker(),
this.parent.getRegionInfo(), a.getRegionInfo(), b.getRegionInfo()); this.parent.getRegionInfo(), a.getRegionInfo(), b.getRegionInfo());
} }
// This is the point of no return. We are committed to the split now. We // This is the point of no return. Adding subsequent edits to .META. as we
// have still the daughter regions to open but meta has been changed. // do below when we do the daugther opens adding each to .META. can fail in
// If we fail from here on out, we cannot rollback so, we'll just abort. // various interesting ways the most interesting of which is a timeout
if (!testing) { // BUT the edits all go through (See HBASE-3872). IF we reach the POWR
// then subsequent failures need to crash out this regionserver; the
// server shutdown processing should be able to fix-up the incomplete split.
this.journal.add(JournalEntry.PONR);
// Open daughters in parallel. // Open daughters in parallel.
DaughterOpener aOpener = new DaughterOpener(server, services, a); DaughterOpener aOpener = new DaughterOpener(server, services, a);
DaughterOpener bOpener = new DaughterOpener(server, services, b); DaughterOpener bOpener = new DaughterOpener(server, services, b);
aOpener.start(); aOpener.start();
bOpener.start(); bOpener.start();
try { try {
aOpener.join(); aOpener.join();
bOpener.join(); bOpener.join();
} catch (InterruptedException e) { } catch (InterruptedException e) {
server.abort("Exception running daughter opens", e); Thread.currentThread().interrupt();
} throw new IOException("Interrupted " + e.getMessage());
}
if (aOpener.getException() != null) {
throw new IOException("Failed " +
aOpener.getName(), aOpener.getException());
}
if (bOpener.getException() != null) {
throw new IOException("Failed " +
bOpener.Name(), bOpener.getException());
} }
// Tell master about split by updating zk. If we fail, abort. // Tell master about split by updating zk. If we fail, abort.
@ -328,12 +348,10 @@ public class SplitTransaction {
if (e instanceof InterruptedException) { if (e instanceof InterruptedException) {
Thread.currentThread().interrupt(); Thread.currentThread().interrupt();
} }
server.abort("Failed telling master about split", e); throw new IOException("Failed telling master about split", e);
} }
} }
// Coprocessor callback // Coprocessor callback
if (this.parent.getCoprocessorHost() != null) { if (this.parent.getCoprocessorHost() != null) {
this.parent.getCoprocessorHost().postSplit(a,b); this.parent.getCoprocessorHost().postSplit(a,b);
@ -345,10 +363,15 @@ public class SplitTransaction {
return new PairOfSameType<HRegion>(a, b); return new PairOfSameType<HRegion>(a, b);
} }
/*
* Open daughter region in its own thread.
* If we fail, abort this hosting server.
*/
class DaughterOpener extends Thread { class DaughterOpener extends Thread {
private final RegionServerServices services; private final RegionServerServices services;
private final Server server; private final Server server;
private final HRegion r; private final HRegion r;
private Throwable t = null;
DaughterOpener(final Server s, final RegionServerServices services, DaughterOpener(final Server s, final RegionServerServices services,
final HRegion r) { final HRegion r) {
@ -359,13 +382,20 @@ public class SplitTransaction {
this.r = r; this.r = r;
} }
/**
* @return Null if open succeeded else exception that causes us fail open.
* Call it after this thread exits else you may get wrong view on result.
*/
Throwable getException() {
return this.t;
}
@Override @Override
public void run() { public void run() {
try { try {
openDaughterRegion(this.server, this.services, r); openDaughterRegion(this.server, this.services, r);
} catch (Throwable t) { } catch (Throwable t) {
this.server.abort("Failed open of daughter " + this.t = t;
this.r.getRegionInfo().getRegionNameAsString(), t);
} }
} }
} }
@ -373,7 +403,7 @@ public class SplitTransaction {
/** /**
* Open daughter regions, add them to online list and update meta. * Open daughter regions, add them to online list and update meta.
* @param server * @param server
* @param services * @param services Can be null when testing.
* @param daughter * @param daughter
* @throws IOException * @throws IOException
* @throws KeeperException * @throws KeeperException
@ -381,20 +411,22 @@ public class SplitTransaction {
void openDaughterRegion(final Server server, void openDaughterRegion(final Server server,
final RegionServerServices services, final HRegion daughter) final RegionServerServices services, final HRegion daughter)
throws IOException, KeeperException { throws IOException, KeeperException {
if (server.isStopped() || services.isStopping()) { boolean stopping = services != null && services.isStopping();
if (server.isStopped() || stopping) {
MetaEditor.addDaughter(server.getCatalogTracker(), MetaEditor.addDaughter(server.getCatalogTracker(),
daughter.getRegionInfo(), null); daughter.getRegionInfo(), null);
LOG.info("Not opening daughter " + LOG.info("Not opening daughter " +
daughter.getRegionInfo().getRegionNameAsString() + daughter.getRegionInfo().getRegionNameAsString() +
" because stopping=" + services.isStopping() + ", stopped=" + " because stopping=" + stopping + ", stopped=" + server.isStopped());
server.isStopped());
return; return;
} }
HRegionInfo hri = daughter.getRegionInfo(); HRegionInfo hri = daughter.getRegionInfo();
LoggingProgressable reporter = LoggingProgressable reporter =
new LoggingProgressable(hri, server.getConfiguration()); new LoggingProgressable(hri, server.getConfiguration());
HRegion r = daughter.openHRegion(reporter); HRegion r = daughter.openHRegion(reporter);
services.postOpenDeployTasks(r, server.getCatalogTracker(), true); if (services != null) {
services.postOpenDeployTasks(r, server.getCatalogTracker(), true);
}
} }
static class LoggingProgressable implements CancelableProgressable { static class LoggingProgressable implements CancelableProgressable {
@ -598,16 +630,19 @@ public class SplitTransaction {
} }
/** /**
* @param or Object that can online/offline parent region. Can be passed null * @param server Hosting server instance (May be null when testing).
* by unit tests. * @param services
* @return The region we were splitting
* @throws IOException If thrown, rollback failed. Take drastic action. * @throws IOException If thrown, rollback failed. Take drastic action.
* @return True if we successfully rolled back, false if we got to the point
* of no return and so now need to abort the server to minimize damage.
*/ */
public void rollback(final Server server, final OnlineRegions or) public boolean rollback(final Server server, final RegionServerServices services)
throws IOException { throws IOException {
boolean result = true;
FileSystem fs = this.parent.getFilesystem(); FileSystem fs = this.parent.getFilesystem();
ListIterator<JournalEntry> iterator = ListIterator<JournalEntry> iterator =
this.journal.listIterator(this.journal.size()); this.journal.listIterator(this.journal.size());
// Iterate in reverse.
while (iterator.hasPrevious()) { while (iterator.hasPrevious()) {
JournalEntry je = iterator.previous(); JournalEntry je = iterator.previous();
switch(je) { switch(je) {
@ -642,13 +677,19 @@ public class SplitTransaction {
break; break;
case OFFLINED_PARENT: case OFFLINED_PARENT:
if (or != null) or.addToOnlineRegions(this.parent); if (services != null) services.addToOnlineRegions(this.parent);
break; break;
case PONR:
// We got to the point-of-no-return so we need to just abort. Return
// immediately.
return false;
default: default:
throw new RuntimeException("Unhandled journal entry: " + je); throw new RuntimeException("Unhandled journal entry: " + je);
} }
} }
return result;
} }
HRegionInfo getFirstDaughter() { HRegionInfo getFirstDaughter() {

View File

@ -113,7 +113,7 @@ public class TestCatalogTracker {
@Test public void testThatIfMETAMovesWeAreNotified() @Test public void testThatIfMETAMovesWeAreNotified()
throws IOException, InterruptedException, KeeperException { throws IOException, InterruptedException, KeeperException {
HConnection connection = Mockito.mock(HConnection.class); HConnection connection = Mockito.mock(HConnection.class);
final CatalogTracker ct = constructAndStartCatalogTracker(connection); constructAndStartCatalogTracker(connection);
try { try {
RootLocationEditor.setRootLocation(this.watcher, RootLocationEditor.setRootLocation(this.watcher,
new ServerName("example.com", 1234, System.currentTimeMillis())); new ServerName("example.com", 1234, System.currentTimeMillis()));

View File

@ -293,6 +293,14 @@ public class TestCatalogJanitor {
assertFalse(janitor.cleanParent(parent, r)); assertFalse(janitor.cleanParent(parent, r));
// Remove the reference file and try again. // Remove the reference file and try again.
assertTrue(fs.delete(p, true)); assertTrue(fs.delete(p, true));
// We will fail!!! Because split b is empty, which is right... we should
// not remove parent if daughters do not exist in fs.
assertFalse(janitor.cleanParent(parent, r));
// Put in place daughter dir for b... that should make it so parent gets
// cleaned up.
storedir = Store.getStoreHomedir(tabledir, splitb.getEncodedName(),
htd.getColumnFamilies()[0].getName());
assertTrue(fs.mkdirs(storedir));
assertTrue(janitor.cleanParent(parent, r)); assertTrue(janitor.cleanParent(parent, r));
} }
} }

View File

@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.regionserver.wal.HLog; import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.PairOfSameType; import org.apache.hadoop.hbase.util.PairOfSameType;
import org.apache.zookeeper.KeeperException;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -84,6 +85,46 @@ public class TestSplitTransaction {
this.fs.delete(this.testdir, true); this.fs.delete(this.testdir, true);
} }
@Test public void testFailAfterPONR() throws IOException, KeeperException {
final int rowcount = TEST_UTIL.loadRegion(this.parent, CF);
assertTrue(rowcount > 0);
int parentRowCount = countRows(this.parent);
assertEquals(rowcount, parentRowCount);
// Start transaction.
SplitTransaction st = prepareGOOD_SPLIT_ROW();
SplitTransaction spiedUponSt = spy(st);
Mockito.doThrow(new MockedFailedDaughterOpen()).
when(spiedUponSt).openDaughterRegion((Server)Mockito.anyObject(),
(RegionServerServices)Mockito.anyObject(), (HRegion)Mockito.anyObject());
// Run the execute. Look at what it returns.
boolean expectedException = false;
Server mockServer = Mockito.mock(Server.class);
when(mockServer.getConfiguration()).thenReturn(TEST_UTIL.getConfiguration());
try {
spiedUponSt.execute(mockServer, null);
} catch (IOException e) {
if (e.getCause() != null &&
e.getCause() instanceof MockedFailedDaughterOpen) {
expectedException = true;
}
}
assertTrue(expectedException);
// Run rollback returns that we should restart.
assertFalse(spiedUponSt.rollback(null, null));
// Make sure that region a and region b are still in the filesystem, that
// they have not been removed; this is supposed to be the case if we go
// past point of no return.
Path tableDir = this.parent.getRegionDir().getParent();
Path daughterADir =
new Path(tableDir, spiedUponSt.getFirstDaughter().getEncodedName());
Path daughterBDir =
new Path(tableDir, spiedUponSt.getSecondDaughter().getEncodedName());
assertTrue(TEST_UTIL.getTestFileSystem().exists(daughterADir));
assertTrue(TEST_UTIL.getTestFileSystem().exists(daughterBDir));
}
/** /**
* Test straight prepare works. Tries to split on {@link #GOOD_SPLIT_ROW} * Test straight prepare works. Tries to split on {@link #GOOD_SPLIT_ROW}
* @throws IOException * @throws IOException
@ -190,7 +231,7 @@ public class TestSplitTransaction {
} }
assertTrue(expectedException); assertTrue(expectedException);
// Run rollback // Run rollback
spiedUponSt.rollback(null, null); assertTrue(spiedUponSt.rollback(null, null));
// Assert I can scan parent. // Assert I can scan parent.
int parentRowCount2 = countRows(this.parent); int parentRowCount2 = countRows(this.parent);
@ -228,6 +269,7 @@ public class TestSplitTransaction {
*/ */
@SuppressWarnings("serial") @SuppressWarnings("serial")
private class MockedFailedDaughterCreation extends IOException {} private class MockedFailedDaughterCreation extends IOException {}
private class MockedFailedDaughterOpen extends IOException {}
private int countRows(final HRegion r) throws IOException { private int countRows(final HRegion r) throws IOException {
int rowcount = 0; int rowcount = 0;