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/09/17 23:10:10 UTC

svn commit: r816382 - in /incubator/cassandra/trunk: src/java/org/apache/cassandra/db/ src/java/org/apache/cassandra/io/ test/unit/org/apache/cassandra/db/

Author: jbellis
Date: Thu Sep 17 21:10:09 2009
New Revision: 816382

URL: http://svn.apache.org/viewvc?rev=816382&view=rev
Log:
Replace PriorityQueue mess with a CompactionIterator that efficiently yields compacted Rows from a set of sstables by feeding CollationIterator into a ReducingIterator transform.  ("Efficiently" means we never deserialize data until it is needed, so the number of sstables that can be compacted at once is  virtually unlimited, and if only one sstable contains a given key that row data will be copied over without an intermediate de/serialize step.) This is a very natural fit for the compaction algorithm and almost entirely gets rid of duplicated code between doFileCompaction and doAntiCompaction.
patch by jbellis; reviewed by goffinet for CASSANDRA-436

Added:
    incubator/cassandra/trunk/src/java/org/apache/cassandra/io/CompactionIterator.java
Removed:
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/FileStructComparator.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/io/FileStruct.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/Memtable.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/io/IteratingRow.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableReader.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyTest.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=816382&r1=816381&r2=816382&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 Thu Sep 17 21:10:09 2009
@@ -35,7 +35,6 @@
 import org.apache.cassandra.io.ICompactSerializer2;
 import org.apache.cassandra.db.filter.QueryPath;
 import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.MarshalException;
 
 
 public final class ColumnFamily implements IColumnContainer
@@ -121,12 +120,13 @@
      *  We need to go through each column
      *  in the column family and resolve it before adding
     */
-    void addColumns(ColumnFamily cf)
+    public void addAll(ColumnFamily cf)
     {
         for (IColumn column : cf.getSortedColumns())
         {
             addColumn(column);
         }
+        delete(cf);
     }
 
     public ICompactSerializer2<IColumn> getColumnSerializer()
@@ -415,8 +415,7 @@
         for (ColumnFamily cf2 : columnFamilies)
         {
             assert cf.name().equals(cf2.name());
-            cf.addColumns(cf2);
-            cf.delete(cf2);
+            cf.addAll(cf2);
         }
         return 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=816382&r1=816381&r2=816382&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 Thu Sep 17 21:10:09 2009
@@ -560,26 +560,6 @@
         CompactionManager.instance().submit(this);
     }
 
-    private PriorityQueue<FileStruct> initializePriorityQueue(Collection<SSTableReader> sstables, List<Range> ranges) throws IOException
-    {
-        PriorityQueue<FileStruct> pq = new PriorityQueue<FileStruct>();
-        if (sstables.size() > 1 || (ranges != null && sstables.size() > 0))
-        {
-            FileStruct fs = null;
-            for (SSTableReader sstable : sstables)
-            {
-                fs = sstable.getFileStruct();
-                fs.advance(true);
-                if (fs.isExhausted())
-                {
-                    continue;
-                }
-                pq.add(fs);
-            }
-        }
-        return pq;
-    }
-
     /*
      * Group files of similar size into buckets.
      */
