[ https://issues.apache.org/jira/browse/CASSANDRA-3638?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Sylvain Lebresne updated CASSANDRA-3638: ---------------------------------------- Attachment: 3638.patch I think you're right. And this go way back I believe, we've never used the stopAt bound to stop iteration early in the range slice case. Patch attached to fix this (against trunk). > It may iterate the whole memtable while just query one row . This seriously > affect the performance . of Cassandra > ------------------------------------------------------------------------------------------------------------------ > > Key: CASSANDRA-3638 > URL: https://issues.apache.org/jira/browse/CASSANDRA-3638 > Project: Cassandra > Issue Type: Bug > Components: Core > Affects Versions: 1.0.0 > Reporter: MaHaiyang > Assignee: Sylvain Lebresne > Attachments: 3638.patch > > > RangeSliceVerbHandler may just only query one row , but cassandra may > iterate the whole memtable . > the problem is in ColumnFamilyStore.getRangeSlice() method . > {color:red} // this iterator may iterate the whole memtable!!{color} > {code:title=ColumnFamilyStore.java|borderStyle=solid} > public List<Row> getRangeSlice(ByteBuffer superColumn, final AbstractBounds > range, int maxResults, IFilter columnFilter) > throws ExecutionException, InterruptedException > { > ... > DecoratedKey startWith = new DecoratedKey(range.left, null); > DecoratedKey stopAt = new DecoratedKey(range.right, null); > QueryFilter filter = new QueryFilter(null, new > QueryPath(columnFamily, superColumn, null), columnFilter); > int gcBefore = (int)(System.currentTimeMillis() / 1000) - > metadata.getGcGraceSeconds(); > List<Row> rows; > ViewFragment view = markReferenced(startWith, stopAt); > try > { > CloseableIterator<Row> iterator = > RowIteratorFactory.getIterator(view.memtables, view.sstables, startWith, > stopAt, filter, getComparator(), this); > rows = new ArrayList<Row>(); > try > { > // pull rows out of the iterator > boolean first = true; > while (iterator.hasNext()) // this iterator may iterate the > whole memtable!! > { > .... > } > } > ..... > } > ..... > return rows; > } > {code} > {color:red} // Just only query one row ,but returned a sublist of > columnFamiles {color} > {code:title=Memtable.java|borderStyle=solid} > // Just only query one row ,but returned a sublist of columnFamiles > public Iterator<Map.Entry<DecoratedKey, ColumnFamily>> > getEntryIterator(DecoratedKey startWith) > { > return columnFamilies.tailMap(startWith).entrySet().iterator(); > } > {code} > {color:red} // entry.getKey() will never bigger or equal to startKey, and > then iterate the whole sublist of memtable {color} > {code:title=RowIteratorFactory.java|borderStyle=solid} > public IColumnIterator computeNext() > { > while (iter.hasNext()) > { > Map.Entry<DecoratedKey, ColumnFamily> entry = iter.next(); > IColumnIterator ici = > filter.getMemtableColumnIterator(entry.getValue(), entry.getKey(), > comparator); > // entry.getKey() will never bigger or equal to startKey, and > then iterate the whole sublist of memtable > if (pred.apply(ici)) > return ici; > } > return endOfData(); > {code} -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira