[
https://issues.apache.org/jira/browse/CASSANDRA-4340?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13403131#comment-13403131
]
Ivan Ganza commented on CASSANDRA-4340:
---------------------------------------
We didn't use row caching on the Quotes CF prior to the upgrade and the
response times were acceptable. We did try enabling row caching on the Quotes
CF after the upgrade. Because Quotes has wide rows, the
SerializingCacheProvider did not help much and resulted in instability. The
ConcurrentLinkedHashmap provider does help, but the cache misses are still in
the 2 second range.
Normally, we query recent data in the Quotes CF. In fact, it is possible to
truncate this CF daily because the older data in it is not useful. On past
versions of Cassandra, the data for our column slices appears to come from
memtables almost exclusively. On 1.1.1, we get great performance right up to
the point in which the first sstable is written to disk. Then query performance
(on cache misses if caching is enabled) degrades gradually throughout the day.
We suspected an issue with the bloom filters, so we tried running nodetool
upgradesstables, but it did not help query performance.
> Cassandra upgrade to 1.1.1 resulted in slow query issue
> -------------------------------------------------------
>
> Key: CASSANDRA-4340
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4340
> Project: Cassandra
> Issue Type: Bug
> Components: Core
> Affects Versions: 1.1.1
> Environment: Ubuntu Linux, Java 7, Hector 1.0-1
> Reporter: Ivan Ganza
> Assignee: Pavel Yaskevich
> Fix For: 1.1.2
>
> Attachments: CassandraIssue.java
>
>
> We have recently introduced Cassandra at the Globe and Mail here in Toronto,
> Canada. We are processing and storing the North American stock-market feed.
> We have found it to work very quickly and things have been looking very good.
> Recently we upgraded to version 1.1.1 and then we have noticed some issues
> occurring.
> I will try to describe it for you here. Basically one operation that we very
> often perform and is very critical is the ability to 'get the latest quote'.
> This would return to you the latest Quote adjusted against exchange delay
> rules. With Cassandra version 1.0.3 we could get a Quote in around 2ms.
> After update we are looking at time of at least 2-3 seconds.
> The way we query the quote is using a REVERSED SuperSliceQuery with
> start=now, end=00:00:00.000 (beginning of day) LIMITED to 1.
> Our investigation leads us to suspect that, since upgrade, Cassandra seems to
> be reading the sstable from disk even when we request a small range of day
> only 5 seconds back. If you look at the output below you can see that the
> query does NOT get slower as the lookback increases from 5 sec, 60 sec, 15
> min, 60 min, and 24 hours.
> We also noticed that the query was very fast for the first five minutes of
> trading, apparently until the first sstable was flushed to disk. After that
> we go into query times of 1-2 seconds or so.
> Query time[lookback=5]:[1711ms]
> Query time[lookback=60]:[1592ms]
> Query time[lookback=900]:[1520ms]
> Query time[lookback=3600]:[1294ms]
> Query time[lookback=86400]:[1391ms]
> We would really appreciate input or help on this.
> Cassandra version: 1.1.1
> Hector version: 1.0-1
> ---
> public void testCassandraIssue() {
> try {
> int[] seconds = new int[]{ 5, 60, 60 * 15, 60 * 60, 60 * 60
> * 24};
> for(int sec : seconds) {
> DateTime start = new DateTime();
> SuperSliceQuery<String, String, String, String>
> superSliceQuery = HFactory.createSuperSliceQuery(keyspaceOperator,
> StringSerializer.get(), StringSerializer.get(), StringSerializer.get(),
> StringSerializer.get());
> superSliceQuery.setKey("101390" + "." +
> testFormatter.print(start));
> superSliceQuery.setColumnFamily("Quotes");
>
> superSliceQuery.setRange(superKeyFormatter.print(start),
>
> superKeyFormatter.print(start.minusSeconds(sec)),
> true,
> 1);
> long theStart = System.currentTimeMillis();
> QueryResult<SuperSlice<String, String, String>>
> result = superSliceQuery.execute();
> long end = System.currentTimeMillis();
> System.out.println("Query time[lookback=" + sec +
> "]:[" + (end - theStart) + "ms]");
> }
> } catch(Exception e) {
> e.printStackTrace();
> fail(e.getMessage());
> }
> }
> ---
> create column family Quotes
> with column_type = Super
> and comparator = BytesType
> and subcomparator = BytesType
> and keys_cached = 7000
> and rows_cached = 0
> and row_cache_save_period = 0
> and key_cache_save_period = 3600
> and memtable_throughput = 255
> and memtable_operations = 0.29
> AND compression_options={sstable_compression:SnappyCompressor,
> chunk_length_kb:64};
--
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