cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Jason Brown (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (CASSANDRA-13444) Fast and garbage-free Streaming Histogram
Date Tue, 16 May 2017 11:13:04 GMT

    [ https://issues.apache.org/jira/browse/CASSANDRA-13444?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16012157#comment-16012157
] 

Jason Brown edited comment on CASSANDRA-13444 at 5/16/17 11:12 AM:
-------------------------------------------------------------------

I've taken a first pass reviewing, and this looks really good :). I'm going to take another
pass or two so I can grok everything, but here are some initial comments:

- please add some more documentation to {{StreamingHistogram}}, especially on the DistanceHolder,
DataHolder, and Spool classes. feel free to expand any of the existing comments, as well.
- What if {{Spool#hash}} wraps around? will that cause a problem in {{tryAddOrAccumulate}}?
admittedly, this is one of the parts I need to read and understand more thoroughly.
- In {{Spool#tryAddOrAccumulate}} why loop for a max of 100 iterations? 
- {{DistanceHolder.getFirstAndRemove()}} may return null, which can cause {{mergeBin}} to
throw an {{NPE}}. what should the correct behavior be here?

While we're improving this class, I don't think it would be much extra work to convert {{DataHolder#keySet}}
and {{StreamingHistogram#getAsMap}} to return an iterator rather than a materialized {{List<Integer>}}
or {{Map<Integer, long[]}}, respectively. At a minimum we could change {{getAsMap}} to
return a {{IntToObjectMap<long[]>}} and avoid all the boxing conversions on that one.
Are you up for doing that?


was (Author: jasobrown):
I've taken a first pass reviewing, and this looks really good :). I'm going to take another
pass or two so I can grok everything, but here are some initial comments:

- please add some more documentation to {{StreamingHistogram}}, especially on the DistanceHolder,
DataHolder, and Spool classes. feel free to expand any of the existing comments, as well.
- What if {{Spool#hash}} wraps around? will that cause a problem in {{tryAddOrAccumulate}}?
admittedly, this is one of the parts I need to read and understand more thoroughly.
- In {{Spool#tryAddOrAccumulate}} why loop for a max of 100 iterations? 
- {{DistanceHolder.getFirstAndRemove()}} may return null, which can cause {{mergeBin}} to
throw an {{NPE}}. what should the correct behavior be here?

While we're improving this class, I don't think it would be much extra work to convert {[DataHolder#keySet}}
and {{StreamingHistogram#getAsMap}} to return an iterator rather than a materialized {{List<Integer>}}
or {{Map<Integer, long[]}}, respectively. At a minimum we could change {{getAsMap}} to
return a {{IntToObjectMap<long[]>}} and avoid all the boxing conversions on that one.
Are you up for doing that?

> Fast and garbage-free Streaming Histogram
> -----------------------------------------
>
>                 Key: CASSANDRA-13444
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-13444
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Compaction
>            Reporter: Fuud
>            Assignee: Fuud
>             Fix For: 4.x
>
>         Attachments: results.csv, results.xlsx
>
>
> StreamingHistogram is cause of high cpu usage and GC pressure.
> It was improved at CASSANDRA-13038 by introducing intermediate buffer to try accumulate
different values into the big map before merging them into smaller one.
> But there was not enought for TTL's distributed within large time. Rounding (also introduced
at 13038) can help but it reduce histogram precision specially in case where TTL's does not
distributed uniformly.
> There are several improvements that can help to reduce cpu and gc usage. Them all included
in the pool-request as separate revisions thus you can test them independently.
> Improvements list:
> # Use Map.computeIfAbsent instead of get->checkIfNull->put chain. In this way "add-or-accumulate"
operation takes one map operation instead of two. But this method (default-defined in interface
Map) is overriden in HashMap but not in TreeMap. Thus I changed spool type to HashMap.
> # As we round incoming values to _roundSeconds_ we can also round value on merge. It
will enlarge hit rate for bin operations.
> # Because we inserted only integers into Histogram and rounding values to integers we
can use *int* type everywhere.
> # Histogram takes huge amount of time merging values. In merge method largest amount
of time taken by finding nearest points. It can be eliminated by holding additional TreeSet
with differences, sorted from smalest to greatest.
> # Because we know max size of _bin_ and _differences_ maps we can replace them with sorted
arrays. Search can be done with _Arrays.binarySearch_ and insertion/deletions can be done
by _System.arraycopy_. Also it helps to merge some operations into one.
> # Because spool map is also limited we can replace it with open address primitive map.
It's finaly reduce allocation rate to zero.
> You can see gain given by each step in the attached file. First number is time for one
benchmark invocation and second - is allocation rate in Mb per operation.
> Dependends of payload time is reduced up to 90%.
> Overall gain:
> |.|.|Payload/SpoolSize|.|.|.|% from original
> |.|.|.|original|.|optimized|
> |.|.|secondInMonth/0|.|.|.|
> |time ms/op|.|.|10747,684|.|5545,063|51,6
> |allocation Mb/op|.|.|2441,38858|.|0,002105713|0
> |.|.|.|.|.|.|
> |.|.|secondInMonth/1000|.|.|.|
> |time ms/op|.|.|8988,578|.|5791,179|64,4
> |allocation Mb/op|.|.|2440,951141|.|0,017715454|0
> |.|.|.|.|.|.|
> |.|.|secondInMonth/10000|.|.|.|
> |time ms/op|.|.|10711,671|.|5765,243|53,8
> |allocation Mb/op|.|.|2437,022537|.|0,264083862|0
> |.|.|.|.|.|.|
> |.|.|secondInMonth/100000|.|.|.|
> |time ms/op|.|.|13001,841|.|5638,069|43,4
> |allocation Mb/op|.|.|2396,947113|.|2,003662109|0,1
> |.|.|.|.|.|.|
> |.|.|secondInDay/0|.|.|.|
> |time ms/op|.|.|10381,833|.|5497,804|53
> |allocation Mb/op|.|.|2441,166107|.|0,002105713|0
> |.|.|.|.|.|.|
> |.|.|secondInDay/1000|.|.|.|
> |time ms/op|.|.|8522,157|.|5929,871|69,6
> |allocation Mb/op|.|.|1973,112381|.|0,017715454|0
> |.|.|.|.|.|.|
> |.|.|secondInDay/10000|.|.|.|
> |time ms/op|.|.|10234,978|.|5480,077|53,5
> |allocation Mb/op|.|.|2306,057404|.|0,262969971|0
> |.|.|.|.|.|.|
> |.|.|secondInDay/100000|.|.|.|
> |time ms/op|.|.|2971,178|.|139,079|4,7
> |allocation Mb/op|.|.|172,1276245|.|2,001721191|1,2
> |.|.|.|.|.|.|
> |.|.|secondIn3Hour/0|.|.|.|
> |time ms/op|.|.|10663,123|.|5605,672|52,6
> |allocation Mb/op|.|.|2439,456818|.|0,002105713|0
> |.|.|.|.|.|.|
> |.|.|secondIn3Hour/1000|.|.|.|
> |time ms/op|.|.|9029,788|.|5838,618|64,7
> |allocation Mb/op|.|.|2331,839249|.|0,180664063|0
> |.|.|.|.|.|.|
> |.|.|secondIn3Hour/10000|.|.|.|
> |time ms/op|.|.|4862,409|.|89,001|1,8
> |allocation Mb/op|.|.|965,4871887|.|0,251711652|0
> |.|.|.|.|.|.|
> |.|.|secondIn3Hour/100000|.|.|.|
> |time ms/op|.|.|1484,454|.|95,044|6,4
> |allocation Mb/op|.|.|153,2464722|.|2,001712809|1,3
> |.|.|.|.|.|.|
> |.|.|secondInMin/0|.|.|.|
> |time ms/op|.|.|875,118|.|424,11|48,5
> |allocation Mb/op|.|.|610,3554993|.|0,001776123|0
> |.|.|.|.|.|.|
> |.|.|secondInMin/1000|.|.|.|
> |time ms/op|.|.|568,7|.|84,208|14,8
> |allocation Mb/op|.|.|0,007598114|.|0,01810023|238,2
> |.|.|.|.|.|.|
> |.|.|secondInMin/10000|.|.|.|
> |time ms/op|.|.|573,595|.|83,862|14,6
> |allocation Mb/op|.|.|0,007597351|.|0,252473872|3323,2
> |.|.|.|.|.|.|
> |.|.|secondInMin/100000|.|.|.|
> |time ms/op|.|.|584,457|.|86,554|14,8
> |allocation Mb/op|.|.|0,007595825|.|2,002506106|26363,2
> You may notice increased allocation rate for secondInMin payload. It is because test
use small values and Integer.valueOf use cache for them. In real case where incoming values
will be timestamps this cache will not work. Also constant memory 2 Mb per StreamingHistogram
is quite good.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


Mime
View raw message