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/08/02 00:30:24 UTC

svn commit: r799943 - in /incubator/cassandra/trunk/src/java/org/apache/cassandra: db/filter/SSTableNamesIterator.java db/filter/SSTableSliceIterator.java io/SSTableReader.java io/SequenceFile.java

Author: jbellis
Date: Sat Aug  1 22:30:23 2009
New Revision: 799943

URL: http://svn.apache.org/viewvc?rev=799943&view=rev
Log:
move next, getColumnGroupReader out of SSTableReader.
patch by jbellis; reviewed by Stu Hood for CASSANDRA-330

Modified:
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableNamesIterator.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableSliceIterator.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableReader.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SequenceFile.java

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableNamesIterator.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableNamesIterator.java?rev=799943&r1=799942&r2=799943&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableNamesIterator.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableNamesIterator.java Sat Aug  1 22:30:23 2009
@@ -6,8 +6,7 @@
 
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.IColumn;
-import org.apache.cassandra.io.SSTableReader;
-import org.apache.cassandra.io.DataInputBuffer;
+import org.apache.cassandra.io.*;
 
 public class SSTableNamesIterator extends SimpleAbstractColumnIterator
 {
@@ -20,7 +19,38 @@
     {
         this.columns = columns;
         SSTableReader ssTable = SSTableReader.open(filename);
-        DataInputBuffer buffer = ssTable.next(key, cfName, columns);
+
+        IFileReader dataReader = null;
+        DataOutputBuffer bufOut = new DataOutputBuffer();
+        DataInputBuffer bufIn = new DataInputBuffer();
+
+        try
+        {
+            dataReader = SequenceFile.bufferedReader(ssTable.getFilename(), 64 * 1024);
+            String decoratedKey = ssTable.getPartitioner().decorateKey(key);
+            long position = ssTable.getPosition(decoratedKey, ssTable.getPartitioner());
+
+            long bytesRead = dataReader.next(decoratedKey, bufOut, cfName, columns, position);
+            if (bytesRead != -1L)
+            {
+                if (bufOut.getLength() > 0)
+                {
+                    bufIn.reset(bufOut.getData(), bufOut.getLength());
+                    /* read the key even though we do not use it */
+                    bufIn.readUTF();
+                    bufIn.readInt();
+                }
+            }
+        }
+        finally
+        {
+            if (dataReader != null)
+            {
+                dataReader.close();
+            }
+        }
+
+        DataInputBuffer buffer = bufIn;
         if (buffer.getLength() > 0)
         {
             cf = ColumnFamily.serializer().deserialize(buffer);

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableSliceIterator.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableSliceIterator.java?rev=799943&r1=799942&r2=799943&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableSliceIterator.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableSliceIterator.java Sat Aug  1 22:30:23 2009
@@ -10,6 +10,7 @@
 import org.apache.cassandra.io.DataInputBuffer;
 import org.apache.cassandra.io.SequenceFile;
 import org.apache.cassandra.io.SSTableReader;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import com.google.common.collect.AbstractIterator;
 
 /**
@@ -32,7 +33,12 @@
     {
         this.isAscending = isAscending;
         SSTableReader ssTable = SSTableReader.open(filename);
-        reader = ssTable.getColumnGroupReader(key, cfName, startColumn, isAscending);
+
+        /* Morph key into actual key based on the partition type. */
+        String decoratedKey = ssTable.getPartitioner().decorateKey(key);
+        long position = ssTable.getPosition(decoratedKey, ssTable.getPartitioner());
+        AbstractType comparator1 = DatabaseDescriptor.getComparator(ssTable.getTableName(), cfName);
+        reader = new SequenceFile.ColumnGroupReader(ssTable.getFilename(), decoratedKey, cfName, comparator1, startColumn, isAscending, position);
         this.comparator = comparator;
         this.startColumn = startColumn;
         curColumnIndex = isAscending ? 0 : -1;

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableReader.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableReader.java?rev=799943&r1=799942&r2=799943&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableReader.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableReader.java Sat Aug  1 22:30:23 2009
@@ -287,60 +287,6 @@
         }
     }
 
-    public DataInputBuffer next(final String clientKey, String cfName, SortedSet<byte[]> columnNames) throws IOException
-    {
-        IFileReader dataReader = null;
-        try
-        {
-            dataReader = SequenceFile.bufferedReader(dataFile, 64 * 1024);
-            String decoratedKey = partitioner.decorateKey(clientKey);
-            long position = getPosition(decoratedKey, partitioner);
-
-            DataOutputBuffer bufOut = new DataOutputBuffer();
-            DataInputBuffer bufIn = new DataInputBuffer();
-            long bytesRead = dataReader.next(decoratedKey, bufOut, cfName, columnNames, position);
-            if (bytesRead != -1L)
-            {
-                if (bufOut.getLength() > 0)
-                {
-                    bufIn.reset(bufOut.getData(), bufOut.getLength());
-                    /* read the key even though we do not use it */
-                    bufIn.readUTF();
-                    bufIn.readInt();
-                }
-            }
-            return bufIn;
-        }
-        finally
-        {
-            if (dataReader != null)
-            {
-                dataReader.close();
-            }
-        }
-    }
-
-    /**
-     * obtain a BlockReader for the getColumnSlice call.
-     */
-    public ColumnGroupReader getColumnGroupReader(String key, String cfName, byte[] startColumn, boolean isAscending) throws IOException
-    {
-        IFileReader dataReader = SequenceFile.bufferedReader(dataFile, 64 * 1024);
-
-        try
-        {
-            /* Morph key into actual key based on the partition type. */
-            String decoratedKey = partitioner.decorateKey(key);
-            long position = getPosition(decoratedKey, partitioner);
-            AbstractType comparator = DatabaseDescriptor.getComparator(getTableName(), cfName);
-            return new ColumnGroupReader(dataFile, decoratedKey, cfName, comparator, startColumn, isAscending, position);
-        }
-        finally
-        {
-            dataReader.close();
-        }
-    }
-
     public void delete() throws IOException
     {
         FileUtils.deleteWithConfirm(new File(dataFile));
@@ -365,7 +311,7 @@
         }
     }
 
-    IPartitioner getPartitioner()
+    public IPartitioner getPartitioner()
     {
         return partitioner;
     }

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SequenceFile.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SequenceFile.java?rev=799943&r1=799942&r2=799943&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SequenceFile.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SequenceFile.java Sat Aug  1 22:30:23 2009
@@ -59,7 +59,7 @@
         private int localDeletionTime_;
         private long markedForDeleteAt_;
 
-        ColumnGroupReader(String filename, String key, String cfName, AbstractType comparator, byte[] startColumn, boolean isAscending, long position) throws IOException
+        public ColumnGroupReader(String filename, String key, String cfName, AbstractType comparator, byte[] startColumn, boolean isAscending, long position) throws IOException
         {
             super(filename, 128 * 1024);
             this.cfName_ = cfName;