Author: junrao
Date: Thu Nov 19 22:22:50 2009
New Revision: 882334
URL: http://svn.apache.org/viewvc?rev=882334&view=rev
Log:
RingCache code breaks; patch by junrao, reviewed by jbellis for issue 566
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/client/RingCache.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/client/RingCache.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/client/RingCache.java?rev=882334&r1=882333&r2=882334&view=diff
==============================================================================
---
incubator/cassandra/trunk/src/java/org/apache/cassandra/client/RingCache.java
(original)
+++
incubator/cassandra/trunk/src/java/org/apache/cassandra/client/RingCache.java
Thu Nov 19 22:22:50 2009
@@ -90,16 +90,8 @@
}
TokenMetadata tokenMetadata = new
TokenMetadata(tokenEndpointMap);
- Class cls =
DatabaseDescriptor.getReplicaPlacementStrategyClass();
- Class [] parameterTypes = new Class[] { TokenMetadata.class,
IPartitioner.class, int.class, int.class};
- try
- {
- nodePicker_ = (AbstractReplicationStrategy)
cls.getConstructor(parameterTypes).newInstance(tokenMetadata, partitioner_,
DatabaseDescriptor.getReplicationFactor(), port_);
- }
- catch (Exception e)
- {
- throw new RuntimeException(e);
- }
+ nodePicker_ =
StorageService.getReplicationStrategy(tokenMetadata, partitioner_);
+
break;
}
catch (TException e)
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=882334&r1=882333&r2=882334&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
Thu Nov 19 22:22:50 2009
@@ -232,19 +232,26 @@
MessagingService.instance().registerVerbHandlers(streamInitiateDoneVerbHandler_,
new Streaming.StreamInitiateDoneVerbHandler());
MessagingService.instance().registerVerbHandlers(streamFinishedVerbHandler_,
new Streaming.StreamFinishedVerbHandler());
+ replicationStrategy_ = getReplicationStrategy(tokenMetadata_,
partitioner_);
+ }
+
+ public static AbstractReplicationStrategy
getReplicationStrategy(TokenMetadata tokenMetadata, IPartitioner partitioner)
+ {
+ AbstractReplicationStrategy replicationStrategy = null;
Class<AbstractReplicationStrategy> cls =
DatabaseDescriptor.getReplicaPlacementStrategyClass();
Class [] parameterTypes = new Class[] { TokenMetadata.class,
IPartitioner.class, int.class};
try
{
Constructor<AbstractReplicationStrategy> constructor =
cls.getConstructor(parameterTypes);
- replicationStrategy_ = constructor.newInstance(tokenMetadata_,
partitioner_, DatabaseDescriptor.getReplicationFactor());
+ replicationStrategy = constructor.newInstance(tokenMetadata,
partitioner, DatabaseDescriptor.getReplicationFactor());
}
catch (Exception e)
{
throw new RuntimeException(e);
}
+ return replicationStrategy;
}
-
+
public void start() throws IOException
{
storageMetadata_ = SystemTable.initMetadata();