@@ -766,150 +746,67 @@
      */
     List<SSTableReader> doFileAntiCompaction(Collection<SSTableReader> sstables, List<Range> ranges, EndPoint target) throws IOException
     {
-        List<SSTableReader> results = new ArrayList<SSTableReader>();
-        long startTime = System.currentTimeMillis();
-        long totalBytesRead = 0;
-        long totalBytesWritten = 0;
-        long totalkeysRead = 0;
-        long totalkeysWritten = 0;
-        String rangeFileLocation;
-        String mergedFileName;
+        logger_.info("AntiCompacting [" + StringUtils.join(sstables, ",") + "]");
         // Calculate the expected compacted filesize
-        long expectedRangeFileSize = getExpectedCompactedFileSize(sstables);
-        /* in the worst case a node will be giving out half of its data so we take a chance */
-        expectedRangeFileSize = expectedRangeFileSize / 2;
-        rangeFileLocation = DatabaseDescriptor.getDataFileLocationForTable(table_, expectedRangeFileSize);
-        // If the compaction file path is null that means we have no space left for this compaction.
-        if (rangeFileLocation == null)
-        {
-            logger_.error("Total bytes to be written for range compaction  ..."
-                          + expectedRangeFileSize + "   is greater than the safe limit of the disk space available.");
-            return results;
-        }
-        PriorityQueue<FileStruct> pq = initializePriorityQueue(sstables, ranges);
-        if (pq.isEmpty())
+        long expectedRangeFileSize = getExpectedCompactedFileSize(sstables) / 2;
+        String compactionFileLocation = DatabaseDescriptor.getDataFileLocationForTable(table_, expectedRangeFileSize);
+        if (compactionFileLocation == null)
         {
-            return results;
+            throw new UnsupportedOperationException("disk full");
         }
+        List<SSTableReader> results = new ArrayList<SSTableReader>();
 
-        mergedFileName = getTempSSTableFileName();
-        SSTableWriter rangeWriter = null;
-        String lastkey = null;
-        List<FileStruct> lfs = new ArrayList<FileStruct>();
-        DataOutputBuffer bufOut = new DataOutputBuffer();
-        int expectedBloomFilterSize = SSTableReader.getApproximateKeyCount(sstables);
-        expectedBloomFilterSize = (expectedBloomFilterSize > 0) ? expectedBloomFilterSize : SSTableReader.indexInterval();
+        long startTime = System.currentTimeMillis();
+        long totalkeysWritten = 0;
+
+        int expectedBloomFilterSize = Math.max(SSTableReader.indexInterval(), SSTableReader.getApproximateKeyCount(sstables) / 2);
         if (logger_.isDebugEnabled())
           logger_.debug("Expected bloom filter size : " + expectedBloomFilterSize);
-        List<ColumnFamily> columnFamilies = new ArrayList<ColumnFamily>();
 
-        while (pq.size() > 0 || lfs.size() > 0)
+        SSTableWriter writer = null;
+        CompactionIterator ci = new CompactionIterator(sstables);
+
+        try
         {
-            FileStruct fs = null;
-            if (pq.size() > 0)
+            if (!ci.hasNext())
             {
-                fs = pq.poll();
+                logger_.warn("Nothing to compact (all files empty or corrupt). This should not happen.");
+                return results;
             }
-            if (fs != null
-                && (lastkey == null || lastkey.equals(fs.getKey())))
-            {
-                // The keys are the same so we need to add this to the
-                // ldfs list
-                lastkey = fs.getKey();
-                lfs.add(fs);
-            }
-            else
+
+            while (ci.hasNext())
             {
-                Collections.sort(lfs, new FileStructComparator());
-                ColumnFamily columnFamily;
-                bufOut.reset();
-                if (lfs.size() > 1)
-                {
-                    for (FileStruct filestruct : lfs)
-                    {
-                        // We want to add only 2 and resolve them right there in order to save on memory footprint
-                        if (columnFamilies.size() > 1)
-                        {
-                            // Now merge the 2 column families
-                            merge(columnFamilies);
-                        }
-                        // deserialize into column families
-                        columnFamilies.add(filestruct.getColumnFamily());
-                    }
-                    // Now after merging all crap append to the sstable
-                    columnFamily = resolveAndRemoveDeleted(columnFamilies);
-                    columnFamilies.clear();
-                    if (columnFamily != null)
-                    {
-                        ColumnFamily.serializer().serializeWithIndexes(columnFamily, bufOut);
-                    }
-                }
-                else
+                CompactionIterator.CompactedRow row = ci.next();
+                if (Range.isTokenInRanges(StorageService.getPartitioner().getToken(row.key), ranges))
                 {
-                    // TODO deserializing only to reserialize is dumb
-                    FileStruct filestruct = lfs.get(0);
-                    ColumnFamily.serializer().serializeWithIndexes(filestruct.getColumnFamily(), bufOut);
-                }
-                if (Range.isTokenInRanges(StorageService.getPartitioner().getToken(lastkey), ranges))
-                {
-                    if (rangeWriter == null)
+                    if (writer == null)
                     {
                         if (target != null)
                         {
-                            rangeFileLocation = rangeFileLocation + File.separator + "bootstrap";
-                        }
-                        FileUtils.createDirectory(rangeFileLocation);
-                        String fname = new File(rangeFileLocation, mergedFileName).getAbsolutePath();
-                        rangeWriter = new SSTableWriter(fname, expectedBloomFilterSize, StorageService.getPartitioner());
-                    }
-                    rangeWriter.append(lastkey, bufOut);
-                }
-                totalkeysWritten++;
-                for (FileStruct filestruct : lfs)
-                {
-                    filestruct.advance(true);
-                    if (filestruct.isExhausted())
-                    {
-                        continue;
-                    }
-                    /* keep on looping until we find a key in the range */
-                    while (!Range.isTokenInRanges(StorageService.getPartitioner().getToken(filestruct.getKey()), ranges))
-                    {
-                        filestruct.advance(true);
-                        if (filestruct.isExhausted())
-                        {
-                            break;
+                            compactionFileLocation = compactionFileLocation + File.separator + "bootstrap";
                         }
+                        FileUtils.createDirectory(compactionFileLocation);
+                        String newFilename = new File(compactionFileLocation, getTempSSTableFileName()).getAbsolutePath();
+                        writer = new SSTableWriter(newFilename, expectedBloomFilterSize, StorageService.getPartitioner());
                     }
-                    if (!filestruct.isExhausted())
-                    {
-                        pq.add(filestruct);
-                    }
-                    totalkeysRead++;
-                }
-                lfs.clear();
-                lastkey = null;
-                if (fs != null)
-                {
-                    // Add back the fs since we processed the rest of
-                    // filestructs
-                    pq.add(fs);
+                    writer.append(row.key, row.buffer);
+                    totalkeysWritten++;
                 }
             }
         }
-
-        if (rangeWriter != null)
+        finally
         {
-            results.add(rangeWriter.closeAndOpenReader());
+            ci.close();
         }
 
-        if (logger_.isDebugEnabled())
+        if (writer != null)
         {
-            logger_.debug("Total time taken for range split   ..." + (System.currentTimeMillis() - startTime));
-            logger_.debug("Total bytes Read for range split  ..." + totalBytesRead);
-            logger_.debug("Total bytes written for range split  ..."
-                          + totalBytesWritten + "   Total keys read ..." + totalkeysRead);
+            results.add(writer.closeAndOpenReader());
+            String format = "AntiCompacted to %s.  %d/%d bytes for %d keys.  Time: %dms.";
+            long dTime = System.currentTimeMillis() - startTime;
+            logger_.info(String.format(format, writer.getFilename(), getTotalBytes(sstables), results.get(0).length(), totalkeysWritten, dTime));
         }
+
         return results;
     }
 
@@ -938,111 +835,59 @@
         }
 
         long startTime = System.currentTimeMillis();
-        long totalBytesRead = 0;
-        long totalkeysRead = 0;
         long totalkeysWritten = 0;
-        PriorityQueue<FileStruct> pq = initializePriorityQueue(sstables, null);
 
-        if (pq.isEmpty())
-        {
-            logger_.warn("Nothing to compact (all files empty or corrupt). This should not happen.");
-            // TODO clean out bad files, if any
-            return 0;
-        }
-
-        int expectedBloomFilterSize = SSTableReader.getApproximateKeyCount(sstables);
-        if (expectedBloomFilterSize < 0)
-            expectedBloomFilterSize = SSTableReader.indexInterval();
-        String newFilename = new File(compactionFileLocation, getTempSSTableFileName()).getAbsolutePath();
-        SSTableWriter writer = new SSTableWriter(newFilename, expectedBloomFilterSize, StorageService.getPartitioner());
-        SSTableReader ssTable = null;
-        String lastkey = null;
-        List<FileStruct> lfs = new ArrayList<FileStruct>();
-        DataOutputBuffer bufOut = new DataOutputBuffer();
+        int expectedBloomFilterSize = Math.max(SSTableReader.indexInterval(), SSTableReader.getApproximateKeyCount(sstables));
         if (logger_.isDebugEnabled())
           logger_.debug("Expected bloom filter size : " + expectedBloomFilterSize);
