Modified: 
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java?rev=882248&r1=882247&r2=882248&view=diff
==============================================================================
--- 
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java
 (original)
+++ 
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java
 Thu Nov 19 19:03:47 2009
@@ -71,7 +71,7 @@
        }
 
     protected Map<String, ColumnFamily> readColumnFamily(List<ReadCommand> 
commands, int consistency_level)
-    throws InvalidRequestException, UnavailableException
+    throws InvalidRequestException, UnavailableException, TimedOutException
     {
         // TODO - Support multiple column families per row, right now row only 
contains 1 column family
         String cfName = commands.get(0).getColumnFamilyName();
@@ -96,10 +96,6 @@
         {
             throw new RuntimeException(e);
         }
-        catch (TimeoutException e)
-        {
-            throw new RuntimeException(e);
-        }
 
         for (Row row: rows)
         {
@@ -170,7 +166,7 @@
     }
 
     private Map<String, List<ColumnOrSuperColumn>> getSlice(List<ReadCommand> 
commands, int consistency_level)
-    throws InvalidRequestException, UnavailableException
+    throws InvalidRequestException, UnavailableException, TimedOutException
     {
         Map<String, ColumnFamily> cfamilies = readColumnFamily(commands, 
consistency_level);
         Map<String, List<ColumnOrSuperColumn>> columnFamiliesMap = new 
HashMap<String, List<ColumnOrSuperColumn>>();
@@ -206,7 +202,7 @@
     }
 
     public List<ColumnOrSuperColumn> get_slice(String keyspace, String key, 
ColumnParent column_parent, SlicePredicate predicate, int consistency_level)
-    throws InvalidRequestException, UnavailableException
+    throws InvalidRequestException, UnavailableException, TimedOutException
     {
         if (logger.isDebugEnabled())
             logger.debug("get_slice");
@@ -214,7 +210,7 @@
     }
     
     public Map<String, List<ColumnOrSuperColumn>> multiget_slice(String 
keyspace, List<String> keys, ColumnParent column_parent, SlicePredicate 
predicate, int consistency_level)
-    throws InvalidRequestException, UnavailableException
+    throws InvalidRequestException, UnavailableException, TimedOutException
     {
         if (logger.isDebugEnabled())
             logger.debug("multiget_slice");
@@ -222,7 +218,7 @@
     }
 
     private Map<String, List<ColumnOrSuperColumn>> 
multigetSliceInternal(String keyspace, List<String> keys, ColumnParent 
column_parent, SlicePredicate predicate, int consistency_level)
-    throws InvalidRequestException, UnavailableException
+    throws InvalidRequestException, UnavailableException, TimedOutException
     {
         ThriftValidation.validateColumnParent(keyspace, column_parent);
         List<ReadCommand> commands = new ArrayList<ReadCommand>();
@@ -251,7 +247,7 @@
     }
 
     public ColumnOrSuperColumn get(String table, String key, ColumnPath 
column_path, int consistency_level)
-    throws InvalidRequestException, NotFoundException, UnavailableException
+    throws InvalidRequestException, NotFoundException, UnavailableException, 
TimedOutException
     {
         if (logger.isDebugEnabled())
             logger.debug("get");
@@ -265,7 +261,7 @@
 
     /** no values will be mapped to keys with no data */
     private Map<String, Collection<IColumn>> multigetColumns(List<ReadCommand> 
commands, int consistency_level)
-    throws InvalidRequestException, UnavailableException
+    throws InvalidRequestException, UnavailableException, TimedOutException
     {
         Map<String, ColumnFamily> cfamilies = readColumnFamily(commands, 
consistency_level);
         Map<String, Collection<IColumn>> columnFamiliesMap = new 
HashMap<String, Collection<IColumn>>();
@@ -300,7 +296,7 @@
 
     /** always returns a ColumnOrSuperColumn for each key, even if there is no 
data for it */
     public Map<String, ColumnOrSuperColumn> multiget(String table, 
List<String> keys, ColumnPath column_path, int consistency_level)
-    throws InvalidRequestException, UnavailableException
+    throws InvalidRequestException, UnavailableException, TimedOutException
     {
         if (logger.isDebugEnabled())
             logger.debug("multiget");
@@ -308,7 +304,7 @@
     }
 
     private Map<String, ColumnOrSuperColumn> multigetInternal(String table, 
List<String> keys, ColumnPath column_path, int consistency_level)
-    throws InvalidRequestException, UnavailableException
+    throws InvalidRequestException, UnavailableException, TimedOutException
     {
         ThriftValidation.validateColumnPath(table, column_path);
 
@@ -358,7 +354,7 @@
     }
 
     public int get_count(String table, String key, ColumnParent column_parent, 
int consistency_level)
-    throws InvalidRequestException, UnavailableException
+    throws InvalidRequestException, UnavailableException, TimedOutException
     {
         if (logger.isDebugEnabled())
             logger.debug("get_count");
@@ -366,7 +362,7 @@
     }
 
     private Map<String, Integer> multigetCountInternal(String table, 
List<String> keys, ColumnParent column_parent, int consistency_level)
-    throws InvalidRequestException, UnavailableException
+    throws InvalidRequestException, UnavailableException, TimedOutException
     {
         // validateColumnParent assumes we require simple columns; g_c_c is 
the only
         // one of the columnParent-taking apis that can also work at the SC 
level.
@@ -403,7 +399,7 @@
     }
 
     public void insert(String table, String key, ColumnPath column_path, 
byte[] value, long timestamp, int consistency_level)
-    throws InvalidRequestException, UnavailableException
+    throws InvalidRequestException, UnavailableException, TimedOutException
     {
         if (logger.isDebugEnabled())
             logger.debug("insert");
@@ -423,7 +419,7 @@
     }
 
     public void batch_insert(String keyspace, String key, Map<String, 
List<ColumnOrSuperColumn>> cfmap, int consistency_level)
-    throws InvalidRequestException, UnavailableException
+    throws InvalidRequestException, UnavailableException, TimedOutException
     {
         if (logger.isDebugEnabled())
             logger.debug("batch_insert");
@@ -451,7 +447,7 @@
     }
 
     public void remove(String table, String key, ColumnPath column_path, long 
timestamp, int consistency_level)
-    throws InvalidRequestException, UnavailableException
+    throws InvalidRequestException, UnavailableException, TimedOutException
     {
         if (logger.isDebugEnabled())
             logger.debug("remove");
@@ -464,7 +460,7 @@
         doInsert(consistency_level, rm);
        }
 
-    private void doInsert(int consistency_level, RowMutation rm) throws 
UnavailableException
+    private void doInsert(int consistency_level, RowMutation rm) throws 
UnavailableException, TimedOutException
     {
         if (consistency_level != ConsistencyLevel.ZERO)
         {
@@ -556,7 +552,7 @@
     }
 
     public List<KeySlice> get_range_slice(String keyspace, ColumnParent 
column_parent, SlicePredicate predicate, String start_key, String finish_key, 
int maxRows, int consistency_level)
-    throws InvalidRequestException, UnavailableException, TException
+            throws InvalidRequestException, UnavailableException, TException, 
TimedOutException
     {
         if (logger.isDebugEnabled())
             logger.debug("range_slice");
@@ -610,7 +606,7 @@
     }
 
     public List<String> get_key_range(String tablename, String columnFamily, 
String startWith, String stopAt, int maxResults, int consistency_level)
-    throws InvalidRequestException, TException, UnavailableException
+            throws InvalidRequestException, TException, UnavailableException, 
TimedOutException
     {
         if (logger.isDebugEnabled())
             logger.debug("get_key_range");

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=882248&r1=882247&r2=882248&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
 Thu Nov 19 19:03:47 2009
@@ -166,7 +166,7 @@
         }
     }
     
-    public static void insertBlocking(final RowMutation rm, int 
consistency_level) throws UnavailableException
+    public static void insertBlocking(final RowMutation rm, int 
consistency_level) throws UnavailableException, TimedOutException
     {
         long startTime = System.currentTimeMillis();
         try
@@ -248,7 +248,7 @@
         }
         catch (TimeoutException e)
         {
-            throw new UnavailableException();
+            throw new TimedOutException();
         }
         catch (IOException e)
         {
@@ -289,13 +289,7 @@
             throw new UnsupportedOperationException("invalid consistency level 
" + consistency_level);
         }
         return blockFor;
-    }
-
-    public static void insertBlocking(RowMutation rm) throws 
UnavailableException
-    {
-        insertBlocking(rm, ConsistencyLevel.QUORUM);
-    }
-    
+    }    
 
     /**
      * Read the data from one replica.  If there is no reply, read the data 
from another.  In the event we get
@@ -304,7 +298,7 @@
      * @return the row associated with command.key
      * @throws Exception
      */
-    private static List<Row> weakReadRemote(List<ReadCommand> commands) throws 
IOException, UnavailableException
+    private static List<Row> weakReadRemote(List<ReadCommand> commands) throws 
IOException, UnavailableException, TimedOutException
     {
         if (logger.isDebugEnabled())
             logger.debug("weakreadremote reading " + 
StringUtils.join(commands, ", "));
@@ -333,8 +327,7 @@
             }
             catch (TimeoutException e)
             {
-                throw new RuntimeException("error reading key " + 
commands.get(commandIndex).key, e);
-                // TODO retry to a different endpoint?
+                throw new TimedOutException();
             }
             DataInputBuffer bufIn = new DataInputBuffer();
             bufIn.reset(body, body.length);
@@ -351,7 +344,7 @@
      * a specific set of column names from a given column family.
      */
     public static List<Row> readProtocol(List<ReadCommand> commands, int 
consistency_level)
-    throws IOException, TimeoutException, UnavailableException
+            throws IOException, UnavailableException, TimedOutException
     {
         long startTime = System.currentTimeMillis();
 
@@ -405,7 +398,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, UnavailableException
+    private static List<Row> strongRead(List<ReadCommand> commands, int 
consistency_level) throws IOException, UnavailableException, TimedOutException
     {
         List<QuorumResponseHandler<Row>> quorumResponseHandlers = new 
ArrayList<QuorumResponseHandler<Row>>();
         List<InetAddress[]> commandEndPoints = new ArrayList<InetAddress[]>();
@@ -464,6 +457,10 @@
                 if (logger.isDebugEnabled())
                     logger.debug("quorumResponseHandler: " + 
(System.currentTimeMillis() - startTime2) + " ms.");
             }
+            catch (TimeoutException e)
+            {
+                throw new TimedOutException();
+            }
             catch (DigestMismatchException ex)
             {
                 if (DatabaseDescriptor.getConsistencyCheck())
@@ -481,6 +478,10 @@
                         if (row != null)
                             rows.add(row);
                     }
+                    catch (TimeoutException e)
+                    {
+                        throw new TimedOutException();
+                    }
                     catch (DigestMismatchException e)
                     {
                         // TODO should this be a thrift exception?
@@ -526,7 +527,7 @@
         return rows;
     }
 
-    static Map<String, Collection<IColumn>> getRangeSlice(RangeSliceCommand 
rawCommand) throws IOException, UnavailableException
+    static Map<String, Collection<IColumn>> getRangeSlice(RangeSliceCommand 
rawCommand) throws IOException, UnavailableException, TimedOutException
     {
         long startTime = System.currentTimeMillis();
         TokenMetadata tokenMetadata = 
StorageService.instance().getTokenMetadata();
@@ -551,7 +552,7 @@
             }
             catch (TimeoutException ex)
             {
-                throw new RuntimeException(ex);
+                throw new TimedOutException();
             }
             RangeSliceReply reply = RangeSliceReply.read(responseBody);
             List<Row> rangeRows = new ArrayList<Row>(reply.rows);
@@ -615,7 +616,7 @@
         return results;
     }
 
-    static List<String> getKeyRange(RangeCommand rawCommand) throws 
IOException, UnavailableException
+    static List<String> getKeyRange(RangeCommand rawCommand) throws 
IOException, UnavailableException, TimedOutException
     {
         long startTime = System.currentTimeMillis();
         TokenMetadata tokenMetadata = 
StorageService.instance().getTokenMetadata();
@@ -641,7 +642,7 @@
             }
             catch (TimeoutException e)
             {
-                throw new RuntimeException(e);
+                throw new TimedOutException();
             }
             RangeReply rangeReply = RangeReply.read(responseBody);
             List<String> rangeKeys = rangeReply.keys;


Reply via email to