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/26 00:16:49 UTC

svn commit: r884332 - in /incubator/cassandra/trunk: src/java/org/apache/cassandra/db/ src/java/org/apache/cassandra/service/ src/java/org/apache/cassandra/tools/ test/unit/org/apache/cassandra/db/

Author: jbellis
Date: Wed Nov 25 23:16:48 2009
New Revision: 884332

URL: http://svn.apache.org/viewvc?rev=884332&view=rev
Log:
make Row contain a single, final CF reference
patch by jbellis; tested by Dan Di Spaltro for CASSANDRA-578

Removed:
    incubator/cassandra/trunk/src/java/org/apache/cassandra/tools/KeyChecker.java
Modified:
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamily.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/HintedHandOffManager.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Row.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/RowMutation.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Table.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/service/ReadResponseResolver.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageProxy.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ReadMessageTest.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RowTest.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/TableTest.java

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamily.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamily.java?rev=884332&r1=884331&r2=884332&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamily.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamily.java Wed Nov 25 23:16:48 2009
@@ -262,7 +262,7 @@
      * This function will calculate the difference between 2 column families.
      * The external input is assumed to be a superset of internal.
      */
-    ColumnFamily diff(ColumnFamily cfComposite)
+    public ColumnFamily diff(ColumnFamily cfComposite)
     {
     	ColumnFamily cfDiff = new ColumnFamily(cfComposite.name(), cfComposite.type_, getComparator(), getSubComparator());
         if (cfComposite.getMarkedForDeleteAt() > getMarkedForDeleteAt())
@@ -390,30 +390,16 @@
         return subtotal;
     }
 
-    /** merge all columnFamilies into a single instance, with only the newest versions of columns preserved. */
-    static ColumnFamily resolve(List<ColumnFamily> columnFamilies)
-    {
-        int size = columnFamilies.size();
-        if (size == 0)
-            return null;
-
-        // start from nothing so that we don't include potential deleted columns from the first instance
-        ColumnFamily cf0 = columnFamilies.get(0);
-        ColumnFamily cf = cf0.cloneMeShallow();
-
-        // merge
-        for (ColumnFamily cf2 : columnFamilies)
-        {
-            assert cf.name().equals(cf2.name());
-            cf.addAll(cf2);
-        }
-        return cf;
-    }
-
     public static AbstractType getComparatorFor(String table, String columnFamilyName, byte[] superColumnName)
     {
         return superColumnName == null
                ? DatabaseDescriptor.getComparator(table, columnFamilyName)
                : DatabaseDescriptor.getSubComparator(table, columnFamilyName);
     }
+
+    public void resolve(ColumnFamily cf)
+    {
+        delete(cf);
+        addAll(cf);
+    }
 }

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=884332&r1=884331&r2=884332&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Wed Nov 25 23:16:48 2009
@@ -1464,10 +1464,8 @@
             columnNameSet.addAll(columnNames);
         for (String key : rr.keys)
         {
-            Row row = new Row(key);
             QueryFilter filter = sliceRange == null ? new NamesQueryFilter(key, queryPath, columnNameSet) : new SliceQueryFilter(key, queryPath, sliceRange.start, sliceRange.finish, sliceRange.reversed, sliceRange.count);
-            row.addColumnFamily(getColumnFamily(filter));
-            rows.add(row);
+            rows.add(new Row(key, getColumnFamily(filter)));
         }
 
         return new RangeSliceReply(rows, rr.rangeCompletedLocally);

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/HintedHandOffManager.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/HintedHandOffManager.java?rev=884332&r1=884331&r2=884332&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/HintedHandOffManager.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/HintedHandOffManager.java Wed Nov 25 23:16:48 2009
@@ -109,8 +109,11 @@
         }
 
         Table table = Table.open(tableName);
-        Row row = table.get(key);
-        RowMutation rm = new RowMutation(tableName, row);
+        RowMutation rm = new RowMutation(tableName, key);
+        for (ColumnFamilyStore cfstore : table.getColumnFamilyStores().values())
+        {
+            rm.add(cfstore.getColumnFamily(new IdentityQueryFilter(key, new QueryPath(cfstore.getColumnFamilyName()))));
+        }
         Message message = rm.makeRowMutationMessage();
         WriteResponseHandler responseHandler = new WriteResponseHandler(1);
         MessagingService.instance().sendRR(message, new InetAddress[] { endPoint }, responseHandler);

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Row.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Row.java?rev=884332&r1=884331&r2=884332&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Row.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Row.java Wed Nov 25 23:16:48 2009
@@ -22,18 +22,12 @@
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.Arrays;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
 
