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/03/27 03:19:11 UTC

svn commit: r758973 - /incubator/cassandra/trunk/src/org/apache/cassandra/db/Row.java

Author: jbellis
Date: Fri Mar 27 02:19:10 2009
New Revision: 758973

URL: http://svn.apache.org/viewvc?rev=758973&view=rev
Log:
add getColumnFamilyNames.  r/m merge() [duplicate of repair]

Modified:
    incubator/cassandra/trunk/src/org/apache/cassandra/db/Row.java

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/db/Row.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/db/Row.java?rev=758973&r1=758972&r2=758973&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/db/Row.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/db/Row.java Fri Mar 27 02:19:10 2009
@@ -21,21 +21,23 @@
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
-import java.io.Serializable;
-import java.util.*;
-import java.util.concurrent.atomic.AtomicInteger;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Hashtable;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.log4j.Logger;
 
 import org.apache.cassandra.io.ICompactSerializer;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.log4j.Logger;
 
-/**
- * Author : Avinash Lakshman ( alakshman@facebook.com) & Prashant Malik ( pmalik@facebook.com )
- */
-public class Row implements Serializable
+public class Row
 {
     private static ICompactSerializer<Row> serializer_;
-	private static Logger logger_ = Logger.getLogger(Row.class);
+    private static Logger logger_ = Logger.getLogger(Row.class);
 
     static
     {
@@ -47,33 +49,31 @@
         return serializer_;
     }
 
-    private String key_;    
+    private String key_;
     private Map<String, ColumnFamily> columnFamilies_ = new Hashtable<String, ColumnFamily>();
-    private transient AtomicInteger size_ = new AtomicInteger(0);
 
-    /* Ctor for JAXB */
     protected Row()
     {
     }
 
     public Row(String key)
     {
-        key_ = key;        
+        key_ = key;
     }
-    
+
     public String key()
     {
         return key_;
     }
-    
+
     void key(String key)
     {
         key_ = key;
     }
-    
-    public ColumnFamily getColumnFamily(String cfName)
+
+    public Set<String> getColumnFamilyNames()
     {
-        return columnFamilies_.get(cfName);
+        return columnFamilies_.keySet();
     }
 
     public Collection<ColumnFamily> getColumnFamilies()
@@ -88,51 +88,28 @@
         return columnFamilies_;
     }
 
+    public ColumnFamily getColumnFamily(String cfName)
+    {
+        return columnFamilies_.get(cfName);
+    }
+
     void addColumnFamily(ColumnFamily columnFamily)
     {
         columnFamilies_.put(columnFamily.name(), columnFamily);
-        size_.addAndGet(columnFamily.size());
     }
 
     void removeColumnFamily(ColumnFamily columnFamily)
     {
         columnFamilies_.remove(columnFamily.name());
         int delta = (-1) * columnFamily.size();
-        size_.addAndGet(delta);
-    }
-
-    public int size()
-    {
-        return size_.get();
     }
 
     public boolean isEmpty()
     {
-    	return ( columnFamilies_.size() == 0 );
-    }
-    
-    /**
-     * This is used as oldRow.merge(newRow). Basically we take the newRow
-     * and merge it into the oldRow.
-    */
-    void merge(Row row)
-    {
-        Map<String, ColumnFamily> columnFamilies = row.getColumnFamilyMap();
-        Set<String> cfNames = columnFamilies.keySet();
-
-        for ( String cfName : cfNames )
-        {
-            ColumnFamily cf = columnFamilies_.get(cfName);
-            if ( cf == null )
-                columnFamilies_.put(cfName, columnFamilies.get(cfName));
-            else
-            {
-                cf.merge(columnFamilies.get(cfName));
-            }
-        }
+        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 fn will make the current row take the latest changes .
@@ -142,12 +119,12 @@
         Map<String, ColumnFamily> columnFamilies = row.getColumnFamilyMap();
         Set<String> cfNames = columnFamilies.keySet();
 
-        for ( String cfName : cfNames )
+        for (String cfName : cfNames)
         {
             ColumnFamily cf = columnFamilies_.get(cfName);
-            if ( cf == null )
+            if (cf == null)
             {
-            	cf = new ColumnFamily(cfName);
+                cf = new ColumnFamily(cfName);
                 columnFamilies_.put(cfName, cf);
             }
             cf.repair(columnFamilies.get(cfName));
@@ -155,7 +132,7 @@
 
     }
 
-    /**
+    /*
      * 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
@@ -166,78 +143,82 @@
     public Row diff(Row row)
     {
         Row rowDiff = new Row(key_);
-    	Map<String, ColumnFamily> columnFamilies = row.getColumnFamilyMap();
+        Map<String, ColumnFamily> columnFamilies = row.getColumnFamilyMap();
         Set<String> cfNames = columnFamilies.keySet();
 
-        for ( String cfName : cfNames )
+        for (String cfName : cfNames)
         {
             ColumnFamily cf = columnFamilies_.get(cfName);
             ColumnFamily cfDiff = null;
-            if ( cf == null )
-            	rowDiff.getColumnFamilyMap().put(cfName, columnFamilies.get(cfName));
+            if (cf == null)
+                rowDiff.getColumnFamilyMap().put(cfName, columnFamilies.get(cfName));
             else
             {
-            	cfDiff = cf.diff(columnFamilies.get(cfName));
-            	if(cfDiff != null)
-            		rowDiff.getColumnFamilyMap().put(cfName, cfDiff);
+                cfDiff = cf.diff(columnFamilies.get(cfName));
+                if (cfDiff != null)
+                    rowDiff.getColumnFamilyMap().put(cfName, cfDiff);
             }
         }
-        if(rowDiff.getColumnFamilyMap().size() != 0)
-        	return rowDiff;
+        if (rowDiff.getColumnFamilyMap().size() != 0)
+            return rowDiff;
         else
-        	return null;
+            return null;
     }
-    
+
     public Row cloneMe()
     {
-    	Row row = new Row(key_);
-    	row.columnFamilies_ = new HashMap<String, ColumnFamily>(columnFamilies_);
-    	return row;
+        Row row = new Row(key_);
+        row.columnFamilies_ = new HashMap<String, ColumnFamily>(columnFamilies_);
+        return row;
     }
 
     public byte[] digest()
     {
         long start = System.currentTimeMillis();
-    	Set<String> cfamilies = columnFamilies_.keySet();
-    	byte[] xorHash = new byte[0];
-    	byte[] tmpHash = new byte[0];
-    	for(String cFamily : cfamilies)
-    	{
-    		if(xorHash.length == 0)
-    		{
-    			xorHash = columnFamilies_.get(cFamily).digest();
-    		}
-    		else
-    		{
-    			tmpHash = columnFamilies_.get(cFamily).digest();
-    			xorHash = FBUtilities.xor(xorHash, tmpHash);
-    		}
-    	}
+        Set<String> cfamilies = columnFamilies_.keySet();
+        byte[] xorHash = ArrayUtils.EMPTY_BYTE_ARRAY;
+        for (String cFamily : cfamilies)
+        {
+            if (xorHash.length == 0)
+            {
+                xorHash = columnFamilies_.get(cFamily).digest();
+            }
+            else
+            {
+                byte[] tmpHash = columnFamilies_.get(cFamily).digest();
+                xorHash = FBUtilities.xor(xorHash, tmpHash);
+            }
+        }
         logger_.info("DIGEST TIME: " + (System.currentTimeMillis() - start)
-                + " ms.");
-    	return xorHash;
+                     + " ms.");
+        return xorHash;
     }
-    
+
     void clear()
-    {        
+    {
         columnFamilies_.clear();
     }
+
+    public String toString()
+    {
+        return "Row(" + key_ + " [" + StringUtils.join(columnFamilies_.values(), ", ") + ")]";
+    }
 }
 
 class RowSerializer implements ICompactSerializer<Row>
 {
     public void serialize(Row row, DataOutputStream dos) throws IOException
     {
-        dos.writeUTF(row.key());        
+        dos.writeUTF(row.key());
         Map<String, ColumnFamily> columnFamilies = row.getColumnFamilyMap();
-        int size = columnFamilies.size();        
+        int size = columnFamilies.size();
         dos.writeInt(size);
-        
-        if ( size > 0 )
-        {   
-        	Set<String> cNames = columnFamilies.keySet();
-            for ( String cName : cNames )
-            {                
+
+        if (size > 0)
+        {
+            Set<String> cNames = columnFamilies.keySet();
+            for (String cName : cNames)
+            {
                 ColumnFamily.serializer().serialize(columnFamilies.get(cName), dos);
             }
         }
@@ -245,13 +226,13 @@
 
     public Row deserialize(DataInputStream dis) throws IOException
     {
-        String key = dis.readUTF();        
-        Row row = new Row(key);        
+        String key = dis.readUTF();
+        Row row = new Row(key);
         int size = dis.readInt();
-        
-        if ( size > 0 )
-        { 
-            for ( int i = 0; i < size; ++i )
+
+        if (size > 0)
+        {
+            for (int i = 0; i < size; ++i)
             {
                 ColumnFamily cf = ColumnFamily.serializer().deserialize(dis);
                 row.addColumnFamily(cf);