HBASE-12785 Use FutureTask to timeout the attempt to get the lock for hbck
This commit is contained in:
parent
e7795007b0
commit
f9873622b8
|
@ -45,8 +45,12 @@ import java.util.concurrent.Callable;
|
||||||
import java.util.concurrent.ConcurrentSkipListMap;
|
import java.util.concurrent.ConcurrentSkipListMap;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
|
import java.util.concurrent.Executors;
|
||||||
import java.util.concurrent.Future;
|
import java.util.concurrent.Future;
|
||||||
|
import java.util.concurrent.FutureTask;
|
||||||
import java.util.concurrent.ScheduledThreadPoolExecutor;
|
import java.util.concurrent.ScheduledThreadPoolExecutor;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.concurrent.TimeoutException;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
|
||||||
|
@ -323,14 +327,9 @@ public class HBaseFsck extends Configured implements Closeable {
|
||||||
this.executor = exec;
|
this.executor = exec;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
private class FileLockCallable implements Callable<FSDataOutputStream> {
|
||||||
* This method maintains a lock using a file. If the creation fails we return null
|
@Override
|
||||||
*
|
public FSDataOutputStream call() throws IOException {
|
||||||
* @return FSDataOutputStream object corresponding to the newly opened lock file
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
private FSDataOutputStream checkAndMarkRunningHbck() throws IOException {
|
|
||||||
long start = EnvironmentEdgeManager.currentTime();
|
|
||||||
try {
|
try {
|
||||||
FileSystem fs = FSUtils.getCurrentFileSystem(getConf());
|
FileSystem fs = FSUtils.getCurrentFileSystem(getConf());
|
||||||
FsPermission defaultPerms = FSUtils.getFilePermissions(fs, getConf(),
|
FsPermission defaultPerms = FSUtils.getFilePermissions(fs, getConf(),
|
||||||
|
@ -348,14 +347,38 @@ public class HBaseFsck extends Configured implements Closeable {
|
||||||
} else {
|
} else {
|
||||||
throw e;
|
throw e;
|
||||||
}
|
}
|
||||||
} finally {
|
}
|
||||||
long duration = EnvironmentEdgeManager.currentTime() - start;
|
}
|
||||||
if (duration > 30000) {
|
}
|
||||||
LOG.warn("Took " + duration + " milliseconds to obtain lock");
|
|
||||||
|
/**
|
||||||
|
* This method maintains a lock using a file. If the creation fails we return null
|
||||||
|
*
|
||||||
|
* @return FSDataOutputStream object corresponding to the newly opened lock file
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
private FSDataOutputStream checkAndMarkRunningHbck() throws IOException {
|
||||||
|
FileLockCallable callable = new FileLockCallable();
|
||||||
|
ExecutorService executor = Executors.newFixedThreadPool(1);
|
||||||
|
FutureTask<FSDataOutputStream> futureTask = new FutureTask<FSDataOutputStream>(callable);
|
||||||
|
executor.execute(futureTask);
|
||||||
|
final int timeoutInSeconds = 30;
|
||||||
|
FSDataOutputStream stream = null;
|
||||||
|
try {
|
||||||
|
stream = futureTask.get(30, TimeUnit.SECONDS);
|
||||||
|
} catch (ExecutionException ee) {
|
||||||
|
LOG.warn("Encountered exception when opening lock file", ee);
|
||||||
|
} catch (InterruptedException ie) {
|
||||||
|
LOG.warn("Interrupted when opening lock file", ie);
|
||||||
|
Thread.currentThread().interrupt();
|
||||||
|
} catch (TimeoutException exception) {
|
||||||
// took too long to obtain lock
|
// took too long to obtain lock
|
||||||
return null;
|
LOG.warn("Took more than " + timeoutInSeconds + " seconds in obtaining lock");
|
||||||
}
|
futureTask.cancel(true);
|
||||||
|
} finally {
|
||||||
|
executor.shutdownNow();
|
||||||
}
|
}
|
||||||
|
return stream;
|
||||||
}
|
}
|
||||||
|
|
||||||
private void unlockHbck() {
|
private void unlockHbck() {
|
||||||
|
|
Loading…
Reference in New Issue