-import org.apache.commons.lang.StringUtils;
 import org.apache.log4j.Logger;
 
 import org.apache.cassandra.io.ICompactSerializer;
-import org.apache.cassandra.io.DataOutputBuffer;
 
 public class Row
 {
@@ -45,97 +39,23 @@
         return serializer;
     }
 
-    public Row(String key)
-    {
-        this.key_ = key;
-    }
-
-    private String key_;
-
-    private Map<String, ColumnFamily> columnFamilies_ = new HashMap<String, ColumnFamily>();
-
-    public String key()
-    {
-        return key_;
-    }
-
-    public Collection<ColumnFamily> getColumnFamilies()
-    {
-        return columnFamilies_.values();
-    }
+    public final String key;
+    public final ColumnFamily cf;
 
-    public ColumnFamily getColumnFamily(String cfName)
+    public Row(String key, ColumnFamily cf)
     {
-        return columnFamilies_.get(cfName);
+        this.key = key;
+        this.cf = cf;
     }
 
-    void addColumnFamily(ColumnFamily columnFamily)
-    {
-        columnFamilies_.put(columnFamily.name(), columnFamily);
-    }
-
-    public boolean isEmpty()
-    {
-        return (columnFamilies_.size() == 0);
-    }
-
-    /*
+    /**
      * This function will repair the current row with the input row
      * what that means is that if there are any differences between the 2 rows then
      * this function will make the current row take the latest changes.
      */
-    public void repair(Row rowOther)
+    public void resolve(Row other)
     {
-        for (ColumnFamily cfOld : rowOther.getColumnFamilies())
-        {
-            ColumnFamily cf = columnFamilies_.get(cfOld.name());
-            if (cf == null)
-            {
-                addColumnFamily(cfOld);
-            }
-            else
-            {
-                columnFamilies_.remove(cf.name());
-                addColumnFamily(ColumnFamily.resolve(Arrays.asList(cfOld, cf)));
-            }
-        }
-    }
-
-    /*
-     * This function will calculate the difference between 2 rows
-     * and return the resultant row. This assumes that the row that
-     * is being submitted is a super set of the current row so
-     * it only calculates additional
-     * difference and does not take care of what needs to be removed from the current row to make
-     * it same as the input row.
-     */
-    public Row diff(Row rowComposite)
-    {
-        Row rowDiff = new Row(key_);
-
-        for (ColumnFamily cfComposite : rowComposite.getColumnFamilies())
-        {
-            ColumnFamily cf = columnFamilies_.get(cfComposite.name());
-            if (cf == null)
-                rowDiff.addColumnFamily(cfComposite);
-            else
-            {
-                ColumnFamily cfDiff = cf.diff(cfComposite);
-                if (cfDiff != null)
-                    rowDiff.addColumnFamily(cfDiff);
-            }
-        }
-        if (rowDiff.getColumnFamilies().isEmpty())
-            return null;
-        else
-            return rowDiff;
-    }
-
-    public Row cloneMe()
-    {
-        Row row = new Row(key_);
-        row.columnFamilies_ = new HashMap<String, ColumnFamily>(columnFamilies_);
-        return row;
+        cf.resolve(other.cf);
     }
 
     public byte[] digest()
@@ -149,18 +69,18 @@
         {
             throw new AssertionError(e);
         }
-
-        for (String cFamily : columnFamilies_.keySet())
-        {
-            columnFamilies_.get(cFamily).updateDigest(digest);
-        }
+        cf.updateDigest(digest);
 
         return digest.digest();
     }
 
+    @Override
     public String toString()
     {
-        return "Row(" + key_ + " [" + StringUtils.join(columnFamilies_.values(), ", ") + "])";
+        return "Row(" +
+               "key='" + key + '\'' +
+               ", cf=" + cf +
+               ')';
     }
 }
 
