|
@@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
import org.apache.hadoop.ipc.RemoteException;
|
|
|
import org.apache.hadoop.util.StopWatch;
|
|
|
-import org.apache.hadoop.util.Time;
|
|
|
+import org.apache.hadoop.util.Timer;
|
|
|
|
|
|
import com.google.common.base.Joiner;
|
|
|
import com.google.common.base.Preconditions;
|
|
@@ -35,6 +35,7 @@ import com.google.common.util.concurrent.ListenableFuture;
|
|
|
import com.google.protobuf.Message;
|
|
|
import com.google.protobuf.TextFormat;
|
|
|
|
|
|
+
|
|
|
/**
|
|
|
* Represents a set of calls for which a quorum of results is needed.
|
|
|
* @param <KEY> a key used to identify each of the outgoing calls
|
|
@@ -60,11 +61,12 @@ class QuorumCall<KEY, RESULT> {
|
|
|
* fraction of the configured timeout for any call.
|
|
|
*/
|
|
|
private static final float WAIT_PROGRESS_WARN_THRESHOLD = 0.7f;
|
|
|
- private final StopWatch quorumStopWatch = new StopWatch();
|
|
|
+ private final StopWatch quorumStopWatch;
|
|
|
+ private final Timer timer;
|
|
|
|
|
|
static <KEY, RESULT> QuorumCall<KEY, RESULT> create(
|
|
|
- Map<KEY, ? extends ListenableFuture<RESULT>> calls) {
|
|
|
- final QuorumCall<KEY, RESULT> qr = new QuorumCall<KEY, RESULT>();
|
|
|
+ Map<KEY, ? extends ListenableFuture<RESULT>> calls, Timer timer) {
|
|
|
+ final QuorumCall<KEY, RESULT> qr = new QuorumCall<KEY, RESULT>(timer);
|
|
|
for (final Entry<KEY, ? extends ListenableFuture<RESULT>> e : calls.entrySet()) {
|
|
|
Preconditions.checkArgument(e.getValue() != null,
|
|
|
"null future for key: " + e.getKey());
|
|
@@ -82,18 +84,53 @@ class QuorumCall<KEY, RESULT> {
|
|
|
}
|
|
|
return qr;
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
+ static <KEY, RESULT> QuorumCall<KEY, RESULT> create(
|
|
|
+ Map<KEY, ? extends ListenableFuture<RESULT>> calls) {
|
|
|
+ return create(calls, new Timer());
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Not intended for outside use.
|
|
|
+ */
|
|
|
private QuorumCall() {
|
|
|
+ this(new Timer());
|
|
|
+ }
|
|
|
+
|
|
|
+ private QuorumCall(Timer timer) {
|
|
|
// Only instantiated from factory method above
|
|
|
+ this.timer = timer;
|
|
|
+ this.quorumStopWatch = new StopWatch(timer);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Used in conjunction with {@link #getQuorumTimeoutIncreaseMillis(long, int)}
|
|
|
+ * to check for pauses.
|
|
|
+ */
|
|
|
private void restartQuorumStopWatch() {
|
|
|
quorumStopWatch.reset().start();
|
|
|
}
|
|
|
|
|
|
- private boolean shouldIncreaseQuorumTimeout(long offset, int millis) {
|
|
|
+ /**
|
|
|
+ * Check for a pause (e.g. GC) since the last time
|
|
|
+ * {@link #restartQuorumStopWatch()} was called. If detected, return the
|
|
|
+ * length of the pause; else, -1.
|
|
|
+ * @param offset Offset the elapsed time by this amount; use if some amount
|
|
|
+ * of pause was expected
|
|
|
+ * @param millis Total length of timeout in milliseconds
|
|
|
+ * @return Length of pause, if detected, else -1
|
|
|
+ */
|
|
|
+ private long getQuorumTimeoutIncreaseMillis(long offset, int millis) {
|
|
|
long elapsed = quorumStopWatch.now(TimeUnit.MILLISECONDS);
|
|
|
- return elapsed + offset > (millis * WAIT_PROGRESS_INFO_THRESHOLD);
|
|
|
+ long pauseTime = elapsed + offset;
|
|
|
+ if (pauseTime > (millis * WAIT_PROGRESS_INFO_THRESHOLD)) {
|
|
|
+ QuorumJournalManager.LOG.info("Pause detected while waiting for " +
|
|
|
+ "QuorumCall response; increasing timeout threshold by pause time " +
|
|
|
+ "of " + pauseTime + " ms.");
|
|
|
+ return pauseTime;
|
|
|
+ } else {
|
|
|
+ return -1;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
|
|
@@ -119,7 +156,7 @@ class QuorumCall<KEY, RESULT> {
|
|
|
int minResponses, int minSuccesses, int maxExceptions,
|
|
|
int millis, String operationName)
|
|
|
throws InterruptedException, TimeoutException {
|
|
|
- long st = Time.monotonicNow();
|
|
|
+ long st = timer.monotonicNow();
|
|
|
long nextLogTime = st + (long)(millis * WAIT_PROGRESS_INFO_THRESHOLD);
|
|
|
long et = st + millis;
|
|
|
while (true) {
|
|
@@ -128,7 +165,7 @@ class QuorumCall<KEY, RESULT> {
|
|
|
if (minResponses > 0 && countResponses() >= minResponses) return;
|
|
|
if (minSuccesses > 0 && countSuccesses() >= minSuccesses) return;
|
|
|
if (maxExceptions >= 0 && countExceptions() > maxExceptions) return;
|
|
|
- long now = Time.monotonicNow();
|
|
|
+ long now = timer.monotonicNow();
|
|
|
|
|
|
if (now > nextLogTime) {
|
|
|
long waited = now - st;
|
|
@@ -154,8 +191,9 @@ class QuorumCall<KEY, RESULT> {
|
|
|
long rem = et - now;
|
|
|
if (rem <= 0) {
|
|
|
// Increase timeout if a full GC occurred after restarting stopWatch
|
|
|
- if (shouldIncreaseQuorumTimeout(0, millis)) {
|
|
|
- et = et + millis;
|
|
|
+ long timeoutIncrease = getQuorumTimeoutIncreaseMillis(0, millis);
|
|
|
+ if (timeoutIncrease > 0) {
|
|
|
+ et += timeoutIncrease;
|
|
|
} else {
|
|
|
throw new TimeoutException();
|
|
|
}
|
|
@@ -165,8 +203,9 @@ class QuorumCall<KEY, RESULT> {
|
|
|
rem = Math.max(rem, 1);
|
|
|
wait(rem);
|
|
|
// Increase timeout if a full GC occurred after restarting stopWatch
|
|
|
- if (shouldIncreaseQuorumTimeout(-rem, millis)) {
|
|
|
- et = et + millis;
|
|
|
+ long timeoutIncrease = getQuorumTimeoutIncreaseMillis(-rem, millis);
|
|
|
+ if (timeoutIncrease > 0) {
|
|
|
+ et += timeoutIncrease;
|
|
|
}
|
|
|
}
|
|
|
}
|