cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jbel...@apache.org
Subject [1/3] git commit: merge from 1.1
Date Tue, 04 Sep 2012 19:08:39 GMT
Updated Branches:
  refs/heads/cassandra-1.1 5c91bd142 -> e1b10590e
  refs/heads/trunk 769fe895a -> 6c8687237


merge from 1.1


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6c868723
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6c868723
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6c868723

Branch: refs/heads/trunk
Commit: 6c8687237cf58fac21078a27ffb32dda4cca5b01
Parents: 769fe89 e1b1059
Author: Jonathan Ellis <jbellis@apache.org>
Authored: Tue Sep 4 14:07:46 2012 -0500
Committer: Jonathan Ellis <jbellis@apache.org>
Committed: Tue Sep 4 14:07:46 2012 -0500

----------------------------------------------------------------------
 CHANGES.txt                                        |    1 +
 .../apache/cassandra/db/CollationController.java   |   18 ++++++++++----
 .../db/columniterator/ISSTableColumnIterator.java  |    8 ++++++
 .../db/columniterator/SSTableNamesIterator.java    |    7 +++++-
 .../db/columniterator/SSTableSliceIterator.java    |   10 +++++++-
 .../org/apache/cassandra/db/filter/IFilter.java    |    9 +++++--
 .../cassandra/db/filter/NamesQueryFilter.java      |    9 +++++--
 .../apache/cassandra/db/filter/QueryFilter.java    |    7 +++--
 .../cassandra/db/filter/SliceQueryFilter.java      |    5 ++-
 9 files changed, 56 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c868723/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index d6b40ba,6c504e8..38678dd
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,55 -1,5 +1,56 @@@
 +1.2-dev
 + * optimize WriteResponse (CASSANDRA-4548)
 + * new metrics (CASSANDRA-4009)
 + * redesign KEYS indexes to avoid read-before-write (CASSANDRA-2897)
 + * debug tracing (CASSANDRA-1123)
 + * parallelize row cache loading (CASSANDRA-4282)
 + * Make compaction, flush JBOD-aware (CASSANDRA-4292)
 + * run local range scans on the read stage (CASSANDRA-3687)
 + * clean up ioexceptions (CASSANDRA-2116)
 + * add disk_failure_policy (CASSANDRA-2118)
 + * Introduce new json format with row level deletion (CASSANDRA-4054)
 + * remove redundant "name" column from schema_keyspaces (CASSANDRA-4433)
 + * improve "nodetool ring" handling of multi-dc clusters (CASSANDRA-3047)
 + * update NTS calculateNaturalEndpoints to be O(N log N) (CASSANDRA-3881)
 + * add UseCondCardMark XX jvm settings on jdk 1.7 (CASSANDRA-4366)
 + * split up rpc timeout by operation type (CASSANDRA-2819)
 + * rewrite key cache save/load to use only sequential i/o (CASSANDRA-3762)
 + * update MS protocol with a version handshake + broadcast address id
 +   (CASSANDRA-4311)
 + * multithreaded hint replay (CASSANDRA-4189)
 + * add inter-node message compression (CASSANDRA-3127)
 + * remove COPP (CASSANDRA-2479)
 + * Track tombstone expiration and compact when tombstone content is
 +   higher than a configurable threshold, default 20% (CASSANDRA-3442, 4234)
 + * update MurmurHash to version 3 (CASSANDRA-2975)
 + * (CLI) track elapsed time for `delete' operation (CASSANDRA-4060)
 + * (CLI) jline version is bumped to 1.0 to properly  support
 +   'delete' key function (CASSANDRA-4132)
 + * Save IndexSummary into new SSTable 'Summary' component (CASSANDRA-2392, 4289)
 + * Add support for range tombstones (CASSANDRA-3708)
 + * Improve MessagingService efficiency (CASSANDRA-3617)
 + * Avoid ID conflicts from concurrent schema changes (CASSANDRA-3794)
 + * Set thrift HSHA server thread limit to unlimited by default (CASSANDRA-4277)
 + * Avoids double serialization of CF id in RowMutation messages
 +   (CASSANDRA-4293)
 + * stream compressed sstables directly with java nio (CASSANDRA-4297)
 + * Support multiple ranges in SliceQueryFilter (CASSANDRA-3885)
 + * Add column metadata to system column families (CASSANDRA-4018)
 + * (cql3) Always use composite types by default (CASSANDRA-4329)
 + * (cql3) Add support for set, map and list (CASSANDRA-3647)
 + * Validate date type correctly (CASSANDRA-4441)
 + * (cql3) Allow definitions with only a PK (CASSANDRA-4361)
 + * (cql3) Add support for row key composites (CASSANDRA-4179)
 + * improve DynamicEndpointSnitch by using reservoir sampling (CASSANDRA-4038)
 + * (cql3) Add support for 2ndary indexes (CASSANDRA-3680)
 + * (cql3) fix defining more than one PK to be invalid (CASSANDRA-4477)
 + * remove schema agreement checking from all external APIs (Thrift, CQL and CQL3) (CASSANDRA-4487)
 + * add Murmur3Partitioner and make it default for new installations (CASSANDRA-3772)
 + * (cql3) update pseudo-map syntax to use map syntax (CASSANDRA-4497)
 +
 +
  1.1.5
+  * fix FD leak in slice queries (CASSANDRA-4571)
   * avoid recursion in leveled compaction (CASSANDRA-4587)
   * increase stack size under Java7 to 180K
   * Log(info) schema changes (CASSANDRA-4547)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c868723/src/java/org/apache/cassandra/db/CollationController.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/CollationController.java
index f4f06df,fb9674f..03876d2
--- a/src/java/org/apache/cassandra/db/CollationController.java
+++ b/src/java/org/apache/cassandra/db/CollationController.java
@@@ -25,7 -28,8 +25,8 @@@ import com.google.common.collect.Maps
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
 -import org.apache.cassandra.db.columniterator.IColumnIterator;
 +import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+ import org.apache.cassandra.db.columniterator.ISSTableColumnIterator;
  import org.apache.cassandra.db.columniterator.SimpleAbstractColumnIterator;
  import org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy;
  import org.apache.cassandra.db.filter.NamesQueryFilter;
@@@ -232,9 -244,8 +233,8 @@@ public class CollationControlle
                      iterators.add(iter);
                  }
              }
 -            
 +
              long mostRecentRowTombstone = Long.MIN_VALUE;
-             Map<OnDiskAtomIterator, Long> iteratorMaxTimes = Maps.newHashMapWithExpectedSize(view.sstables.size());
              for (SSTableReader sstable : view.sstables)
              {
                  // if we've already seen a row tombstone with a timestamp greater 
@@@ -242,8 -253,8 +242,8 @@@
                  if (sstable.getMaxTimestamp() < mostRecentRowTombstone)
                      continue;
  
 -                IColumnIterator iter = filter.getSSTableColumnIterator(sstable);
 +                OnDiskAtomIterator iter = filter.getSSTableColumnIterator(sstable);
-                 iteratorMaxTimes.put(iter, sstable.getMaxTimestamp());
+                 iterators.add(iter);
                  if (iter.getColumnFamily() != null)
                  {
                      ColumnFamily cf = iter.getColumnFamily();
@@@ -258,10 -269,19 +258,18 @@@
              // If we saw a row tombstone, do a second pass through the iterators we
              // obtained from the sstables and drop any whose maxTimestamp < that of the
              // row tombstone
-             for (Map.Entry<OnDiskAtomIterator, Long> entry : iteratorMaxTimes.entrySet())
 -            if (mostRecentRowTombstone > Long.MIN_VALUE)
              {
-                 if (entry.getValue() >= mostRecentRowTombstone)
-                     iterators.add(entry.getKey());
 -                Iterator<IColumnIterator> it = iterators.iterator();
++                Iterator<OnDiskAtomIterator> it = iterators.iterator();
+                 while (it.hasNext())
+                 {
 -                    IColumnIterator iter = it.next();
++                    OnDiskAtomIterator iter = it.next();
+                     if ((iter instanceof ISSTableColumnIterator)
+                         && ((ISSTableColumnIterator) iter).getSStable().getMaxTimestamp()
< mostRecentRowTombstone)
+                     {
+                         FileUtils.closeQuietly(iter);
+                         it.remove();
+                     }
+                 }
              }
  
              // we need to distinguish between "there is no data at all for this row" (BF
will let us rebuild that efficiently)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c868723/src/java/org/apache/cassandra/db/columniterator/ISSTableColumnIterator.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/columniterator/ISSTableColumnIterator.java
index 0000000,4da4c0a..a9c072d
mode 000000,100644..100644
--- a/src/java/org/apache/cassandra/db/columniterator/ISSTableColumnIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/ISSTableColumnIterator.java
@@@ -1,0 -1,8 +1,8 @@@
+ package org.apache.cassandra.db.columniterator;
+ 
+ import org.apache.cassandra.io.sstable.SSTableReader;
+ 
 -public interface ISSTableColumnIterator extends IColumnIterator
++public interface ISSTableColumnIterator extends OnDiskAtomIterator
+ {
+     public SSTableReader getSStable();
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c868723/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java
index 2cb3435,23036df..3ba5143
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java
@@@ -39,14 -43,15 +39,14 @@@ import org.apache.cassandra.io.util.Fil
  import org.apache.cassandra.utils.ByteBufferUtil;
  import org.apache.cassandra.utils.Filter;
  
- public class SSTableNamesIterator extends SimpleAbstractColumnIterator implements OnDiskAtomIterator
+ public class SSTableNamesIterator extends SimpleAbstractColumnIterator implements ISSTableColumnIterator
  {
 -    private static Logger logger = LoggerFactory.getLogger(SSTableNamesIterator.class);
 -
      private ColumnFamily cf;
 -    private Iterator<IColumn> iter;
 +    private final SSTableReader sstable;
 +    private FileDataInput fileToClose;
 +    private Iterator<OnDiskAtom> iter;
      public final SortedSet<ByteBuffer> columns;
      public final DecoratedKey key;
 -    private final SSTableReader sstable;
  
      public SSTableNamesIterator(SSTableReader sstable, DecoratedKey key, SortedSet<ByteBuffer>
columns)
      {
@@@ -93,67 -102,27 +93,72 @@@
          }
      }
  
 +    private FileDataInput createFileDataInput(long position)
 +    {
 +        fileToClose = sstable.getFileDataInput(position);
 +        return fileToClose;
 +    }
 +
+     public SSTableReader getSStable()
+     {
+         return sstable;
+     }
+ 
 -    private void read(SSTableReader sstable, FileDataInput file)
 +    private void read(SSTableReader sstable, FileDataInput file, RowIndexEntry indexEntry)
      throws IOException
      {
 -        Filter bf = IndexHelper.defreezeBloomFilter(file, sstable.descriptor.usesOldBloomFilter);
 -        List<IndexHelper.IndexInfo> indexList = IndexHelper.deserializeIndex(file);
 +        Filter bf;
 +        List<IndexHelper.IndexInfo> indexList;
 +
 +        // If the entry is not indexed or the index is not promoted, read from the row start
 +        if (!indexEntry.isIndexed())
 +        {
 +            if (file == null)
 +                file = createFileDataInput(indexEntry.position);
 +            else
 +                file.seek(indexEntry.position);
  
 -        // we can stop early if bloom filter says none of the columns actually exist --
but,
 -        // we can't stop before initializing the cf above, in case there's a relevant tombstone
 -        ColumnFamilySerializer serializer = ColumnFamily.serializer();
 -        try {
 -            cf = serializer.deserializeFromSSTableNoColumns(ColumnFamily.create(sstable.metadata),
file);
 -        } catch (Exception e) {
 -            throw new IOException
 -                (serializer + " failed to deserialize " + sstable.getColumnFamilyName()
+ " with " + sstable.metadata + " from " + file, e);
 +            DecoratedKey keyInDisk = SSTableReader.decodeKey(sstable.partitioner,
 +                                                             sstable.descriptor,
 +                                                             ByteBufferUtil.readWithShortLength(file));
 +            assert keyInDisk.equals(key) : String.format("%s != %s in %s", keyInDisk, key,
file.getPath());
 +            SSTableReader.readRowSize(file, sstable.descriptor);
          }
  
 +        if (sstable.descriptor.version.hasPromotedIndexes)
 +        {
 +            bf = indexEntry.isIndexed() ? indexEntry.bloomFilter() : null;
 +            indexList = indexEntry.columnsIndex();
 +        }
 +        else
 +        {
 +            assert file != null;
 +            bf = IndexHelper.defreezeBloomFilter(file, sstable.descriptor.version.filterType);
 +            indexList = IndexHelper.deserializeIndex(file);
 +        }
 +
 +        if (!indexEntry.isIndexed())
 +        {
 +            // we can stop early if bloom filter says none of the columns actually exist
-- but,
 +            // we can't stop before initializing the cf above, in case there's a relevant
tombstone
 +            ColumnFamilySerializer serializer = ColumnFamily.serializer;
 +            try
 +            {
 +                cf = ColumnFamily.create(sstable.metadata);
 +                cf.delete(DeletionInfo.serializer().deserializeFromSSTable(file, sstable.descriptor.version));
 +            }
 +            catch (Exception e)
 +            {
 +                throw new IOException(serializer + " failed to deserialize " + sstable.getColumnFamilyName()
+ " with " + sstable.metadata + " from " + file, e);
 +            }
 +        }
 +        else
 +        {
 +            cf = ColumnFamily.create(sstable.metadata);
 +            cf.delete(indexEntry.deletionInfo());
 +        }
 +
 +        List<OnDiskAtom> result = new ArrayList<OnDiskAtom>();
          List<ByteBuffer> filteredColumnNames = new ArrayList<ByteBuffer>(columns.size());
          for (ByteBuffer name : columns)
          {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c868723/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java
index 1dde4c9,38335f4..bbd0f74
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java
@@@ -30,16 -36,37 +30,18 @@@ import org.apache.cassandra.io.util.Fil
  /**
   *  A Column Iterator over SSTable
   */
- public class SSTableSliceIterator implements OnDiskAtomIterator
+ public class SSTableSliceIterator implements ISSTableColumnIterator
  {
 -    private final FileDataInput fileToClose;
 -    private IColumnIterator reader;
 +    private final OnDiskAtomIterator reader;
 +    private final DecoratedKey key;
+     private final SSTableReader sstable;
 -    private DecoratedKey key;
  
 -    public SSTableSliceIterator(SSTableReader sstable, DecoratedKey key, ByteBuffer startColumn,
ByteBuffer finishColumn, boolean reversed)
 +    public SSTableSliceIterator(SSTableReader sstable, DecoratedKey key, ColumnSlice[] slices,
boolean reversed)
      {
+         this.sstable = sstable;
          this.key = key;
 -        fileToClose = sstable.getFileDataInput(this.key);
 -        if (fileToClose == null)
 -            return;
 -
 -        try
 -        {
 -            DecoratedKey keyInDisk = SSTableReader.decodeKey(sstable.partitioner,
 -                                                             sstable.descriptor,
 -                                                             ByteBufferUtil.readWithShortLength(fileToClose));
 -            assert keyInDisk.equals(key)
 -                   : String.format("%s != %s in %s", keyInDisk, key, fileToClose.getPath());
 -            SSTableReader.readRowSize(fileToClose, sstable.descriptor);
 -        }
 -        catch (IOException e)
 -        {
 -            sstable.markSuspect();
 -            throw new IOError(e);
 -        }
 -
 -        reader = createReader(sstable, fileToClose, startColumn, finishColumn, reversed);
 +        RowIndexEntry indexEntry = sstable.getPosition(key, SSTableReader.Operator.EQ);
 +        this.reader = indexEntry == null ? null : createReader(sstable, indexEntry, null,
slices, reversed);
      }
  
      /**
@@@ -54,19 -81,26 +56,25 @@@
       * @param finishColumn The end of the slice
       * @param reversed Results are returned in reverse order iff reversed is true.
       */
 -    public SSTableSliceIterator(SSTableReader sstable, FileDataInput file, DecoratedKey
key, ByteBuffer startColumn, ByteBuffer finishColumn, boolean reversed)
 +    public SSTableSliceIterator(SSTableReader sstable, FileDataInput file, DecoratedKey
key, ColumnSlice[] slices, boolean reversed, RowIndexEntry indexEntry)
      {
+         this.sstable = sstable;
          this.key = key;
 -        fileToClose = null;
 -        reader = createReader(sstable, file, startColumn, finishColumn, reversed);
 +        reader = createReader(sstable, indexEntry, file, slices, reversed);
      }
  
 -    private static IColumnIterator createReader(SSTableReader sstable, FileDataInput file,
ByteBuffer startColumn, ByteBuffer finishColumn, boolean reversed)
 +    private static OnDiskAtomIterator createReader(SSTableReader sstable, RowIndexEntry
indexEntry, FileDataInput file, ColumnSlice[] slices, boolean reversed)
      {
 -        return startColumn.remaining() == 0 && !reversed
 -                 ? new SimpleSliceReader(sstable, file, finishColumn)
 -                 : new IndexedSliceReader(sstable, file, startColumn, finishColumn, reversed);
 +        return slices.length == 1 && slices[0].start.remaining() == 0 &&
!reversed
 +             ? new SimpleSliceReader(sstable, indexEntry, file, slices[0].finish)
 +             : new IndexedSliceReader(sstable, indexEntry, file, slices, reversed);
      }
  
+     public SSTableReader getSStable()
+     {
+         return sstable;
+     }
+ 
      public DecoratedKey getKey()
      {
          return key;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c868723/src/java/org/apache/cassandra/db/filter/IFilter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/filter/IFilter.java
index 937f98b,355d3c6..309461f
--- a/src/java/org/apache/cassandra/db/filter/IFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/IFilter.java
@@@ -7,24 -9,24 +7,27 @@@
   * "License"); you may not use this file except in compliance
   * with the License.  You may obtain a copy of the License at
   *
 - *   http://www.apache.org/licenses/LICENSE-2.0
 - *
 - * Unless required by applicable law or agreed to in writing,
 - * software distributed under the License is distributed on an
 - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
 - * KIND, either express or implied.  See the License for the
 - * specific language governing permissions and limitations
 - * under the License.
 + *     http://www.apache.org/licenses/LICENSE-2.0
   *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
   */
 +package org.apache.cassandra.db.filter;
  
- import java.io.*;
++import java.io.DataInput;
++import java.io.DataOutput;
++import java.io.IOException;
  import java.util.Comparator;
  import java.util.Iterator;
  
  import org.apache.cassandra.db.*;
 -import org.apache.cassandra.db.columniterator.IColumnIterator;
+ import org.apache.cassandra.db.columniterator.ISSTableColumnIterator;
 +import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
  import org.apache.cassandra.db.marshal.AbstractType;
 +import org.apache.cassandra.io.IVersionedSerializer;
  import org.apache.cassandra.io.sstable.SSTableReader;
  import org.apache.cassandra.io.util.FileDataInput;
  
@@@ -50,13 -52,13 +53,13 @@@ public interface IFilte
       * @param file Already opened file data input, saves us opening another one
       * @param key The key of the row we are about to iterate over
       */
-     public abstract OnDiskAtomIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput
file, DecoratedKey key, RowIndexEntry indexEntry);
 -    public abstract ISSTableColumnIterator getSSTableColumnIterator(SSTableReader sstable,
FileDataInput file, DecoratedKey<?> key);
++    public abstract ISSTableColumnIterator getSSTableColumnIterator(SSTableReader sstable,
FileDataInput file, DecoratedKey key, RowIndexEntry indexEntry);
  
      /**
       * returns an iterator that returns columns from the given SSTable
       * matching the Filter criteria in sorted order.
       */
-     public abstract OnDiskAtomIterator getSSTableColumnIterator(SSTableReader sstable, DecoratedKey
key);
 -    public abstract ISSTableColumnIterator getSSTableColumnIterator(SSTableReader sstable,
DecoratedKey<?> key);
++    public abstract ISSTableColumnIterator getSSTableColumnIterator(SSTableReader sstable,
DecoratedKey key);
  
      /**
       * collects columns from reducedColumns into returnCF.  Termination is determined

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c868723/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java
index f52d9d4,6db0aee..fb8c3aa
--- a/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java
@@@ -7,17 -9,18 +7,19 @@@
   * "License"); you may not use this file except in compliance
   * with the License.  You may obtain a copy of the License at
   *
 - *   http://www.apache.org/licenses/LICENSE-2.0
 - *
 - * Unless required by applicable law or agreed to in writing,
 - * software distributed under the License is distributed on an
 - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
 - * KIND, either express or implied.  See the License for the
 - * specific language governing permissions and limitations
 - * under the License.
 + *     http://www.apache.org/licenses/LICENSE-2.0
   *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
   */
 +package org.apache.cassandra.db.filter;
  
- import java.io.*;
 -
++import java.io.DataInput;
++import java.io.DataOutput;
++import java.io.IOException;
  import java.nio.ByteBuffer;
  import java.util.Comparator;
  import java.util.Iterator;
@@@ -27,13 -29,12 +29,14 @@@ import java.util.TreeSet
  import org.apache.commons.lang.StringUtils;
  
  import org.apache.cassandra.db.*;
 -import org.apache.cassandra.db.columniterator.IColumnIterator;
+ import org.apache.cassandra.db.columniterator.ISSTableColumnIterator;
 +import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
  import org.apache.cassandra.db.columniterator.SSTableNamesIterator;
  import org.apache.cassandra.db.marshal.AbstractType;
 +import org.apache.cassandra.io.IVersionedSerializer;
  import org.apache.cassandra.io.sstable.SSTableReader;
  import org.apache.cassandra.io.util.FileDataInput;
 +import org.apache.cassandra.utils.ByteBufferUtil;
  import org.apache.cassandra.utils.FBUtilities;
  
  public class NamesQueryFilter implements IFilter
@@@ -57,14 -56,14 +60,14 @@@
          return Memtable.getNamesIterator(key, cf, this);
      }
  
-     public OnDiskAtomIterator getSSTableColumnIterator(SSTableReader sstable, DecoratedKey
key)
 -    public ISSTableColumnIterator getSSTableColumnIterator(SSTableReader sstable, DecoratedKey<?>
key)
++    public ISSTableColumnIterator getSSTableColumnIterator(SSTableReader sstable, DecoratedKey
key)
      {
          return new SSTableNamesIterator(sstable, key, columns);
      }
  
-     public OnDiskAtomIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput
file, DecoratedKey key, RowIndexEntry indexEntry)
 -    public ISSTableColumnIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput
file, DecoratedKey<?> key)
++    public ISSTableColumnIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput
file, DecoratedKey key, RowIndexEntry indexEntry)
      {
 -        return new SSTableNamesIterator(sstable, file, key, columns);
 +        return new SSTableNamesIterator(sstable, file, key, columns, indexEntry);
      }
  
      public SuperColumn filterSuperColumn(SuperColumn superColumn, int gcBefore)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c868723/src/java/org/apache/cassandra/db/filter/QueryFilter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/filter/QueryFilter.java
index 786033c,08a4f70..6d073c1
--- a/src/java/org/apache/cassandra/db/filter/QueryFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/QueryFilter.java
@@@ -21,11 -24,18 +21,12 @@@ import java.io.IOException
  import java.nio.ByteBuffer;
  import java.util.*;
  
 -import org.slf4j.Logger;
 -import org.slf4j.LoggerFactory;
 -
  import org.apache.cassandra.db.*;
- import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 -import org.apache.cassandra.db.columniterator.IColumnIterator;
+ import org.apache.cassandra.db.columniterator.ISSTableColumnIterator;
  import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
 -import org.apache.cassandra.db.marshal.AbstractType;
++import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
  import org.apache.cassandra.io.sstable.SSTableReader;
  import org.apache.cassandra.io.util.FileDataInput;
 -import org.apache.cassandra.thrift.SlicePredicate;
 -import org.apache.cassandra.thrift.SliceRange;
  import org.apache.cassandra.utils.CloseableIterator;
  import org.apache.cassandra.utils.MergeIterator;
  
@@@ -68,19 -80,11 +69,19 @@@ public class QueryFilte
          return superFilter.getSSTableColumnIterator(sstable, key);
      }
  
-     public OnDiskAtomIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput
file, DecoratedKey key, RowIndexEntry indexEntry)
 -    public ISSTableColumnIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput
file, DecoratedKey<?> key)
++    public ISSTableColumnIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput
file, DecoratedKey key, RowIndexEntry indexEntry)
      {
          if (path.superColumnName == null)
 -            return filter.getSSTableColumnIterator(sstable, file, key);
 -        return superFilter.getSSTableColumnIterator(sstable, file, key);
 +            return filter.getSSTableColumnIterator(sstable, file, key, indexEntry);
 +        return superFilter.getSSTableColumnIterator(sstable, file, key, indexEntry);
 +    }
 +
 +    public void collateOnDiskAtom(final ColumnFamily returnCF, List<? extends CloseableIterator<OnDiskAtom>>
toCollate, final int gcBefore)
 +    {
 +        List<CloseableIterator<IColumn>> filteredIterators = new ArrayList<CloseableIterator<IColumn>>(toCollate.size());
 +        for (CloseableIterator<OnDiskAtom> iter : toCollate)
 +            filteredIterators.add(gatherTombstones(returnCF, iter));
 +        collateColumns(returnCF, filteredIterators, gcBefore);
      }
  
      // TODO move gcBefore into a field

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6c868723/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
index 9b0e3ef,e749719..6b36f4e
--- a/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
@@@ -31,14 -33,12 +31,15 @@@ import org.slf4j.Logger
  import org.slf4j.LoggerFactory;
  
  import org.apache.cassandra.db.*;
 -import org.apache.cassandra.db.columniterator.IColumnIterator;
 +import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+ import org.apache.cassandra.db.columniterator.ISSTableColumnIterator;
  import org.apache.cassandra.db.columniterator.SSTableSliceIterator;
  import org.apache.cassandra.db.marshal.AbstractType;
 +import org.apache.cassandra.db.marshal.CompositeType;
 +import org.apache.cassandra.io.IVersionedSerializer;
  import org.apache.cassandra.io.sstable.SSTableReader;
  import org.apache.cassandra.io.util.FileDataInput;
 +import org.apache.cassandra.net.MessagingService;
  
  public class SliceQueryFilter implements IFilter
  {
@@@ -88,14 -62,14 +89,14 @@@
          return Memtable.getSliceIterator(key, cf, this);
      }
  
-     public OnDiskAtomIterator getSSTableColumnIterator(SSTableReader sstable, DecoratedKey
key)
 -    public ISSTableColumnIterator getSSTableColumnIterator(SSTableReader sstable, DecoratedKey<?>
key)
++    public ISSTableColumnIterator getSSTableColumnIterator(SSTableReader sstable, DecoratedKey
key)
      {
 -        return new SSTableSliceIterator(sstable, key, start, finish, reversed);
 +        return new SSTableSliceIterator(sstable, key, slices, reversed);
      }
  
-     public OnDiskAtomIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput
file, DecoratedKey key, RowIndexEntry indexEntry)
 -    public ISSTableColumnIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput
file, DecoratedKey<?> key)
++    public ISSTableColumnIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput
file, DecoratedKey key, RowIndexEntry indexEntry)
      {
 -        return new SSTableSliceIterator(sstable, file, key, start, finish, reversed);
 +        return new SSTableSliceIterator(sstable, file, key, slices, reversed, indexEntry);
      }
  
      public SuperColumn filterSuperColumn(SuperColumn superColumn, int gcBefore)


Mime
View raw message