Author: jbellis
Date: Tue Feb 1 12:38:36 2011
New Revision: 1066031
URL: http://svn.apache.org/viewvc?rev=1066031&view=rev
Log:
include digest responses in dynamic snitch latencies
patch by jbellis; reviewed by stuhood for CASSANDRA-2085
Modified:
cassandra/branches/cassandra-0.6/CHANGES.txt
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/MessagingService.java
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/ConsistencyChecker.java
Modified: cassandra/branches/cassandra-0.6/CHANGES.txt
URL:
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/CHANGES.txt?rev=1066031&r1=1066030&r2=1066031&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.6/CHANGES.txt (original)
+++ cassandra/branches/cassandra-0.6/CHANGES.txt Tue Feb 1 12:38:36 2011
@@ -1,3 +1,7 @@
+0.6.12
+ * include digest responses in dynamic snitch latencies (CASSANDRA-2085)
+
+
0.6.11
* reduce garbage generated by MessagingService to prevent load spikes
(CASSANDRA-2058)
Modified:
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/MessagingService.java
URL:
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/MessagingService.java?rev=1066031&r1=1066030&r2=1066031&view=diff
==============================================================================
---
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/MessagingService.java
(original)
+++
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/net/MessagingService.java
Tue Feb 1 12:38:36 2011
@@ -43,6 +43,7 @@ import org.apache.cassandra.io.util.Data
import org.apache.cassandra.locator.ILatencySubscriber;
import org.apache.cassandra.net.io.SerializerType;
import org.apache.cassandra.net.sink.SinkManager;
+import org.apache.cassandra.service.ConsistencyChecker;
import org.apache.cassandra.service.GCInspector;
import org.apache.cassandra.service.QuorumResponseHandler;
import org.apache.cassandra.service.StorageService;
@@ -134,7 +135,9 @@ public class MessagingService
*/
public void maybeAddLatency(IMessageCallback cb, InetAddress address,
double latency)
{
- if (cb instanceof QuorumResponseHandler || cb instanceof AsyncResult)
+ if (cb instanceof QuorumResponseHandler
+ || cb instanceof AsyncResult
+ || cb instanceof ConsistencyChecker.DigestResponseHandler)
addLatency(address, latency);
}
Modified:
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/ConsistencyChecker.java
URL:
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/ConsistencyChecker.java?rev=1066031&r1=1066030&r2=1066031&view=diff
==============================================================================
---
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/ConsistencyChecker.java
(original)
+++
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/ConsistencyChecker.java
Tue Feb 1 12:38:36 2011
@@ -60,7 +60,7 @@ import org.apache.cassandra.utils.Wrappe
* (3) processes full-read responses and invokes resolve. The actual sending
of messages
* repairing out-of-date or missing data is handled by ReadResponseResolver.
*/
-class ConsistencyChecker implements Runnable
+public class ConsistencyChecker implements Runnable
{
private static Logger logger_ =
LoggerFactory.getLogger(ConsistencyChecker.class);
@@ -109,7 +109,7 @@ class ConsistencyChecker implements Runn
return readCommand;
}
- class DigestResponseHandler implements IAsyncCallback
+ public class DigestResponseHandler implements IAsyncCallback
{
private boolean repairInvoked;
private final byte[] dataDigest = ColumnFamily.digest(row_.cf);