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/11/04 18:07:43 UTC
svn commit: r832797 -
/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
Author: jbellis
Date: Wed Nov 4 17:07:42 2009
New Revision: 832797
URL: http://svn.apache.org/viewvc?rev=832797&view=rev
Log:
don't removeDeleted on the whole CF before filtering what the request was for; it's expensive. also, fixes subcolumn queries being counted twice in readStats
patch by jbellis; reviewed by gdusbabek for CASSANDRA-510
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
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=832797&r1=832796&r2=832797&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 Wed Nov 4 17:07:42 2009
@@ -1102,31 +1102,41 @@
assert columnFamily_.equals(filter.getColumnFamilyName());
long start = System.currentTimeMillis();
+ try
+ {
+ // if we are querying subcolumns of a supercolumn, fetch the supercolumn with NQF, then filter in-memory.
+ if (filter.path.superColumnName != null)
+ {
+ QueryFilter nameFilter = new NamesQueryFilter(filter.key, new QueryPath(columnFamily_), filter.path.superColumnName);
+ ColumnFamily cf = getColumnFamilyInternal(nameFilter, getDefaultGCBefore());
+ if (cf == null || cf.getColumnCount() == 0)
+ return cf;
+
+ assert cf.getSortedColumns().size() == 1;
+ SuperColumn sc = (SuperColumn)cf.getSortedColumns().iterator().next();
+ SuperColumn scFiltered = filter.filterSuperColumn(sc, gcBefore);
+ ColumnFamily cfFiltered = cf.cloneMeShallow();
+ cfFiltered.addColumn(scFiltered);
+ return removeDeleted(cfFiltered, gcBefore);
+ }
- // if we are querying subcolumns of a supercolumn, fetch the supercolumn with NQF, then filter in-memory.
- if (filter.path.superColumnName != null)
+ return removeDeleted(getColumnFamilyInternal(filter, gcBefore), gcBefore);
+ }
+ finally
{
- QueryFilter nameFilter = new NamesQueryFilter(filter.key, new QueryPath(columnFamily_), filter.path.superColumnName);
- ColumnFamily cf = getColumnFamily(nameFilter);
- if (cf == null || cf.getColumnCount() == 0)
- return cf;
-
- assert cf.getSortedColumns().size() == 1;
- SuperColumn sc = (SuperColumn)cf.getSortedColumns().iterator().next();
- SuperColumn scFiltered = filter.filterSuperColumn(sc, gcBefore);
- ColumnFamily cfFiltered = cf.cloneMeShallow();
- cfFiltered.addColumn(scFiltered);
readStats_.add(System.currentTimeMillis() - start);
- return cfFiltered;
}
+ }
+ private ColumnFamily getColumnFamilyInternal(QueryFilter filter, int gcBefore) throws IOException
+ {
// we are querying top-level columns, do a merging fetch with indexes.
List<ColumnIterator> iterators = new ArrayList<ColumnIterator>();
try
{
final ColumnFamily returnCF;
ColumnIterator iter;
-
+
/* add the current memtable */
Table.flusherLock_.readLock().lock();
try
@@ -1137,7 +1147,7 @@
finally
{
Table.flusherLock_.readLock().unlock();
- }
+ }
iterators.add(iter);
/* add the memtables being flushed */
@@ -1166,8 +1176,7 @@
return null;
filter.collectCollatedColumns(returnCF, collated, gcBefore);
-
- return removeDeleted(returnCF, gcBefore); // collect does a first pass but doesn't try to recognize e.g. the entire CF being tombstoned
+ return returnCF;
}
finally
{
@@ -1183,8 +1192,6 @@
logger_.error("error closing " + ci, th);
}
}
-
- readStats_.add(System.currentTimeMillis() - start);
}
}