add core name to some of the recovery logging

move getRecentUpdates into try block - we should continue with no updates if there is an exception getting them
move cancelRecovery call into recovery wait sync block

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1371030 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Mark Robert Miller 2012-08-09 03:20:24 +00:00
parent e0dbd4c4f6
commit b2ab339643
2 changed files with 28 additions and 24 deletions

View File

@ -100,7 +100,7 @@ public class RecoveryStrategy extends Thread implements SafeStopThread {
private void recoveryFailed(final SolrCore core,
final ZkController zkController, final String baseUrl,
final String shardZkNodeName, final CoreDescriptor cd) throws KeeperException, InterruptedException {
SolrException.log(log, "Recovery failed - I give up.");
SolrException.log(log, "Recovery failed - I give up. Core:" + coreName);
try {
zkController.publish(cd, ZkStateReader.RECOVERY_FAILED);
} finally {
@ -115,7 +115,7 @@ public class RecoveryStrategy extends Thread implements SafeStopThread {
ZkCoreNodeProps leaderCNodeProps = new ZkCoreNodeProps(leaderprops);
String leaderUrl = leaderCNodeProps.getCoreUrl();
log.info("Attempting to replicate from " + leaderUrl);
log.info("Attempting to replicate from " + leaderUrl + ". Core:" + coreName);
// if we are the leader, either we are trying to recover faster
// then our ephemeral timed out or we are the only node
@ -205,7 +205,7 @@ public class RecoveryStrategy extends Thread implements SafeStopThread {
SolrQueryResponse rsp = new SolrQueryResponse();
SolrRequestInfo.setRequestInfo(new SolrRequestInfo(req, rsp));
log.info("Starting recovery process. recoveringAfterStartup=" + recoveringAfterStartup);
log.info("Starting recovery process. Core:" + coreName + " - recoveringAfterStartup=" + recoveringAfterStartup);
try {
doRecovery(core);
@ -233,7 +233,7 @@ public class RecoveryStrategy extends Thread implements SafeStopThread {
UpdateLog ulog;
ulog = core.getUpdateHandler().getUpdateLog();
if (ulog == null) {
SolrException.log(log, "No UpdateLog found - cannot recover");
SolrException.log(log, "No UpdateLog found - cannot recover. Core:" + coreName);
recoveryFailed(core, zkController, baseUrl, coreZkNodeName,
core.getCoreDescriptor());
return;
@ -241,14 +241,17 @@ public class RecoveryStrategy extends Thread implements SafeStopThread {
List<Long> recentVersions;
UpdateLog.RecentUpdates recentUpdates = ulog.getRecentUpdates();
UpdateLog.RecentUpdates recentUpdates = null;
try {
recentUpdates = ulog.getRecentUpdates();
recentVersions = recentUpdates.getVersions(ulog.numRecordsToKeep);
} catch (Throwable t) {
SolrException.log(log, "Corrupt tlog - ignoring", t);
SolrException.log(log, "Corrupt tlog - ignoring. Core:" + coreName, t);
recentVersions = new ArrayList<Long>(0);
}finally {
recentUpdates.close();
} finally {
if (recentUpdates != null) {
recentUpdates.close();
}
}
List<Long> startingVersions = ulog.getStartingVersions();
@ -282,7 +285,7 @@ public class RecoveryStrategy extends Thread implements SafeStopThread {
// last operation at the time of startup had the GAP flag set...
// this means we were previously doing a full index replication
// that probably didn't complete and buffering updates in the meantime.
log.info("Looks like a previous replication recovery did not complete - skipping peer sync");
log.info("Looks like a previous replication recovery did not complete - skipping peer sync. Core:" + coreName);
firstTime = false; // skip peersync
}
}
@ -308,7 +311,7 @@ public class RecoveryStrategy extends Thread implements SafeStopThread {
// first thing we just try to sync
if (firstTime) {
firstTime = false; // only try sync the first time through the loop
log.info("Attempting to PeerSync from " + leaderUrl + " recoveringAfterStartup="+recoveringAfterStartup);
log.info("Attempting to PeerSync from " + leaderUrl + " Core:" + coreName + " - recoveringAfterStartup="+recoveringAfterStartup);
// System.out.println("Attempting to PeerSync from " + leaderUrl
// + " i am:" + zkController.getNodeName());
PeerSync peerSync = new PeerSync(core,
@ -319,7 +322,7 @@ public class RecoveryStrategy extends Thread implements SafeStopThread {
SolrQueryRequest req = new LocalSolrQueryRequest(core,
new ModifiableSolrParams());
core.getUpdateHandler().commit(new CommitUpdateCommand(req, false));
log.info("PeerSync Recovery was successful - registering as Active");
log.info("PeerSync Recovery was successful - registering as Active. Core:" + coreName);
// System.out
// .println("Sync Recovery was successful - registering as Active "
// + zkController.getNodeName());
@ -348,11 +351,11 @@ public class RecoveryStrategy extends Thread implements SafeStopThread {
return;
}
log.info("PeerSync Recovery was not successful - trying replication");
log.info("PeerSync Recovery was not successful - trying replication. Core:" + coreName);
}
//System.out.println("Sync Recovery was not successful - trying replication");
log.info("Starting Replication Recovery");
log.info("Begin buffering updates");
log.info("Starting Replication Recovery. Core:" + coreName);
log.info("Begin buffering updates. Core:" + coreName);
ulog.bufferUpdates();
replayed = false;
@ -364,7 +367,7 @@ public class RecoveryStrategy extends Thread implements SafeStopThread {
replay(ulog);
replayed = true;
log.info("Replication Recovery was successful - registering as Active");
log.info("Replication Recovery was successful - registering as Active. Core:" + coreName);
// if there are pending recovery requests, don't advert as active
zkController.publish(core.getCoreDescriptor(), ZkStateReader.ACTIVE);
close = true;
@ -387,7 +390,7 @@ public class RecoveryStrategy extends Thread implements SafeStopThread {
}
} catch (Throwable t) {
log.error("Error while trying to recover.", t);
log.error("Error while trying to recover. Core:" + coreName, t);
}
if (!successfulRecovery) {
@ -396,13 +399,13 @@ public class RecoveryStrategy extends Thread implements SafeStopThread {
// Or do a fall off retry...
try {
log.error("Recovery failed - trying again...");
log.error("Recovery failed - trying again... Core:" + coreName);
retries++;
if (retries >= MAX_RETRIES) {
if (retries == INTERRUPTED) {
} else {
log.error("Recovery failed - max retries exceeded.");
log.error("Recovery failed - max retries exceeded. Core:" + coreName);
recoveryFailed(core, zkController, baseUrl, coreZkNodeName,
core.getCoreDescriptor());
}
@ -410,7 +413,7 @@ public class RecoveryStrategy extends Thread implements SafeStopThread {
}
} catch (Exception e) {
log.error("", e);
log.error("Core:" + coreName, e);
}
try {
@ -422,13 +425,13 @@ public class RecoveryStrategy extends Thread implements SafeStopThread {
}
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
log.warn("Recovery was interrupted", e);
log.warn("Recovery was interrupted. Core:" + coreName, e);
retries = INTERRUPTED;
}
}
}
log.info("Finished recovery process");
log.info("Finished recovery process. Core:" + coreName);
}
@ -437,9 +440,9 @@ public class RecoveryStrategy extends Thread implements SafeStopThread {
Future<RecoveryInfo> future = ulog.applyBufferedUpdates();
if (future == null) {
// no replay needed\
log.info("No replay needed");
log.info("No replay needed. Core:" + coreName);
} else {
log.info("Replaying buffered documents");
log.info("Replaying buffered documents. Core:" + coreName);
// wait for replay
future.get();
}

View File

@ -198,8 +198,9 @@ public final class DefaultSolrCoreState extends SolrCoreState {
return;
}
cancelRecovery();
synchronized (recoveryLock) {
cancelRecovery();
while (recoveryRunning) {
try {
recoveryLock.wait(1000);