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 2010/06/16 17:26:43 UTC

svn commit: r955264 - /cassandra/trunk/src/java/org/apache/cassandra/db/ColumnIndexer.java

Author: jbellis
Date: Wed Jun 16 15:26:43 2010
New Revision: 955264

URL: http://svn.apache.org/viewvc?rev=955264&view=rev
Log:
make single pass over columns for indexing
patch by jbellis; reviewed by Stu Hood for CASSANDRA-16


Modified:
    cassandra/trunk/src/java/org/apache/cassandra/db/ColumnIndexer.java

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/ColumnIndexer.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnIndexer.java?rev=955264&r1=955263&r2=955264&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/ColumnIndexer.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/ColumnIndexer.java Wed Jun 16 15:26:43 2010
@@ -18,18 +18,17 @@
 
 package org.apache.cassandra.db;
 
+import java.io.IOError;
 import java.io.IOException;
 import java.io.DataOutput;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
-import java.util.Iterator;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.io.sstable.IndexHelper;
 import org.apache.cassandra.utils.BloomFilter;
-import org.apache.cassandra.db.marshal.AbstractType;
 
 
 /**
@@ -45,36 +44,20 @@ public class ColumnIndexer
 	 * @throws IOException
 	 */
     public static void serialize(ColumnFamily columnFamily, DataOutput dos)
-	{
-        Collection<IColumn> columns = columnFamily.getSortedColumns();
-        BloomFilter bf = createColumnBloomFilter(columns);                    
-        /* Write out the bloom filter. */
-        DataOutputBuffer bufOut = new DataOutputBuffer();
+    {
         try
         {
-            BloomFilter.serializer().serialize(bf, bufOut);
-            /* write the length of the serialized bloom filter. */
-            dos.writeInt(bufOut.getLength());
-            /* write out the serialized bytes. */
-            dos.write(bufOut.getData(), 0, bufOut.getLength());
-
-            /* Do the indexing */
-            doIndexing(columnFamily.getComparator(), columns, dos);
+            serializeInternal(columnFamily, dos);
         }
         catch (IOException e)
         {
-            throw new RuntimeException(e);
+            throw new IOError(e);
         }
-	}
-    
-    /**
-     * Create a bloom filter that contains the subcolumns and the columns that
-     * make up this Column Family.
-     * @param columns columns of the ColumnFamily
-     * @return BloomFilter with the summarized information.
-     */
-    private static BloomFilter createColumnBloomFilter(Collection<IColumn> columns)
+    }
+
+    public static void serializeInternal(ColumnFamily columnFamily, DataOutput dos) throws IOException
     {
+        Collection<IColumn> columns = columnFamily.getSortedColumns();
         int columnCount = 0;
         for (IColumn column : columns)
         {
@@ -82,37 +65,14 @@ public class ColumnIndexer
         }
 
         BloomFilter bf = BloomFilter.getFilter(columnCount, 4);
-        for (IColumn column : columns)
-        {
-            bf.add(column.name());
-            /* If this is SuperColumn type Column Family we need to get the subColumns too. */
-            if (column instanceof SuperColumn)
-            {
-                Collection<IColumn> subColumns = column.getSubColumns();
-                for (IColumn subColumn : subColumns)
-                {
-                    bf.add(subColumn.name());
-                }
-            }
-        }
-        return bf;
-    }
 
-    /**
-     * Given the collection of columns in the Column Family,
-     * the name index is generated and written into the provided
-     * stream
-     * @param columns for whom the name index needs to be generated
-     * @param dos stream into which the serialized name index needs
-     *            to be written.
-     * @throws IOException
-     */
-    private static void doIndexing(AbstractType comparator, Collection<IColumn> columns, DataOutput dos) throws IOException
-    {
         if (columns.isEmpty())
         {
+            // write empty bloom filter and index
+            writeBloomFilter(dos, bf);
             dos.writeInt(0);
-            return;            
+
+            return;
         }
 
         /*
@@ -122,14 +82,24 @@ public class ColumnIndexer
          * We do this so that we don't read all the columns into memory.
         */
         List<IndexHelper.IndexInfo> indexList = new ArrayList<IndexHelper.IndexInfo>();
-        
+
         int endPosition = 0, startPosition = -1;
         int indexSizeInBytes = 0;
-        IColumn column = null, firstColumn = null;
+        IColumn lastColumn = null, firstColumn = null;
         /* column offsets at the right thresholds into the index map. */
-        for (Iterator<IColumn> it = columns.iterator(); it.hasNext();)
+        for (IColumn column : columns)
         {
-            column = it.next();
+            bf.add(column.name());
+            /* If this is SuperColumn type Column Family we need to get the subColumns too. */
+            if (column instanceof SuperColumn)
+            {
+                Collection<IColumn> subColumns = column.getSubColumns();
+                for (IColumn subColumn : subColumns)
+                {
+                    bf.add(subColumn.name());
+                }
+            }
+
             if (firstColumn == null)
             {
                 firstColumn = column;
@@ -144,20 +114,35 @@ public class ColumnIndexer
                 indexSizeInBytes += cIndexInfo.serializedSize();
                 firstColumn = null;
             }
+
+            lastColumn = column;
         }
         // the last column may have fallen on an index boundary already.  if not, index it explicitly.
-        if (indexList.isEmpty() || comparator.compare(indexList.get(indexList.size() - 1).lastName, column.name()) != 0)
+        if (indexList.isEmpty() || columnFamily.getComparator().compare(indexList.get(indexList.size() - 1).lastName, lastColumn.name()) != 0)
         {
-            IndexHelper.IndexInfo cIndexInfo = new IndexHelper.IndexInfo(firstColumn.name(), column.name(), startPosition, endPosition - startPosition);
+            IndexHelper.IndexInfo cIndexInfo = new IndexHelper.IndexInfo(firstColumn.name(), lastColumn.name(), startPosition, endPosition - startPosition);
             indexList.add(cIndexInfo);
             indexSizeInBytes += cIndexInfo.serializedSize();
         }
 
+        /* Write out the bloom filter. */
+        writeBloomFilter(dos, bf);
+
+        // write the index
         assert indexSizeInBytes > 0;
         dos.writeInt(indexSizeInBytes);
         for (IndexHelper.IndexInfo cIndexInfo : indexList)
         {
             cIndexInfo.serialize(dos);
         }
+	}
+
+    private static void writeBloomFilter(DataOutput dos, BloomFilter bf) throws IOException
+    {
+        DataOutputBuffer bufOut = new DataOutputBuffer();
+        BloomFilter.serializer().serialize(bf, bufOut);
+        dos.writeInt(bufOut.getLength());
+        dos.write(bufOut.getData(), 0, bufOut.getLength());
     }
+
 }