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/04/24 19:52:08 UTC

svn commit: r768381 [1/3] - in /incubator/cassandra/trunk: interface/ src/org/apache/cassandra/cli/ src/org/apache/cassandra/db/ src/org/apache/cassandra/service/ test/org/apache/cassandra/db/

Author: jbellis
Date: Fri Apr 24 17:52:07 2009
New Revision: 768381

URL: http://svn.apache.org/viewvc?rev=768381&view=rev
Log:
move CF checking into CassandraServer.  begin to expose InvalidRequestException.  r/m unused "del" cfmaps from batch methods.
patch by jbellis; reviewed by Jun Rao for #66

Added:
    incubator/cassandra/trunk/src/org/apache/cassandra/db/TableNotDefinedException.java
    incubator/cassandra/trunk/src/org/apache/cassandra/service/InvalidRequestException.java
      - copied, changed from r768137, incubator/cassandra/trunk/src/org/apache/cassandra/service/CassandraException.java
    incubator/cassandra/trunk/src/org/apache/cassandra/service/NotFoundException.java
Modified:
    incubator/cassandra/trunk/interface/cassandra.thrift
    incubator/cassandra/trunk/src/org/apache/cassandra/cli/CliClient.java
    incubator/cassandra/trunk/src/org/apache/cassandra/db/ColumnFamilyNotDefinedException.java
    incubator/cassandra/trunk/src/org/apache/cassandra/db/IScanner.java
    incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadCommand.java
    incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadRepairVerbHandler.java
    incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadVerbHandler.java
    incubator/cassandra/trunk/src/org/apache/cassandra/db/RowMutation.java
    incubator/cassandra/trunk/src/org/apache/cassandra/db/RowMutationVerbHandler.java
    incubator/cassandra/trunk/src/org/apache/cassandra/db/Scanner.java
    incubator/cassandra/trunk/src/org/apache/cassandra/db/Table.java
    incubator/cassandra/trunk/src/org/apache/cassandra/service/Cassandra.java
    incubator/cassandra/trunk/src/org/apache/cassandra/service/CassandraException.java
    incubator/cassandra/trunk/src/org/apache/cassandra/service/CassandraServer.java
    incubator/cassandra/trunk/src/org/apache/cassandra/service/CqlResult_t.java
    incubator/cassandra/trunk/src/org/apache/cassandra/service/StorageProxy.java
    incubator/cassandra/trunk/src/org/apache/cassandra/service/batch_mutation_super_t.java
    incubator/cassandra/trunk/src/org/apache/cassandra/service/batch_mutation_t.java
    incubator/cassandra/trunk/src/org/apache/cassandra/service/superColumn_t.java
    incubator/cassandra/trunk/test/org/apache/cassandra/db/ColumnFamilyStoreTest.java

Modified: incubator/cassandra/trunk/interface/cassandra.thrift
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/interface/cassandra.thrift?rev=768381&r1=768380&r2=768381&view=diff
==============================================================================
--- incubator/cassandra/trunk/interface/cassandra.thrift (original)
+++ incubator/cassandra/trunk/interface/cassandra.thrift Fri Apr 24 17:52:07 2009
@@ -1,13 +1,18 @@
-#!/usr/local/bin/thrift --java --php
+#!/usr/local/bin/thrift --java --php --py
 
 #
 # Interface definition for Cassandra Service
 #
 
+namespace java org.apache.cassandra.service
+namespace py org.apache.cassandra
+namespace cpp org.apache.cassandra
+namespace php org.apache.cassandra
 
-namespace java  org.apache.cassandra.service
-namespace cpp   org.apache.cassandra
-php_namespace   cassandra
+
+#
+# structures
+#
 
 struct column_t {
    1: string                        columnName,
@@ -21,7 +26,6 @@
    1: string                        table,
    2: string                        key,
    3: column_family_map             cfmap,
-   4: column_family_map             cfmapdel,
 }
 
 struct superColumn_t {
@@ -35,7 +39,6 @@
    1: string                        table,
    2: string                        key,
    3: superColumn_family_map        cfmap,
-   4: superColumn_family_map        cfmapdel,
 }
 
 
@@ -47,30 +50,71 @@
    3: resultSet_t                   resultSet,
 }
 
+
+#
+# Exceptions
+#
+
+# a specific column was requested that does not exist
+exception NotFoundException {
+}
+
+# invalid request (table / CF does not exist, etc.)
+exception InvalidRequestException {
+    1: string why
+}
+
+# internal server error
 exception CassandraException {
-  1: string error
- }
+    1: string why
+}
+
 
