HBASE-6748 Endless recursive of deleteNode happened in SplitLogManager#DeleteAsyncCallback (Jeffrey Zhong)

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1433733 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Zhihong Yu 2013-01-15 23:18:56 +00:00
parent 5f60754608
commit 10c1bceda4
2 changed files with 47 additions and 10 deletions

View File

@ -26,6 +26,7 @@ import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.S
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@ -42,9 +43,9 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.Chore;
import org.apache.hadoop.hbase.SplitLogCounters;
import org.apache.hadoop.hbase.DeserializationException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.SplitLogCounters;
import org.apache.hadoop.hbase.SplitLogTask;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.master.SplitLogManager.TaskFinisher.Status;
@ -125,6 +126,8 @@ public class SplitLogManager extends ZooKeeperListener {
private volatile Set<ServerName> deadWorkers = null;
private final Object deadWorkersLock = new Object();
private Set<String> failedDeletions = null;
/**
* Wrapper around {@link #SplitLogManager(ZooKeeperWatcher zkw, Configuration conf,
* Stoppable stopper, MasterServices master, ServerName serverName, TaskFinisher tf)}
@ -182,6 +185,8 @@ public class SplitLogManager extends ZooKeeperListener {
this.serverName = serverName;
this.timeoutMonitor =
new TimeoutMonitor(conf.getInt("hbase.splitlog.manager.timeoutmonitor.period", 1000), stopper);
this.failedDeletions = Collections.synchronizedSet(new HashSet<String>());
}
public void finishInitialization(boolean masterRecovery) {
@ -436,11 +441,12 @@ public class SplitLogManager extends ZooKeeperListener {
}
}
}
// delete the task node in zk. Keep trying indefinitely - its an async
// delete the task node in zk. It's an async
// call and no one is blocked waiting for this node to be deleted. All
// task names are unique (log.<timestamp>) there is no risk of deleting
// a future task.
deleteNode(path, Long.MAX_VALUE);
// if a deletion fails, TimeoutMonitor will retry the same deletion later
deleteNode(path, zkretries);
return;
}
@ -549,6 +555,21 @@ public class SplitLogManager extends ZooKeeperListener {
}
}
/**
* Helper function to check whether to abandon retries in ZooKeeper AsyncCallback functions
* @param statusCode integer value of a ZooKeeper exception code
* @param action description message about the retried action
* @return true when need to abandon retries otherwise false
*/
private boolean needAbandonRetries(int statusCode, String action) {
if (statusCode == KeeperException.Code.SESSIONEXPIRED.intValue()) {
LOG.error("ZK session expired. Master is expected to shut down. Abandoning retries for "
+ "action=" + action);
return true;
}
return false;
}
private void heartbeat(String path, int new_version, ServerName workerName) {
Task task = findOrCreateOrphanTask(path);
if (new_version != task.last_version) {
@ -680,8 +701,7 @@ public class SplitLogManager extends ZooKeeperListener {
}
private void deleteNodeFailure(String path) {
LOG.fatal("logic failure, failing to delete a node should never happen " +
"because delete has infinite retries");
LOG.info("Failed to delete node " + path + " and will retry soon.");
return;
}
@ -1023,6 +1043,16 @@ public class SplitLogManager extends ZooKeeperListener {
SplitLogCounters.tot_mgr_resubmit_unassigned.incrementAndGet();
LOG.debug("resubmitting unassigned task(s) after timeout");
}
// Retry previously failed deletes
if (failedDeletions.size() > 0) {
List<String> tmpPaths = new ArrayList<String>(failedDeletions);
for (String tmpPath : tmpPaths) {
// deleteNode is an async call
deleteNode(tmpPath, zkretries);
}
failedDeletions.removeAll(tmpPaths);
}
}
}
@ -1037,6 +1067,10 @@ public class SplitLogManager extends ZooKeeperListener {
public void processResult(int rc, String path, Object ctx, String name) {
SplitLogCounters.tot_mgr_node_create_result.incrementAndGet();
if (rc != 0) {
if (needAbandonRetries(rc, "Create znode " + path)) {
createNodeFailure(path);
return;
}
if (rc == KeeperException.Code.NODEEXISTS.intValue()) {
// What if there is a delete pending against this pre-existing
// znode? Then this soon-to-be-deleted task znode must be in TASK_DONE
@ -1076,8 +1110,7 @@ public class SplitLogManager extends ZooKeeperListener {
Stat stat) {
SplitLogCounters.tot_mgr_get_data_result.incrementAndGet();
if (rc != 0) {
if (rc == KeeperException.Code.SESSIONEXPIRED.intValue()) {
LOG.error("ZK session expired. Master is expected to shut down. Abandoning retries.");
if (needAbandonRetries(rc, "GetData from znode " + path)) {
return;
}
if (rc == KeeperException.Code.NONODE.intValue()) {
@ -1131,6 +1164,10 @@ public class SplitLogManager extends ZooKeeperListener {
public void processResult(int rc, String path, Object ctx) {
SplitLogCounters.tot_mgr_node_delete_result.incrementAndGet();
if (rc != 0) {
if (needAbandonRetries(rc, "Delete znode " + path)) {
failedDeletions.add(path);
return;
}
if (rc != KeeperException.Code.NONODE.intValue()) {
SplitLogCounters.tot_mgr_node_delete_err.incrementAndGet();
Long retry_count = (Long) ctx;
@ -1138,6 +1175,7 @@ public class SplitLogManager extends ZooKeeperListener {
path + " remaining retries=" + retry_count);
if (retry_count == 0) {
LOG.warn("delete failed " + path);
failedDeletions.add(path);
deleteNodeFailure(path);
} else {
deleteNode(path, retry_count - 1);
@ -1169,8 +1207,7 @@ public class SplitLogManager extends ZooKeeperListener {
@Override
public void processResult(int rc, String path, Object ctx, String name) {
if (rc != 0) {
if (rc == KeeperException.Code.SESSIONEXPIRED.intValue()) {
LOG.error("ZK session expired. Master is expected to shut down. Abandoning retries.");
if (needAbandonRetries(rc, "CreateRescan znode " + path)) {
return;
}
Long retry_count = (Long)ctx;

View File

@ -69,7 +69,7 @@ import org.apache.zookeeper.data.Stat;
public class RecoverableZooKeeper {
private static final Log LOG = LogFactory.getLog(RecoverableZooKeeper.class);
// the actual ZooKeeper client instance
private ZooKeeper zk;
volatile private ZooKeeper zk;
private final RetryCounterFactory retryCounterFactory;
// An identifier of this process in the cluster
private final String identifier;