HBASE-11217 Race between SplitLogManager task creation + TimeoutMonitor
This commit is contained in:
parent
75d1431a28
commit
92b2c86776
|
@ -123,7 +123,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
private long resubmit_threshold;
|
||||
private long timeout;
|
||||
private long unassignedTimeout;
|
||||
private long lastNodeCreateTime = Long.MAX_VALUE;
|
||||
private long lastTaskCreateTime = Long.MAX_VALUE;
|
||||
public boolean ignoreZKDeleteForTesting = false;
|
||||
private volatile long lastRecoveringNodeCreationTime = 0;
|
||||
// When lastRecoveringNodeCreationTime is older than the following threshold, we'll check
|
||||
|
@ -247,7 +247,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
* @throws IOException
|
||||
* if there was an error while splitting any log file
|
||||
* @return cumulative size of the logfiles split
|
||||
* @throws IOException
|
||||
* @throws IOException
|
||||
*/
|
||||
public long splitLogDistributed(final Path logDir) throws IOException {
|
||||
List<Path> logDirs = new ArrayList<Path>();
|
||||
|
@ -348,7 +348,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
} catch (IOException ioe) {
|
||||
FileStatus[] files = fs.listStatus(logDir);
|
||||
if (files != null && files.length > 0) {
|
||||
LOG.warn("returning success without actually splitting and " +
|
||||
LOG.warn("returning success without actually splitting and " +
|
||||
"deleting all the log files in path " + logDir);
|
||||
} else {
|
||||
LOG.warn("Unable to delete log src dir. Ignoring. " + logDir, ioe);
|
||||
|
@ -366,7 +366,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
|
||||
/**
|
||||
* Add a task entry to splitlog znode if it is not already there.
|
||||
*
|
||||
*
|
||||
* @param taskname the path of the log to be split
|
||||
* @param batch the batch this task belongs to
|
||||
* @return true if a new entry is created, false if it is already there.
|
||||
|
@ -376,6 +376,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
// This is a znode path under the splitlog dir with the rest of the path made up of an
|
||||
// url encoding of the passed in log to split.
|
||||
String path = ZKSplitLog.getEncodedNodeName(watcher, taskname);
|
||||
lastTaskCreateTime = EnvironmentEdgeManager.currentTimeMillis();
|
||||
Task oldtask = createTaskIfAbsent(path, batch);
|
||||
if (oldtask == null) {
|
||||
// publish the task in zk
|
||||
|
@ -505,7 +506,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
if(isMetaRecovery != null) {
|
||||
if ((isMetaRecovery && !region.equalsIgnoreCase(metaEncodeRegionName))
|
||||
|| (!isMetaRecovery && region.equalsIgnoreCase(metaEncodeRegionName))) {
|
||||
// skip non-meta regions when recovering the meta region or
|
||||
// skip non-meta regions when recovering the meta region or
|
||||
// skip the meta region when recovering user regions
|
||||
continue;
|
||||
}
|
||||
|
@ -689,7 +690,6 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
}
|
||||
|
||||
private void createNodeSuccess(String path) {
|
||||
lastNodeCreateTime = EnvironmentEdgeManager.currentTimeMillis();
|
||||
LOG.debug("put up splitlog task at znode " + path);
|
||||
getDataSetWatch(path, zkretries);
|
||||
}
|
||||
|
@ -940,7 +940,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
/**
|
||||
* signal the workers that a task was resubmitted by creating the
|
||||
* RESCAN node.
|
||||
* @throws KeeperException
|
||||
* @throws KeeperException
|
||||
*/
|
||||
private void createRescanNode(long retries) {
|
||||
// The RESCAN node will be deleted almost immediately by the
|
||||
|
@ -950,6 +950,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
// might miss the watch-trigger that creation of RESCAN node provides.
|
||||
// Since the TimeoutMonitor will keep resubmitting UNASSIGNED tasks
|
||||
// therefore this behavior is safe.
|
||||
lastTaskCreateTime = EnvironmentEdgeManager.currentTimeMillis();
|
||||
SplitLogTask slt = new SplitLogTask.Done(this.serverName);
|
||||
this.watcher.getRecoverableZooKeeper().getZooKeeper().
|
||||
create(ZKSplitLog.getRescanNode(watcher), slt.toByteArray(),
|
||||
|
@ -958,7 +959,6 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
}
|
||||
|
||||
private void createRescanSuccess(String path) {
|
||||
lastNodeCreateTime = EnvironmentEdgeManager.currentTimeMillis();
|
||||
SplitLogCounters.tot_mgr_rescan.incrementAndGet();
|
||||
getDataSetWatch(path, zkretries);
|
||||
}
|
||||
|
@ -1217,7 +1217,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
// when SplitLogWorker recovers a region by directly replaying unflushed WAL edits,
|
||||
// last flushed sequence Id changes when newly assigned RS flushes writes to the region.
|
||||
// If the newly assigned RS fails again(a chained RS failures scenario), the last flushed
|
||||
// sequence Id name space (sequence Id only valid for a particular RS instance), changes
|
||||
// sequence Id name space (sequence Id only valid for a particular RS instance), changes
|
||||
// when different newly assigned RS flushes the region.
|
||||
// Therefore, in this mode we need to fetch last sequence Ids from ZK where we keep history of
|
||||
// last flushed sequence Id for each failed RS instance.
|
||||
|
@ -1410,7 +1410,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
// master should spawn both a manager and a worker thread to guarantee
|
||||
// that there is always one worker in the system
|
||||
if (tot > 0 && !found_assigned_task &&
|
||||
((EnvironmentEdgeManager.currentTimeMillis() - lastNodeCreateTime) >
|
||||
((EnvironmentEdgeManager.currentTimeMillis() - lastTaskCreateTime) >
|
||||
unassignedTimeout)) {
|
||||
for (Map.Entry<String, Task> e : tasks.entrySet()) {
|
||||
String path = e.getKey();
|
||||
|
@ -1518,15 +1518,10 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
}
|
||||
if (rc == KeeperException.Code.NONODE.intValue()) {
|
||||
SplitLogCounters.tot_mgr_get_data_nonode.incrementAndGet();
|
||||
// The task znode has been deleted. Must be some pending delete
|
||||
// that deleted the task. Assume success because a task-znode is
|
||||
// is only deleted after TaskFinisher is successful.
|
||||
LOG.warn("task znode " + path + " vanished.");
|
||||
try {
|
||||
getDataSetWatchSuccess(path, null, Integer.MIN_VALUE);
|
||||
} catch (DeserializationException e) {
|
||||
LOG.warn("Deserialization problem", e);
|
||||
}
|
||||
LOG.warn("task znode " + path + " vanished or not created yet.");
|
||||
// ignore since we should not end up in a case where there is in-memory task,
|
||||
// but no znode. The only case is between the time task is created in-memory
|
||||
// and the znode is created. See HBASE-11217.
|
||||
return;
|
||||
}
|
||||
Long retry_count = (Long) ctx;
|
||||
|
@ -1675,7 +1670,7 @@ public class SplitLogManager extends ZooKeeperListener {
|
|||
TerminationStatus(String msg) {
|
||||
statusMsg = msg;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return statusMsg;
|
||||
|
|
Loading…
Reference in New Issue