diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml index e7cc7fd..ff62e6f 100644 --- a/hbase-common/src/main/resources/hbase-default.xml +++ b/hbase-common/src/main/resources/hbase-default.xml @@ -1028,4 +1028,21 @@ possible configurations would overwhelm and obscure the important. hbase.server.compactchecker.interval.multiplier multiplied by hbase.server.thread.wakefrequency. + + hbase.lease.recovery.timeout + 90000 + + How long we wait on dfs lease recovery in total before giving up. + + + + hbase.lease.recovery.dfs.socket.timeout + 61000 + + How long between dfs recover lease invocations. Should be just larger than how long + it takes the namenode to timeout trying to reach a datanode; usually + dfs.socket.timeout. If HBase asked hdfs its cluster configs, we would not need + this config. See the end of HBASE-8389 for more. + + diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java index ba45de3..82a1a18 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java @@ -46,32 +46,65 @@ public class FSHDFSUtils extends FSUtils { */ @Override public void recoverFileLease(final FileSystem fs, final Path p, - Configuration conf, CancelableProgressable reporter) throws IOException { + Configuration conf, CancelableProgressable reporter) + throws IOException { // lease recovery not needed for local file system case. - if (!(fs instanceof DistributedFileSystem)) { - return; - } - DistributedFileSystem dfs = (DistributedFileSystem) fs; + if (!(fs instanceof DistributedFileSystem)) return; + recoverDFSFileLease((DistributedFileSystem)fs, p, conf, reporter); + } - LOG.info("Recovering file " + p); + /* + * Run the dfs recover lease. recoverLease is asynchronous. It returns: + * -false when it starts the lease recovery (i.e. lease recovery not *yet* done) + * - true when the lease recovery has succeeded or the file is closed. + * But, we have to be careful. Each time we call recoverLease, it starts the recover lease + * process over from the beginning. We could put ourselves in a situation where we are + * doing nothing but starting a recovery, interrupting it to start again, and so on. + * The findings over in HBASE-8354 have it that the namenode will try to recover the lease + * on the file's primary node. If all is well, it should return near immediately. But, + * as is common, it is the very primary node that has crashed and so the namenode will be + * stuck waiting on a socket timeout before it will ask another datanode to start the + * recovery. It does not help if we call recoverLease in the meantime and in particular, + * subsequent to the socket timeout, a recoverLease invocation will cause us to start + * over from square one (possibly waiting on socket timeout against primary node). So, + * in the below, we do the following: + * 1. Call recoverLease. + * 2. If it returns true, break. + * 3. If it returns false, wait a few seconds and then call it again. + * 4. If it returns true, break. + * 5. If it returns false, wait for what we think the datanode socket timeout is + * (configurable) and then try again. + * 6. If it returns true, break. + * 7. If it returns false, repeat starting at step 5. above. + * + * If HDFS-4525 is available, call it every second and we might be able to exit early. + */ + void recoverDFSFileLease(final DistributedFileSystem dfs, final Path p, final Configuration conf, + final CancelableProgressable reporter) + throws IOException { + LOG.info("Recovering dfs file " + p); long startWaiting = EnvironmentEdgeManager.currentTimeMillis(); // Default is 15 minutes. It's huge, but the idea is that if we have a major issue, HDFS - // usually needs 10 minutes before marking the nodes as dead. So we're putting ourselves - // beyond that limit 'to be safe'. + // usually needs 10 minutes before marking the nodes as dead. So we're putting ourselves + // beyond that limit 'to be safe'. long recoveryTimeout = conf.getInt("hbase.lease.recovery.timeout", 900000) + startWaiting; boolean recovered = false; + // This setting should be what the cluster dfs heartbeat is set to. + long firstPause = conf.getInt("hbase.lease.recovery.first.pause", 3000); + // This should be set to how long it'll take for us to timeout against primary datanode if it + // is dead. We set it to 61 seconds, 1 second than the default READ_TIMEOUT in HDFS, the + // default value for DFS_CLIENT_SOCKET_TIMEOUT_KEY. + long subsequentPause = conf.getInt("hbase.lease.recovery.dfs.socket.timeout", 61 * 1000); int nbAttempt = 0; while (!recovered) { nbAttempt++; try { - // recoverLease is asynchronous. We expect it to return true at the first call if the - // file is closed. So, it returns: - // - false when it starts the lease recovery (i.e. lease recovery not *yet* done - // - true when the lease recovery has succeeded or the file is closed. recovered = dfs.recoverLease(p); LOG.info("Attempt " + nbAttempt + " to recoverLease on file " + p + - " returned " + recovered + ", trying for " + - (EnvironmentEdgeManager.currentTimeMillis() - startWaiting) + "ms"); + " returned " + recovered + " after " + + (EnvironmentEdgeManager.currentTimeMillis() - startWaiting) + "ms"); + // Break the loop if successfully recovered. + if (recovered) break; } catch (IOException e) { if (e instanceof LeaseExpiredException && e.getMessage().contains("File does not exist")) { // This exception comes out instead of FNFE, fix it @@ -82,24 +115,26 @@ public class FSHDFSUtils extends FSUtils { LOG.warn("Got IOException on attempt " + nbAttempt + " to recover lease for file " + p + ", retrying.", e); } - if (!recovered) { - if (reporter != null && !reporter.progress()) { - throw new InterruptedIOException("Operation is cancelled"); - } - // try at least twice. - if (nbAttempt > 2 && recoveryTimeout < EnvironmentEdgeManager.currentTimeMillis()) { - LOG.error("Can't recoverLease after " + nbAttempt + " attempts and " + + if (reporter != null && !reporter.progress()) { + throw new InterruptedIOException("Operation is cancelled"); + } + try { + if (nbAttempt == 1) { + Thread.sleep(firstPause); + } else { + if (recoveryTimeout < EnvironmentEdgeManager.currentTimeMillis()) { + LOG.error("Can't recoverLease after " + nbAttempt + " attempts and " + (EnvironmentEdgeManager.currentTimeMillis() - startWaiting) + "ms " + " for " + p + " - continuing without the lease, but we could have a data loss."); - } else { - try { - Thread.sleep(nbAttempt < 3 ? 500 : 1000); - } catch (InterruptedException ie) { - InterruptedIOException iioe = new InterruptedIOException(); - iioe.initCause(ie); - throw iioe; + break; } + // TODO: Add in isFileClosed tickling. + Thread.sleep(subsequentPause); } + } catch (InterruptedException ie) { + InterruptedIOException iioe = new InterruptedIOException(); + iioe.initCause(ie); + throw iioe; } } }