-        List<ColumnFamily> columnFamilies = new ArrayList<ColumnFamily>();
 
-        while (pq.size() > 0 || lfs.size() > 0)
+        SSTableWriter writer;
+        CompactionIterator ci = new CompactionIterator(sstables);
+
+        try
         {
-            FileStruct fs = null;
-            if (pq.size() > 0)
+            if (!ci.hasNext())
             {
-                fs = pq.poll();
+                logger_.warn("Nothing to compact (all files empty or corrupt). This should not happen.");
+                return 0;
             }
-            if (fs != null
-                && (lastkey == null || lastkey.equals(fs.getKey())))
-            {
-                // The keys are the same so we need to add this to the
-                // ldfs list
-                lastkey = fs.getKey();
-                lfs.add(fs);
-            }
-            else
-            {
-                Collections.sort(lfs, new FileStructComparator());
-                ColumnFamily columnFamily;
-                bufOut.reset();
-                if (lfs.size() > 1)
-                {
-                    for (FileStruct filestruct : lfs)
-                    {
-                        // We want to add only 2 and resolve them right there in order to save on memory footprint
-                        if (columnFamilies.size() > 1)
-                        {
-                            merge(columnFamilies);
-                        }
-                        // deserialize into column families
-                        columnFamilies.add(filestruct.getColumnFamily());
-                    }
-                    // Now after merging all crap append to the sstable
-                    columnFamily = resolveAndRemoveDeleted(columnFamilies);
-                    columnFamilies.clear();
-                    if (columnFamily != null)
-                    {
-                        ColumnFamily.serializer().serializeWithIndexes(columnFamily, bufOut);
-                    }
-                }
-                else
-                {
-                    // TODO deserializing only to reserialize is dumb
-                    FileStruct filestruct = lfs.get(0);
-                    ColumnFamily.serializer().serializeWithIndexes(filestruct.getColumnFamily(), bufOut);
-                }
 
-                writer.append(lastkey, bufOut);
-                totalkeysWritten++;
+            String newFilename = new File(compactionFileLocation, getTempSSTableFileName()).getAbsolutePath();
+            writer = new SSTableWriter(newFilename, expectedBloomFilterSize, StorageService.getPartitioner());
 
-                for (FileStruct filestruct : lfs)
-                {
-                    filestruct.advance(true);
-                    if (filestruct.isExhausted())
-                    {
-                        continue;
-                    }
-                    pq.add(filestruct);
-                    totalkeysRead++;
-                }
-                lfs.clear();
-                lastkey = null;
-                if (fs != null)
-                {
-                    /* Add back the fs since we processed the rest of filestructs */
-                    pq.add(fs);
-                }
+            while (ci.hasNext())
+            {
+                CompactionIterator.CompactedRow row = ci.next();
+                writer.append(row.key, row.buffer);
+                totalkeysWritten++;
             }
         }
-        ssTable = writer.closeAndOpenReader();
+        finally
+        {
+            ci.close();
+        }
+
+        SSTableReader ssTable = writer.closeAndOpenReader();
         ssTables_.add(ssTable);
         ssTables_.markCompacted(sstables);
         CompactionManager.instance().submit(ColumnFamilyStore.this);
 
-        String format = "Compacted to %s.  %d/%d bytes for %d/%d keys read/written.  Time: %dms.";
+        String format = "Compacted to %s.  %d/%d bytes for %d keys.  Time: %dms.";
         long dTime = System.currentTimeMillis() - startTime;
-        logger_.info(String.format(format, writer.getFilename(), totalBytesRead, ssTable.length(), totalkeysRead, totalkeysWritten, dTime));
+        logger_.info(String.format(format, writer.getFilename(), getTotalBytes(sstables), ssTable.length(), totalkeysWritten, dTime));
         return sstables.size();
     }
 