-service Cassandra  {
+#
+# service api
+#
+
+service Cassandra {
+  list<column_t> get_slice(string tablename,string key,string columnFamily_column, i32 start = -1 , i32 count = -1) 
+  throws (1: InvalidRequestException ire, 2: NotFoundException nfe, 3: CassandraException ce),
+  
+  list<column_t> get_slice_by_names(string tablename,string key,string columnFamily, list<string> columnNames) 
+  throws (1: InvalidRequestException ire, 2: NotFoundException nfe, 3: CassandraException ce),
+  
+  column_t       get_column(string tablename,string key,string columnFamily_column) 
+  throws (1: InvalidRequestException ire, 2: NotFoundException nfe, 3: CassandraException ce),
+
+  i32            get_column_count(string tablename,string key,string columnFamily_column) 
+  throws (1: InvalidRequestException ire, 2: CassandraException ce),
 
-  list<column_t> get_slice(string tablename,string key,string columnFamily_column, i32 start = -1 , i32 count = -1) throws (1:CassandraException e),
-  list<column_t> get_slice_by_names(string tablename,string key,string columnFamily, list<string> columnNames) throws (1:CassandraException e),
-  column_t       get_column(string tablename,string key,string columnFamily_column) throws (1:CassandraException e),
-  i32            get_column_count(string tablename,string key,string columnFamily_column) throws (1:CassandraException e),
   async void     insert(string tablename,string key,string columnFamily_column, binary cellData,i64 timestamp),
+
   async void     batch_insert(batch_mutation_t batchMutation),
-  bool           batch_insert_blocking(batch_mutation_t batchMutation) throws (1:CassandraException e),
-  bool           remove(string tablename,string key,string columnFamily_column, i64 timestamp, bool block),
-  list<column_t> get_columns_since(string tablename, string key, string columnFamily_column, i64 timeStamp) throws (1:CassandraException e),
-
-  list<superColumn_t> get_slice_super(string tablename, string key, string columnFamily_superColumnName, i32 start = -1 , i32 count = -1) throws (1:CassandraException e),
-  list<superColumn_t> get_slice_super_by_names(string tablename, string key, string columnFamily, list<string> superColumnNames) throws (1:CassandraException e),
-  superColumn_t       get_superColumn(string tablename,string key,string columnFamily) throws (1:CassandraException e),
-  async void          batch_insert_superColumn(batch_mutation_super_t batchMutationSuper),
-  bool                batch_insert_superColumn_blocking(batch_mutation_super_t batchMutationSuper) throws (1:CassandraException e),
 
-  // Get the specified key into buffercache fData = true means get the data also for the key in buffercache.
-  async void          touch(string key, bool fData),
+  bool           batch_insert_blocking(batch_mutation_t batchMutation)
+  throws (1: InvalidRequestException ire),
+
+  bool           remove(string tablename,string key,string columnFamily_column, i64 timestamp, bool block)
+  throws (1: InvalidRequestException ire),
+
+  list<column_t> get_columns_since(string tablename, string key, string columnFamily_column, i64 timeStamp) 
+  throws (1: InvalidRequestException ire, 2: NotFoundException nfe, 3: CassandraException ce),
+
+  list<superColumn_t> get_slice_super(string tablename, string key, string columnFamily_superColumnName, i32 start = -1 , i32 count = -1) 
+  throws (1: InvalidRequestException ire, 2: CassandraException ce),
+
+  list<superColumn_t> get_slice_super_by_names(string tablename,string key,string columnFamily, list<string> superColumnNames) 
+  throws (1: InvalidRequestException ire, 2: CassandraException ce),
+
+  superColumn_t  get_superColumn(string tablename,string key,string columnFamily) 
+  throws (1: InvalidRequestException ire, 2: NotFoundException nfe, 3: CassandraException ce),
+
+  async void     batch_insert_superColumn(batch_mutation_super_t batchMutationSuper),
+
+  bool           batch_insert_superColumn_blocking(batch_mutation_super_t batchMutationSuper)
+  throws (1: InvalidRequestException ire),
+
+  async void     touch(string key,bool fData),
 
   /////////////////////////////////////////////////////////////////////////////////////
   // The following are beta APIs being introduced for CLI and/or CQL support.        //

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/cli/CliClient.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/cli/CliClient.java?rev=768381&r1=768380&r2=768381&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/cli/CliClient.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/cli/CliClient.java Fri Apr 24 17:52:07 2009
@@ -25,6 +25,8 @@
 import org.apache.cassandra.service.CassandraException;
 import org.apache.cassandra.service.CqlResult_t;
 import org.apache.cassandra.service.column_t;
+import org.apache.cassandra.service.NotFoundException;
+import org.apache.cassandra.service.InvalidRequestException;
 import org.apache.cassandra.service.Cassandra.Client;
 import org.apache.cassandra.utils.LogUtil;
 
@@ -43,7 +45,7 @@
     }
 
     // Execute a CLI Statement 
-    public void executeCLIStmt(String stmt) throws TException 
+    public void executeCLIStmt(String stmt) throws TException, NotFoundException, InvalidRequestException
     {
         CommonTree ast = null;
 
@@ -118,7 +120,7 @@
     }
 
     // Execute GET statement
-    private void executeGet(CommonTree ast) throws TException
+    private void executeGet(CommonTree ast) throws TException, NotFoundException, InvalidRequestException
     {
         if (!CliMain.isConnected())
             return;

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/db/ColumnFamilyNotDefinedException.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/db/ColumnFamilyNotDefinedException.java?rev=768381&r1=768380&r2=768381&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/db/ColumnFamilyNotDefinedException.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/db/ColumnFamilyNotDefinedException.java Fri Apr 24 17:52:07 2009
@@ -18,12 +18,14 @@
 
 package org.apache.cassandra.db;
 
+import org.apache.cassandra.service.InvalidRequestException;
+
 
 /**
  * Author : Avinash Lakshman ( alakshman@facebook.com) & Prashant Malik ( pmalik@facebook.com )
  */
 
-public class ColumnFamilyNotDefinedException extends Exception
+public class ColumnFamilyNotDefinedException extends InvalidRequestException
 {
     public ColumnFamilyNotDefinedException(String message)
     {

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/db/IScanner.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/db/IScanner.java?rev=768381&r1=768380&r2=768381&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/db/IScanner.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/db/IScanner.java Fri Apr 24 17:52:07 2009
@@ -7,5 +7,5 @@
 {
     public boolean hasNext() throws IOException;
     public T next() throws IOException;
-    public void fetch(String key, String cf) throws IOException, ColumnFamilyNotDefinedException;    
+    public void fetch(String key, String cf) throws IOException;    
 }

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadCommand.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadCommand.java?rev=768381&r1=768380&r2=768381&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadCommand.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadCommand.java Fri Apr 24 17:52:07 2009
@@ -126,7 +126,7 @@
         return new ReadCommand(table, key, columnFamilyColumn, start, count, sinceTimestamp, columnNames);
     }
 
-    public Row getRow(Table table) throws IOException, ColumnFamilyNotDefinedException
+    public Row getRow(Table table) throws IOException
     {
         if (!columnNames.isEmpty())
         {

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadRepairVerbHandler.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadRepairVerbHandler.java?rev=768381&r1=768380&r2=768381&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadRepairVerbHandler.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadRepairVerbHandler.java Fri Apr 24 17:52:07 2009
@@ -50,10 +50,6 @@
             RowMutation rm = rmMsg.getRowMutation();
             rm.apply();                                   
         }
-        catch( ColumnFamilyNotDefinedException ex )
-        {
-            logger_.debug(LogUtil.throwableToString(ex));
-        }
         catch ( IOException e )
         {
             logger_.debug(LogUtil.throwableToString(e));            

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadVerbHandler.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadVerbHandler.java?rev=768381&r1=768380&r2=768381&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadVerbHandler.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/db/ReadVerbHandler.java Fri Apr 24 17:52:07 2009
@@ -114,10 +114,6 @@
         {
             logger_.info( LogUtil.throwableToString(ex) );
         }
-        catch ( ColumnFamilyNotDefinedException ex)
-        {
-            logger_.info( LogUtil.throwableToString(ex) );
-        }
     }
     
     private void doReadRepair(Row row, ReadCommand readCommand)

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/db/RowMutation.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/db/RowMutation.java?rev=768381&r1=768380&r2=768381&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/db/RowMutation.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/db/RowMutation.java Fri Apr 24 17:52:07 2009
@@ -102,7 +102,7 @@
         return cf.split(":");
     }
 
-    String table()
+    public String table()
     {
         return table_;
     }
@@ -112,7 +112,12 @@
         return key_;
     }
 
-    void addHints(String hint) throws IOException, ColumnFamilyNotDefinedException
+    public Set<String> columnFamilyNames()
+    {
+        return modifications_.keySet();
+    }
+
+    void addHints(String hint) throws IOException
     {
         String cfName = Table.hints_ + ":" + hint;
         add(cfName, ArrayUtils.EMPTY_BYTE_ARRAY, 0);
@@ -222,7 +227,7 @@
      * This is equivalent to calling commit. Applies the changes to
      * to the table that is obtained by calling Table.open().
     */
-    public void apply() throws IOException, ColumnFamilyNotDefinedException
+    public void apply() throws IOException
     {
         Row row = new Row(key_);
         apply(row);
@@ -231,14 +236,13 @@
     /*
      * Allows RowMutationVerbHandler to optimize by re-using a single Row object.
     */
-    void apply(Row emptyRow) throws IOException, ColumnFamilyNotDefinedException
+    void apply(Row emptyRow) throws IOException
     {
         assert emptyRow.getColumnFamilyMap().size() == 0;
         Table table = Table.open(table_);
         for (String cfName : modifications_.keySet())
         {
-            if (!table.isValidColumnFamily(cfName))
-                throw new ColumnFamilyNotDefinedException("Column Family " + cfName + " has not been defined.");
+            assert table.isValidColumnFamily(cfName);
             emptyRow.addColumnFamily(modifications_.get(cfName));
         }
         table.apply(emptyRow);
@@ -248,14 +252,13 @@
      * This is equivalent to calling commit. Applies the changes to
      * to the table that is obtained by calling Table.open().
     */
-    void load(Row row) throws IOException, ColumnFamilyNotDefinedException, ExecutionException, InterruptedException
+    void load(Row row) throws IOException, ExecutionException, InterruptedException
     {
         Table table = Table.open(table_);
         Set<String> cfNames = modifications_.keySet();
         for (String cfName : cfNames)
         {
-            if (!table.isValidColumnFamily(cfName))
-                throw new ColumnFamilyNotDefinedException("Column Family " + cfName + " has not been defined.");
+            assert table.isValidColumnFamily(cfName);
             row.addColumnFamily(modifications_.get(cfName));
         }
         table.load(row);

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/db/RowMutationVerbHandler.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/db/RowMutationVerbHandler.java?rev=768381&r1=768380&r2=768381&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/db/RowMutationVerbHandler.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/db/RowMutationVerbHandler.java Fri Apr 24 17:52:07 2009
@@ -94,11 +94,6 @@
             logger_.debug("Mutation applied in " + (end - start) + "ms.  Sending response to " +  message.getFrom() + " for key :" + rm.key());
             MessagingService.getMessagingInstance().sendOneWay(responseMessage, message.getFrom());
         }
-        catch(ColumnFamilyNotDefinedException ex)
-        {
-            // TODO shouldn't this be checked before it's sent to us?
-            logger_.warn("column family not defined, and no way to tell the client", ex);
-        }
         catch (IOException e)
         {
             logger_.error("Error in row mutation", e);

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/db/Scanner.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/db/Scanner.java?rev=768381&r1=768380&r2=768381&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/db/Scanner.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/db/Scanner.java Fri Apr 24 17:52:07 2009
@@ -57,9 +57,8 @@
      * @param key key we are interested in.
      * @param cf column family we are interested in.
      * @throws IOException
-     * @throws ColumnFamilyNotDefinedException
      */
-    public void fetch(String key, String cf) throws IOException, ColumnFamilyNotDefinedException
+    public void fetch(String key, String cf) throws IOException
     {        
         if ( cf != null )
         {

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/db/Table.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/db/Table.java?rev=768381&r1=768380&r2=768381&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/db/Table.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/db/Table.java Fri Apr 24 17:52:07 2009
@@ -43,6 +43,7 @@
 import org.apache.cassandra.net.io.IStreamComplete;
 import org.apache.cassandra.net.io.StreamContextManager;
 import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.service.CassandraServer;
 import org.apache.cassandra.utils.BasicUtilities;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.FileUtils;
@@ -655,28 +656,22 @@
     /**
      * Selects the specified column family for the specified key.
     */
-    public ColumnFamily get(String key, String cf) throws ColumnFamilyNotDefinedException, IOException
+    public ColumnFamily get(String key, String cf) throws IOException
     {
         String[] values = RowMutation.getColumnAndColumnFamily(cf);
         long start = System.currentTimeMillis();
         ColumnFamilyStore cfStore = columnFamilyStores_.get(values[0]);
-        if ( cfStore != null )
-        {
-            ColumnFamily columnFamily = cfStore.getColumnFamily(key, cf, new IdentityFilter());
-            long timeTaken = System.currentTimeMillis() - start;
-            dbAnalyticsSource_.updateReadStatistics(timeTaken);
-            return columnFamily;
-        }
-        else
-        {
-            throw new ColumnFamilyNotDefinedException("Column family " + cf + " has not been defined");
-        }
+        assert cfStore != null : "Column family " + cf + " has not been defined";
+        ColumnFamily columnFamily = cfStore.getColumnFamily(key, cf, new IdentityFilter());
+        long timeTaken = System.currentTimeMillis() - start;
+        dbAnalyticsSource_.updateReadStatistics(timeTaken);
+        return columnFamily;
     }
 
     /**
      * Selects only the specified column family for the specified key.
     */
-    public Row getRow(String key, String cf) throws ColumnFamilyNotDefinedException, IOException
+    public Row getRow(String key, String cf) throws IOException
     {
         Row row = new Row(key);
         ColumnFamily columnFamily = get(key, cf);
@@ -688,54 +683,46 @@
     /**
      * Selects only the specified column family for the specified key.
     */
-    public Row getRow(String key, String cf, int start, int count) throws ColumnFamilyNotDefinedException, IOException
+    public Row getRow(String key, String cf, int start, int count) throws IOException
     {
         Row row = new Row(key);
         String[] values = RowMutation.getColumnAndColumnFamily(cf);
         ColumnFamilyStore cfStore = columnFamilyStores_.get(values[0]);
         long start1 = System.currentTimeMillis();
-        if ( cfStore != null )
+        assert cfStore != null : "Column family " + cf + " has not been defined";
+        ColumnFamily columnFamily = cfStore.getColumnFamily(key, cf, new IdentityFilter());
+        if ( columnFamily != null )
         {
-            ColumnFamily columnFamily = cfStore.getColumnFamily(key, cf, new IdentityFilter());
-            if ( columnFamily != null )
+
+            ColumnFamily filteredCf = null;
+            if ((count <=0 || count == Integer.MAX_VALUE) && start <= 0) //Don't need to filter
             {
-                
-                ColumnFamily filteredCf = null;
-                if ((count <=0 || count == Integer.MAX_VALUE) && start <= 0) //Don't need to filter
-                {
-                    filteredCf = columnFamily;
-                }
-                else 
-                {
-                    filteredCf = new CountFilter(count, start).filter(cf, columnFamily);                    
-                }
-                row.addColumnFamily(filteredCf);
+                filteredCf = columnFamily;
+            }
+            else
+            {
+                filteredCf = new CountFilter(count, start).filter(cf, columnFamily);
             }
-            long timeTaken = System.currentTimeMillis() - start1;
-            dbAnalyticsSource_.updateReadStatistics(timeTaken);
-            return row;
+            row.addColumnFamily(filteredCf);
         }
-        else
-            throw new ColumnFamilyNotDefinedException("Column family " + cf + " has not been defined");
+        long timeTaken = System.currentTimeMillis() - start1;
+        dbAnalyticsSource_.updateReadStatistics(timeTaken);
+        return row;
     }
     
-    public Row getRow(String key, String cf, long sinceTimeStamp) throws ColumnFamilyNotDefinedException, IOException
+    public Row getRow(String key, String cf, long sinceTimeStamp) throws IOException
     {
         Row row = new Row(key);
         String[] values = RowMutation.getColumnAndColumnFamily(cf);
         ColumnFamilyStore cfStore = columnFamilyStores_.get(values[0]);
         long start1 = System.currentTimeMillis();
-        if ( cfStore != null )
-        {
-            ColumnFamily columnFamily = cfStore.getColumnFamily(key, cf, new TimeFilter(sinceTimeStamp));
-            if ( columnFamily != null )
-                row.addColumnFamily(columnFamily);
-            long timeTaken = System.currentTimeMillis() - start1;
-            dbAnalyticsSource_.updateReadStatistics(timeTaken);
-            return row;
-        }
-        else
-            throw new ColumnFamilyNotDefinedException("Column family " + cf + " has not been defined");
+        assert cfStore != null : "Column family " + cf + " has not been defined";
+        ColumnFamily columnFamily = cfStore.getColumnFamily(key, cf, new TimeFilter(sinceTimeStamp));
+        if ( columnFamily != null )
+            row.addColumnFamily(columnFamily);
+        long timeTaken = System.currentTimeMillis() - start1;
+        dbAnalyticsSource_.updateReadStatistics(timeTaken);
+        return row;
     }
 
     /**

Added: incubator/cassandra/trunk/src/org/apache/cassandra/db/TableNotDefinedException.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/db/TableNotDefinedException.java?rev=768381&view=auto
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/db/TableNotDefinedException.java (added)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/db/TableNotDefinedException.java Fri Apr 24 17:52:07 2009
@@ -0,0 +1,11 @@
+package org.apache.cassandra.db;
+
+import org.apache.cassandra.service.InvalidRequestException;
+
+public class TableNotDefinedException extends InvalidRequestException
+{
+    public TableNotDefinedException(String why)
+    {
+        super(why);
+    }
+}