Author: jbellis
Date: Wed Mar 31 19:25:22 2010
New Revision: 929671

URL: http://svn.apache.org/viewvc?rev=929671&view=rev
Log:
centralize calls to DatabaseDescriptor.getConsistencyCheck() in StorageProxy.  
patch by jbellis

Modified:
    
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/ReadVerbHandler.java
    
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/StorageProxy.java

Modified: 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/ReadVerbHandler.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/ReadVerbHandler.java?rev=929671&r1=929670&r2=929671&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/ReadVerbHandler.java
 (original)
+++ 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/db/ReadVerbHandler.java
 Wed Mar 31 19:25:22 2010
@@ -99,7 +99,7 @@ public class ReadVerbHandler implements 
                 List<InetAddress> endpoints = 
StorageService.instance.getLiveNaturalEndpoints(command.table, command.key);
                 /* Remove the local storage endpoint from the list. */
                 endpoints.remove(FBUtilities.getLocalAddress());
-                if (endpoints.size() > 0 && 
DatabaseDescriptor.getConsistencyCheck())
+                if (endpoints.size() > 0)
                     StorageService.instance.doConsistencyCheck(row, endpoints, 
command);
             }
         }

Modified: 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/StorageProxy.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/StorageProxy.java?rev=929671&r1=929670&r2=929671&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/StorageProxy.java
 (original)
+++ 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/service/StorageProxy.java
 Wed Mar 31 19:25:22 2010
@@ -356,7 +356,8 @@ public class StorageProxy implements Sto
 
             if (logger.isDebugEnabled())
                 logger.debug("weakreadremote reading " + command + " from " + 
message.getMessageId() + "@" + endPoint);
-            message.setHeader(ReadCommand.DO_REPAIR, 
ReadCommand.DO_REPAIR.getBytes());
+            if (DatabaseDescriptor.getConsistencyCheck())
+                message.setHeader(ReadCommand.DO_REPAIR, 
ReadCommand.DO_REPAIR.getBytes());
             iars.add(MessagingService.instance.sendRR(message, endPoint));
         }
 
@@ -752,19 +753,21 @@ public class StorageProxy implements Sto
 
         public Object call() throws IOException
         {
-            List<InetAddress> endpoints = 
StorageService.instance.getLiveNaturalEndpoints(command.table, command.key);
-            /* Remove the local storage endpoint from the list. */
-            endpoints.remove(FBUtilities.getLocalAddress());
-
             if (logger.isDebugEnabled())
                 logger.debug("weakreadlocal reading " + command);
 
             Table table = Table.open(command.table);
             Row row = command.getRow(table);
 
-            // Do the consistency checks in the background and return the non 
NULL row
-            if (endpoints.size() > 0 && 
DatabaseDescriptor.getConsistencyCheck())
-                StorageService.instance.doConsistencyCheck(row, endpoints, 
command);
+            // Do the consistency checks in the background
+            if (DatabaseDescriptor.getConsistencyCheck())
+            {
+                List<InetAddress> endpoints = 
StorageService.instance.getLiveNaturalEndpoints(command.table, command.key);
+                /* Remove the local storage endpoint from the list. */
+                endpoints.remove(FBUtilities.getLocalAddress());
+                if (endpoints.size() > 0)
+                    StorageService.instance.doConsistencyCheck(row, endpoints, 
command);
+            }
 
             return row;
         }


Reply via email to