+    private long getTotalBytes(Iterable<SSTableReader> sstables)
+    {
+        long sum = 0;
+        for (SSTableReader sstable : sstables)
+        {
+            sum += sstable.length();
+        }
+        return sum;
+    }
+
     public static List<Memtable> getUnflushedMemtables(String cfName)
     {
         return new ArrayList<Memtable>(getMemtablesPendingFlushNotNull(cfName));
@@ -1341,23 +1186,24 @@
         // sstables
         for (SSTableReader sstable : ssTables_)
         {
-            final SSTableScanner fs = sstable.getScanner();
-            fs.seekTo(startWith);
-            iterators.add(new Iterator<String>()
+            final SSTableScanner scanner = sstable.getScanner();
+            scanner.seekTo(startWith);
+            Iterator<String> iter = new Iterator<String>()
             {
                 public boolean hasNext()
                 {
-                    return fs.hasNext();
+                    return scanner.hasNext();
                 }
                 public String next()
                 {
-                    return fs.next().getKey();
+                    return scanner.next().getKey();
                 }
                 public void remove()
                 {
                     throw new UnsupportedOperationException();
                 }
-            });
+            };
+            iterators.add(iter);
         }
 
         Iterator<String> collated = IteratorUtils.collatedIterator(comparator, iterators);

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Memtable.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Memtable.java?rev=816382&r1=816381&r2=816382&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Memtable.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Memtable.java Thu Sep 17 21:10:09 2009
@@ -153,7 +153,7 @@
         {
             int oldSize = oldCf.size();
             int oldObjectCount = oldCf.getColumnCount();
-            oldCf.addColumns(columnFamily);
+            oldCf.addAll(columnFamily);
             int newSize = oldCf.size();
             int newObjectCount = oldCf.getColumnCount();
             resolveSize(oldSize, newSize);

Added: incubator/cassandra/trunk/src/java/org/apache/cassandra/io/CompactionIterator.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/io/CompactionIterator.java?rev=816382&view=auto
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/io/CompactionIterator.java (added)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/io/CompactionIterator.java Thu Sep 17 21:10:09 2009
@@ -0,0 +1,113 @@
+package org.apache.cassandra.io;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Comparator;
+
+import org.apache.commons.collections.iterators.CollatingIterator;
+
+import org.apache.cassandra.utils.ReducingIterator;
+import org.apache.cassandra.db.ColumnFamily;
+
+public class CompactionIterator extends ReducingIterator<IteratingRow, CompactionIterator.CompactedRow> implements Closeable
+{
+    private final List<IteratingRow> rows = new ArrayList<IteratingRow>();
+
+    @SuppressWarnings("unchecked")
+    public CompactionIterator(Iterable<SSTableReader> sstables) throws IOException
+    {
+        super(getCollatingIterator(sstables));
+    }
+
+    @SuppressWarnings("unchecked")
+    private static CollatingIterator getCollatingIterator(Iterable<SSTableReader> sstables) throws IOException
+    {
+        // CollatingIterator has a bug that causes NPE when you try to use default comparator. :(
+        CollatingIterator iter = new CollatingIterator(new Comparator()
+        {
+            public int compare(Object o1, Object o2)
+            {
+                return ((Comparable)o1).compareTo(o2);
+            }
+        });
+        for (SSTableReader sstable : sstables)
+        {
+            iter.addIterator(sstable.getScanner());
+        }
+        return iter;
+    }
+
+    @Override
+    protected boolean isEqual(IteratingRow o1, IteratingRow o2)
+    {
+        return o1.getKey().equals(o2.getKey());
+    }
+
+    public void reduce(IteratingRow current)
+    {
+        rows.add(current);
+    }
+
+    protected CompactedRow getReduced()
+    {
+        try
+        {
+            return getReducedRaw();
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    protected CompactedRow getReducedRaw() throws IOException
+    {
+        DataOutputBuffer buffer = new DataOutputBuffer();
+        String key = rows.get(0).getKey();
+        if (rows.size() > 1)
+        {
+            ColumnFamily cf = null;
+            for (IteratingRow row : rows)
+            {
+                if (cf == null)
+                {
+                    cf = row.getColumnFamily();
+                }
+                else
+                {
+                    cf.addAll(row.getColumnFamily());
+                }
+            }
+            ColumnFamily.serializer().serializeWithIndexes(cf, buffer);
+        }
+        else
+        {
+            assert rows.size() == 1;
+            rows.get(0).echoData(buffer);
+        }
+        rows.clear();
+        return new CompactedRow(key, buffer);
+    }
+
+    public void close() throws IOException
+    {
+        for (Object o : ((CollatingIterator)source).getIterators())
+        {
+            ((SSTableScanner)o).close();
+        }
+    }
+
+    public static class CompactedRow
+    {
+        public final String key;
+        public final DataOutputBuffer buffer;
+
+        public CompactedRow(String key, DataOutputBuffer buffer)
+        {
+            this.key = key;
+            this.buffer = buffer;
+        }
+    }
+}

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/io/IteratingRow.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/io/IteratingRow.java?rev=816382&r1=816381&r2=816382&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/io/IteratingRow.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/io/IteratingRow.java Thu Sep 17 21:10:09 2009
@@ -37,7 +37,6 @@
 {
     private final String key;
     private final long finishedAt;
-    private final ColumnFamily emptyColumnFamily;
     private final BufferedRandomAccessFile file;
     private SSTableReader sstable;
     private long dataStart;
@@ -51,10 +50,6 @@
         int dataSize = file.readInt();
         dataStart = file.getFilePointer();
         finishedAt = dataStart + dataSize;
-        // legacy stuff to support FileStruct:
-        IndexHelper.skipBloomFilterAndIndex(file);
-        emptyColumnFamily = ColumnFamily.serializer().deserializeFromSSTableNoColumns(sstable.makeColumnFamily(), file);
-        file.readInt();
     }
 
     public String getKey()
@@ -62,11 +57,6 @@
         return key;
     }
 
-    public ColumnFamily getEmptyColumnFamily()
-    {
-        return emptyColumnFamily;
-    }
-
     public void echoData(DataOutput out) throws IOException
     {
         file.seek(dataStart);

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=816382&r1=816381&r2=816382&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 Thu Sep 17 21:10:09 2009
@@ -333,11 +333,6 @@
         return partitioner;
     }
 
-    public FileStruct getFileStruct() throws IOException
-    {
-        return new FileStruct(this);
-    }
-
     public SSTableScanner getScanner() throws IOException
     {
         return new SSTableScanner(this);

Modified: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java?rev=816382&r1=816381&r2=816382&view=diff
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java (original)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java Thu Sep 17 21:10:09 2009
@@ -20,8 +20,6 @@
 
 import java.io.IOException;
 import java.util.Arrays;
-import java.util.HashSet;
-import java.util.Random;
 import java.util.TreeMap;
 
 import org.junit.Test;
@@ -125,8 +123,8 @@
         cf_old.addColumn(QueryPath.column("col2".getBytes()), val2, 1);
         cf_old.addColumn(QueryPath.column("col3".getBytes()), val2, 2);
 
-        cf_result.addColumns(cf_new);
-        cf_result.addColumns(cf_old);
+        cf_result.addAll(cf_new);
+        cf_result.addAll(cf_old);
 
         assert 3 == cf_result.getColumnCount() : "Count is " + cf_new.getColumnCount();
         //addcolumns will only add if timestamp >= old timestamp