diff --git a/CHANGES.txt b/CHANGES.txt index 439e5f693ef..519888d369a 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -304,7 +304,6 @@ Release 0.91.0 - Unreleased HBASE-4010 HMaster.createTable could be heavily optimized HBASE-3506 Ability to disable, drop and enable tables using regex expression (Joey Echeverria via Ted Yu) - HBASE-4003 Cleanup Calls Conservatively On Timeout (Karthick Sankarachary) HBASE-3516 Coprocessors: add test cases for loading coprocessor jars (Mingjie Lai via garyh) HBASE-4036 Implementing a MultipleColumnPrefixFilter (Anirudh Todi) diff --git a/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java b/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java index fd0bc31bd70..e2be68d4f09 100644 --- a/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java +++ b/src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java @@ -167,11 +167,9 @@ public class HBaseClient { Writable value; // value, null if error IOException error; // exception, null if value boolean done; // true when call is done - long startTime; protected Call(Writable param) { this.param = param; - this.startTime = System.currentTimeMillis(); synchronized (HBaseClient.this) { this.id = counter++; } @@ -203,10 +201,6 @@ public class HBaseClient { this.value = value; callComplete(); } - - public long getStartTime() { - return this.startTime; - } } /** Thread that reads responses and notifies callers. Each connection owns a @@ -580,7 +574,7 @@ public class HBaseClient { // since we expect certain responses to not make it by the specified // {@link ConnectionId#rpcTimeout}. closeException = ste; - cleanupCalls(remoteId.rpcTimeout); + cleanupCalls(); } else { // Since the server did not respond within the default ping interval // time, treat this as a fatal condition and close this connection @@ -641,21 +635,15 @@ public class HBaseClient { /* Cleanup all calls and mark them as done */ private void cleanupCalls() { - cleanupCalls(0); - } - - private synchronized void cleanupCalls(long rpcTimeout) { Iterator> itor = calls.entrySet().iterator() ; while (itor.hasNext()) { Call c = itor.next().getValue(); - if (System.currentTimeMillis() - c.getStartTime() > rpcTimeout) { - c.setException(closeException); // local exception - // Notify the open calls, so they are aware of what just happened - synchronized (c) { - c.notifyAll(); - } - itor.remove(); + c.setException(closeException); // local exception + // Notify the open calls, so they are aware of what just happened + synchronized (c) { + c.notifyAll(); } + itor.remove(); } } }