[
https://issues.apache.org/jira/browse/CASSANDRA-2843?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13059947#comment-13059947
]
Jonathan Ellis commented on CASSANDRA-2843:
-------------------------------------------
bq. a TreeMap based implementation is slightly slower than the ArrayBacked one
but not by a lot
If the main benefit is avoiding synchronization, shouldn't we just stick w/
TreeMap for simplicity?
bq. That is, if you only add columns with different names, the ArrayBacked
implementation is faster, but not dramatically so
That's odd, because adding in sorted order is actually worst-case for CSLM, and
that's what we do on reads. I would expect we could do better than just the
synchronization difference. Did you try wide slices too?
FWIW, back in CASSANDRA-633 I tried something similar on the write path, with
the additional difference that it would only sort lazily (i.e. on flush, for an
insert-mostly workload). That didn't seem to be much improvement either.
> better performance on long row read
> -----------------------------------
>
> Key: CASSANDRA-2843
> URL: https://issues.apache.org/jira/browse/CASSANDRA-2843
> Project: Cassandra
> Issue Type: New Feature
> Reporter: Yang Yang
> Attachments: 2843.patch, fast_cf_081_trunk.diff
>
>
> currently if a row contains > 1000 columns, the run time becomes considerably
> slow (my test of
> a row with 30 00 columns (standard, regular) each with 8 bytes in name, and
> 40 bytes in value, is about 16ms.
> this is all running in memory, no disk read is involved.
> through debugging we can find
> most of this time is spent on
> [Wall Time] org.apache.cassandra.db.Table.getRow(QueryFilter)
> [Wall Time]
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(QueryFilter,
> ColumnFamily)
> [Wall Time]
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(QueryFilter, int,
> ColumnFamily)
> [Wall Time]
> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(QueryFilter,
> int, ColumnFamily)
> [Wall Time]
> org.apache.cassandra.db.filter.QueryFilter.collectCollatedColumns(ColumnFamily,
> Iterator, int)
> [Wall Time]
> org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns(IColumnContainer,
> Iterator, int)
> [Wall Time] org.apache.cassandra.db.ColumnFamily.addColumn(IColumn)
> ColumnFamily.addColumn() is slow because it inserts into an internal
> concurrentSkipListMap() that maps column names to values.
> this structure is slow for two reasons: it needs to do synchronization; it
> needs to maintain a more complex structure of map.
> but if we look at the whole read path, thrift already defines the read output
> to be List<ColumnOrSuperColumn> so it does not make sense to use a luxury map
> data structure in the interium and finally convert it to a list. on the
> synchronization side, since the return CF is never going to be
> shared/modified by other threads, we know the access is always single thread,
> so no synchronization is needed.
> but these 2 features are indeed needed for ColumnFamily in other cases,
> particularly write. so we can provide a different ColumnFamily to
> CFS.getTopLevelColumnFamily(), so getTopLevelColumnFamily no longer always
> creates the standard ColumnFamily, but take a provided returnCF, whose cost
> is much cheaper.
> the provided patch is for demonstration now, will work further once we agree
> on the general direction.
> CFS, ColumnFamily, and Table are changed; a new FastColumnFamily is
> provided. the main work is to let the FastColumnFamily use an array for
> internal storage. at first I used binary search to insert new columns in
> addColumn(), but later I found that even this is not necessary, since all
> calling scenarios of ColumnFamily.addColumn() has an invariant that the
> inserted columns come in sorted order (I still have an issue to resolve
> descending or ascending now, but ascending works). so the current logic is
> simply to compare the new column against the end column in the array, if
> names not equal, append, if equal, reconcile.
> slight temporary hacks are made on getTopLevelColumnFamily so we have 2
> flavors of the method, one accepting a returnCF. but we could definitely
> think about what is the better way to provide this returnCF.
> this patch compiles fine, no tests are provided yet. but I tested it in my
> application, and the performance improvement is dramatic: it offers about 50%
> reduction in read time in the 3000-column case.
> thanks
> Yang
--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira