You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2009/11/03 15:48:19 UTC
svn commit: r832439 - in
/incubator/cassandra/trunk/src/java/org/apache/cassandra: locator/ service/
Author: jbellis
Date: Tue Nov 3 14:48:18 2009
New Revision: 832439
URL: http://svn.apache.org/viewvc?rev=832439&view=rev
Log:
only quorum timeout should be unavailableexception; others should be internal errors. also remove un-thrown InvalidRequest exception from some throws clauses. patch by jbellis
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterShardStategy.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/DatacenterQuorumResponseHandler.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/DatacenterQuorumSyncResponseHandler.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/QuorumResponseHandler.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageProxy.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java?rev=832439&r1=832438&r2=832439&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java Tue Nov 3 14:48:18 2009
@@ -58,7 +58,7 @@
public abstract ArrayList<InetAddress> getNaturalEndpoints(Token token, TokenMetadata metadata);
- public <T> QuorumResponseHandler<T> getResponseHandler(IResponseResolver<T> responseResolver, int blockFor, int consistency_level) throws InvalidRequestException
+ public <T> QuorumResponseHandler<T> getResponseHandler(IResponseResolver<T> responseResolver, int blockFor, int consistency_level)
{
return new QuorumResponseHandler<T>(blockFor, responseResolver);
}
Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterShardStategy.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterShardStategy.java?rev=832439&r1=832438&r2=832439&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterShardStategy.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/locator/DatacenterShardStategy.java Tue Nov 3 14:48:18 2009
@@ -200,7 +200,6 @@
*/
@Override
public <T> QuorumResponseHandler<T> getResponseHandler(IResponseResolver<T> responseResolver, int blockFor, int consistency_level)
- throws InvalidRequestException
{
if (consistency_level == ConsistencyLevel.DCQUORUM)
{
Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/service/DatacenterQuorumResponseHandler.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/DatacenterQuorumResponseHandler.java?rev=832439&r1=832438&r2=832439&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/service/DatacenterQuorumResponseHandler.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/service/DatacenterQuorumResponseHandler.java Tue Nov 3 14:48:18 2009
@@ -24,7 +24,6 @@
private InetAddress localEndpoint;
public DatacenterQuorumResponseHandler(int blockFor, IResponseResolver<T> responseResolver)
- throws InvalidRequestException
{
// Response is been managed by the map so the waitlist size really doesnt matter.
super(blockFor, responseResolver);
Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/service/DatacenterQuorumSyncResponseHandler.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/DatacenterQuorumSyncResponseHandler.java?rev=832439&r1=832438&r2=832439&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/service/DatacenterQuorumSyncResponseHandler.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/service/DatacenterQuorumSyncResponseHandler.java Tue Nov 3 14:48:18 2009
@@ -21,7 +21,6 @@
private final Map<String, Integer> responseCounts;
public DatacenterQuorumSyncResponseHandler(Map<String, Integer> responseCounts, IResponseResolver<T> responseResolver)
- throws InvalidRequestException
{
// Response is been managed by the map so make it 1 for the superclass.
super(1, responseResolver);
Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/service/QuorumResponseHandler.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/QuorumResponseHandler.java?rev=832439&r1=832438&r2=832439&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/service/QuorumResponseHandler.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/service/QuorumResponseHandler.java Tue Nov 3 14:48:18 2009
@@ -44,7 +44,7 @@
private IResponseResolver<T> responseResolver_;
private long startTime_;
- public QuorumResponseHandler(int responseCount, IResponseResolver<T> responseResolver) throws InvalidRequestException
+ public QuorumResponseHandler(int responseCount, IResponseResolver<T> responseResolver)
{
assert 1 <= responseCount && responseCount <= DatabaseDescriptor.getReplicationFactor()
: "invalid response count " + responseCount;
Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageProxy.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageProxy.java?rev=832439&r1=832438&r2=832439&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageProxy.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageProxy.java Tue Nov 3 14:48:18 2009
@@ -178,11 +178,14 @@
}
}
}
- catch (Exception e)
+ catch (TimeoutException e)
{
- logger.error("error writing key " + rm.key(), e);
throw new UnavailableException();
}
+ catch (Exception e)
+ {
+ throw new RuntimeException("error writing key " + rm.key(), e);
+ }
finally
{
writeStats.add(System.currentTimeMillis() - startTime);
@@ -347,7 +350,7 @@
* 7. else carry out read repair by getting data from all the nodes.
// 5. return success
*/
- private static List<Row> strongRead(List<ReadCommand> commands, int consistency_level) throws IOException, TimeoutException, InvalidRequestException, UnavailableException
+ private static List<Row> strongRead(List<ReadCommand> commands, int consistency_level) throws IOException, TimeoutException, UnavailableException
{
List<QuorumResponseHandler<Row>> quorumResponseHandlers = new ArrayList<QuorumResponseHandler<Row>>();
List<InetAddress[]> commandEndPoints = new ArrayList<InetAddress[]>();
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=832439&r1=832438&r2=832439&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 Tue Nov 3 14:48:18 2009
@@ -868,7 +868,7 @@
return tokens;
}
- public <T> QuorumResponseHandler<T> getResponseHandler(IResponseResolver<T> responseResolver, int blockFor, int consistency_level) throws InvalidRequestException, UnavailableException
+ public <T> QuorumResponseHandler<T> getResponseHandler(IResponseResolver<T> responseResolver, int blockFor, int consistency_level)
{
return replicationStrategy_.getResponseHandler(responseResolver, blockFor, consistency_level);
}