accumulo-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From els...@apache.org
Subject [2/5] accumulo git commit: ACCUMULO-3853 Addendum -- update queueSize in ZTC and do all updates while holding lock.
Date Wed, 27 May 2015 20:54:47 GMT
ACCUMULO-3853 Addendum -- update queueSize in ZTC and do all updates while holding lock.

Missed an area where the sendQueue was modified in another class.
Also fixed all invocations to only update the size when the lock
is held just to be sure the changes to the queue and size are
made in tandem.


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/b477dbcd
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/b477dbcd
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/b477dbcd

Branch: refs/heads/1.7
Commit: b477dbcdb6fb9f2ae34b31a47d6e105097f2e873
Parents: 7e5145a
Author: Josh Elser <elserj@apache.org>
Authored: Wed May 27 16:25:15 2015 -0400
Committer: Josh Elser <elserj@apache.org>
Committed: Wed May 27 16:49:17 2015 -0400

----------------------------------------------------------------------
 .../main/java/org/apache/accumulo/tracer/AsyncSpanReceiver.java  | 4 ++--
 .../src/main/java/org/apache/accumulo/tracer/ZooTraceClient.java | 1 +
 2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/b477dbcd/server/tracer/src/main/java/org/apache/accumulo/tracer/AsyncSpanReceiver.java
----------------------------------------------------------------------
diff --git a/server/tracer/src/main/java/org/apache/accumulo/tracer/AsyncSpanReceiver.java
b/server/tracer/src/main/java/org/apache/accumulo/tracer/AsyncSpanReceiver.java
index d3a2fc5..28a9088 100644
--- a/server/tracer/src/main/java/org/apache/accumulo/tracer/AsyncSpanReceiver.java
+++ b/server/tracer/src/main/java/org/apache/accumulo/tracer/AsyncSpanReceiver.java
@@ -113,8 +113,8 @@ public abstract class AsyncSpanReceiver<SpanKey,Destination> implements
SpanRece
         synchronized (sendQueue) {
           sendQueue.remove();
           sendQueue.notifyAll();
+          sendQueueSize.decrementAndGet();
         }
-        sendQueueSize.decrementAndGet();
         continue;
       }
       SpanKey dest = getSpanKey(s.data);
@@ -132,8 +132,8 @@ public abstract class AsyncSpanReceiver<SpanKey,Destination> implements
SpanRece
           synchronized (sendQueue) {
             sendQueue.remove();
             sendQueue.notifyAll();
+            sendQueueSize.decrementAndGet();
           }
-          sendQueueSize.decrementAndGet();
           sent = true;
         } catch (Exception ex) {
           log.warn("Got error sending to " + dest + ", refreshing client", ex);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b477dbcd/server/tracer/src/main/java/org/apache/accumulo/tracer/ZooTraceClient.java
----------------------------------------------------------------------
diff --git a/server/tracer/src/main/java/org/apache/accumulo/tracer/ZooTraceClient.java b/server/tracer/src/main/java/org/apache/accumulo/tracer/ZooTraceClient.java
index 3cfc5bc..aa5a9ee 100644
--- a/server/tracer/src/main/java/org/apache/accumulo/tracer/ZooTraceClient.java
+++ b/server/tracer/src/main/java/org/apache/accumulo/tracer/ZooTraceClient.java
@@ -133,6 +133,7 @@ public class ZooTraceClient extends SendSpansViaThrift implements Watcher
{
         synchronized (sendQueue) {
           sendQueue.clear();
           sendQueue.notifyAll();
+          sendQueueSize.set(0);
         }
       }
     } else {


Mime
View raw message