cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jbel...@apache.org
Subject svn commit: r771325 - in /incubator/cassandra/trunk/src/java/org/apache/cassandra/service: ConsistencyManager.java StorageService.java
Date Mon, 04 May 2009 14:59:51 GMT
Author: jbellis
Date: Mon May  4 14:59:51 2009
New Revision: 771325

URL: http://svn.apache.org/viewvc?rev=771325&view=rev
Log:
do not leave variables uninitialized in ConsistencyManager constructor.  fixes regression
from #95.  patch by jbellis for CASSANDRA-124

Modified:
    incubator/cassandra/trunk/src/java/org/apache/cassandra/service/ConsistencyManager.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/service/ConsistencyManager.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/ConsistencyManager.java?rev=771325&r1=771324&r2=771325&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/service/ConsistencyManager.java
(original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/service/ConsistencyManager.java
Mon May  4 14:59:51 2009
@@ -145,16 +145,18 @@
 			}
 		}
 	}
+
 	private static long scheduledTimeMillis_ = 600;
 	private static ICachetable<String, String> readRepairTable_ = new Cachetable<String,
String>(scheduledTimeMillis_);
-	private Row row_;
-	protected List<EndPoint> replicas_;
-	
-	private ReadCommand readCommand_;
-	
-    public ConsistencyManager(Row row_, List<EndPoint> replicas_, ReadCommand readCommand)
+	private final Row row_;
+	protected final List<EndPoint> replicas_;
+	private final ReadCommand readCommand_;
+
+    public ConsistencyManager(Row row, List<EndPoint> replicas, ReadCommand readCommand)
     {
-        this.readCommand_ = readCommand;
+        row_ = row;
+        replicas_ = replicas;
+        readCommand_ = readCommand;
     }
 
 	public void run()
@@ -165,7 +167,7 @@
 		{
 			Message messageDigestOnly = readCommandDigestOnly.makeReadMessage();
 			IAsyncCallback digestResponseHandler = new DigestResponseHandler();
-			MessagingService.getMessagingInstance().sendRR(messageDigestOnly, replicas_.toArray(new
EndPoint[0]), digestResponseHandler);
+			MessagingService.getMessagingInstance().sendRR(messageDigestOnly, replicas_.toArray(new
EndPoint[replicas_.size()]), digestResponseHandler);
 		}
 		catch ( IOException ex )
 		{

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java?rev=771325&r1=771324&r2=771325&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java Mon
May  4 14:59:51 2009
@@ -549,9 +549,9 @@
      * sure that the N replicas are in sync. We do this in the
      * background when we do not care much about consistency.
      */
-    public void doConsistencyCheck(Row row, List<EndPoint> endpoints, ReadCommand message)
+    public void doConsistencyCheck(Row row, List<EndPoint> endpoints, ReadCommand command)
     {
-        Runnable consistencySentinel = new ConsistencyManager(row.cloneMe(), endpoints, message);
+        Runnable consistencySentinel = new ConsistencyManager(row.cloneMe(), endpoints, command);
         consistencyManager_.submit(consistencySentinel);
     }
 



Mime
View raw message