[jira] [Commented] (CASSANDRA-13652) Deadlock in AbstractCommitLogSegmentManager
[ https://issues.apache.org/jira/browse/CASSANDRA-13652?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16087047#comment-16087047 ] Fuud commented on CASSANDRA-13652: -- Yes. Seems good. > Deadlock in AbstractCommitLogSegmentManager > --- > > Key: CASSANDRA-13652 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13652 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Fuud > > AbstractCommitLogManager uses LockSupport.(un)park incorreclty. It invokes > unpark without checking if manager thread was parked in approriate place. > For example, logging frameworks uses queues and queues uses ReadWriteLock's > that uses LockSupport. Therefore AbstractCommitLogManager.wakeManager can > wake thread inside Lock and manager thread will sleep forever at park() > method (because unpark permit was already consumed inside lock). > For examle stack traces: > {code} > "MigrationStage:1" id=412 state=WAITING > at sun.misc.Unsafe.park(Native Method) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304) > at > org.apache.cassandra.utils.concurrent.WaitQueue$AbstractSignal.awaitUninterruptibly(WaitQueue.java:279) > at > org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager.awaitAvailableSegment(AbstractCommitLogSegmentManager.java:263) > at > org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager.advanceAllocatingFrom(AbstractCommitLogSegmentManager.java:237) > at > org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager.forceRecycleAll(AbstractCommitLogSegmentManager.java:279) > at > org.apache.cassandra.db.commitlog.CommitLog.forceRecycleAllSegments(CommitLog.java:210) > at org.apache.cassandra.config.Schema.dropView(Schema.java:708) > at > org.apache.cassandra.schema.SchemaKeyspace.lambda$updateKeyspace$23(SchemaKeyspace.java:1361) > at > org.apache.cassandra.schema.SchemaKeyspace$$Lambda$382/1123232162.accept(Unknown > Source) > at java.util.LinkedHashMap$LinkedValues.forEach(LinkedHashMap.java:608) > at > java.util.Collections$UnmodifiableCollection.forEach(Collections.java:1080) > at > org.apache.cassandra.schema.SchemaKeyspace.updateKeyspace(SchemaKeyspace.java:1361) > at > org.apache.cassandra.schema.SchemaKeyspace.mergeSchema(SchemaKeyspace.java:1332) > at > org.apache.cassandra.schema.SchemaKeyspace.mergeSchemaAndAnnounceVersion(SchemaKeyspace.java:1282) > - locked java.lang.Class@cc38904 > at > org.apache.cassandra.db.DefinitionsUpdateVerbHandler$1.runMayThrow(DefinitionsUpdateVerbHandler.java:51) > at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor$LocalSessionWrapper.run(DebuggableThreadPoolExecutor.java:322) > at > com.ringcentral.concurrent.executors.MonitoredRunnable.run(MonitoredRunnable.java:36) > at MON_R_MigrationStage.run(NamedRunnableFactory.java:67) > at > com.ringcentral.concurrent.executors.MonitoredThreadPoolExecutor$MdcAwareRunnable.run(MonitoredThreadPoolExecutor.java:114) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79) > at > org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$61/179045.run(Unknown > Source) > at java.lang.Thread.run(Thread.java:745) > "COMMIT-LOG-ALLOCATOR:1" id=80 state=WAITING > at sun.misc.Unsafe.park(Native Method) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304) > at > org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager$1.runMayThrow(AbstractCommitLogSegmentManager.java:128) > at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79) > at > org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$61/179045.run(Unknown > Source) > at java.lang.Thread.run(Thread.java:745) > {code} > Solution is to use Semaphore instead of low-level LockSupport. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-13652) Deadlock in AbstractCommitLogSegmentManager
[ https://issues.apache.org/jira/browse/CASSANDRA-13652?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16083616#comment-16083616 ] Fuud commented on CASSANDRA-13652: -- [~aweisberg] >>It's technically ok if other usages of park are woken because spurious >>wakeups are part of the specification so other usages should handle it. Yes. But such other useages will eat permit and Manager Thread will be blocked in our code. > Deadlock in AbstractCommitLogSegmentManager > --- > > Key: CASSANDRA-13652 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13652 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Fuud > > AbstractCommitLogManager uses LockSupport.(un)park incorreclty. It invokes > unpark without checking if manager thread was parked in approriate place. > For example, logging frameworks uses queues and queues uses ReadWriteLock's > that uses LockSupport. Therefore AbstractCommitLogManager.wakeManager can > wake thread inside Lock and manager thread will sleep forever at park() > method (because unpark permit was already consumed inside lock). > For examle stack traces: > {code} > "MigrationStage:1" id=412 state=WAITING > at sun.misc.Unsafe.park(Native Method) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304) > at > org.apache.cassandra.utils.concurrent.WaitQueue$AbstractSignal.awaitUninterruptibly(WaitQueue.java:279) > at > org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager.awaitAvailableSegment(AbstractCommitLogSegmentManager.java:263) > at > org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager.advanceAllocatingFrom(AbstractCommitLogSegmentManager.java:237) > at > org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager.forceRecycleAll(AbstractCommitLogSegmentManager.java:279) > at > org.apache.cassandra.db.commitlog.CommitLog.forceRecycleAllSegments(CommitLog.java:210) > at org.apache.cassandra.config.Schema.dropView(Schema.java:708) > at > org.apache.cassandra.schema.SchemaKeyspace.lambda$updateKeyspace$23(SchemaKeyspace.java:1361) > at > org.apache.cassandra.schema.SchemaKeyspace$$Lambda$382/1123232162.accept(Unknown > Source) > at java.util.LinkedHashMap$LinkedValues.forEach(LinkedHashMap.java:608) > at > java.util.Collections$UnmodifiableCollection.forEach(Collections.java:1080) > at > org.apache.cassandra.schema.SchemaKeyspace.updateKeyspace(SchemaKeyspace.java:1361) > at > org.apache.cassandra.schema.SchemaKeyspace.mergeSchema(SchemaKeyspace.java:1332) > at > org.apache.cassandra.schema.SchemaKeyspace.mergeSchemaAndAnnounceVersion(SchemaKeyspace.java:1282) > - locked java.lang.Class@cc38904 > at > org.apache.cassandra.db.DefinitionsUpdateVerbHandler$1.runMayThrow(DefinitionsUpdateVerbHandler.java:51) > at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor$LocalSessionWrapper.run(DebuggableThreadPoolExecutor.java:322) > at > com.ringcentral.concurrent.executors.MonitoredRunnable.run(MonitoredRunnable.java:36) > at MON_R_MigrationStage.run(NamedRunnableFactory.java:67) > at > com.ringcentral.concurrent.executors.MonitoredThreadPoolExecutor$MdcAwareRunnable.run(MonitoredThreadPoolExecutor.java:114) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79) > at > org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$61/179045.run(Unknown > Source) > at java.lang.Thread.run(Thread.java:745) > "COMMIT-LOG-ALLOCATOR:1" id=80 state=WAITING > at sun.misc.Unsafe.park(Native Method) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304) > at > org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager$1.runMayThrow(AbstractCommitLogSegmentManager.java:128) > at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79) > at > org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$61/179045.run(Unknown > Source) > at java.lang.Thread.run(Thread.java:745) > {code} > Solution is to use Semaphore instead of low-level LockSupport. -- This message was sent by Atlassian JIRA (v6.4.14#64029)
[jira] [Commented] (CASSANDRA-13652) Deadlock in AbstractCommitLogSegmentManager
[ https://issues.apache.org/jira/browse/CASSANDRA-13652?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16081873#comment-16081873 ] Fuud commented on CASSANDRA-13652: -- Just to keep all things together, copy from mailing list http://www.mail-archive.com/dev@cassandra.apache.org/msg11313.html - Hello, I found possible deadlock in AbstractCommitLogSegmentManager. The root cause is incorrect use of LockSupport.park/unpark pair. Unpark should be invoked only if caller is sure that thread was parked in appropriate place. Otherwice permission given by calling unpark can be consumed by other structures (for example - inside ReadWriteLock). Jira: https://issues.apache.org/jira/browse/CASSANDRA-13652 I suggest simplest solution: change LockSupport to Semaphore. PR: https://github.com/apache/cassandra/pull/127 Also I suggest another solution with SynchronousQueue-like structure to move available segment from Manager Thread to consumers. With theese changes code became more clear and straightforward. PR https://github.com/apache/cassandra/pull/129 We can not use j.u.c.SynchronousQueue because we need to support shutdown and there is only way to terminate SynchronousQueue.put is to call Thread.interrupt(). But C* uses nio and it does not expect ClosedByInterruptException during IO operations. Thus we can not interrupt Manager Thread. I implemented o.a.c.u.c.Transferer that supports shutdown and restart (needed for tests). https://github.com/Fuud/cassandra/blob/e1a695874dc24e532ae21ef627e852bf999a75f3/src/java/org/apache/cassandra/utils/concurrent/Transferer.java Also I modified o.a.c.d.c.SimpleCachedBufferPool to support waiting for free space. Please feel free to ask any questions. Thank you. Feodor Bobin fuudtorrent...@gmail.com > Deadlock in AbstractCommitLogSegmentManager > --- > > Key: CASSANDRA-13652 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13652 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Fuud > > AbstractCommitLogManager uses LockSupport.(un)park incorreclty. It invokes > unpark without checking if manager thread was parked in approriate place. > For example, logging frameworks uses queues and queues uses ReadWriteLock's > that uses LockSupport. Therefore AbstractCommitLogManager.wakeManager can > wake thread inside Lock and manager thread will sleep forever at park() > method (because unpark permit was already consumed inside lock). > For examle stack traces: > {code} > "MigrationStage:1" id=412 state=WAITING > at sun.misc.Unsafe.park(Native Method) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304) > at > org.apache.cassandra.utils.concurrent.WaitQueue$AbstractSignal.awaitUninterruptibly(WaitQueue.java:279) > at > org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager.awaitAvailableSegment(AbstractCommitLogSegmentManager.java:263) > at > org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager.advanceAllocatingFrom(AbstractCommitLogSegmentManager.java:237) > at > org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager.forceRecycleAll(AbstractCommitLogSegmentManager.java:279) > at > org.apache.cassandra.db.commitlog.CommitLog.forceRecycleAllSegments(CommitLog.java:210) > at org.apache.cassandra.config.Schema.dropView(Schema.java:708) > at > org.apache.cassandra.schema.SchemaKeyspace.lambda$updateKeyspace$23(SchemaKeyspace.java:1361) > at > org.apache.cassandra.schema.SchemaKeyspace$$Lambda$382/1123232162.accept(Unknown > Source) > at java.util.LinkedHashMap$LinkedValues.forEach(LinkedHashMap.java:608) > at > java.util.Collections$UnmodifiableCollection.forEach(Collections.java:1080) > at > org.apache.cassandra.schema.SchemaKeyspace.updateKeyspace(SchemaKeyspace.java:1361) > at > org.apache.cassandra.schema.SchemaKeyspace.mergeSchema(SchemaKeyspace.java:1332) > at > org.apache.cassandra.schema.SchemaKeyspace.mergeSchemaAndAnnounceVersion(SchemaKeyspace.java:1282) > - locked java.lang.Class@cc38904 > at > org.apache.cassandra.db.DefinitionsUpdateVerbHandler$1.runMayThrow(DefinitionsUpdateVerbHandler.java:51) > at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor$LocalSessionWrapper.run(DebuggableThreadPoolExecutor.java:322) > at > com.ringcentral.concurrent.executors.MonitoredRunnable.run(MonitoredRunnable.java:36) > at MON_R_MigrationStage.run(NamedRunnableFactory.java:67) > at >
[jira] [Created] (CASSANDRA-13652) Deadlock in AbstractCompactionManager
Fuud created CASSANDRA-13652: Summary: Deadlock in AbstractCompactionManager Key: CASSANDRA-13652 URL: https://issues.apache.org/jira/browse/CASSANDRA-13652 Project: Cassandra Issue Type: Bug Components: Core Reporter: Fuud AbstractCommitLogManager uses LockSupport.(un)park incorreclty. It invokes unpark without checking if manager thread was parked in approriate place. For example, logging frameworks uses queues and queues uses ReadWriteLock's that uses LockSupport. Therefore AbstractCommitLogManager.wakeManager can wake thread inside Lock and manager thread will sleep forever at park() method (because unpark permit was already consumed inside lock). For examle stack traces: {code} "MigrationStage:1" id=412 state=WAITING at sun.misc.Unsafe.park(Native Method) at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304) at org.apache.cassandra.utils.concurrent.WaitQueue$AbstractSignal.awaitUninterruptibly(WaitQueue.java:279) at org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager.awaitAvailableSegment(AbstractCommitLogSegmentManager.java:263) at org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager.advanceAllocatingFrom(AbstractCommitLogSegmentManager.java:237) at org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager.forceRecycleAll(AbstractCommitLogSegmentManager.java:279) at org.apache.cassandra.db.commitlog.CommitLog.forceRecycleAllSegments(CommitLog.java:210) at org.apache.cassandra.config.Schema.dropView(Schema.java:708) at org.apache.cassandra.schema.SchemaKeyspace.lambda$updateKeyspace$23(SchemaKeyspace.java:1361) at org.apache.cassandra.schema.SchemaKeyspace$$Lambda$382/1123232162.accept(Unknown Source) at java.util.LinkedHashMap$LinkedValues.forEach(LinkedHashMap.java:608) at java.util.Collections$UnmodifiableCollection.forEach(Collections.java:1080) at org.apache.cassandra.schema.SchemaKeyspace.updateKeyspace(SchemaKeyspace.java:1361) at org.apache.cassandra.schema.SchemaKeyspace.mergeSchema(SchemaKeyspace.java:1332) at org.apache.cassandra.schema.SchemaKeyspace.mergeSchemaAndAnnounceVersion(SchemaKeyspace.java:1282) - locked java.lang.Class@cc38904 at org.apache.cassandra.db.DefinitionsUpdateVerbHandler$1.runMayThrow(DefinitionsUpdateVerbHandler.java:51) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor$LocalSessionWrapper.run(DebuggableThreadPoolExecutor.java:322) at com.ringcentral.concurrent.executors.MonitoredRunnable.run(MonitoredRunnable.java:36) at MON_R_MigrationStage.run(NamedRunnableFactory.java:67) at com.ringcentral.concurrent.executors.MonitoredThreadPoolExecutor$MdcAwareRunnable.run(MonitoredThreadPoolExecutor.java:114) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79) at org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$61/179045.run(Unknown Source) at java.lang.Thread.run(Thread.java:745) "COMMIT-LOG-ALLOCATOR:1" id=80 state=WAITING at sun.misc.Unsafe.park(Native Method) at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304) at org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager$1.runMayThrow(AbstractCommitLogSegmentManager.java:128) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79) at org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$61/179045.run(Unknown Source) at java.lang.Thread.run(Thread.java:745) {code} Solution is to use Semaphore instead of low-level LockSupport. -- This message was sent by Atlassian JIRA (v6.4.14#64029) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-13444) Fast and garbage-free Streaming Histogram
[ https://issues.apache.org/jira/browse/CASSANDRA-13444?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16038805#comment-16038805 ] Fuud commented on CASSANDRA-13444: -- Thank you > 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.0 > > 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/1|.|.|.| > |time ms/op|.|.|10711,671|.|5765,243|53,8 > |allocation Mb/op|.|.|2437,022537|.|0,264083862|0 > |.|.|.|.|.|.| > |.|.|secondInMonth/10|.|.|.| > |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/1|.|.|.| > |time ms/op|.|.|10234,978|.|5480,077|53,5 > |allocation Mb/op|.|.|2306,057404|.|0,262969971|0 > |.|.|.|.|.|.| > |.|.|secondInDay/10|.|.|.| > |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/1|.|.|.| > |time ms/op|.|.|4862,409|.|89,001|1,8 > |allocation Mb/op|.|.|965,4871887|.|0,251711652|0 > |.|.|.|.|.|.| > |.|.|secondIn3Hour/10|.|.|.| > |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/1|.|.|.| > |time ms/op|.|.|573,595|.|83,862|14,6 > |allocation Mb/op|.|.|0,007597351|.|0,252473872|3323,2 > |.|.|.|.|.|.| >
[jira] [Commented] (CASSANDRA-13444) Fast and garbage-free Streaming Histogram
[ https://issues.apache.org/jira/browse/CASSANDRA-13444?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16031102#comment-16031102 ] Fuud commented on CASSANDRA-13444: -- [~jasobrown] Anything else preventing PR from merge? > 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/1|.|.|.| > |time ms/op|.|.|10711,671|.|5765,243|53,8 > |allocation Mb/op|.|.|2437,022537|.|0,264083862|0 > |.|.|.|.|.|.| > |.|.|secondInMonth/10|.|.|.| > |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/1|.|.|.| > |time ms/op|.|.|10234,978|.|5480,077|53,5 > |allocation Mb/op|.|.|2306,057404|.|0,262969971|0 > |.|.|.|.|.|.| > |.|.|secondInDay/10|.|.|.| > |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/1|.|.|.| > |time ms/op|.|.|4862,409|.|89,001|1,8 > |allocation Mb/op|.|.|965,4871887|.|0,251711652|0 > |.|.|.|.|.|.| > |.|.|secondIn3Hour/10|.|.|.| > |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/1|.|.|.| > |time ms/op|.|.|573,595|.|83,862|14,6 > |allocation
[jira] [Comment Edited] (CASSANDRA-13444) Fast and garbage-free Streaming Histogram
[ https://issues.apache.org/jira/browse/CASSANDRA-13444?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16025999#comment-16025999 ] Fuud edited comment on CASSANDRA-13444 at 5/26/17 8:42 AM: --- Sorry. It was mistake to amend commit during the review. Bug was fixed + equals method fixed + small refactoring to keep size() calculation clear. Diff: {code} diff --git a/src/java/org/apache/cassandra/utils/streamhist/StreamingTombstoneHistogramBuilder.java b/src/java/org/apache/cassandra/utils/streamhist/StreamingTombstoneHistogramBuilder.java index 8affb4e..21b3bef 100755 --- a/src/java/org/apache/cassandra/utils/streamhist/StreamingTombstoneHistogramBuilder.java +++ b/src/java/org/apache/cassandra/utils/streamhist/StreamingTombstoneHistogramBuilder.java @@ -404,6 +404,13 @@ public class StreamingTombstoneHistogramBuilder } } +public int size() +{ +int[] accumulator = new int[1]; +forEach((point, value) -> accumulator[0]++); +return accumulator[0]; +} + public double sum(int b) { double sum = 0; @@ -480,7 +487,21 @@ public class StreamingTombstoneHistogramBuilder @Override public boolean equals(Object o) { -return o instanceof DataHolder && Arrays.equals(data, ((DataHolder) o).data); +if (!(o instanceof DataHolder)) +return false; + + +final DataHolder other = ((DataHolder) o); + +if (this.size()!=other.size()) +return false; + +for (int i=0; iaccumulator[0]++); -return accumulator[0]; +return this.bin.size(); } public void forEach(HistogramDataConsumer histogramDataConsumer) throws E @@ -90,7 +88,10 @@ public class TombstoneHistogram in.readInt(); // max bin size int size = in.readInt(); DataHolder dataHolder = new DataHolder(size, 1); -dataHolder.addValue((int)in.readDouble(), (int)in.readLong()); +for (int i = 0; i < size; i++) +{ +dataHolder.addValue((int)in.readDouble(), (int)in.readLong()); +} return new TombstoneHistogram(dataHolder); } {code} was (Author: fuud): Sorry. It was mistake to amend commit during the review. Bug was fixed + equals method fixed + small refactoring to keep size() calculation clear. Diff: ``` diff --git a/src/java/org/apache/cassandra/utils/streamhist/StreamingTombstoneHistogramBuilder.java b/src/java/org/apache/cassandra/utils/streamhist/StreamingTombstoneHistogramBuilder.java index 8affb4e..21b3bef 100755 --- a/src/java/org/apache/cassandra/utils/streamhist/StreamingTombstoneHistogramBuilder.java +++ b/src/java/org/apache/cassandra/utils/streamhist/StreamingTombstoneHistogramBuilder.java @@ -404,6 +404,13 @@ public class StreamingTombstoneHistogramBuilder } } +public int size() +{ +int[] accumulator = new int[1]; +forEach((point, value) -> accumulator[0]++); +return accumulator[0]; +} + public double sum(int b) { double sum = 0; @@ -480,7 +487,21 @@ public class StreamingTombstoneHistogramBuilder @Override public boolean equals(Object o) { -return o instanceof DataHolder && Arrays.equals(data, ((DataHolder) o).data); +if (!(o instanceof DataHolder)) +return false; + + +final DataHolder other = ((DataHolder) o); + +if (this.size()!=other.size()) +return false; + +for (int i=0; i
[jira] [Commented] (CASSANDRA-13444) Fast and garbage-free Streaming Histogram
[ https://issues.apache.org/jira/browse/CASSANDRA-13444?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16025999#comment-16025999 ] Fuud commented on CASSANDRA-13444: -- Sorry. It was mistake to amend commit during the review. Bug was fixed + equals method fixed + small refactoring to keep size() calculation clear. Diff: ``` diff --git a/src/java/org/apache/cassandra/utils/streamhist/StreamingTombstoneHistogramBuilder.java b/src/java/org/apache/cassandra/utils/streamhist/StreamingTombstoneHistogramBuilder.java index 8affb4e..21b3bef 100755 --- a/src/java/org/apache/cassandra/utils/streamhist/StreamingTombstoneHistogramBuilder.java +++ b/src/java/org/apache/cassandra/utils/streamhist/StreamingTombstoneHistogramBuilder.java @@ -404,6 +404,13 @@ public class StreamingTombstoneHistogramBuilder } } +public int size() +{ +int[] accumulator = new int[1]; +forEach((point, value) -> accumulator[0]++); +return accumulator[0]; +} + public double sum(int b) { double sum = 0; @@ -480,7 +487,21 @@ public class StreamingTombstoneHistogramBuilder @Override public boolean equals(Object o) { -return o instanceof DataHolder && Arrays.equals(data, ((DataHolder) o).data); +if (!(o instanceof DataHolder)) +return false; + + +final DataHolder other = ((DataHolder) o); + +if (this.size()!=other.size()) +return false; + +for (int i=0; iaccumulator[0]++); -return accumulator[0]; +return this.bin.size(); } public void forEach(HistogramDataConsumer histogramDataConsumer) throws E @@ -90,7 +88,10 @@ public class TombstoneHistogram in.readInt(); // max bin size int size = in.readInt(); DataHolder dataHolder = new DataHolder(size, 1); -dataHolder.addValue((int)in.readDouble(), (int)in.readLong()); +for (int i = 0; i < size; i++) +{ +dataHolder.addValue((int)in.readDouble(), (int)in.readLong()); +} return new TombstoneHistogram(dataHolder); } ``` > 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
[jira] [Commented] (CASSANDRA-13444) Fast and garbage-free Streaming Histogram
[ https://issues.apache.org/jira/browse/CASSANDRA-13444?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16022785#comment-16022785 ] Fuud commented on CASSANDRA-13444: -- Sorry. Fixed, last commit was amended. > 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/1|.|.|.| > |time ms/op|.|.|10711,671|.|5765,243|53,8 > |allocation Mb/op|.|.|2437,022537|.|0,264083862|0 > |.|.|.|.|.|.| > |.|.|secondInMonth/10|.|.|.| > |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/1|.|.|.| > |time ms/op|.|.|10234,978|.|5480,077|53,5 > |allocation Mb/op|.|.|2306,057404|.|0,262969971|0 > |.|.|.|.|.|.| > |.|.|secondInDay/10|.|.|.| > |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/1|.|.|.| > |time ms/op|.|.|4862,409|.|89,001|1,8 > |allocation Mb/op|.|.|965,4871887|.|0,251711652|0 > |.|.|.|.|.|.| > |.|.|secondIn3Hour/10|.|.|.| > |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/1|.|.|.| > |time ms/op|.|.|573,595|.|83,862|14,6 > |allocation Mb/op|.|.|0,007597351|.|0,252473872|3323,2 >
[jira] [Commented] (CASSANDRA-13444) Fast and garbage-free Streaming Histogram
[ https://issues.apache.org/jira/browse/CASSANDRA-13444?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16022384#comment-16022384 ] Fuud commented on CASSANDRA-13444: -- >>there's nothing specific tombstones in this, so maybe remove that from the >>class names[...] I do not like name StreamingHistogram because now it is not 'streaming'. May be just Histogram or TimeHistogram (because it has roundSeconds parameter)? >> let's rename PointAndValueConsumer to ... HistogramConsumer? HistogramDataConsumer seems more clear. >>add a comment to PointAndValueConsumer [...] done >> TombstoneHistogram#maxBinSize [...]. Let's just add a note to the member >> field that is only there for historical purposes. We can dump this field >> if/when we update serialization format. It is not needed. We can write histogram.size as maxBinSize to maintain backward compatibility and preserve format. I removed this field. >> TombstoneHistogram#roundSeconds is unused removed >>TombstoneHistogram#size 'acc' was renamed, we can not use primitive int because we will not be able to increment from lambda. >>Lastly, I think we can optimize HistogramSerializer#deserialize() [...] >>feeding the values directly into the DataHolder Yes. Since Histogram is immutable we can do this. Also we can create DataHolder of size 'size' instead of 'maxBinSize'. > 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/1|.|.|.| > |time ms/op|.|.|10711,671|.|5765,243|53,8 > |allocation Mb/op|.|.|2437,022537|.|0,264083862|0 > |.|.|.|.|.|.| > |.|.|secondInMonth/10|.|.|.| > |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/1|.|.|.| > |time ms/op|.|.|10234,978|.|5480,077|53,5 > |allocation Mb/op|.|.|2306,057404|.|0,262969971|0
[jira] [Comment Edited] (CASSANDRA-13444) Fast and garbage-free Streaming Histogram
[ https://issues.apache.org/jira/browse/CASSANDRA-13444?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16020734#comment-16020734 ] Fuud edited comment on CASSANDRA-13444 at 5/23/17 7:13 AM: --- Good concern. I merged your commit and added some comments. Also I moved this stuff into separate package and splitted classes. Is it ok? >>[...]avoid all the boxing conversions on that one. Are you up for doing that? Done. Unused methods was deleted, used methods was rewritten with forEach method. >>why loop for a max of 100 iterations? It is not clear how many collisions we should allow before consider to flush Spool. But cluster with 100 elements is seems large enough to consider map as wasted. >>What if Spool#hash wraps around? You mean, what is result became negative? Yes it is expected to be negative in (around) of half of cases. Up bits will be erased in tryAddOrAccumulate method and cell will be calculated properly. >>StreamingHistogramSerializer refactoring. Yes it can be done but I do not know how to do it properly. Do we need to update sstableupgrate tool too? was (Author: fuud): Good consern. I merged your commit and added some comments. Also I moved this stuff into separate package and splitted classes. Is it ok? >>[...]avoid all the boxing conversions on that one. Are you up for doing that? Done. Unused methods was deleted, used methods was rewritten with forEach method. >>why loop for a max of 100 iterations? It is not clear how many collisions we should allow before consider to flush Spool. But cluster with 100 elements is seems large enough to consider map as wasted. >>What if Spool#hash wraps around? You mean, what is result became negative? Yes it is expected to be negative in (around) of half of cases. Up bits will be erased in tryAddOrAccumulate method and cell will be calculated properly. >>StreamingHistogramSerializer refactoring. Yes it can be done but I do not know how to do it properly. Do we need to update sstableupgrate tool too? > 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 > |.|.|.|.|.|.| >
[jira] [Commented] (CASSANDRA-13444) Fast and garbage-free Streaming Histogram
[ https://issues.apache.org/jira/browse/CASSANDRA-13444?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16020734#comment-16020734 ] Fuud commented on CASSANDRA-13444: -- Good consern. I merged your commit and added some comments. Also I moved this stuff into separate package and splitted classes. Is it ok? >>[...]avoid all the boxing conversions on that one. Are you up for doing that? Done. Unused methods was deleted, used methods was rewritten with forEach method. >>why loop for a max of 100 iterations? It is not clear how many collisions we should allow before consider to flush Spool. But cluster with 100 elements is seems large enough to consider map as wasted. >>What if Spool#hash wraps around? You mean, what is result became negative? Yes it is expected to be negative in (around) of half of cases. Up bits will be erased in tryAddOrAccumulate method and cell will be calculated properly. >>StreamingHistogramSerializer refactoring. Yes it can be done but I do not know how to do it properly. Do we need to update sstableupgrate tool too? > 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/1|.|.|.| > |time ms/op|.|.|10711,671|.|5765,243|53,8 > |allocation Mb/op|.|.|2437,022537|.|0,264083862|0 > |.|.|.|.|.|.| > |.|.|secondInMonth/10|.|.|.| > |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/1|.|.|.| > |time ms/op|.|.|10234,978|.|5480,077|53,5 > |allocation Mb/op|.|.|2306,057404|.|0,262969971|0 > |.|.|.|.|.|.| > |.|.|secondInDay/10|.|.|.| > |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 >
[jira] [Commented] (CASSANDRA-13444) Fast and garbage-free Streaming Histogram
[ https://issues.apache.org/jira/browse/CASSANDRA-13444?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16012181#comment-16012181 ] Fuud commented on CASSANDRA-13444: -- Thank you for review. I'll handle issues in near future. > 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/1|.|.|.| > |time ms/op|.|.|10711,671|.|5765,243|53,8 > |allocation Mb/op|.|.|2437,022537|.|0,264083862|0 > |.|.|.|.|.|.| > |.|.|secondInMonth/10|.|.|.| > |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/1|.|.|.| > |time ms/op|.|.|10234,978|.|5480,077|53,5 > |allocation Mb/op|.|.|2306,057404|.|0,262969971|0 > |.|.|.|.|.|.| > |.|.|secondInDay/10|.|.|.| > |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/1|.|.|.| > |time ms/op|.|.|4862,409|.|89,001|1,8 > |allocation Mb/op|.|.|965,4871887|.|0,251711652|0 > |.|.|.|.|.|.| > |.|.|secondIn3Hour/10|.|.|.| > |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/1|.|.|.| > |time ms/op|.|.|573,595|.|83,862|14,6 > |allocation
[jira] [Commented] (CASSANDRA-13444) Fast and garbage-free Streaming Histogram
[ https://issues.apache.org/jira/browse/CASSANDRA-13444?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15980926#comment-15980926 ] Fuud commented on CASSANDRA-13444: -- any updates? > 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 > 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/1|.|.|.| > |time ms/op|.|.|10711,671|.|5765,243|53,8 > |allocation Mb/op|.|.|2437,022537|.|0,264083862|0 > |.|.|.|.|.|.| > |.|.|secondInMonth/10|.|.|.| > |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/1|.|.|.| > |time ms/op|.|.|10234,978|.|5480,077|53,5 > |allocation Mb/op|.|.|2306,057404|.|0,262969971|0 > |.|.|.|.|.|.| > |.|.|secondInDay/10|.|.|.| > |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/1|.|.|.| > |time ms/op|.|.|4862,409|.|89,001|1,8 > |allocation Mb/op|.|.|965,4871887|.|0,251711652|0 > |.|.|.|.|.|.| > |.|.|secondIn3Hour/10|.|.|.| > |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/1|.|.|.| > |time ms/op|.|.|573,595|.|83,862|14,6 > |allocation Mb/op|.|.|0,007597351|.|0,252473872|3323,2 > |.|.|.|.|.|.| > |.|.|secondInMin/10|.|.|.| > |time ms/op|.|.|584,457|.|86,554|14,8 >
[jira] [Created] (CASSANDRA-13456) Needs better logging for timeout/failures
Fuud created CASSANDRA-13456: Summary: Needs better logging for timeout/failures Key: CASSANDRA-13456 URL: https://issues.apache.org/jira/browse/CASSANDRA-13456 Project: Cassandra Issue Type: Bug Reporter: Fuud When read was failed due to timeout, Cassandra reports in logs "Timeout; reveived 1 of 3 responses". Same information is passed to clients. But this information is not enoght to get list of slow nodes. Better to have detailed message in debug log: "Timeout; reveived 1 of 4 responses. Requested but not responded node: [, ], Failed nodes: []" I implemented such behavior by patching ReadCallback, AbstractWriteResponseHandler, DatacenterSyncWriteResponseHandler and WriteResponseHandler. It handles all cases except Paxos. But I want to implement solid solution that handle all cases in same way. But before I will start I want to know: are there any objections against such logging? -- This message was sent by Atlassian JIRA (v6.3.15#6346)
[jira] [Commented] (CASSANDRA-13444) Fast and garbage-free Streaming Histogram
[ https://issues.apache.org/jira/browse/CASSANDRA-13444?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15967288#comment-15967288 ] Fuud commented on CASSANDRA-13444: -- [~jjirsa] please review > 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 > 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/1|.|.|.| > |time ms/op|.|.|10711,671|.|5765,243|53,8 > |allocation Mb/op|.|.|2437,022537|.|0,264083862|0 > |.|.|.|.|.|.| > |.|.|secondInMonth/10|.|.|.| > |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/1|.|.|.| > |time ms/op|.|.|10234,978|.|5480,077|53,5 > |allocation Mb/op|.|.|2306,057404|.|0,262969971|0 > |.|.|.|.|.|.| > |.|.|secondInDay/10|.|.|.| > |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/1|.|.|.| > |time ms/op|.|.|4862,409|.|89,001|1,8 > |allocation Mb/op|.|.|965,4871887|.|0,251711652|0 > |.|.|.|.|.|.| > |.|.|secondIn3Hour/10|.|.|.| > |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/1|.|.|.| > |time ms/op|.|.|573,595|.|83,862|14,6 > |allocation Mb/op|.|.|0,007597351|.|0,252473872|3323,2 > |.|.|.|.|.|.| > |.|.|secondInMin/10|.|.|.| > |time
[jira] [Updated] (CASSANDRA-13444) Fast and garbage-free Streaming Histogram
[ https://issues.apache.org/jira/browse/CASSANDRA-13444?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Fuud updated CASSANDRA-13444: - Description: 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/1|.|.|.| |time ms/op|.|.|10711,671|.|5765,243|53,8 |allocation Mb/op|.|.|2437,022537|.|0,264083862|0 |.|.|.|.|.|.| |.|.|secondInMonth/10|.|.|.| |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/1|.|.|.| |time ms/op|.|.|10234,978|.|5480,077|53,5 |allocation Mb/op|.|.|2306,057404|.|0,262969971|0 |.|.|.|.|.|.| |.|.|secondInDay/10|.|.|.| |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/1|.|.|.| |time ms/op|.|.|4862,409|.|89,001|1,8 |allocation Mb/op|.|.|965,4871887|.|0,251711652|0 |.|.|.|.|.|.| |.|.|secondIn3Hour/10|.|.|.| |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/1|.|.|.| |time ms/op|.|.|573,595|.|83,862|14,6 |allocation Mb/op|.|.|0,007597351|.|0,252473872|3323,2 |.|.|.|.|.|.| |.|.|secondInMin/10|.|.|.| |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. was: 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
[jira] [Updated] (CASSANDRA-13444) Fast and garbage-free Streaming Histogram
[ https://issues.apache.org/jira/browse/CASSANDRA-13444?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Fuud updated CASSANDRA-13444: - Attachment: results.xlsx results.csv > 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 > 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. > 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/1|.|.|.| > |time ms/op|.|.|10711,671|.|5765,243|53,8 > |allocation Mb/op|.|.|2437,022537|.|0,264083862|0 > |.|.|.|.|.|.| > |.|.|secondInMonth/10|.|.|.| > |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/1|.|.|.| > |time ms/op|.|.|10234,978|.|5480,077|53,5 > |allocation Mb/op|.|.|2306,057404|.|0,262969971|0 > |.|.|.|.|.|.| > |.|.|secondInDay/10|.|.|.| > |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/1|.|.|.| > |time ms/op|.|.|4862,409|.|89,001|1,8 > |allocation Mb/op|.|.|965,4871887|.|0,251711652|0 > |.|.|.|.|.|.| > |.|.|secondIn3Hour/10|.|.|.| > |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/1|.|.|.| > |time ms/op|.|.|573,595|.|83,862|14,6 > |allocation Mb/op|.|.|0,007597351|.|0,252473872|3323,2 > |.|.|.|.|.|.| > |.|.|secondInMin/10|.|.|.| > |time ms/op|.|.|584,457|.|86,554|14,8 > |allocation
[jira] [Created] (CASSANDRA-13444) Fast and garbage-free Streaming Histogram
Fuud created CASSANDRA-13444: Summary: 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 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. 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/1|.|.|.| |time ms/op|.|.|10711,671|.|5765,243|53,8 |allocation Mb/op|.|.|2437,022537|.|0,264083862|0 |.|.|.|.|.|.| |.|.|secondInMonth/10|.|.|.| |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/1|.|.|.| |time ms/op|.|.|10234,978|.|5480,077|53,5 |allocation Mb/op|.|.|2306,057404|.|0,262969971|0 |.|.|.|.|.|.| |.|.|secondInDay/10|.|.|.| |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/1|.|.|.| |time ms/op|.|.|4862,409|.|89,001|1,8 |allocation Mb/op|.|.|965,4871887|.|0,251711652|0 |.|.|.|.|.|.| |.|.|secondIn3Hour/10|.|.|.| |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/1|.|.|.| |time ms/op|.|.|573,595|.|83,862|14,6 |allocation Mb/op|.|.|0,007597351|.|0,252473872|3323,2 |.|.|.|.|.|.| |.|.|secondInMin/10|.|.|.| |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)
[jira] [Commented] (CASSANDRA-13393) Invalid row cache size (in MB) is reported by JMX and NodeTool
[ https://issues.apache.org/jira/browse/CASSANDRA-13393?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15960390#comment-15960390 ] Fuud commented on CASSANDRA-13393: -- Patch: https://github.com/apache/cassandra/pull/105 > Invalid row cache size (in MB) is reported by JMX and NodeTool > -- > > Key: CASSANDRA-13393 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13393 > Project: Cassandra > Issue Type: Bug > Components: Tools >Reporter: Fuud >Priority: Minor > Labels: lhf > > Row Cache size is reported in entries but should be reported in bytes (as > KeyCache do). > It happens because incorrect OHCProvider.OHCacheAdapter.weightedSize method. > Currently it returns cache size but should return ohCache.memUsed() -- This message was sent by Atlassian JIRA (v6.3.15#6346)
[jira] [Created] (CASSANDRA-13393) Invalid row cache size (in MB) is reported by JMX and NodeTool
Fuud created CASSANDRA-13393: Summary: Invalid row cache size (in MB) is reported by JMX and NodeTool Key: CASSANDRA-13393 URL: https://issues.apache.org/jira/browse/CASSANDRA-13393 Project: Cassandra Issue Type: Bug Components: Tools Reporter: Fuud Priority: Minor Row Cache size is reported in entries but should be reported in bytes (as KeyCache do). It happens because incorrect OHCProvider.OHCacheAdapter.weightedSize method. Currently it returns cache size but should return ohCache.memUsed() -- This message was sent by Atlassian JIRA (v6.3.15#6346)
[jira] [Created] (CASSANDRA-13329) max_hints_delivery_threads does not work
Fuud created CASSANDRA-13329: Summary: max_hints_delivery_threads does not work Key: CASSANDRA-13329 URL: https://issues.apache.org/jira/browse/CASSANDRA-13329 Project: Cassandra Issue Type: Bug Reporter: Fuud HintsDispatchExecutor creates JMXEnabledThreadPoolExecutor with corePoolSize == 1 and maxPoolSize==max_hints_delivery_threads and unbounded LinkedBlockingQueue. In this configuration additional threads will not be created. Same problem with PerSSTableIndexWriter. -- This message was sent by Atlassian JIRA (v6.3.15#6346)