cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Sylvain Lebresne (JIRA)" <j...@apache.org>
Subject [jira] [Updated] (CASSANDRA-2843) better performance on long row read
Date Tue, 05 Jul 2011 15:03:16 GMT

     [ https://issues.apache.org/jira/browse/CASSANDRA-2843?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]

Sylvain Lebresne updated CASSANDRA-2843:
----------------------------------------

    Attachment: 2843.patch

I do think that using a specific implementation for the backing map of a ColumnFamily during
is a good idea. It is clear that avoiding synchronization will be faster, and given the type
of operations we do during reads (insertion in sorted order and iteration), an ArrayList backed
solution is sure to be faster too. I will also be much gentle on the GC that the linked list
ConcurrentSkipListMap uses. I think that all those will help even with relatively small reads.
So let's focus on that for this ticket and let other potential improvement to other ticket,
especially if it is unclear they bear any noticeable speedup.

So focusing on the patch itself:
* We really shouldn't "simply" extend ColumnFamily as the patch does. This is quite frankly
ugly and will be a maintenance nightmare (you'll have to check you did overwrite every function
that touch the map (which is not the case in the patch) and every update to ColumnFamily have
to be aware that it should update FastColumnFamily as well).
* The implementation of FastColumnFamily should really be a fully functionnal ColumnFamily
implementation (albeit not synchronized). That is, we can't assume that addition will always
be in strict increasing order, otherwise again this will be too hard to use.
* The addAll function can be optimized given that both input are sorted.  Granted, I don't
think it is used in the read path, but I think that the new ColumnFamily implementation could
advantageously be used during compaction (by preCompactedRow typically, and possibly other
places where concurrent access is not an issue) where this would matter.

Attaching a version of the patch (2843.patch) that tries to address all the remarks above.
The patch is against trunk (not 0.8 branch), because it build on the recently committed refactor
of ColumnFamily. It refactors ColumnFamily (AbstractColumnContainer actually) to allow for
a pluggable backing column map. The ConcurrentSkipListMap implemn is name ThreadSafeColumnMap
and the new one is called ArrayBackedColumnMap (which I prefer to FastSomething since it's
not a very helpful name).

On the ColumnFamilyStore side, instead of feeding the returnCF to getTopLevelColumns, I pass
along a factory (that each backing implementation provides). The main goal was to avoid creating
a columnFamily when it's useless (if row cache is enabled on the CF -- btw, this ticket only
improve on read for column family with no cache).

Micro benchmarks does show that on the operation involved during a read (addition of column
+ iteration), the ArrayBacked implementation is faster than the ConcurrentSkipListMap based
one. Interestingly though, this is mainly true when some reconciliation of columns happens.
That is, if you only add columns with different names, the ArrayBacked implementation is faster,
but not dramatically so. If you start adding column that have to be resolved, the ArrayBacked
implementation becomes much faster, even with a reasonably small number of columns (inserting
100 columns with only 10 unique column names, the ArrayBacked is already >30% faster).
And this mostly due to the overhead of synchronization (of replace()): a TreeMap based implementation
is slightly slower than the ArrayBacked one but not by a lot and thus is much faster than
the ConcurrentSkipListMap implementation.

The attached patch should be ready for review (though it could probably use a few unit test
for the new ArrayBacked implementation).


> 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

        

Mime
View raw message