@@ -168,28 +88,12 @@
 {
     public void serialize(Row row, DataOutputStream dos) throws IOException
     {
-        dos.writeUTF(row.key());
-        Collection<ColumnFamily> columnFamilies = row.getColumnFamilies();
-        int size = columnFamilies.size();
-        dos.writeInt(size);
-
-        for (ColumnFamily cf : columnFamilies)
-        {
-            ColumnFamily.serializer().serialize(cf, dos);
-        }
+        dos.writeUTF(row.key);
+        ColumnFamily.serializer().serialize(row.cf, dos);
     }
 
     public Row deserialize(DataInputStream dis) throws IOException
     {
-        String key = dis.readUTF();
-        Row row = new Row(key);
-        int size = dis.readInt();
-
-        for (int i = 0; i < size; ++i)
-        {
-            ColumnFamily cf = ColumnFamily.serializer().deserialize(dis);
-            row.addColumnFamily(cf);
-        }
-        return row;
+        return new Row(dis.readUTF(), ColumnFamily.serializer().deserialize(dis));
     }
 }

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/RowMutation.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/RowMutation.java?rev=884332&r1=884331&r2=884332&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/RowMutation.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/RowMutation.java Wed Nov 25 23:16:48 2009
@@ -71,11 +71,8 @@
     public RowMutation(String table, Row row)
     {
         table_ = table;
-        key_ = row.key();
-        for (ColumnFamily cf : row.getColumnFamilies())
-        {
-            add(cf);
-        }
+        key_ = row.key;
+        add(row.cf);
     }
 
     protected RowMutation(String table, String key, Map<String, ColumnFamily> modifications)

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Table.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Table.java?rev=884332&r1=884331&r2=884332&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Table.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Table.java Wed Nov 25 23:16:48 2009
@@ -394,25 +394,6 @@
     }
 
     /**
-     * Selects the row associated with the given key.
-    */
-    @Deprecated // CF should be our atom of work, not Row
-    public Row get(String key) throws IOException
-    {
-        Row row = new Row(key);
-        for (String columnFamily : getColumnFamilies())
-        {
-            ColumnFamily cf = get(key, columnFamily);
-            if (cf != null)
-            {
-                row.addColumnFamily(cf);
-            }
-        }
-        return row;
-    }
-
-
-    /**
      * Selects the specified column family for the specified key.
     */
     @Deprecated // single CFs could be larger than memory
@@ -423,27 +404,11 @@
         return cfStore.getColumnFamily(new IdentityQueryFilter(key, new QueryPath(cfName)));
     }
 
-    /**
-     * Selects only the specified column family for the specified key.
-    */
-    @Deprecated
-    public Row getRow(String key, String cfName) throws IOException
-    {
-        Row row = new Row(key);
-        ColumnFamily columnFamily = get(key, cfName);
-        if ( columnFamily != null )
-        	row.addColumnFamily(columnFamily);
-        return row;
-    }
-    
     public Row getRow(QueryFilter filter) throws IOException
     {
         ColumnFamilyStore cfStore = columnFamilyStores_.get(filter.getColumnFamilyName());
-        Row row = new Row(filter.key);
         ColumnFamily columnFamily = cfStore.getColumnFamily(filter);
-        if (columnFamily != null)
-            row.addColumnFamily(columnFamily);
-        return row;
+        return new Row(filter.key, columnFamily);
     }
 
     /**

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=884332&r1=884331&r2=884332&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 Wed Nov 25 23:16:48 2009
@@ -22,7 +22,6 @@
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.util.*;
-import java.util.concurrent.TimeoutException;
 
 import org.apache.log4j.Logger;
 
@@ -99,7 +98,7 @@
 
         for (Row row: rows)
         {
-            columnFamilyKeyMap.put(row.key(), row.getColumnFamily(cfName));
+            columnFamilyKeyMap.put(row.key, row.cf);
         }
         return columnFamilyKeyMap;
 	}

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/service/ReadResponseResolver.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/ReadResponseResolver.java?rev=884332&r1=884331&r2=884332&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/service/ReadResponseResolver.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/service/ReadResponseResolver.java Wed Nov 25 23:16:48 2009
@@ -22,8 +22,6 @@
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
 
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.ReadResponse;
@@ -65,7 +63,7 @@
 	public Row resolve(List<Message> responses) throws DigestMismatchException, IOException
     {
         long startTime = System.currentTimeMillis();
-		Row retRow = null;
+		Row resolved = null;
 		List<Row> rowList = new ArrayList<Row>();
 		List<InetAddress> endPoints = new ArrayList<InetAddress>();
 		String key = null;
@@ -93,7 +91,7 @@
             {
                 rowList.add(result.row());
                 endPoints.add(response.getFrom());
-                key = result.row().key();
+                key = result.row().key;
             }
         }
 		// If there was a digest query compare it with all the data digests 
@@ -105,7 +103,7 @@
                 if (!Arrays.equals(row.digest(), digest))
                 {
                     /* Wrap the key as the context in this exception */
-                    String s = String.format("Mismatch for key %s (%s vs %s)", row.key(), FBUtilities.bytesToHex(row.digest()), FBUtilities.bytesToHex(digest));
+                    String s = String.format("Mismatch for key %s (%s vs %s)", row.key, FBUtilities.bytesToHex(row.digest()), FBUtilities.bytesToHex(digest));
                     throw new DigestMismatchException(s);
                 }
             }
@@ -114,37 +112,33 @@
         /* If the rowList is empty then we had some exception above. */
         if (rowList.size() == 0)
         {
-            return retRow;
+            return resolved;
         }
 
         /* Now calculate the resolved row */
-        retRow = new Row(key);
-        for (int i = 0; i < rowList.size(); i++)
+        resolved = new Row(key, rowList.get(0).cf);
+        for (Row other : rowList.subList(1, rowList.size()))
         {
-            retRow.repair(rowList.get(i));
+            resolved.resolve(other);
         }
 
-        // At  this point  we have the return row .
-        // Now we need to calculate the difference
-        // so that we can schedule read repairs
+        // At this point we have the return row;
+        // Now we need to calculate the difference so that we can schedule read repairs
         for (int i = 0; i < rowList.size(); i++)
         {
             // since retRow is the resolved row it can be used as the super set
-            Row diffRow = rowList.get(i).diff(retRow);
-            if (diffRow == null) // no repair needs to happen
+            ColumnFamily diffCf = rowList.get(i).cf.diff(resolved.cf);
+            if (diffCf == null) // no repair needs to happen
                 continue;
             // create the row mutation message based on the diff and schedule a read repair
             RowMutation rowMutation = new RowMutation(table, key);
-            for (ColumnFamily cf : diffRow.getColumnFamilies())
-            {
-                rowMutation.add(cf);
-            }
+            rowMutation.add(diffCf);
             RowMutationMessage rowMutationMessage = new RowMutationMessage(rowMutation);
             ReadRepairManager.instance().schedule(endPoints.get(i), rowMutationMessage);
         }
         if (logger_.isDebugEnabled())
             logger_.debug("resolve: " + (System.currentTimeMillis() - startTime) + " ms.");
-		return retRow;
+		return resolved;
 	}
 
 	public boolean isDataPresent(List<Message> responses)

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=884332&r1=884331&r2=884332&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 Wed Nov 25 23:16:48 2009
@@ -84,7 +84,7 @@
     {
         public int compare(Row r1, Row r2)
         {
-            return keyComparator.compare(r1.key(), r2.key());
+            return keyComparator.compare(r1.key, r2.key);
         }
     };
 
@@ -562,22 +562,22 @@
             {
                 if (allRows.size() > 0)
                 {
-                    if (keyComparator.compare(rangeRows.get(rangeRows.size() - 1).key(), allRows.first().key()) <= 0)
+                    if (keyComparator.compare(rangeRows.get(rangeRows.size() - 1).key, allRows.first().key) <= 0)
                     {
                         // unlikely, but possible
-                        if (rangeRows.get(rangeRows.size() - 1).equals(allRows.first().key()))
+                        if (rangeRows.get(rangeRows.size() - 1).equals(allRows.first().key))
                         {
                             rangeRows.remove(rangeRows.size() - 1);
                         }
                         // put all from rangeRows into allRows.
                         allRows.addAll(rangeRows);
                     }
-                    else if (keyComparator.compare(allRows.last().key(), rangeRows.get(0).key()) <= 0)
+                    else if (keyComparator.compare(allRows.last().key, rangeRows.get(0).key) <= 0)
                     {
                         // common case. deal with simple start/end key overlaps
-                        if (allRows.last().key().equals(rangeRows.get(0)))
+                        if (allRows.last().key.equals(rangeRows.get(0)))
                         {
-                            allRows.remove(allRows.last().key());
+                            allRows.remove(allRows.last().key);
                         }
                         allRows.addAll(rangeRows); // todo: check logic.
                     }
@@ -609,8 +609,7 @@
         for (Row row : allRows)
         {
             // for now, assume only one cf per row, since that is all we can specify in the Command.
-            ColumnFamily cf = row.getColumnFamilies().iterator().next();
-            results.put(row.key(),cf.getSortedColumns());
+            results.put(row.key, row.cf.getSortedColumns());
         }
         rangeStats.add(System.currentTimeMillis() - startTime);
         return results;

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=884332&r1=884331&r2=884332&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 Wed Nov 25 23:16:48 2009
@@ -353,8 +353,7 @@
      */
     public void doConsistencyCheck(Row row, List<InetAddress> endpoints, ReadCommand command)
     {
-        Runnable consistencySentinel = new ConsistencyManager(command.table, row.cloneMe(), endpoints, command);
-        consistencyManager_.submit(consistencySentinel);
+        consistencyManager_.submit(new ConsistencyManager(command.table, row, endpoints, command));
     }
 
     public Map<Range, List<String>> getRangeToEndPointMap()

Modified: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ReadMessageTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ReadMessageTest.java?rev=884332&r1=884331&r2=884332&view=diff
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ReadMessageTest.java (original)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ReadMessageTest.java Wed Nov 25 23:16:48 2009
@@ -81,8 +81,7 @@
 
         ReadCommand command = new SliceByNamesReadCommand("Keyspace1", "key1", new QueryPath("Standard1"), Arrays.asList("Column1".getBytes()));
         Row row = command.getRow(table);
-        ColumnFamily cf = row.getColumnFamily("Standard1");
-        IColumn col = cf.getColumn("Column1".getBytes());
+        IColumn col = row.cf.getColumn("Column1".getBytes());
         assert Arrays.equals(col.value(), "abcd".getBytes());  
     }
 }

Modified: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RowTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RowTest.java?rev=884332&r1=884331&r2=884332&view=diff
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RowTest.java (original)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RowTest.java Wed Nov 25 23:16:48 2009
@@ -23,7 +23,6 @@
 import org.junit.Test;
 
 import static junit.framework.Assert.assertEquals;
-import org.apache.cassandra.db.filter.QueryPath;
 import org.apache.cassandra.db.marshal.AsciiType;
 import static org.apache.cassandra.Util.column;
 
@@ -58,26 +57,20 @@
     }
 
     @Test
-    public void testRepair()
+    public void testResolve()
     {
-        Row row1 = new Row("");
         ColumnFamily cf1 = ColumnFamily.create("Keyspace1", "Standard1");
         cf1.addColumn(column("one", "A", 0));
-        row1.addColumnFamily(cf1);
+        Row row1 = new Row("", cf1);
 
-        Row row2 = new Row("");
         ColumnFamily cf2 = ColumnFamily.create("Keyspace1", "Standard1");
         cf2.addColumn(column("one", "B", 1));
         cf2.addColumn(column("two", "C", 1));
-        ColumnFamily cf3 = ColumnFamily.create("Keyspace2", "Standard3");
-        cf3.addColumn(column("three", "D", 1));
-        row2.addColumnFamily(cf2);
-        row2.addColumnFamily(cf3);
+        Row row2 = new Row("", cf2);
 
-        row1.repair(row2);
-        cf1 = row1.getColumnFamily("Standard1");
+        row1.resolve(row2);
+        cf1 = row1.cf;
         assert Arrays.equals(cf1.getColumn("one".getBytes()).value(), "B".getBytes());
-        assert Arrays.equals(cf2.getColumn("two".getBytes()).value(), "C".getBytes());
-        assert row1.getColumnFamily("Standard3") != null;
+        assert Arrays.equals(cf1.getColumn("two".getBytes()).value(), "C".getBytes());
     }
 }

Modified: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/TableTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/TableTest.java?rev=884332&r1=884331&r2=884332&view=diff
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/TableTest.java (original)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/TableTest.java Wed Nov 25 23:16:48 2009
@@ -58,13 +58,6 @@
     }
 
     @Test
-    public void testOpen() throws Throwable {
-        Table table = Table.open("Mailbox");
-        Row row = table.get("35300190:1");
-        assertNotNull(row);
-    }
-
-    @Test
     public void testGetRowNoColumns() throws Throwable
     {
         final Table table = Table.open("Keyspace2");