[jira] [Updated] (CASSANDRA-7282) Faster Memtable map
[ https://issues.apache.org/jira/browse/CASSANDRA-7282?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benedict updated CASSANDRA-7282: Description: Currently we maintain a ConcurrentSkipLastMap of DecoratedKey - Partition in our memtables. Maintaining this is an O(lg(n)) operation; since the vast majority of users use a hash partitioner, it occurs to me we could maintain a hybrid ordered list / hash map. The list would impose the normal order on the collection, but a hash index would live alongside as part of the same data structure, simply mapping into the list and permitting O(1) lookups and inserts. I've chosen to implement this initial version as a linked-list node per item, but we can optimise this in future by storing fatter nodes that permit a cache-line's worth of hashes to be checked at once, further reducing the constant factor costs for lookups. was: Currently we maintain a ConcurrentSkipLastMap of DecoratedKey - Partition in our memtables. Maintaining this is an O(lg(n)) operation; since the vast majority of users use a hash partitioner, it occurs to me we could maintain a hybrid skip list / hash map. The skip list would impose the order on the collection, but a hash index would live alongside as part of the same data structure, simply mapping into the skip list and permitting O(1) lookups. It should be possible to define the hash map to also permit O(1) inserts. Our decorated keys are in fact perfectly designed for this scheme. At the same time, we can potentially improve the data locality in the skip list by baking the initial 64 token bits directly into the structure, and storing multiple values per skip list entry to improve cache performance, bringing down memory and constant factor cpu overheads. Faster Memtable map --- Key: CASSANDRA-7282 URL: https://issues.apache.org/jira/browse/CASSANDRA-7282 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Benedict Assignee: Benedict Fix For: 3.0 Currently we maintain a ConcurrentSkipLastMap of DecoratedKey - Partition in our memtables. Maintaining this is an O(lg(n)) operation; since the vast majority of users use a hash partitioner, it occurs to me we could maintain a hybrid ordered list / hash map. The list would impose the normal order on the collection, but a hash index would live alongside as part of the same data structure, simply mapping into the list and permitting O(1) lookups and inserts. I've chosen to implement this initial version as a linked-list node per item, but we can optimise this in future by storing fatter nodes that permit a cache-line's worth of hashes to be checked at once, further reducing the constant factor costs for lookups. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7359) Optimize locking in PaxosState
[ https://issues.apache.org/jira/browse/CASSANDRA-7359?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14019171#comment-14019171 ] Benedict commented on CASSANDRA-7359: - It's probably a lot easier to either 1) simply expand the bucket range; or 2) use a ConcurrentHashMap and perform putIfAbsent with the exact hash, so that there is no limit to the number of concurrent operations Optimize locking in PaxosState -- Key: CASSANDRA-7359 URL: https://issues.apache.org/jira/browse/CASSANDRA-7359 Project: Cassandra Issue Type: Improvement Reporter: sankalp kohli Priority: Minor In PaxosState, we want to lock on same rows and have created 1024 size array with java Objects in them to be used for locking. We should replace these Objects with some Lock so that we can know whether there is contention trying to acquire a lock for different rows. We can achieve that by also storing the hash of the row which acquired the lock. This will tell us if this needs to be improved. Here is an improvement which I was thinking about. Say two rows A and B map to the same 1024 bucket which we have. A get the lock and B has to wait. Here B can check if his hash is different and create a new object and chain it to the other one. This looks close to a hashMap with chaining for same key. The hard part will be removing the entries no longer being used. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7359) Optimize locking in PaxosState
[ https://issues.apache.org/jira/browse/CASSANDRA-7359?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benedict updated CASSANDRA-7359: Attachment: 7359.txt par example; there are of course lots of tweaks/variants of this Optimize locking in PaxosState -- Key: CASSANDRA-7359 URL: https://issues.apache.org/jira/browse/CASSANDRA-7359 Project: Cassandra Issue Type: Improvement Reporter: sankalp kohli Priority: Minor Attachments: 7359.txt In PaxosState, we want to lock on same rows and have created 1024 size array with java Objects in them to be used for locking. We should replace these Objects with some Lock so that we can know whether there is contention trying to acquire a lock for different rows. We can achieve that by also storing the hash of the row which acquired the lock. This will tell us if this needs to be improved. Here is an improvement which I was thinking about. Say two rows A and B map to the same 1024 bucket which we have. A get the lock and B has to wait. Here B can check if his hash is different and create a new object and chain it to the other one. This looks close to a hashMap with chaining for same key. The hard part will be removing the entries no longer being used. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7282) Faster Memtable map
[ https://issues.apache.org/jira/browse/CASSANDRA-7282?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14019210#comment-14019210 ] Benedict commented on CASSANDRA-7282: - There was a minor bug with compatability mode for non-murmur3 partitioners. Unfortunately it looks like the unit tests and dtests all specify ByteOrderedPartitioner, making this difficult to test thoroughly across all database features. It works fine with stress, but I will need to write some thorough bespoke unit tests. Faster Memtable map --- Key: CASSANDRA-7282 URL: https://issues.apache.org/jira/browse/CASSANDRA-7282 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Benedict Assignee: Benedict Labels: performance Fix For: 3.0 Currently we maintain a ConcurrentSkipLastMap of DecoratedKey - Partition in our memtables. Maintaining this is an O(lg(n)) operation; since the vast majority of users use a hash partitioner, it occurs to me we could maintain a hybrid ordered list / hash map. The list would impose the normal order on the collection, but a hash index would live alongside as part of the same data structure, simply mapping into the list and permitting O(1) lookups and inserts. I've chosen to implement this initial version as a linked-list node per item, but we can optimise this in future by storing fatter nodes that permit a cache-line's worth of hashes to be checked at once, further reducing the constant factor costs for lookups. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7359) Optimize locking in PaxosState
[ https://issues.apache.org/jira/browse/CASSANDRA-7359?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14019790#comment-14019790 ] Benedict commented on CASSANDRA-7359: - I agree that 1 is likely sufficient, at least for throughput. It's possible you'll keep latency down slightly with 2, but with a fast network this effect will most likely be drowned out by gc etc Optimize locking in PaxosState -- Key: CASSANDRA-7359 URL: https://issues.apache.org/jira/browse/CASSANDRA-7359 Project: Cassandra Issue Type: Improvement Reporter: sankalp kohli Assignee: Benedict Priority: Minor Attachments: 7359.txt In PaxosState, we want to lock on same rows and have created 1024 size array with java Objects in them to be used for locking. We should replace these Objects with some Lock so that we can know whether there is contention trying to acquire a lock for different rows. We can achieve that by also storing the hash of the row which acquired the lock. This will tell us if this needs to be improved. Here is an improvement which I was thinking about. Say two rows A and B map to the same 1024 bucket which we have. A get the lock and B has to wait. Here B can check if his hash is different and create a new object and chain it to the other one. This looks close to a hashMap with chaining for same key. The hard part will be removing the entries no longer being used. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7361) Cassandra locks up in full GC when you assign the entire heap to row cache
[ https://issues.apache.org/jira/browse/CASSANDRA-7361?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14020008#comment-14020008 ] Benedict commented on CASSANDRA-7361: - It is only offheap of JNA is installed, which it may not be. There is also a small per-row on-heap overhead even if the offheap row cache is enabled (though the docs could be much clearer on this). Cassandra locks up in full GC when you assign the entire heap to row cache -- Key: CASSANDRA-7361 URL: https://issues.apache.org/jira/browse/CASSANDRA-7361 Project: Cassandra Issue Type: Bug Components: Core Environment: Ubuntu, RedHat, JDK 1.7 Reporter: Jacek Furmankiewicz Priority: Minor We have a long running batch load process, which runs for many hours. Massive amount of writes, in large mutation batches (we increase the thrift frame size to 45 MB). Everything goes well, but after about 3 hrs of processing everything locks up. We start getting NoHostsAvailable exceptions on the Java application side (with Astyanax as our driver), eventually socket timeouts. Looking at Cassandra, we can see that it is using nearly the full 8GB of heap and unable to free it. It spends most of its time in full GC, but the amount of memory does not go down. Here is a long sample from jstat to show this over an extended time period e.g. http://aep.appspot.com/display/NqqEagzGRLO_pCP2q8hZtitnuVU/ This continues even after we shut down our app. Nothing is connected to Cassandra any more, yet it is still stuck in full GC and cannot free up memory. Running nodetool tpstats shows that nothing is pending, all seems OK: {quote} Pool NameActive Pending Completed Blocked All time blocked ReadStage 0 0 69555935 0 0 RequestResponseStage 0 0 0 0 0 MutationStage 0 0 73123690 0 0 ReadRepairStage 0 0 0 0 0 ReplicateOnWriteStage 0 0 0 0 0 GossipStage 0 0 0 0 0 CacheCleanupExecutor 0 0 0 0 0 MigrationStage0 0 46 0 0 MemoryMeter 0 0 1125 0 0 FlushWriter 0 0824 0 30 ValidationExecutor0 0 0 0 0 InternalResponseStage 0 0 23 0 0 AntiEntropyStage 0 0 0 0 0 MemtablePostFlusher 0 0 1783 0 0 MiscStage 0 0 0 0 0 PendingRangeCalculator0 0 1 0 0 CompactionExecutor0 0 74330 0 0 commitlog_archiver0 0 0 0 0 HintedHandoff 0 0 0 0 0 Message type Dropped RANGE_SLICE 0 READ_REPAIR 0 PAGED_RANGE 0 BINARY 0 READ 585 MUTATION 75775 _TRACE 0 REQUEST_RESPONSE 0 COUNTER_MUTATION 0 {quote} We had this happen on 2 separate boxes, one with 2.0.6, the other with 2.0.8. Right now this is a total blocker for us. We are unable to process the customer data and have to abort in the middle of large processing. This is a new customer, so we did not have a chance to see if this occurred with 1.1 or 1.2 in the past (we moved to 2.0 recently). We have the Cassandra process still running, pls let us know if there is anything else we could run to give you more insight. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7361) Cassandra locks up in full GC when you assign the entire heap to row cache
[ https://issues.apache.org/jira/browse/CASSANDRA-7361?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14020072#comment-14020072 ] Benedict commented on CASSANDRA-7361: - bq. Actually we use Unsafe for the row cache; JNA is not required. Good point. The docs are incorrect then [http://www.datastax.com/documentation/cassandra/2.0/cassandra/operations/ops_tune_jvm_c.html]. I've emailed docs to let them know. Cassandra locks up in full GC when you assign the entire heap to row cache -- Key: CASSANDRA-7361 URL: https://issues.apache.org/jira/browse/CASSANDRA-7361 Project: Cassandra Issue Type: Bug Components: Core Environment: Ubuntu, RedHat, JDK 1.7 Reporter: Jacek Furmankiewicz Priority: Minor We have a long running batch load process, which runs for many hours. Massive amount of writes, in large mutation batches (we increase the thrift frame size to 45 MB). Everything goes well, but after about 3 hrs of processing everything locks up. We start getting NoHostsAvailable exceptions on the Java application side (with Astyanax as our driver), eventually socket timeouts. Looking at Cassandra, we can see that it is using nearly the full 8GB of heap and unable to free it. It spends most of its time in full GC, but the amount of memory does not go down. Here is a long sample from jstat to show this over an extended time period e.g. http://aep.appspot.com/display/NqqEagzGRLO_pCP2q8hZtitnuVU/ This continues even after we shut down our app. Nothing is connected to Cassandra any more, yet it is still stuck in full GC and cannot free up memory. Running nodetool tpstats shows that nothing is pending, all seems OK: {quote} Pool NameActive Pending Completed Blocked All time blocked ReadStage 0 0 69555935 0 0 RequestResponseStage 0 0 0 0 0 MutationStage 0 0 73123690 0 0 ReadRepairStage 0 0 0 0 0 ReplicateOnWriteStage 0 0 0 0 0 GossipStage 0 0 0 0 0 CacheCleanupExecutor 0 0 0 0 0 MigrationStage0 0 46 0 0 MemoryMeter 0 0 1125 0 0 FlushWriter 0 0824 0 30 ValidationExecutor0 0 0 0 0 InternalResponseStage 0 0 23 0 0 AntiEntropyStage 0 0 0 0 0 MemtablePostFlusher 0 0 1783 0 0 MiscStage 0 0 0 0 0 PendingRangeCalculator0 0 1 0 0 CompactionExecutor0 0 74330 0 0 commitlog_archiver0 0 0 0 0 HintedHandoff 0 0 0 0 0 Message type Dropped RANGE_SLICE 0 READ_REPAIR 0 PAGED_RANGE 0 BINARY 0 READ 585 MUTATION 75775 _TRACE 0 REQUEST_RESPONSE 0 COUNTER_MUTATION 0 {quote} We had this happen on 2 separate boxes, one with 2.0.6, the other with 2.0.8. Right now this is a total blocker for us. We are unable to process the customer data and have to abort in the middle of large processing. This is a new customer, so we did not have a chance to see if this occurred with 1.1 or 1.2 in the past (we moved to 2.0 recently). We have the Cassandra process still running, pls let us know if there is anything else we could run to give you more insight. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7361) Cassandra locks up in full GC when you assign the entire heap to row cache
[ https://issues.apache.org/jira/browse/CASSANDRA-7361?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14020074#comment-14020074 ] Benedict commented on CASSANDRA-7361: - bq. Should I log a separate issue for not finding JNA even though it is installed? Neither of these currently warrant a JIRA ticket. These issues should be investigated on the user IRC or mailing lists to establish if there is in fact a bug. It is most likely both of these are a configuration issue. Cassandra locks up in full GC when you assign the entire heap to row cache -- Key: CASSANDRA-7361 URL: https://issues.apache.org/jira/browse/CASSANDRA-7361 Project: Cassandra Issue Type: Bug Components: Core Environment: Ubuntu, RedHat, JDK 1.7 Reporter: Jacek Furmankiewicz Priority: Minor We have a long running batch load process, which runs for many hours. Massive amount of writes, in large mutation batches (we increase the thrift frame size to 45 MB). Everything goes well, but after about 3 hrs of processing everything locks up. We start getting NoHostsAvailable exceptions on the Java application side (with Astyanax as our driver), eventually socket timeouts. Looking at Cassandra, we can see that it is using nearly the full 8GB of heap and unable to free it. It spends most of its time in full GC, but the amount of memory does not go down. Here is a long sample from jstat to show this over an extended time period e.g. http://aep.appspot.com/display/NqqEagzGRLO_pCP2q8hZtitnuVU/ This continues even after we shut down our app. Nothing is connected to Cassandra any more, yet it is still stuck in full GC and cannot free up memory. Running nodetool tpstats shows that nothing is pending, all seems OK: {quote} Pool NameActive Pending Completed Blocked All time blocked ReadStage 0 0 69555935 0 0 RequestResponseStage 0 0 0 0 0 MutationStage 0 0 73123690 0 0 ReadRepairStage 0 0 0 0 0 ReplicateOnWriteStage 0 0 0 0 0 GossipStage 0 0 0 0 0 CacheCleanupExecutor 0 0 0 0 0 MigrationStage0 0 46 0 0 MemoryMeter 0 0 1125 0 0 FlushWriter 0 0824 0 30 ValidationExecutor0 0 0 0 0 InternalResponseStage 0 0 23 0 0 AntiEntropyStage 0 0 0 0 0 MemtablePostFlusher 0 0 1783 0 0 MiscStage 0 0 0 0 0 PendingRangeCalculator0 0 1 0 0 CompactionExecutor0 0 74330 0 0 commitlog_archiver0 0 0 0 0 HintedHandoff 0 0 0 0 0 Message type Dropped RANGE_SLICE 0 READ_REPAIR 0 PAGED_RANGE 0 BINARY 0 READ 585 MUTATION 75775 _TRACE 0 REQUEST_RESPONSE 0 COUNTER_MUTATION 0 {quote} We had this happen on 2 separate boxes, one with 2.0.6, the other with 2.0.8. Right now this is a total blocker for us. We are unable to process the customer data and have to abort in the middle of large processing. This is a new customer, so we did not have a chance to see if this occurred with 1.1 or 1.2 in the past (we moved to 2.0 recently). We have the Cassandra process still running, pls let us know if there is anything else we could run to give you more insight. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7359) Optimize locking in PaxosState
[ https://issues.apache.org/jira/browse/CASSANDRA-7359?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14020268#comment-14020268 ] Benedict commented on CASSANDRA-7359: - LGTM. Committed Optimize locking in PaxosState -- Key: CASSANDRA-7359 URL: https://issues.apache.org/jira/browse/CASSANDRA-7359 Project: Cassandra Issue Type: Improvement Reporter: sankalp kohli Assignee: Aleksey Yeschenko Priority: Minor Fix For: 2.1.0 Attachments: 7359-2.txt, 7359.txt In PaxosState, we want to lock on same rows and have created 1024 size array with java Objects in them to be used for locking. We should replace these Objects with some Lock so that we can know whether there is contention trying to acquire a lock for different rows. We can achieve that by also storing the hash of the row which acquired the lock. This will tell us if this needs to be improved. Here is an improvement which I was thinking about. Say two rows A and B map to the same 1024 bucket which we have. A get the lock and B has to wait. Here B can check if his hash is different and create a new object and chain it to the other one. This looks close to a hashMap with chaining for same key. The hard part will be removing the entries no longer being used. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7361) Cassandra locks up in full GC when you assign the entire heap to row cache
[ https://issues.apache.org/jira/browse/CASSANDRA-7361?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14020822#comment-14020822 ] Benedict commented on CASSANDRA-7361: - As I attempted to clarify, there is not insignificant on-heap overhead associated with even the off-heap row cache. Nowhere is it indicated that this is designed to free you from heap limitations, only that it reduces the heap burden. If your rows are small (100 bytes) you will find that the on-heap overhead is in fact *larger* than the data stored offheap. So the offheap limit you place on the data should be determined by the size of the rows it will typically cache. It is likely the documentation should be improved here, and it may be worth emailing d...@datastax.com with this suggestion; it is also possible a different improvement ticket could be filed asking to support a configurable limit to the on-heap overhead as well as the off-heap data, however this would not be prioritized highly at the moment since tuning your off-heap limit correctly would suffice to ensure correct behaviour. Cassandra locks up in full GC when you assign the entire heap to row cache -- Key: CASSANDRA-7361 URL: https://issues.apache.org/jira/browse/CASSANDRA-7361 Project: Cassandra Issue Type: Bug Components: Core Environment: Ubuntu, RedHat, JDK 1.7 Reporter: Jacek Furmankiewicz Priority: Minor Attachments: histogram.png, leaks_report.png, top_consumers.png We have a long running batch load process, which runs for many hours. Massive amount of writes, in large mutation batches (we increase the thrift frame size to 45 MB). Everything goes well, but after about 3 hrs of processing everything locks up. We start getting NoHostsAvailable exceptions on the Java application side (with Astyanax as our driver), eventually socket timeouts. Looking at Cassandra, we can see that it is using nearly the full 8GB of heap and unable to free it. It spends most of its time in full GC, but the amount of memory does not go down. Here is a long sample from jstat to show this over an extended time period e.g. http://aep.appspot.com/display/NqqEagzGRLO_pCP2q8hZtitnuVU/ This continues even after we shut down our app. Nothing is connected to Cassandra any more, yet it is still stuck in full GC and cannot free up memory. Running nodetool tpstats shows that nothing is pending, all seems OK: {quote} Pool NameActive Pending Completed Blocked All time blocked ReadStage 0 0 69555935 0 0 RequestResponseStage 0 0 0 0 0 MutationStage 0 0 73123690 0 0 ReadRepairStage 0 0 0 0 0 ReplicateOnWriteStage 0 0 0 0 0 GossipStage 0 0 0 0 0 CacheCleanupExecutor 0 0 0 0 0 MigrationStage0 0 46 0 0 MemoryMeter 0 0 1125 0 0 FlushWriter 0 0824 0 30 ValidationExecutor0 0 0 0 0 InternalResponseStage 0 0 23 0 0 AntiEntropyStage 0 0 0 0 0 MemtablePostFlusher 0 0 1783 0 0 MiscStage 0 0 0 0 0 PendingRangeCalculator0 0 1 0 0 CompactionExecutor0 0 74330 0 0 commitlog_archiver0 0 0 0 0 HintedHandoff 0 0 0 0 0 Message type Dropped RANGE_SLICE 0 READ_REPAIR 0 PAGED_RANGE 0 BINARY 0 READ 585 MUTATION 75775 _TRACE 0 REQUEST_RESPONSE 0 COUNTER_MUTATION 0 {quote} We had this happen on 2 separate boxes, one with 2.0.6, the other with 2.0.8. Right now this is a total blocker for us. We are unable to process the customer
[jira] [Comment Edited] (CASSANDRA-7361) Cassandra locks up in full GC when you assign the entire heap to row cache
[ https://issues.apache.org/jira/browse/CASSANDRA-7361?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14020822#comment-14020822 ] Benedict edited comment on CASSANDRA-7361 at 6/7/14 2:48 PM: - As I attempted to clarify, there is not insignificant on-heap overhead associated with even the off-heap row cache. Nowhere is it indicated that this is designed to free you from heap limitations, only that it reduces the heap burden. If your rows are small (150 bytes) you will find that the on-heap overhead is in fact *larger* than the data stored offheap. So the offheap limit you place on the data should be determined by the size of the rows it will typically cache. It is likely the documentation should be improved here, and it may be worth emailing d...@datastax.com with this suggestion; it is also possible a different improvement ticket could be filed asking to support a configurable limit to the on-heap overhead as well as the off-heap data, however this would not be prioritized highly at the moment since tuning your off-heap limit correctly would suffice to ensure correct behaviour. was (Author: benedict): As I attempted to clarify, there is not insignificant on-heap overhead associated with even the off-heap row cache. Nowhere is it indicated that this is designed to free you from heap limitations, only that it reduces the heap burden. If your rows are small (100 bytes) you will find that the on-heap overhead is in fact *larger* than the data stored offheap. So the offheap limit you place on the data should be determined by the size of the rows it will typically cache. It is likely the documentation should be improved here, and it may be worth emailing d...@datastax.com with this suggestion; it is also possible a different improvement ticket could be filed asking to support a configurable limit to the on-heap overhead as well as the off-heap data, however this would not be prioritized highly at the moment since tuning your off-heap limit correctly would suffice to ensure correct behaviour. Cassandra locks up in full GC when you assign the entire heap to row cache -- Key: CASSANDRA-7361 URL: https://issues.apache.org/jira/browse/CASSANDRA-7361 Project: Cassandra Issue Type: Bug Components: Core Environment: Ubuntu, RedHat, JDK 1.7 Reporter: Jacek Furmankiewicz Priority: Minor Attachments: histogram.png, leaks_report.png, top_consumers.png We have a long running batch load process, which runs for many hours. Massive amount of writes, in large mutation batches (we increase the thrift frame size to 45 MB). Everything goes well, but after about 3 hrs of processing everything locks up. We start getting NoHostsAvailable exceptions on the Java application side (with Astyanax as our driver), eventually socket timeouts. Looking at Cassandra, we can see that it is using nearly the full 8GB of heap and unable to free it. It spends most of its time in full GC, but the amount of memory does not go down. Here is a long sample from jstat to show this over an extended time period e.g. http://aep.appspot.com/display/NqqEagzGRLO_pCP2q8hZtitnuVU/ This continues even after we shut down our app. Nothing is connected to Cassandra any more, yet it is still stuck in full GC and cannot free up memory. Running nodetool tpstats shows that nothing is pending, all seems OK: {quote} Pool NameActive Pending Completed Blocked All time blocked ReadStage 0 0 69555935 0 0 RequestResponseStage 0 0 0 0 0 MutationStage 0 0 73123690 0 0 ReadRepairStage 0 0 0 0 0 ReplicateOnWriteStage 0 0 0 0 0 GossipStage 0 0 0 0 0 CacheCleanupExecutor 0 0 0 0 0 MigrationStage0 0 46 0 0 MemoryMeter 0 0 1125 0 0 FlushWriter 0 0824 0 30 ValidationExecutor0 0 0 0 0 InternalResponseStage 0 0 23 0 0 AntiEntropyStage 0 0 0 0 0 MemtablePostFlusher 0 0
[jira] [Commented] (CASSANDRA-7365) some compactions do not works under windows (file in use during rename)
[ https://issues.apache.org/jira/browse/CASSANDRA-7365?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14027350#comment-14027350 ] Benedict commented on CASSANDRA-7365: - Simplest thing is to disable the feature on Windows some compactions do not works under windows (file in use during rename) --- Key: CASSANDRA-7365 URL: https://issues.apache.org/jira/browse/CASSANDRA-7365 Project: Cassandra Issue Type: Bug Components: Core Environment: jdk7, cassandra-2.1rc1, os windows 32 bit Reporter: Radim Kolar Assignee: Joshua McKenzie Labels: Windows Fix For: 2.1.1 Attachments: cassandra.yaml, system.log compaction do not works under windows due to file rename fails: (Pro es nemß p°Ýstup k souboru, neboŁ jej prßvý vyu×Ývß jinř proces = process can not access file because its in use by another process). Not all compactions are broken. compactions done during server startup on system tables works fine. INFO 18:30:27 Completed flushing c:\cassandra-2.1\data\system\compactions_in_progress-55080ab05d9c388690a4acb25fe1f77b\system-compactions_in_progress-ka-6-Dat.db (42 bytes) for commitlog position ReplayPosition(segmentId=1402165543361, psition=8024611) ERROR 18:30:27 Exception in thread hread[CompactionExecutor:5,1,RMI Runtime] java.lang.RuntimeException: Failed to rename c:\cassandra-2.1\data\test\sipdb-5 f51090ee6511e3815625991ef2b954\test-sipdb-tmp-ka-7-Index.db to c:\cassandra-2.1 data\test\sipdb-58f51090ee6511e3815625991ef2b954\test-sipdb-ka-7-Index.db at org.apache.cassandra.io.util.FileUtils.renameWithConfirm(FileUtils.j va:167) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.util.FileUtils.renameWithConfirm(FileUtils.j va:151) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableWriter.rename(SSTableWriter.j va:512) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableWriter.rename(SSTableWriter.j va:504) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableWriter.close(SSTableWriter.ja a:479) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableWriter.closeAndOpenReader(SST bleWriter.java:427) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableWriter.closeAndOpenReader(SST bleWriter.java:422) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableRewriter.finish(SSTableRewrit r.java:312) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableRewriter.finish(SSTableRewrit r.java:306) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.compaction.CompactionTask.runWith(Compaction ask.java:188) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAware unnable.java:48) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java: 8) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(Co pactionTask.java:74) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(Ab tractCompactionTask.java:59) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompa tionTask.run(CompactionManager.java:235) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0 rc1] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:4 1) ~[na:1.7.0_60] at java.util.concurrent.FutureTask.run(FutureTask.java:262) ~[na:1.7.0_ 0] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor java:1145) ~[na:1.7.0_60] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecuto .java:615) [na:1.7.0_60] at java.lang.Thread.run(Thread.java:745) [na:1.7.0_60] Caused by: java.nio.file.FileSystemException: c:\cassandra-2.1\data\test\sipdb- 8f51090ee6511e3815625991ef2b954\test-sipdb-tmp-ka-7-Index.db - c:\cassandra-2. \data\test\sipdb-58f51090ee6511e3815625991ef2b954\test-sipdb-ka-7-Index.db: Pro es nemß p°Ýstup k souboru, neboŁ jej prßvý vyu×Ývß jinř proces. at sun.nio.fs.WindowsException.translateToIOException(WindowsException. ava:86) ~[na:1.7.0_60] at sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.ja a:97) ~[na:1.7.0_60] at
[jira] [Updated] (CASSANDRA-5483) Repair tracing
[ https://issues.apache.org/jira/browse/CASSANDRA-5483?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benedict updated CASSANDRA-5483: Attachment: 5483-v15.patch Agreed. Uploaded a patch that has identical behaviour, but is a bit easier to understand. Repair tracing -- Key: CASSANDRA-5483 URL: https://issues.apache.org/jira/browse/CASSANDRA-5483 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Yuki Morishita Assignee: Ben Chan Priority: Minor Labels: repair Attachments: 5483-full-trunk.txt, 5483-v06-04-Allow-tracing-ttl-to-be-configured.patch, 5483-v06-05-Add-a-command-column-to-system_traces.events.patch, 5483-v06-06-Fix-interruption-in-tracestate-propagation.patch, 5483-v07-07-Better-constructor-parameters-for-DebuggableThreadPoolExecutor.patch, 5483-v07-08-Fix-brace-style.patch, 5483-v07-09-Add-trace-option-to-a-more-complete-set-of-repair-functions.patch, 5483-v07-10-Correct-name-of-boolean-repairedAt-to-fullRepair.patch, 5483-v08-11-Shorten-trace-messages.-Use-Tracing-begin.patch, 5483-v08-12-Trace-streaming-in-Differencer-StreamingRepairTask.patch, 5483-v08-13-sendNotification-of-local-traces-back-to-nodetool.patch, 5483-v08-14-Poll-system_traces.events.patch, 5483-v08-15-Limit-trace-notifications.-Add-exponential-backoff.patch, 5483-v09-16-Fix-hang-caused-by-incorrect-exit-code.patch, 5483-v10-17-minor-bugfixes-and-changes.patch, 5483-v10-rebased-and-squashed-471f5cc.patch, 5483-v11-01-squashed.patch, 5483-v11-squashed-nits.patch, 5483-v12-02-cassandra-yaml-ttl-doc.patch, 5483-v13-608fb03-May-14-trace-formatting-changes.patch, 5483-v14-01-squashed.patch, 5483-v15.patch, ccm-repair-test, cqlsh-left-justify-text-columns.patch, prerepair-vs-postbuggedrepair.diff, test-5483-system_traces-events.txt, trunk@4620823-5483-v02-0001-Trace-filtering-and-tracestate-propagation.patch, trunk@4620823-5483-v02-0002-Put-a-few-traces-parallel-to-the-repair-logging.patch, tr...@8ebeee1-5483-v01-001-trace-filtering-and-tracestate-propagation.txt, tr...@8ebeee1-5483-v01-002-simple-repair-tracing.txt, v02p02-5483-v03-0003-Make-repair-tracing-controllable-via-nodetool.patch, v02p02-5483-v04-0003-This-time-use-an-EnumSet-to-pass-boolean-repair-options.patch, v02p02-5483-v05-0003-Use-long-instead-of-EnumSet-to-work-with-JMX.patch I think it would be nice to log repair stats and results like query tracing stores traces to system keyspace. With it, you don't have to lookup each log file to see what was the status and how it performed the repair you invoked. Instead, you can query the repair log with session ID to see the state and stats of all nodes involved in that repair session. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Comment Edited] (CASSANDRA-5483) Repair tracing
[ https://issues.apache.org/jira/browse/CASSANDRA-5483?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14027754#comment-14027754 ] Benedict edited comment on CASSANDRA-5483 at 6/11/14 1:34 PM: -- Agreed. Uploaded a patch that has identical behaviour, but is a bit easier to understand. I should note I've made no attempt to corroborate this behaviour is sensible; I've only simplified it :) was (Author: benedict): Agreed. Uploaded a patch that has identical behaviour, but is a bit easier to understand. Repair tracing -- Key: CASSANDRA-5483 URL: https://issues.apache.org/jira/browse/CASSANDRA-5483 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Yuki Morishita Assignee: Ben Chan Priority: Minor Labels: repair Attachments: 5483-full-trunk.txt, 5483-v06-04-Allow-tracing-ttl-to-be-configured.patch, 5483-v06-05-Add-a-command-column-to-system_traces.events.patch, 5483-v06-06-Fix-interruption-in-tracestate-propagation.patch, 5483-v07-07-Better-constructor-parameters-for-DebuggableThreadPoolExecutor.patch, 5483-v07-08-Fix-brace-style.patch, 5483-v07-09-Add-trace-option-to-a-more-complete-set-of-repair-functions.patch, 5483-v07-10-Correct-name-of-boolean-repairedAt-to-fullRepair.patch, 5483-v08-11-Shorten-trace-messages.-Use-Tracing-begin.patch, 5483-v08-12-Trace-streaming-in-Differencer-StreamingRepairTask.patch, 5483-v08-13-sendNotification-of-local-traces-back-to-nodetool.patch, 5483-v08-14-Poll-system_traces.events.patch, 5483-v08-15-Limit-trace-notifications.-Add-exponential-backoff.patch, 5483-v09-16-Fix-hang-caused-by-incorrect-exit-code.patch, 5483-v10-17-minor-bugfixes-and-changes.patch, 5483-v10-rebased-and-squashed-471f5cc.patch, 5483-v11-01-squashed.patch, 5483-v11-squashed-nits.patch, 5483-v12-02-cassandra-yaml-ttl-doc.patch, 5483-v13-608fb03-May-14-trace-formatting-changes.patch, 5483-v14-01-squashed.patch, 5483-v15.patch, ccm-repair-test, cqlsh-left-justify-text-columns.patch, prerepair-vs-postbuggedrepair.diff, test-5483-system_traces-events.txt, trunk@4620823-5483-v02-0001-Trace-filtering-and-tracestate-propagation.patch, trunk@4620823-5483-v02-0002-Put-a-few-traces-parallel-to-the-repair-logging.patch, tr...@8ebeee1-5483-v01-001-trace-filtering-and-tracestate-propagation.txt, tr...@8ebeee1-5483-v01-002-simple-repair-tracing.txt, v02p02-5483-v03-0003-Make-repair-tracing-controllable-via-nodetool.patch, v02p02-5483-v04-0003-This-time-use-an-EnumSet-to-pass-boolean-repair-options.patch, v02p02-5483-v05-0003-Use-long-instead-of-EnumSet-to-work-with-JMX.patch I think it would be nice to log repair stats and results like query tracing stores traces to system keyspace. With it, you don't have to lookup each log file to see what was the status and how it performed the repair you invoked. Instead, you can query the repair log with session ID to see the state and stats of all nodes involved in that repair session. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-5483) Repair tracing
[ https://issues.apache.org/jira/browse/CASSANDRA-5483?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14028104#comment-14028104 ] Benedict commented on CASSANDRA-5483: - Like I said, I only replicated functionality - the exponential backoff is maintained, it's only defined more simply (as far as I could read you doubled the wait time every second timeout, which is what happens now also). As the code is defined now we explicitly create a repair session and spawn a specific thread to process it, so we have a guarantee there's only one thread. It doesn't matter if the consumer is in its method when notifyAll is called; if it isn't it will receive the notification as soon as it next enters the method. When I have some time I'll take a look and see if there's an alternative approach. Repair tracing -- Key: CASSANDRA-5483 URL: https://issues.apache.org/jira/browse/CASSANDRA-5483 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Yuki Morishita Assignee: Ben Chan Priority: Minor Labels: repair Attachments: 5483-full-trunk.txt, 5483-v06-04-Allow-tracing-ttl-to-be-configured.patch, 5483-v06-05-Add-a-command-column-to-system_traces.events.patch, 5483-v06-06-Fix-interruption-in-tracestate-propagation.patch, 5483-v07-07-Better-constructor-parameters-for-DebuggableThreadPoolExecutor.patch, 5483-v07-08-Fix-brace-style.patch, 5483-v07-09-Add-trace-option-to-a-more-complete-set-of-repair-functions.patch, 5483-v07-10-Correct-name-of-boolean-repairedAt-to-fullRepair.patch, 5483-v08-11-Shorten-trace-messages.-Use-Tracing-begin.patch, 5483-v08-12-Trace-streaming-in-Differencer-StreamingRepairTask.patch, 5483-v08-13-sendNotification-of-local-traces-back-to-nodetool.patch, 5483-v08-14-Poll-system_traces.events.patch, 5483-v08-15-Limit-trace-notifications.-Add-exponential-backoff.patch, 5483-v09-16-Fix-hang-caused-by-incorrect-exit-code.patch, 5483-v10-17-minor-bugfixes-and-changes.patch, 5483-v10-rebased-and-squashed-471f5cc.patch, 5483-v11-01-squashed.patch, 5483-v11-squashed-nits.patch, 5483-v12-02-cassandra-yaml-ttl-doc.patch, 5483-v13-608fb03-May-14-trace-formatting-changes.patch, 5483-v14-01-squashed.patch, 5483-v15.patch, ccm-repair-test, cqlsh-left-justify-text-columns.patch, prerepair-vs-postbuggedrepair.diff, test-5483-system_traces-events.txt, trunk@4620823-5483-v02-0001-Trace-filtering-and-tracestate-propagation.patch, trunk@4620823-5483-v02-0002-Put-a-few-traces-parallel-to-the-repair-logging.patch, tr...@8ebeee1-5483-v01-001-trace-filtering-and-tracestate-propagation.txt, tr...@8ebeee1-5483-v01-002-simple-repair-tracing.txt, v02p02-5483-v03-0003-Make-repair-tracing-controllable-via-nodetool.patch, v02p02-5483-v04-0003-This-time-use-an-EnumSet-to-pass-boolean-repair-options.patch, v02p02-5483-v05-0003-Use-long-instead-of-EnumSet-to-work-with-JMX.patch I think it would be nice to log repair stats and results like query tracing stores traces to system keyspace. With it, you don't have to lookup each log file to see what was the status and how it performed the repair you invoked. Instead, you can query the repair log with session ID to see the state and stats of all nodes involved in that repair session. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6995) Execute local ONE/LOCAL_ONE reads on request thread instead of dispatching to read stage
[ https://issues.apache.org/jira/browse/CASSANDRA-6995?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14030128#comment-14030128 ] Benedict commented on CASSANDRA-6995: - We basically implemented this in CASSANDRA-4718, so closing it Execute local ONE/LOCAL_ONE reads on request thread instead of dispatching to read stage Key: CASSANDRA-6995 URL: https://issues.apache.org/jira/browse/CASSANDRA-6995 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Jason Brown Assignee: Jason Brown Priority: Minor Labels: performance Fix For: 3.0 Attachments: 6995-v1.diff, syncread-stress.txt When performing a read local to a coordinator node, AbstractReadExecutor will create a new SP.LocalReadRunnable and drop it into the read stage for asynchronous execution. If you are using a client that intelligently routes read requests to a node holding the data for a given request, and are using CL.ONE/LOCAL_ONE, the enqueuing SP.LocalReadRunnable and waiting for the context switches (and possible NUMA misses) adds unneccesary latency. We can reduce that latency and improve throughput by avoiding the queueing and thread context switching by simply executing the SP.LocalReadRunnable synchronously in the request thread. Testing on a three node cluster (each with 32 cpus, 132 GB ram) yields ~10% improvement in throughput and ~20% speedup on avg/95/99 percentiles (99.9% was about 5-10% improvement). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Resolved] (CASSANDRA-6995) Execute local ONE/LOCAL_ONE reads on request thread instead of dispatching to read stage
[ https://issues.apache.org/jira/browse/CASSANDRA-6995?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benedict resolved CASSANDRA-6995. - Resolution: Fixed Execute local ONE/LOCAL_ONE reads on request thread instead of dispatching to read stage Key: CASSANDRA-6995 URL: https://issues.apache.org/jira/browse/CASSANDRA-6995 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Jason Brown Assignee: Jason Brown Priority: Minor Labels: performance Fix For: 3.0 Attachments: 6995-v1.diff, syncread-stress.txt When performing a read local to a coordinator node, AbstractReadExecutor will create a new SP.LocalReadRunnable and drop it into the read stage for asynchronous execution. If you are using a client that intelligently routes read requests to a node holding the data for a given request, and are using CL.ONE/LOCAL_ONE, the enqueuing SP.LocalReadRunnable and waiting for the context switches (and possible NUMA misses) adds unneccesary latency. We can reduce that latency and improve throughput by avoiding the queueing and thread context switching by simply executing the SP.LocalReadRunnable synchronously in the request thread. Testing on a three node cluster (each with 32 cpus, 132 GB ram) yields ~10% improvement in throughput and ~20% speedup on avg/95/99 percentiles (99.9% was about 5-10% improvement). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Resolved] (CASSANDRA-7045) Investigate changes to threading model for network consumers to improve performance for small messages
[ https://issues.apache.org/jira/browse/CASSANDRA-7045?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benedict resolved CASSANDRA-7045. - Resolution: Duplicate CASSANDRA-4718 has delivered most of the benefit I had envisaged with this ticket, and is likely the best we'll manage without major application changes, so closing as duplicate. Investigate changes to threading model for network consumers to improve performance for small messages -- Key: CASSANDRA-7045 URL: https://issues.apache.org/jira/browse/CASSANDRA-7045 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Benedict Assignee: Benedict Labels: performance See CASSANDRA-7029 for original discussion. We can (maybe) optimise the threading model for network consumers: since we generally process very small messages (200 bytes not at all implausible), the thread signalling costs on the processing thread can actually dramatically impede throughput. In general it costs ~10micros to signal (and passing the message to another thread for processing in the current model requires signalling). For 200-byte messages this caps our throughput at 20MB/s. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7371) DELETEs get lost
[ https://issues.apache.org/jira/browse/CASSANDRA-7371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14032680#comment-14032680 ] Benedict commented on CASSANDRA-7371: - Two minor comments: # In addAllWithSizeDelta, I would only copy the deletionInfo on the first iteration (if there's a race we'll copy multiple times) # In RangeTombstoneList.copy we should just allocate an empty array for starts/ends, as we overwrite the data we copy anyway (this is slightly cheaper, as the arrays are zeroed before allocation) Otherwise LGTM DELETEs get lost Key: CASSANDRA-7371 URL: https://issues.apache.org/jira/browse/CASSANDRA-7371 Project: Cassandra Issue Type: Bug Components: Core Environment: 2.1 git branch since merge commit 4722fe70aa9ae1b62772cfa1a1de58ef289445d5 (RefCount native frames from netty to avoid corruption bugs) Reporter: Robert Stupp Assignee: T Jake Luciani Priority: Blocker Fix For: 2.1.0 Attachments: 7371.txt, Cassandra7371.java The mentioned commit introduced a bug which is not easy to reproduce: Workload description: - One INSERT into a table - multiple concurrent SELECTs against different tables (one select returns a result) - One UPDATE against the same table as the INSERT - (same) multiple concurrent SELECTs against different tables (one select returns a result) - One DELETE against the same table as the INSERT - (same) multiple concurrent SELECTs against different tables Expected is that the last bunch of SELECTs returns no result. But since commit SHA the DELETE gets not processed. To clarify - the DELETE is not delayed - it is not executed at all. Checked against a single node C* cluster. Does only affect unreleased 2.1 - not 2.0 nor 1.2. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Assigned] (CASSANDRA-7403) Reconciliation doesn't consider fields specific to expiring cells
[ https://issues.apache.org/jira/browse/CASSANDRA-7403?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benedict reassigned CASSANDRA-7403: --- Assignee: Benedict Reconciliation doesn't consider fields specific to expiring cells -- Key: CASSANDRA-7403 URL: https://issues.apache.org/jira/browse/CASSANDRA-7403 Project: Cassandra Issue Type: Bug Components: Core Reporter: Sam Tunnicliffe Assignee: Benedict Reconciling 2 ExpiringColumns which are equal in every way except for the localExpirationTime field will always favour the instance on which reconcile is called as fields specific to expiration are not considered. This is actually beneficial in pre-2.1 versions as in AtomicSortedColumns.Holder.addColumn we call reconcile on the new column, which 'wins' the reconcilliation and so the localExpirationTime is effectively extended. From 2.1 onwards, reconcile is actually called on the existing value (in BTreeSortedColumns.ColumnUpdater) and so it wins the reconcilliation and the ttl doesn't get extended. The same thing happens in the iterator returned from MergeIterator.Reducer.getReducer() so we see the same behaviour when merging cells from the multiple SSTables and/or memtables. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7062) Extension of static columns for compound cluster keys
[ https://issues.apache.org/jira/browse/CASSANDRA-7062?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14035976#comment-14035976 ] Benedict commented on CASSANDRA-7062: - I am not very keen on this idea, as it makes future storage changes potentially very challenging. With simple static rows we can always separate their storage from the main tabular data if we need to, but arbitrary nesting of data substantially complicates the data model. It may be something we want to support at some point, but I'd rather not tackle it now. Extension of static columns for compound cluster keys - Key: CASSANDRA-7062 URL: https://issues.apache.org/jira/browse/CASSANDRA-7062 Project: Cassandra Issue Type: New Feature Reporter: Constance Eustace CASSANDRA-6561 implemented static columns for a given partition key. What this is proposing for a compound cluster key is a static column that is static at intermediate parts of a compound cluster key. This example shows a table modelling a moderately complex EAV pattern : {code} CREATE TABLE t ( entityID text, propertyName text, valueIndex text, entityName text static (entityID), propertyType text static (entityID, propertyName), propertyRelations Listtext static (entityID, propertyName), data text, PRIMARY KEY (entityID, (propertyName,valueIndex)) ) {code} So in this example has the following static columns: - the entityName column behaves exactly as CASSANDRA-6561 details, so all cluster rows have the same value - the propertyType and propertyRelations columns are static with respect to the remaining parts of the cluster key (that is, across all valueIndex values for a given propertyName), so an update to those values for an entityID and a propertyName will be shared/constant by all the value rows... Is this a relatively simple extension of the same mechanism in -6561, or is this a whoa, you have no idea what you are proposing? Sample data: Mary and Jane aren't married... {code} INSERT INTO t (entityID, entityName, propertyName, propertyType, valueIndex, data) VALUES ('0001','MARY MATALIN','married','SingleValue','0','false'); INSERT INTO t (entityID, entityName, propertyName, propertyType, valueIndex, data) VALUES ('0002','JANE JOHNSON','married','SingleValue','0','false'); INSERT INTO t (entityID, entityName, propertyName, propertyType, valueIndex) VALUES ('0001','MARY MATALIN','kids','NOVALUE',''); INSERT INTO t (entityID, entityName, propertyName, propertyType, valueIndex) VALUES ('0002','JANE JOHNSON','kids','NOVALUE',''); {code} {code} SELECT * FROM t: 0001 MARY MATALIN married SingleValue 0 false 0001 MARY MATALIN kids NOVALUE null 0002 JANE JOHNSON married SingleValue 0 false 0002 JANE JOHNSON kids NOVALUE null {code} Then mary and jane get married (so the entityName column that is static on the partition key is updated just like CASSANDRA-6561 ) {code} INSERT INTO t (entityID, entityName, propertyName, propertyType, valueIndex, data) VALUES ('0001','MARY SMITH','married','SingleValue','0','TRUE'); INSERT INTO t (entityID, entityName, propertyName, propertyType, valueIndex, data) VALUES ('0002','JANE JONES','married','SingleValue','0','TRUE'); {code} {code} SELECT * FROM t: 0001 MARY SMITH married SingleValue 0 TRUE 0001 MARY SMITH kids NOVALUE null 0002 JANE JONES married SingleValue 0 TRUE 0002 JANE JONES kids NOVALUE null {code} Then mary and jane have a kid, so we add another value to the kids attribute: {code} INSERT INTO t (entityID, propertyName, propertyType, valueIndex,data) VALUES ('0001','kids','SingleValue','0','JIM-BOB'); INSERT INTO t (entityID, propertyName, propertyType, valueIndex,data) VALUES ('0002','kids','SingleValue','0','JENNY'); {code} {code} SELECT * FROM t: 0001 MARY SMITH married SingleValue 0 TRUE 0001 MARY SMITH kids SingleValuenull 0001 MARY SMITH kids SingleValue 0 JIM-BOB 0002 JANE JONES married SingleValue 0 TRUE 0002 JANE JONES kids SingleValuenull 0002 JANE JONES kids SingleValue 0 JENNY {code} Then Mary has ANOTHER kid, which demonstrates the partially static column relative to the cluster key, as ALL value rows for the property 'kids' get updated to the new value: {code} INSERT INTO t (entityID, propertyName, propertyType, valueIndex,data) VALUES ('0001','kids','MultiValue','1','HARRY'); {code} {code} SELECT * FROM t: 0001 MARY SMITH married SingleValue 0 TRUE 0001 MARY SMITH kids MultiValue null 0001 MARY SMITH kids MultiValue 0 JIM-BOB 0001 MARY SMITH kids MultiValue 1 HARRY
[jira] [Resolved] (CASSANDRA-6843) Reclaim overwritten off-heap memtable space early
[ https://issues.apache.org/jira/browse/CASSANDRA-6843?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benedict resolved CASSANDRA-6843. - Resolution: Later Reclaim overwritten off-heap memtable space early - Key: CASSANDRA-6843 URL: https://issues.apache.org/jira/browse/CASSANDRA-6843 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Benedict Assignee: Benedict Labels: performance Fix For: 3.0 Perform GC on the off-heap memtable arenas -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6755) Optimise CellName/Composite comparisons for NativeCell
[ https://issues.apache.org/jira/browse/CASSANDRA-6755?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14036173#comment-14036173 ] Benedict commented on CASSANDRA-6755: - [~tjake] I think you're still planning on dropping a version with inlining right? Optimise CellName/Composite comparisons for NativeCell -- Key: CASSANDRA-6755 URL: https://issues.apache.org/jira/browse/CASSANDRA-6755 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: T Jake Luciani Priority: Minor Labels: performance Fix For: 2.1.0 Attachments: 6755.txt As discussed in CASSANDRA-6694, to reduce temporary garbage generation we should minimise the incidence of CellName component extraction. The biggest win will be to perform comparisons on Cell where possible, instead of CellName, so that Native*Cell can use its extra information to avoid creating any ByteBuffer objects -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6108) Create timeid64 type
[ https://issues.apache.org/jira/browse/CASSANDRA-6108?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14036186#comment-14036186 ] Benedict commented on CASSANDRA-6108: - It seems this implementation has some slightly weird behaviour: if more than 1000 timeuuids are produced in a single millisecond (potentially decoupled from actually committing them to the database), updates from another client can get overridden. Say a single client produces 100k updates before submitting them to be processed asynchronously, these updates will create a 100ms window during which any other client's writes will lose despite occurring later. This seems a pretty dangerous tradeoff, and one user is likely to expect. Create timeid64 type Key: CASSANDRA-6108 URL: https://issues.apache.org/jira/browse/CASSANDRA-6108 Project: Cassandra Issue Type: New Feature Components: API, Core Reporter: Jonathan Ellis Assignee: Sylvain Lebresne Priority: Minor Fix For: 2.1.1 As discussed in CASSANDRA-6106, we could create a 64-bit type with 48 bits of timestamp and 16 bites of unique coordinator id. This would give us a unique-per-cluster value that could be used as a more compact replacement for many TimeUUID uses. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Comment Edited] (CASSANDRA-6108) Create timeid64 type
[ https://issues.apache.org/jira/browse/CASSANDRA-6108?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14036186#comment-14036186 ] Benedict edited comment on CASSANDRA-6108 at 6/18/14 7:11 PM: -- It seems this implementation has some slightly weird behaviour: if more than 1000 timeuuids are produced in a single millisecond (potentially decoupled from actually committing them to the database), updates from another client can get overridden. Say a single client produces 100k updates before submitting them to be processed asynchronously, these updates will create a 100ms window during which any other client's writes will lose despite occurring later. This seems a pretty dangerous tradeoff, and one the user is unlikely to expect. was (Author: benedict): It seems this implementation has some slightly weird behaviour: if more than 1000 timeuuids are produced in a single millisecond (potentially decoupled from actually committing them to the database), updates from another client can get overridden. Say a single client produces 100k updates before submitting them to be processed asynchronously, these updates will create a 100ms window during which any other client's writes will lose despite occurring later. This seems a pretty dangerous tradeoff, and one user is likely to expect. Create timeid64 type Key: CASSANDRA-6108 URL: https://issues.apache.org/jira/browse/CASSANDRA-6108 Project: Cassandra Issue Type: New Feature Components: API, Core Reporter: Jonathan Ellis Assignee: Sylvain Lebresne Priority: Minor Fix For: 2.1.1 As discussed in CASSANDRA-6106, we could create a 64-bit type with 48 bits of timestamp and 16 bites of unique coordinator id. This would give us a unique-per-cluster value that could be used as a more compact replacement for many TimeUUID uses. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6843) Reclaim overwritten off-heap memtable space early
[ https://issues.apache.org/jira/browse/CASSANDRA-6843?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14036193#comment-14036193 ] Benedict commented on CASSANDRA-6843: - Closing as later, as for 3.0/3.1 we are likely to be revisiting memtable design along with new storage format changes (ticket to follow), and so since this is unlikely to make it into 2.1 it is probably a wasted effort at this point, as the code will change under it. Will revisit once the dust settles. Reclaim overwritten off-heap memtable space early - Key: CASSANDRA-6843 URL: https://issues.apache.org/jira/browse/CASSANDRA-6843 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Benedict Assignee: Benedict Labels: performance Fix For: 3.0 Perform GC on the off-heap memtable arenas -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7275) Errors in FlushRunnable may leave threads hung
[ https://issues.apache.org/jira/browse/CASSANDRA-7275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14038635#comment-14038635 ] Benedict commented on CASSANDRA-7275: - This will break the commit log, by causing records to be discarded out of order. The future does not get placed on the executor until the onSuccess is called now, so run order is simply in order of flush completion and no longer in order of submission, but the only point of separating the work onto the postFlush is to ensure it is run in submission order (but not before the flush is finished - see the comment that is now attached to onSuccess). The simplest correct solution is probably to annotate the post flush runnable with a state variable indicating success/failure, which is set before the latch is triggered. If you're modifying these parts of the code where correctness is paramount and not always obvious, it would be great if you could explicitly run it past a third set of eyes, as I only happened to spot this in the commits@ feeds, and as it's a concurrency bug could easily have not been spotted. Although we could no doubt craft a specific test to look for this scenario, and perhaps we should. Errors in FlushRunnable may leave threads hung -- Key: CASSANDRA-7275 URL: https://issues.apache.org/jira/browse/CASSANDRA-7275 Project: Cassandra Issue Type: Bug Components: Core Reporter: Tyler Hobbs Assignee: Yuki Morishita Priority: Minor Fix For: 1.2.17, 2.0.9 Attachments: 0001-Move-latch.countDown-into-finally-block.patch, 7252-2.0-v2.txt In Memtable.FlushRunnable, the CountDownLatch will never be counted down if there are errors, which results in hanging any threads that are waiting for the flush to complete. For example, an error like this causes the problem: {noformat} ERROR [FlushWriter:474] 2014-05-20 12:10:31,137 CassandraDaemon.java (line 198) Exception in thread Thread[FlushWriter:474,5,main] java.lang.IllegalArgumentException at java.nio.Buffer.position(Unknown Source) at org.apache.cassandra.db.marshal.AbstractCompositeType.getBytes(AbstractCompositeType.java:64) at org.apache.cassandra.db.marshal.AbstractCompositeType.getWithShortLength(AbstractCompositeType.java:72) at org.apache.cassandra.db.marshal.AbstractCompositeType.split(AbstractCompositeType.java:138) at org.apache.cassandra.io.sstable.ColumnNameHelper.minComponents(ColumnNameHelper.java:103) at org.apache.cassandra.db.ColumnFamily.getColumnStats(ColumnFamily.java:439) at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:194) at org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:397) at org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:350) at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source) at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source) at java.lang.Thread.run(Unknown Source) {noformat} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7421) CompoundSparseCellName/WithCollection report smaller heap sizes then actual
[ https://issues.apache.org/jira/browse/CASSANDRA-7421?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benedict updated CASSANDRA-7421: Attachment: 7421v2.patch Does need tidying up, but we should avoid using ObjectSizes.measure() for two reasons: # The ColumnIdentifier for compound sparse tables are always shared / declared (to my knowledge - only sparse tables permit dynamic column names) # It's very expensive, so even if they weren't we should prefer columnName.unsharedHeapSize() I've taken this opportunity to finally tidy up the mismatched naming of unshared vs excess heap size methods (some were renamed as part of bikeshedding their first commit, but some were missed) CompoundSparseCellName/WithCollection report smaller heap sizes then actual Key: CASSANDRA-7421 URL: https://issues.apache.org/jira/browse/CASSANDRA-7421 Project: Cassandra Issue Type: Bug Components: Core Reporter: Dave Brosius Assignee: Dave Brosius Priority: Trivial Fix For: 2.1 rc2 Attachments: 7421.txt, 7421v2.patch Class relies on sizing of super class, which doesn't take into consideration it's own foot print, along with it's excess fields. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6917) enum data type
[ https://issues.apache.org/jira/browse/CASSANDRA-6917?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14039250#comment-14039250 ] Benedict commented on CASSANDRA-6917: - My main goal with enums is only uniquely representing a string value efficiently. Supporting custom orderings on the data might be a possibility for enums defined _up front_, however in this case I want to support denormalising arbitrary string data, the universe of which could be moderately large (certainly 100k+) and is not necessarily known in advance. An enum that must be defined up front with a predetermined ordering is frankly just as easy to implement client-side, so whilst it might be a nice feature to support eventually, I consider it out of scope for this ticket, and I think guaranteeing any specific order may be undesirable for write performance. enum data type -- Key: CASSANDRA-6917 URL: https://issues.apache.org/jira/browse/CASSANDRA-6917 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Benedict Priority: Minor Labels: performance It seems like it would be useful to support an enum data type, that automatically converts string data from the user into a fixed-width data type with guaranteed uniqueness across the cluster. This data would be replicated to all nodes for lookup, but ideally would use only the keyspace RF to determine nodes for coordinating quorum writes/consistency. This would not only permit improved local disk and inter-node network IO for symbology information (e.g. stock tickers, ISINs, etc), but also potentially for column identifiers also, which are currently stored as their full string representation. It should be possible then with later updates to propagate the enum map (lazily) to clients through the native protocol, reducing network IO further. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6477) Global indexes
[ https://issues.apache.org/jira/browse/CASSANDRA-6477?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14039757#comment-14039757 ] Benedict commented on CASSANDRA-6477: - There's a possible simplification of my proposed algorithm: if we start by forcing writes/reads to quorum if they touch a global index, we could simply proxy each write deterministically to one other replica, so long as we wait for both writes to complete before a single source/table/undelrying replica reports success. We need to bikeshed nomenclature a bit as well: we already have the term primary replica reserved for regular replication. It will help is we can disambiguate between this and the source/underlying table replica(s). So we'd have primary table replica, secondary table replica, primary index replica, secondary index replica? Or something along those lines. Global indexes -- Key: CASSANDRA-6477 URL: https://issues.apache.org/jira/browse/CASSANDRA-6477 Project: Cassandra Issue Type: New Feature Components: API, Core Reporter: Jonathan Ellis Fix For: 3.0 Local indexes are suitable for low-cardinality data, where spreading the index across the cluster is a Good Thing. However, for high-cardinality data, local indexes require querying most nodes in the cluster even if only a handful of rows is returned. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6108) Create timeid64 type
[ https://issues.apache.org/jira/browse/CASSANDRA-6108?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14040168#comment-14040168 ] Benedict commented on CASSANDRA-6108: - Is this approach inherently incompatible with client-provided-timestamps? As far as replacing timestamps are concerned, anyway; not necessarily as a datatype. I think solving this problem properly is going to be very challenging, but I'd like to propose the following (rough sketch) of a solution. Note that this doesn't solve timeid64, as much as mostly-unique cluster-wide timestamps in 64-bits or less that can be generated by the client: # I propose each client auto-generates a 20-bit id on startup. We can try to make this guaranteed unique, but I think a random number is probably sufficient. # We define rolling epochs, each ~6 days apart, which is ~half the addressable ms interval in 32-bits, i.e. given any full ms time we split into its most recent epoch plus its delta from that epoch. # Each client then produces a timestamp that is 32-bits of current time (in millis) since the most recent epoch, a local monotonically increasing 14-bit value that is reset each ms, and their unique id On the cluster we ensure memtables are flushed at least once per epoch, with the epoch appearing in the metadata, and we consider a full timestamp to be a composite of the timestamp stored combined with the epoch. Once the data is fully repaired prior to an epoch we can optionally save 32-bits per cell by stripping out the per-node and monotonically increasing timestamp values on compaction. The added complexity, as far as I can tell, will be in repairs, hints and compaction which need to ensure they compare a 96-bit timestamp instead of a 64-bit one. But in compaction at least this might actually simplify matters, as reconcile knows in advance which sstables it prefers data from. It's a pretty non-trivial change, and needs some further thought, but I think only non-trivial solutions are probably going to work for this non-trivial problem. Some possible safety optimisations with this solution might include refusing client timestamps that are not within some sensible skew from now, e.g. within 1 day, or 1 hour, giving a high degree of confidence the cluster is sufficiently in sync, since old timestamps should only appear during client retries, which should not be so badly delayed. We could also move to micros time if some users require it with this solution (which no doubt some will), with narrower epochs. Create timeid64 type Key: CASSANDRA-6108 URL: https://issues.apache.org/jira/browse/CASSANDRA-6108 Project: Cassandra Issue Type: New Feature Components: API, Core Reporter: Jonathan Ellis Assignee: Sylvain Lebresne Priority: Minor Fix For: 2.1.1 As discussed in CASSANDRA-6106, we could create a 64-bit type with 48 bits of timestamp and 16 bites of unique coordinator id. This would give us a unique-per-cluster value that could be used as a more compact replacement for many TimeUUID uses. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6108) Create timeid64 type
[ https://issues.apache.org/jira/browse/CASSANDRA-6108?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14040527#comment-14040527 ] Benedict commented on CASSANDRA-6108: - How do we safely manage the life cycle of the ids? We have no idea if the client is still alive or not Create timeid64 type Key: CASSANDRA-6108 URL: https://issues.apache.org/jira/browse/CASSANDRA-6108 Project: Cassandra Issue Type: New Feature Components: API, Core Reporter: Jonathan Ellis Assignee: Sylvain Lebresne Priority: Minor Fix For: 2.1.1 As discussed in CASSANDRA-6106, we could create a 64-bit type with 48 bits of timestamp and 16 bites of unique coordinator id. This would give us a unique-per-cluster value that could be used as a more compact replacement for many TimeUUID uses. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7395) Support for pure user-defined functions (UDF)
[ https://issues.apache.org/jira/browse/CASSANDRA-7395?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14040559#comment-14040559 ] Benedict commented on CASSANDRA-7395: - 0.5ms is a much higher delay than you should expect for handing work off to another thread, however I don't see how moving the UDF evaluation to another thread helps you stop the work it's doing. It does help interrupt the user's query more quickly, but the work of the query is still ongoing, so interrupting it from the user's perspective would be misleading. Support for pure user-defined functions (UDF) - Key: CASSANDRA-7395 URL: https://issues.apache.org/jira/browse/CASSANDRA-7395 Project: Cassandra Issue Type: New Feature Components: API, Core Reporter: Jonathan Ellis Fix For: 3.0 Attachments: 7395.diff We have some tickets for various aspects of UDF (CASSANDRA-4914, CASSANDRA-5970, CASSANDRA-4998) but they all suffer from various degrees of ocean-boiling. Let's start with something simple: allowing pure user-defined functions in the SELECT clause of a CQL query. That's it. By pure I mean, must depend only on the input parameters. No side effects. No exposure to C* internals. Column values in, result out. http://en.wikipedia.org/wiki/Pure_function -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6108) Create timeid64 type
[ https://issues.apache.org/jira/browse/CASSANDRA-6108?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14040659#comment-14040659 ] Benedict commented on CASSANDRA-6108: - I'm comfortable with that approach; sticking to 64-bit is certainly not essential. Although it might require a bit of tweaking TimeUUID generation for the fact that we can have multiple clients being created/terminated near in time and on the same node (say, by using SecureRandom instead for the clock value, or System.nanoTime()+System.currentTimeMillis()), esp. as makeNode() has only 16-bits of uniqueness which is less collision-free for clients than for nodes. I'm not 100% sure it's better than the approach I suggested, however: storing a lookup table inside of each sstable is unlikely to be expensive initially, but since the universe of ids will grow linearly with cluster up time, this could get out of hand before long. We could settle for a hybrid approach where we set this value to zero for all repaired sstables though. I'd still prefer not to have to perform a lookup each time I want to query the timestamp, though, as this is an extra indirection and makes simple delta encoding of timestamps trickier. But it may be the best overall solution for implementation complexity vs functionality. Create timeid64 type Key: CASSANDRA-6108 URL: https://issues.apache.org/jira/browse/CASSANDRA-6108 Project: Cassandra Issue Type: New Feature Components: API, Core Reporter: Jonathan Ellis Assignee: Sylvain Lebresne Priority: Minor Fix For: 2.1.1 As discussed in CASSANDRA-6106, we could create a 64-bit type with 48 bits of timestamp and 16 bites of unique coordinator id. This would give us a unique-per-cluster value that could be used as a more compact replacement for many TimeUUID uses. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Comment Edited] (CASSANDRA-6108) Create timeid64 type
[ https://issues.apache.org/jira/browse/CASSANDRA-6108?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14040659#comment-14040659 ] Benedict edited comment on CASSANDRA-6108 at 6/23/14 11:55 AM: --- I'm comfortable with that approach; sticking to 64-bit is certainly not essential. Although it might require a bit of tweaking TimeUUID generation for the fact that we can have multiple clients being created/terminated near in time and on the same node (say, by using SecureRandom instead for the clock value, or System.nanoTime()+System.currentTimeMillis()), esp. as makeNode() has only 16-bits of uniqueness which is less collision-free for clients than for nodes. I'm not 100% sure it's better than the approach I suggested, however: storing a lookup table inside of each sstable is unlikely to be expensive initially, but since the universe of ids will grow linearly with cluster up time, this could get out of hand before long. We could settle for a hybrid approach where we set this value to zero for all repaired sstables though. I'd still prefer not to have to perform a lookup each time I want to query the timestamp, as this is an extra indirection and makes simple delta encoding of timestamps trickier, meaning tables are unlikely to be as compact. But it may be the best overall solution for implementation complexity vs functionality. was (Author: benedict): I'm comfortable with that approach; sticking to 64-bit is certainly not essential. Although it might require a bit of tweaking TimeUUID generation for the fact that we can have multiple clients being created/terminated near in time and on the same node (say, by using SecureRandom instead for the clock value, or System.nanoTime()+System.currentTimeMillis()), esp. as makeNode() has only 16-bits of uniqueness which is less collision-free for clients than for nodes. I'm not 100% sure it's better than the approach I suggested, however: storing a lookup table inside of each sstable is unlikely to be expensive initially, but since the universe of ids will grow linearly with cluster up time, this could get out of hand before long. We could settle for a hybrid approach where we set this value to zero for all repaired sstables though. I'd still prefer not to have to perform a lookup each time I want to query the timestamp, though, as this is an extra indirection and makes simple delta encoding of timestamps trickier. But it may be the best overall solution for implementation complexity vs functionality. Create timeid64 type Key: CASSANDRA-6108 URL: https://issues.apache.org/jira/browse/CASSANDRA-6108 Project: Cassandra Issue Type: New Feature Components: API, Core Reporter: Jonathan Ellis Assignee: Sylvain Lebresne Priority: Minor Fix For: 2.1.1 As discussed in CASSANDRA-6106, we could create a 64-bit type with 48 bits of timestamp and 16 bites of unique coordinator id. This would give us a unique-per-cluster value that could be used as a more compact replacement for many TimeUUID uses. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6108) Create timeid64 type
[ https://issues.apache.org/jira/browse/CASSANDRA-6108?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14040713#comment-14040713 ] Benedict commented on CASSANDRA-6108: - bq. True, though at least for the 'node' part of the timeuuid, that's not suppose to change for a given client host, so in most installations, their might not be much growth over time. True, but not sure I'd like to rely on that. 2^14 * unique client ids is still a lot of ids. We could decompose into unique client ids + 14 bits and hope the unique client id set is relatively static, but this is kind of ugly to persist, and further limits possibilities for compressing, as we probably have to store a short for every timestamp without question. If anybody ever deploys clients internet-wide, this would break badly. Also, our client-id probably needs to have some extra random salt to the IP at startup in case clients could be connecting over NAT. Create timeid64 type Key: CASSANDRA-6108 URL: https://issues.apache.org/jira/browse/CASSANDRA-6108 Project: Cassandra Issue Type: New Feature Components: API, Core Reporter: Jonathan Ellis Assignee: Sylvain Lebresne Priority: Minor Fix For: 2.1.1 As discussed in CASSANDRA-6106, we could create a 64-bit type with 48 bits of timestamp and 16 bites of unique coordinator id. This would give us a unique-per-cluster value that could be used as a more compact replacement for many TimeUUID uses. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6755) Optimise CellName/Composite comparisons for NativeCell
[ https://issues.apache.org/jira/browse/CASSANDRA-6755?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14040815#comment-14040815 ] Benedict commented on CASSANDRA-6755: - I've pushed a branch [here|https://github.com/belliottsmith/cassandra/tree/6755-bes] The changes largely involve extracting all extra costs to be incurred once per set-of comparisons (i.e. once per search), instead of once per component per item. I've also fixed handling of non-byteorder-comparable types, and I've removed @Inline calls from everywhere not related directly to this ticket, as any introductions elsewhere should probably have their own discussion. This branch is now slightly faster for the simple types case, and (whilst I haven't yet tested it) should be faster still for composite types. At the same time, I've also slightly reduced the instruction overhead from inlining by restoring the old loop behaviour for the final 8-bytes in FastByteOperations.compare, and *most importantly* (for review purposes) I have imposed as a precondition for calling compare() that an empty composite is never provided (since in almost all cases one wasn't already). [~slebresne] I'd appreciate a quick review of just that particular part of the change, to confirm it makes sense. Mostly it was only happening in validation logic, ironically. Finally, I also slightly optimised the cost of cloning components back onto the heap, though I didn't see any net effect from this. Optimise CellName/Composite comparisons for NativeCell -- Key: CASSANDRA-6755 URL: https://issues.apache.org/jira/browse/CASSANDRA-6755 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: T Jake Luciani Priority: Minor Labels: performance Fix For: 2.1.1 Attachments: 6755.txt, 6755v2.txt As discussed in CASSANDRA-6694, to reduce temporary garbage generation we should minimise the incidence of CellName component extraction. The biggest win will be to perform comparisons on Cell where possible, instead of CellName, so that Native*Cell can use its extra information to avoid creating any ByteBuffer objects -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7395) Support for pure user-defined functions (UDF)
[ https://issues.apache.org/jira/browse/CASSANDRA-7395?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14040910#comment-14040910 ] Benedict commented on CASSANDRA-7395: - In C* that would be a very long time to communicate between threads. There are very few processes that steal CPU for an entire quanta, which is when you would see those kidns of delays. It's still illadvised though :) Support for pure user-defined functions (UDF) - Key: CASSANDRA-7395 URL: https://issues.apache.org/jira/browse/CASSANDRA-7395 Project: Cassandra Issue Type: New Feature Components: API, Core Reporter: Jonathan Ellis Fix For: 3.0 Attachments: 7395.diff We have some tickets for various aspects of UDF (CASSANDRA-4914, CASSANDRA-5970, CASSANDRA-4998) but they all suffer from various degrees of ocean-boiling. Let's start with something simple: allowing pure user-defined functions in the SELECT clause of a CQL query. That's it. By pure I mean, must depend only on the input parameters. No side effects. No exposure to C* internals. Column values in, result out. http://en.wikipedia.org/wiki/Pure_function -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7436) Dtest for tracking metrics regressions
[ https://issues.apache.org/jira/browse/CASSANDRA-7436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14041153#comment-14041153 ] Benedict commented on CASSANDRA-7436: - I think I'd prefer to work this into a performance regression suite off the back of the performance test framework, where we collect JMX stats along with our other data for a standardized set of stress runs. We can then post process the results to check it's within tolerance of prior runs. I suspect this would be too noisy for a dtest, and not actually that useful without a long enough test to create a continuum of values. Dtest for tracking metrics regressions -- Key: CASSANDRA-7436 URL: https://issues.apache.org/jira/browse/CASSANDRA-7436 Project: Cassandra Issue Type: Test Reporter: Ryan McGuire We should create a dtest that runs a small load test and collects a bunch of the JMX metrics and introduce assertions to ensure they are roughly the same every time. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7436) Dtest for tracking metrics regressions
[ https://issues.apache.org/jira/browse/CASSANDRA-7436?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14041173#comment-14041173 ] Benedict commented on CASSANDRA-7436: - I just think you'll struggle to get a viable baseline that doesn't break regularly, which might cause unnecessary overhead in trying to get it fixed - but the principle is sound Dtest for tracking metrics regressions -- Key: CASSANDRA-7436 URL: https://issues.apache.org/jira/browse/CASSANDRA-7436 Project: Cassandra Issue Type: Test Reporter: Ryan McGuire Assignee: Ryan McGuire We should create a dtest that runs a small load test and collects a bunch of the JMX metrics and introduce assertions to ensure they are roughly the same every time. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6755) Optimise CellName/Composite comparisons for NativeCell
[ https://issues.apache.org/jira/browse/CASSANDRA-6755?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14041180#comment-14041180 ] Benedict commented on CASSANDRA-6755: - Yes, unfortunately I just can't get it quite as fast; I think this is most likely down to an increase in virtual invocation costs, as the introduction of the parallel NativeCell tree breaks the VM's opportunities for fast-pathing everything. Although I haven't had the time to truly test the hypothesis, there aren't many other possibilities. I'll give [~slebresne] a 24hr opportunity to veto the simple cell comparison changes if he wants to before committing with your nits. Optimise CellName/Composite comparisons for NativeCell -- Key: CASSANDRA-6755 URL: https://issues.apache.org/jira/browse/CASSANDRA-6755 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: T Jake Luciani Priority: Minor Labels: performance Fix For: 2.1.1 Attachments: 6755.txt, 6755v2.txt As discussed in CASSANDRA-6694, to reduce temporary garbage generation we should minimise the incidence of CellName component extraction. The biggest win will be to perform comparisons on Cell where possible, instead of CellName, so that Native*Cell can use its extra information to avoid creating any ByteBuffer objects -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-7437) Drop Keyspace and ColumnFamily can raise assertions inside of forceRecycleAll still
Benedict created CASSANDRA-7437: --- Summary: Drop Keyspace and ColumnFamily can raise assertions inside of forceRecycleAll still Key: CASSANDRA-7437 URL: https://issues.apache.org/jira/browse/CASSANDRA-7437 Project: Cassandra Issue Type: Bug Components: Core Reporter: Benedict Assignee: Benedict Priority: Minor Fix For: 2.1 rc2 I've noticed on unit test output that there are still assertions being raised here, so I've taken a torch to the code path to make damned certain it cannot happen in future # We now wait for all running reads on a column family or writes on the keyspace during a dropCf call # We wait for all appends to the prior commit log segments before recycling them # We pass the list of dropped Cfs into the CL.forceRecycle call so that they can be markedClean definitely after they have been marked finished # Finally, to prevent any possibility of this still happening causing any negative consequences, I've suppressed the assertion in favour of an error log message, as the assertion would break correct program flow for the drop and potentially result in undefined behaviour (in actuality there is the slightest possibility still of a race condition on read of a secondary index that causes a repair driven write, but this is a really tiny race window, as I force wait for all reads after unlinking the CF, so it would have to be a read that grabbed the CFS reference before it was dropped, but hadn't quite started its read op yet). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7437) Drop Keyspace and ColumnFamily can raise assertions inside of forceRecycleAll still
[ https://issues.apache.org/jira/browse/CASSANDRA-7437?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14041235#comment-14041235 ] Benedict commented on CASSANDRA-7437: - Patch [here|https://github.com/belliottsmith/cassandra/tree/7437] Drop Keyspace and ColumnFamily can raise assertions inside of forceRecycleAll still --- Key: CASSANDRA-7437 URL: https://issues.apache.org/jira/browse/CASSANDRA-7437 Project: Cassandra Issue Type: Bug Components: Core Reporter: Benedict Assignee: Benedict Priority: Minor Fix For: 2.1 rc2 I've noticed on unit test output that there are still assertions being raised here, so I've taken a torch to the code path to make damned certain it cannot happen in future # We now wait for all running reads on a column family or writes on the keyspace during a dropCf call # We wait for all appends to the prior commit log segments before recycling them # We pass the list of dropped Cfs into the CL.forceRecycle call so that they can be markedClean definitely after they have been marked finished # Finally, to prevent any possibility of this still happening causing any negative consequences, I've suppressed the assertion in favour of an error log message, as the assertion would break correct program flow for the drop and potentially result in undefined behaviour (in actuality there is the slightest possibility still of a race condition on read of a secondary index that causes a repair driven write, but this is a really tiny race window, as I force wait for all reads after unlinking the CF, so it would have to be a read that grabbed the CFS reference before it was dropped, but hadn't quite started its read op yet). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6890) Standardize on a single read path
[ https://issues.apache.org/jira/browse/CASSANDRA-6890?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14041287#comment-14041287 ] Benedict commented on CASSANDRA-6890: - Shall we close this as Later then? Standardize on a single read path - Key: CASSANDRA-6890 URL: https://issues.apache.org/jira/browse/CASSANDRA-6890 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Joshua McKenzie Assignee: Joshua McKenzie Labels: performance Fix For: 3.0 Attachments: 6890_v1.txt, mmap_gc.jpg, mmap_jstat.txt, mmap_perf.txt, nommap_gc.jpg, nommap_jstat.txt Since we actively unmap unreferenced SSTR's and also copy data out of those readers on the read path, the current memory mapped i/o is a lot of complexity for very little payoff. Clean out the mmapp'ed i/o on the read path. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Assigned] (CASSANDRA-7029) Investigate alternative transport protocols for both client and inter-server communications
[ https://issues.apache.org/jira/browse/CASSANDRA-7029?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benedict reassigned CASSANDRA-7029: --- Assignee: (was: Benedict) Investigate alternative transport protocols for both client and inter-server communications --- Key: CASSANDRA-7029 URL: https://issues.apache.org/jira/browse/CASSANDRA-7029 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Benedict Labels: performance Fix For: 3.0 There are a number of reasons to think we can do better than TCP for our communications: 1) We can actually tolerate sporadic small message losses, so guaranteed delivery isn't essential (although for larger messages it probably is) 2) As shown in \[1\] and \[2\], Linux can behave quite suboptimally with regard to TCP message delivery when the system is under load. Judging from the theoretical description, this is likely to apply even when the system-load is not high, but the number of processes to schedule is high. Cassandra generally has a lot of threads to schedule, so this is quite pertinent for us. UDP performs substantially better here. 3) Even when the system is not under load, UDP has a lower CPU burden, and that burden is constant regardless of the number of connections it processes. 4) On a simple benchmark on my local PC, using non-blocking IO for UDP and busy spinning on IO I can actually push 20-40% more throughput through loopback (where TCP should be optimal, as no latency), even for very small messages. Since we can see networking taking multiple CPUs' worth of time during a stress test, using a busy-spin for ~100micros after last message receipt is almost certainly acceptable, especially as we can (ultimately) process inter-server and client communications on the same thread/socket in this model. 5) We can optimise the threading model heavily: since we generally process very small messages (200 bytes not at all implausible), the thread signalling costs on the processing thread can actually dramatically impede throughput. In general it costs ~10micros to signal (and passing the message to another thread for processing in the current model requires signalling). For 200-byte messages this caps our throughput at 20MB/s. I propose to knock up a highly naive UDP-based connection protocol with super-trivial congestion control over the course of a few days, with the only initial goal being maximum possible performance (not fairness, reliability, or anything else), and trial it in Netty (possibly making some changes to Netty to mitigate thread signalling costs). The reason for knocking up our own here is to get a ceiling on what the absolute limit of potential for this approach is. Assuming this pans out with performance gains in C* proper, we then look to contributing to/forking the udt-java project and see how easy it is to bring performance in line with what we can get with our naive approach (I don't suggest starting here, as the project is using blocking old-IO, and modifying it with latency in mind may be challenging, and we won't know for sure what the best case scenario is). \[1\] http://test-docdb.fnal.gov/0016/001648/002/Potential%20Performance%20Bottleneck%20in%20Linux%20TCP.PDF \[2\] http://cd-docdb.fnal.gov/cgi-bin/RetrieveFile?docid=1968;filename=Performance%20Analysis%20of%20Linux%20Networking%20-%20Packet%20Receiving%20(Official).pdf;version=2 Further related reading: http://public.dhe.ibm.com/software/commerce/doc/mft/cdunix/41/UDTWhitepaper.pdf https://mospace.umsystem.edu/xmlui/bitstream/handle/10355/14482/ChoiUndPerTcp.pdf?sequence=1 https://access.redhat.com/site/documentation/en-US/JBoss_Enterprise_Web_Platform/5/html/Administration_And_Configuration_Guide/jgroups-perf-udpbuffer.html http://citeseerx.ist.psu.edu/viewdoc/download?doi=10.1.1.153.3762rep=rep1type=pdf -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Assigned] (CASSANDRA-7061) High accuracy, low overhead local read/write tracing
[ https://issues.apache.org/jira/browse/CASSANDRA-7061?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benedict reassigned CASSANDRA-7061: --- Assignee: (was: Benedict) High accuracy, low overhead local read/write tracing Key: CASSANDRA-7061 URL: https://issues.apache.org/jira/browse/CASSANDRA-7061 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Benedict Fix For: 3.0 External profilers are pretty inadequate for getting accurate information at the granularity we're working at: tracing is too high overhead, so measures something completely different, and sampling suffers from bias of attribution due to the way the stack traces are retrieved. Hyperthreading can make this even worse. I propose to introduce an extremely low overhead tracing feature that must be enabled with a system property that will trace operations within the node only, so that we can perform various accurate low level analyses of performance. This information will include threading info, so that we can trace hand off delays and actual active time spent processing an operation. With the property disabled there will be no increased burden of tracing, however I hope to keep the total trace burden to less than one microsecond, and any single trace command to a few tens of nanos. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6591) un-deprecate cache recentHitRate and expose in o.a.c.metrics
[ https://issues.apache.org/jira/browse/CASSANDRA-6591?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14041400#comment-14041400 ] Benedict commented on CASSANDRA-6591: - Apologies for the delay on this. I think this ticket does slightly fit into a wider discussion, though here we're using a Meter instead of a Histogram/Timer (where codahale is much more broken by default). The Meters *are* still costlier to keep around and maintain than we might otherwise like: each meter maintains a LongAdder (which is in principle to avoid contention, but for our use case just means more cpu-cache misses), but more importantly maintains *four* LongAdders per meter, so each time we attempt to hit the cache we must update at least 8 LongAdders (since we update two meters here). I would prefer to see us moving to a scheme that uses only two AtomicLong for this data, but I'm not sure we need to hold up this ticket for that. However I would say we only need to maintain *either* hits *or* misses, since even for a rate property, misses = total - hits, so we can define our misses as a simple Gauge returning requests - hits un-deprecate cache recentHitRate and expose in o.a.c.metrics Key: CASSANDRA-6591 URL: https://issues.apache.org/jira/browse/CASSANDRA-6591 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Chris Burroughs Assignee: Chris Burroughs Priority: Minor Attachments: j6591-1.2-v1.txt, j6591-1.2-v2.txt, j6591-1.2-v3.txt recentHitRate metrics were not added as part of CASSANDRA-4009 because there is not an obvious way to do it with the Metrics library. Instead hitRate was added as an all time measurement since node restart. This does allow changes in cache rate (aka production performance problems) to be detected. Ideally there would be 1/5/15 moving averages for the hit rate, but I'm not sure how to calculate that. Instead I propose updating recentHitRate on a fixed interval and exposing that as a Gauge. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Comment Edited] (CASSANDRA-6591) un-deprecate cache recentHitRate and expose in o.a.c.metrics
[ https://issues.apache.org/jira/browse/CASSANDRA-6591?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14041400#comment-14041400 ] Benedict edited comment on CASSANDRA-6591 at 6/23/14 10:12 PM: --- Apologies for the delay on this. I think this ticket does slightly fit into a wider discussion, though here we're using a Meter instead of a Histogram/Timer (where codahale is much more broken by default). The Meters _are_ still costlier to keep around and maintain than we might otherwise like: each meter maintains a LongAdder (which is in principle to avoid contention, but for our use case just means more cpu-cache misses), but more importantly maintains _four_ LongAdders per meter, so each time we attempt to hit the cache we must update at least 8 LongAdders (since we update two meters here). I would prefer to see us moving to a scheme that uses only two AtomicLong for this data, but I'm not sure we need to hold up this ticket for that. However I would say we only need to maintain _either_ hits _or_ misses, since even for a rate property, misses = total - hits, so we can define our misses as a simple Gauge returning requests - hits was (Author: benedict): Apologies for the delay on this. I think this ticket does slightly fit into a wider discussion, though here we're using a Meter instead of a Histogram/Timer (where codahale is much more broken by default). The Meters *are* still costlier to keep around and maintain than we might otherwise like: each meter maintains a LongAdder (which is in principle to avoid contention, but for our use case just means more cpu-cache misses), but more importantly maintains *four* LongAdders per meter, so each time we attempt to hit the cache we must update at least 8 LongAdders (since we update two meters here). I would prefer to see us moving to a scheme that uses only two AtomicLong for this data, but I'm not sure we need to hold up this ticket for that. However I would say we only need to maintain *either* hits *or* misses, since even for a rate property, misses = total - hits, so we can define our misses as a simple Gauge returning requests - hits un-deprecate cache recentHitRate and expose in o.a.c.metrics Key: CASSANDRA-6591 URL: https://issues.apache.org/jira/browse/CASSANDRA-6591 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Chris Burroughs Assignee: Chris Burroughs Priority: Minor Attachments: j6591-1.2-v1.txt, j6591-1.2-v2.txt, j6591-1.2-v3.txt recentHitRate metrics were not added as part of CASSANDRA-4009 because there is not an obvious way to do it with the Metrics library. Instead hitRate was added as an all time measurement since node restart. This does allow changes in cache rate (aka production performance problems) to be detected. Ideally there would be 1/5/15 moving averages for the hit rate, but I'm not sure how to calculate that. Instead I propose updating recentHitRate on a fixed interval and exposing that as a Gauge. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7029) Investigate alternative transport protocols for both client and inter-server communications
[ https://issues.apache.org/jira/browse/CASSANDRA-7029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14041447#comment-14041447 ] Benedict commented on CASSANDRA-7029: - bq. Also we might want to batch multiple messages in OutboundTcpConnection and send it together. This already effectively happens, but only if there are multiple messages waiting to send. If/when we move to a more async approach we might be able to more easily batch similarly to how we do in 2.1 for our native client connections, but I'm cautious about doing that with sync IO as it could potentially behave negatively with very large node counts. I'm not sure if we'll want to do move to async IO for internode comms anytime soon, though, as it is much slower for small node counts. bq. This will help a lot as I have seen lot of dropped messages when under load. Out of interest, do you mean logged-by-Cassandra dropped, or dropped-by-the-network? Investigate alternative transport protocols for both client and inter-server communications --- Key: CASSANDRA-7029 URL: https://issues.apache.org/jira/browse/CASSANDRA-7029 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Benedict Labels: performance Fix For: 3.0 There are a number of reasons to think we can do better than TCP for our communications: 1) We can actually tolerate sporadic small message losses, so guaranteed delivery isn't essential (although for larger messages it probably is) 2) As shown in \[1\] and \[2\], Linux can behave quite suboptimally with regard to TCP message delivery when the system is under load. Judging from the theoretical description, this is likely to apply even when the system-load is not high, but the number of processes to schedule is high. Cassandra generally has a lot of threads to schedule, so this is quite pertinent for us. UDP performs substantially better here. 3) Even when the system is not under load, UDP has a lower CPU burden, and that burden is constant regardless of the number of connections it processes. 4) On a simple benchmark on my local PC, using non-blocking IO for UDP and busy spinning on IO I can actually push 20-40% more throughput through loopback (where TCP should be optimal, as no latency), even for very small messages. Since we can see networking taking multiple CPUs' worth of time during a stress test, using a busy-spin for ~100micros after last message receipt is almost certainly acceptable, especially as we can (ultimately) process inter-server and client communications on the same thread/socket in this model. 5) We can optimise the threading model heavily: since we generally process very small messages (200 bytes not at all implausible), the thread signalling costs on the processing thread can actually dramatically impede throughput. In general it costs ~10micros to signal (and passing the message to another thread for processing in the current model requires signalling). For 200-byte messages this caps our throughput at 20MB/s. I propose to knock up a highly naive UDP-based connection protocol with super-trivial congestion control over the course of a few days, with the only initial goal being maximum possible performance (not fairness, reliability, or anything else), and trial it in Netty (possibly making some changes to Netty to mitigate thread signalling costs). The reason for knocking up our own here is to get a ceiling on what the absolute limit of potential for this approach is. Assuming this pans out with performance gains in C* proper, we then look to contributing to/forking the udt-java project and see how easy it is to bring performance in line with what we can get with our naive approach (I don't suggest starting here, as the project is using blocking old-IO, and modifying it with latency in mind may be challenging, and we won't know for sure what the best case scenario is). \[1\] http://test-docdb.fnal.gov/0016/001648/002/Potential%20Performance%20Bottleneck%20in%20Linux%20TCP.PDF \[2\] http://cd-docdb.fnal.gov/cgi-bin/RetrieveFile?docid=1968;filename=Performance%20Analysis%20of%20Linux%20Networking%20-%20Packet%20Receiving%20(Official).pdf;version=2 Further related reading: http://public.dhe.ibm.com/software/commerce/doc/mft/cdunix/41/UDTWhitepaper.pdf https://mospace.umsystem.edu/xmlui/bitstream/handle/10355/14482/ChoiUndPerTcp.pdf?sequence=1 https://access.redhat.com/site/documentation/en-US/JBoss_Enterprise_Web_Platform/5/html/Administration_And_Configuration_Guide/jgroups-perf-udpbuffer.html http://citeseerx.ist.psu.edu/viewdoc/download?doi=10.1.1.153.3762rep=rep1type=pdf -- This message was sent by Atlassian JIRA
[jira] [Updated] (CASSANDRA-7437) Drop Keyspace and ColumnFamily can raise assertions inside of forceRecycleAll still
[ https://issues.apache.org/jira/browse/CASSANDRA-7437?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benedict updated CASSANDRA-7437: Description: I've noticed on unit test output that there are still assertions being raised here, so I've taken a torch to the code path to make damned certain it cannot happen in future # We now wait for all running reads on a column family or writes on the keyspace during a dropCf call # We wait for all appends to the prior commit log segments before recycling them # We pass the list of dropped Cfs into the CL.forceRecycle call so that they can be markedClean definitely after they have been marked finished # Finally, to prevent any possibility of this still happening causing any negative consequences, I've suppressed the assertion in favour of an error log message, as the assertion would break correct program flow for the drop and potentially result in undefined behaviour -(in actuality there is the slightest possibility still of a race condition on read of a secondary index that causes a repair driven write, but this is a really tiny race window, as I force wait for all reads after unlinking the CF, so it would have to be a read that grabbed the CFS reference before it was dropped, but hadn't quite started its read op yet).- In fact this is also safe, as these modifications all grab a write op from the Keyspace, which has to happen before they get the CFS, and also because we drop the data before waiting for reads to finish on the CFS. was: I've noticed on unit test output that there are still assertions being raised here, so I've taken a torch to the code path to make damned certain it cannot happen in future # We now wait for all running reads on a column family or writes on the keyspace during a dropCf call # We wait for all appends to the prior commit log segments before recycling them # We pass the list of dropped Cfs into the CL.forceRecycle call so that they can be markedClean definitely after they have been marked finished # Finally, to prevent any possibility of this still happening causing any negative consequences, I've suppressed the assertion in favour of an error log message, as the assertion would break correct program flow for the drop and potentially result in undefined behaviour (in actuality there is the slightest possibility still of a race condition on read of a secondary index that causes a repair driven write, but this is a really tiny race window, as I force wait for all reads after unlinking the CF, so it would have to be a read that grabbed the CFS reference before it was dropped, but hadn't quite started its read op yet). Drop Keyspace and ColumnFamily can raise assertions inside of forceRecycleAll still --- Key: CASSANDRA-7437 URL: https://issues.apache.org/jira/browse/CASSANDRA-7437 Project: Cassandra Issue Type: Bug Components: Core Reporter: Benedict Assignee: Benedict Priority: Minor Fix For: 2.1 rc2 I've noticed on unit test output that there are still assertions being raised here, so I've taken a torch to the code path to make damned certain it cannot happen in future # We now wait for all running reads on a column family or writes on the keyspace during a dropCf call # We wait for all appends to the prior commit log segments before recycling them # We pass the list of dropped Cfs into the CL.forceRecycle call so that they can be markedClean definitely after they have been marked finished # Finally, to prevent any possibility of this still happening causing any negative consequences, I've suppressed the assertion in favour of an error log message, as the assertion would break correct program flow for the drop and potentially result in undefined behaviour -(in actuality there is the slightest possibility still of a race condition on read of a secondary index that causes a repair driven write, but this is a really tiny race window, as I force wait for all reads after unlinking the CF, so it would have to be a read that grabbed the CFS reference before it was dropped, but hadn't quite started its read op yet).- In fact this is also safe, as these modifications all grab a write op from the Keyspace, which has to happen before they get the CFS, and also because we drop the data before waiting for reads to finish on the CFS. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-3852) use LIFO queueing policy when queue size exceeds thresholds
[ https://issues.apache.org/jira/browse/CASSANDRA-3852?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14042788#comment-14042788 ] Benedict commented on CASSANDRA-3852: - I think I have a preference for a configurable threshold below which we always serve requests in FIFO, however if a request isn't serviced in that window it falls onto a later queue which is only serviced once the in-time queue is exhausted. This way we maintain our current behaviour but degrade gracefully otherwise. This shouldn't be too difficult to programmatically encapsulate in a queue implementation, however with 3.0 we hope to have per-disk access coordination (CASSANDRA-7040), and it's likely any solution will look quite different under that scheme. So it's probably best to let that settle down first unless we want to target 2.1 with this use LIFO queueing policy when queue size exceeds thresholds --- Key: CASSANDRA-3852 URL: https://issues.apache.org/jira/browse/CASSANDRA-3852 Project: Cassandra Issue Type: Improvement Reporter: Peter Schuller Assignee: Peter Schuller Labels: performance A strict FIFO policy for queueing (between stages) is detrimental to latency and forward progress. Whenever a node is saturated beyond incoming request rate, *all* requests become slow. If it is consistently saturated, you start effectively timing out on *all* requests. A much better strategy from the point of view of latency is to serve a subset requests quickly, and letting some time out, rather than letting all either time out or be slow. Care must be taken such that: * We still guarantee that requests are processed reasonably timely (we couldn't go strict LIFO for example as that would result in requests getting stuck potentially forever on a loaded node). * Maybe, depending on the previous point's solution, ensure that some requests bypass the policy and get prioritized (e.g., schema migrations, or anything internal to a node). A possible implementation is to go LIFO whenever there are requests in the queue that are older than N milliseconds (or a certain queue size, etc). Benefits: * All cases where the client is directly, or is indirectly affecting through other layers, a system which has limited concurrency (e.g., thread pool size of X to serve some incoming request rate), it is *much* better for a few requests to time out while most are serviced quickly, than for all requests to become slow, as it doesn't explode concurrency. Think any random non-super-advanced php app, ruby web app, java servlet based app, etc. Essentially, it optimizes very heavily for improved average latencies. * Systems with strict p95/p99/p999 requirements on latencies should greatly benefit from such a policy. For example, suppose you have a system at 85% of capacity, and it takes a write spike (or has a hiccup like GC pause, blocking on a commit log write, etc). Suppose the hiccup racks up 500 ms worth of requests. At 15% margin at steady state, that takes 500ms * 100/15 = 3.2 seconds to recover. Instead of *all* requests for an entire 3.2 second window being slow, we'd serve requests quickly for 2.7 of those seconds, with the incoming requests during that 500 ms interval being the ones primarily affected. The flip side though is that once you're at the point where more than N percent of requests end up having to wait for others to take LIFO priority, the p(100-N) latencies will actually be *worse* than without this change (but at this point you have to consider what the root reason for those pXX requirements are). * In the case of complete saturation, it allows forward progress. Suppose you're taking 25% more traffic than you are able to handle. Instead of getting backed up and ending up essentially timing out *every single request*, you will succeed in processing up to 75% of them (I say up to because it depends; for example on a {{QUORUM}} request you need at least two of the requests from the co-ordinator to succeed so the percentage is brought down) and allowing clients to make forward progress and get work done, rather than being stuck. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-7447) New Storage Format
Benedict created CASSANDRA-7447: --- Summary: New Storage Format Key: CASSANDRA-7447 URL: https://issues.apache.org/jira/browse/CASSANDRA-7447 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Benedict Assignee: Benedict Fix For: 3.0 Attachments: ngcc-storage.odp h2. Storage Format Proposal C* has come a long way over the past few years, and unfortunately our storage format hasn't kept pace with the data models we are now encouraging people to utilise. This ticket proposes a collections of storage primitives that can be combined to serve these data models more optimally. It would probably help to first state the data model at the most abstract level. We have a fixed three-tier structure: We have the partition key, the clustering columns, and the data columns. Each have their own characteristics and so require their own specialised treatment. I should note that these changes will necessarily be delivered in stages, and that we will be making some assumptions about what the most useful features to support initially will be. Any features not supported will require sticking with the old format until we extend support to all C* functionality. h3. Partition Key * This really has two components: the partition, and the value. Although the partition is primarily used to distribute across nodes, it can also be used to optimise lookups for a given key within a node * Generally partitioning is by hash, and for the moment I want to focus this ticket on the assumption that this is the case * Given this, it makes sense to optimise our storage format to permit O(1) searching of a given partition. It may be possible to achieve this with little overhead based on the fact we store the hashes in order and know they are approximately randomly distributed, as this effectively forms an immutable contiguous split-ordered list (see Shalev/Shavit, or CASSANDRA-7282), so we only need to store an amount of data based on how imperfectly distributed the hashes are, or at worst a single value per block. * This should completely obviate the need for a separate key-cache, which will be relegated to supporting the old storage format only h3. Primary Key / Clustering Columns * Given we have a hierarchical data model, I propose the use of a cache-oblivious trie * The main advantage of the trie is that it is extremely compact and _supports optimally efficient merges with other tries_ so that we can support more efficient reads when multiple sstables are touched * The trie will be preceded by a small amount of related data; the full partition key, a timestamp epoch (for offset-encoding timestamps) and any other partition level optimisation data, such as (potentially) a min/max timestamp to abort merges earlier * Initially I propose to limit the trie to byte-order comparable data types only (the number of which we can expand through translations of the important types that are not currently) * Crucially the trie will also encapsulate any range tombstones, so that these are merged early in the process and avoids re-iterating the same data * Results in true bidirectional streaming without having to read entire range into memory h3. Values There are generally two approaches to storing rows of data: columnar, or row-oriented. The above two data structures can be combined with a value storage scheme that is based on either. However, given the current model we have of reading large 64Kb blocks for any read, I am inclined to focus on columnar support first, as this delivers order-of-magnitude benefits to those users with the correct workload, while for most workloads our 64Kb blocks are large enough to store row-oriented data in a column-oriented fashion without any performance degradation (I'm happy to consign very large row support to phase 2). Since we will most likely target both behaviours eventually, I am currently inclined to suggest that static columns, sets and maps be targeted for a row-oriented release, as they don't naturally fit in a columnar layout without secondary heap-blocks. This may be easier than delivering heap-blocks also, as it keeps both implementations relatively clean. This is certainly open to debate, and I have no doubt there will be conflicting opinions here. Focusing on our columnar layout, the goals are: * Support sparse and dense column storage * Efficient compression of tombstones, timestamps, ttls, etc. into near-zero space based on offset/delta/bitmap encoding * Normalisation of column names once per file * Per-file block-layout index, defining how each block's data is encoded, so we can index directly within a block for dense fields (permitting more efficient page cache utilisation) * Configurable grouping of fields per block, so that we can get closer to row-oriented or
[jira] [Commented] (CASSANDRA-7056) Add RAMP transactions
[ https://issues.apache.org/jira/browse/CASSANDRA-7056?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14042898#comment-14042898 ] Benedict commented on CASSANDRA-7056: - We were discussing this internally just a few days ago. I'm very keen to see this introduced, as I think it could have tremendous potential. There has been a side discussion about CASSANDRA-6108 and whether this would make an implementation simpler, by virtue of providing a unique commit id that is more robust than a server-generated timestamp, however I am of the opinion this could be worked in later. Using the timestamp either way certainly seems the easiest solution, it will just benefit from improved timestamps when we get them. One important question for me is if we maintain a separate expired-read-buffer from the write-buffer; optimally we would clear records from the write buffer as soon as they make it into memtables, only we them need to track values that are overwritten in a separate read-buffer. It might be slightly easier to simply keep them longer in the write-buffer, however this could lead to significantly larger memory overheads, as we keep all writes twice (instead of only those that are overwritten) Either way, I'm currently of the opinion we should target either a very narrow expired-read-buffer window, or one with a fixed size, so that we can keep a tight bound on the resources dedicated to these transactions. We also need to take care with how we safely inform a reader that their read could not be safely serviced from this window so that they may retry, and to fail if reads consistently fail to reach consensus. There are some related problems as well, namely how we expose this functionality to the user. Currently we have no concept of a batched read so this might need protocol support, but that's probably a separate discussion/problem. As far as _writes_ are concerned, I'd be inclined to simply replace current LOGGED batches entirely. Add RAMP transactions - Key: CASSANDRA-7056 URL: https://issues.apache.org/jira/browse/CASSANDRA-7056 Project: Cassandra Issue Type: Wish Components: Core Reporter: Tupshin Harper Priority: Minor We should take a look at [RAMP|http://www.bailis.org/blog/scalable-atomic-visibility-with-ramp-transactions/] transactions, and figure out if they can be used to provide more efficient LWT (or LWT-like) operations. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7056) Add RAMP transactions
[ https://issues.apache.org/jira/browse/CASSANDRA-7056?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14043225#comment-14043225 ] Benedict commented on CASSANDRA-7056: - bq. Should we just make it automatic for IN queries? That would leave the option of doing a bunch of asynchronous SELECTs if you wanted to opt out. This is probably the easiest first step, but batches support hitting multiple tables simultaneously, so ideally we would support reads that do the same. Add RAMP transactions - Key: CASSANDRA-7056 URL: https://issues.apache.org/jira/browse/CASSANDRA-7056 Project: Cassandra Issue Type: Wish Components: Core Reporter: Tupshin Harper Priority: Minor We should take a look at [RAMP|http://www.bailis.org/blog/scalable-atomic-visibility-with-ramp-transactions/] transactions, and figure out if they can be used to provide more efficient LWT (or LWT-like) operations. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7438) Serializing Row cache alternative (Fully off heap)
[ https://issues.apache.org/jira/browse/CASSANDRA-7438?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benedict updated CASSANDRA-7438: Labels: performance (was: ) Serializing Row cache alternative (Fully off heap) -- Key: CASSANDRA-7438 URL: https://issues.apache.org/jira/browse/CASSANDRA-7438 Project: Cassandra Issue Type: Improvement Components: Core Environment: Linux Reporter: Vijay Assignee: Vijay Labels: performance Fix For: 3.0 Currently SerializingCache is partially off heap, keys are still stored in JVM heap as BB, * There is a higher GC costs for a reasonably big cache. * Some users have used the row cache efficiently in production for better results, but this requires careful tunning. * Overhead in Memory for the cache entries are relatively high. So the proposal for this ticket is to move the LRU cache logic completely off heap and use JNI to interact with cache. We might want to ensure that the new implementation match the existing API's (ICache), and the implementation needs to have safe memory access, low overhead in memory and less memcpy's (As much as possible). We might also want to make this cache configurable. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2
[ https://issues.apache.org/jira/browse/CASSANDRA-7443?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14043384#comment-14043384 ] Benedict commented on CASSANDRA-7443: - A quick comment: we may need to make pluggability a bit higher in the stack than sstable reader/writer; CcollationController should probably be involved in this step. A related goal to keep in mind, whilst not the explicit goal of this ticket, is making replacing ColumnFamily with a new internal resultset/iterator abstraction a little easier. We should position ourselves to be able to tackle that after this (or one more follow up) ticket. We will eventually want to make writes configurable before they hit memtables as well, though that can probably be bumped to a later ticket also. SSTable Pluggability v2 --- Key: CASSANDRA-7443 URL: https://issues.apache.org/jira/browse/CASSANDRA-7443 Project: Cassandra Issue Type: Bug Components: Core Reporter: T Jake Luciani Assignee: T Jake Luciani Fix For: 3.0 As part of a wider effort to improve the performance of our storage engine we will need to support basic pluggability of the SSTable reader/writer. We primarily need this to support the current SSTable format and new SSTable format in the same version. This will also let us encapsulate the changes in a single layer vs forcing the whole engine to change at once. We previously discussed how to accomplish this in CASSANDRA-3067 -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2
[ https://issues.apache.org/jira/browse/CASSANDRA-7443?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14043399#comment-14043399 ] Benedict commented on CASSANDRA-7443: - We will have to dice up CFS somewhat no doubt, yes - as soon as we start messing with CC and Memtables, CFS is in the mix too. With some of these changes and compaction etc., I'm sort of tempted to wait until we have a new storage API (which may follow this ticket)... we need to try and do this piecemeal as it is going to be a difficult project. Let's make it minimally abstract, then iterate from there :) Specifically for compaction, you can already drop a custom compaction strategy (which permits, e.g., turning it off), but we probably don't want to reimplement LCS/STCS etc. for each storage engine that needs them. SSTable Pluggability v2 --- Key: CASSANDRA-7443 URL: https://issues.apache.org/jira/browse/CASSANDRA-7443 Project: Cassandra Issue Type: Bug Components: Core Reporter: T Jake Luciani Assignee: T Jake Luciani Fix For: 3.0 As part of a wider effort to improve the performance of our storage engine we will need to support basic pluggability of the SSTable reader/writer. We primarily need this to support the current SSTable format and new SSTable format in the same version. This will also let us encapsulate the changes in a single layer vs forcing the whole engine to change at once. We previously discussed how to accomplish this in CASSANDRA-3067 -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7443) SSTable Pluggability v2
[ https://issues.apache.org/jira/browse/CASSANDRA-7443?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14043478#comment-14043478 ] Benedict commented on CASSANDRA-7443: - To clarify, and I hope put everyone at ease, I have no interest in explicitly supporting an officially guaranteed pluggable storage API - but in making it easy for us to swap out the implementation we do necessarily need to make it somewhat pluggable internally SSTable Pluggability v2 --- Key: CASSANDRA-7443 URL: https://issues.apache.org/jira/browse/CASSANDRA-7443 Project: Cassandra Issue Type: Bug Components: Core Reporter: T Jake Luciani Assignee: T Jake Luciani Fix For: 3.0 As part of a wider effort to improve the performance of our storage engine we will need to support basic pluggability of the SSTable reader/writer. We primarily need this to support the current SSTable format and new SSTable format in the same version. This will also let us encapsulate the changes in a single layer vs forcing the whole engine to change at once. We previously discussed how to accomplish this in CASSANDRA-3067 -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7056) Add RAMP transactions
[ https://issues.apache.org/jira/browse/CASSANDRA-7056?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14043841#comment-14043841 ] Benedict commented on CASSANDRA-7056: - You will read timestamp 2 as the latest value, and will request the latest value as of that timestamp, which will be 1.5 for C; since 1.5 has to be visible (if it was written with RAMP transactions and you've seen it, it's visible) you'll get correct behaviour. If it isn't written with a RAMP transaction, it's undefined which you see, and that is also correct. Add RAMP transactions - Key: CASSANDRA-7056 URL: https://issues.apache.org/jira/browse/CASSANDRA-7056 Project: Cassandra Issue Type: Wish Components: Core Reporter: Tupshin Harper Priority: Minor We should take a look at [RAMP|http://www.bailis.org/blog/scalable-atomic-visibility-with-ramp-transactions/] transactions, and figure out if they can be used to provide more efficient LWT (or LWT-like) operations. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7029) Investigate alternative transport protocols for both client and inter-server communications
[ https://issues.apache.org/jira/browse/CASSANDRA-7029?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14044056#comment-14044056 ] Benedict commented on CASSANDRA-7029: - [~kohlisankalp] in that case it's more likely you're seeing a bottleneck in the read/write stages, rather than networking overheads. You should find 2.1 is a lot better with the read/write stages already if it's CPU bound, though these messages can be down to pending flushes or any number of causes. Investigate alternative transport protocols for both client and inter-server communications --- Key: CASSANDRA-7029 URL: https://issues.apache.org/jira/browse/CASSANDRA-7029 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Benedict Labels: performance Fix For: 3.0 There are a number of reasons to think we can do better than TCP for our communications: 1) We can actually tolerate sporadic small message losses, so guaranteed delivery isn't essential (although for larger messages it probably is) 2) As shown in \[1\] and \[2\], Linux can behave quite suboptimally with regard to TCP message delivery when the system is under load. Judging from the theoretical description, this is likely to apply even when the system-load is not high, but the number of processes to schedule is high. Cassandra generally has a lot of threads to schedule, so this is quite pertinent for us. UDP performs substantially better here. 3) Even when the system is not under load, UDP has a lower CPU burden, and that burden is constant regardless of the number of connections it processes. 4) On a simple benchmark on my local PC, using non-blocking IO for UDP and busy spinning on IO I can actually push 20-40% more throughput through loopback (where TCP should be optimal, as no latency), even for very small messages. Since we can see networking taking multiple CPUs' worth of time during a stress test, using a busy-spin for ~100micros after last message receipt is almost certainly acceptable, especially as we can (ultimately) process inter-server and client communications on the same thread/socket in this model. 5) We can optimise the threading model heavily: since we generally process very small messages (200 bytes not at all implausible), the thread signalling costs on the processing thread can actually dramatically impede throughput. In general it costs ~10micros to signal (and passing the message to another thread for processing in the current model requires signalling). For 200-byte messages this caps our throughput at 20MB/s. I propose to knock up a highly naive UDP-based connection protocol with super-trivial congestion control over the course of a few days, with the only initial goal being maximum possible performance (not fairness, reliability, or anything else), and trial it in Netty (possibly making some changes to Netty to mitigate thread signalling costs). The reason for knocking up our own here is to get a ceiling on what the absolute limit of potential for this approach is. Assuming this pans out with performance gains in C* proper, we then look to contributing to/forking the udt-java project and see how easy it is to bring performance in line with what we can get with our naive approach (I don't suggest starting here, as the project is using blocking old-IO, and modifying it with latency in mind may be challenging, and we won't know for sure what the best case scenario is). \[1\] http://test-docdb.fnal.gov/0016/001648/002/Potential%20Performance%20Bottleneck%20in%20Linux%20TCP.PDF \[2\] http://cd-docdb.fnal.gov/cgi-bin/RetrieveFile?docid=1968;filename=Performance%20Analysis%20of%20Linux%20Networking%20-%20Packet%20Receiving%20(Official).pdf;version=2 Further related reading: http://public.dhe.ibm.com/software/commerce/doc/mft/cdunix/41/UDTWhitepaper.pdf https://mospace.umsystem.edu/xmlui/bitstream/handle/10355/14482/ChoiUndPerTcp.pdf?sequence=1 https://access.redhat.com/site/documentation/en-US/JBoss_Enterprise_Web_Platform/5/html/Administration_And_Configuration_Guide/jgroups-perf-udpbuffer.html http://citeseerx.ist.psu.edu/viewdoc/download?doi=10.1.1.153.3762rep=rep1type=pdf -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7465) DecoratedKey assertion error on reads
[ https://issues.apache.org/jira/browse/CASSANDRA-7465?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14046693#comment-14046693 ] Benedict commented on CASSANDRA-7465: - Ah, yes this is a more subtle version of the netty buffer recycling bug. It's a case where on _read_ we save data sliced directly from the netty buffer. This problem will affect the row cache as well. We need to think if there's a systematic way we can be certain all bytebuffers we slice expire with the session... it's a tricky problem DecoratedKey assertion error on reads - Key: CASSANDRA-7465 URL: https://issues.apache.org/jira/browse/CASSANDRA-7465 Project: Cassandra Issue Type: Bug Components: Core Environment: 3 nodes Oracle Linux Server 6.3 kernel ver 2.6.39 Reporter: Jason Brown Priority: Blocker Fix For: 2.1.0 Getting the following exception when running read stress: {code}WARN [SharedPool-Worker-31] 2014-06-27 21:25:51,391 AbstractTracingAwareExecutorService.java:166 - Uncaught exception on thread Thread[SharedPool-Worker-31,10,main]: {} java.lang.AssertionError: DecoratedKey(-5397116645141815707, 30303031393143364639) != DecoratedKey(-5397116645141815707, 30303031343439443233) in /u/sdd/cassandra-jasobrown/data/Keyspace1/Standard1-6ab9bd90fe3b11e385edff96c2ef2fd6/Keyspace1-Standard1-ka-73-Data.db at org.apache.cassandra.db.columniterator.SSTableNamesIterator.read(SSTableNamesIterator.java:111) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at org.apache.cassandra.db.columniterator.SSTableNamesIterator.init(SSTableNamesIterator.java:59) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at org.apache.cassandra.db.filter.NamesQueryFilter.getSSTableColumnIterator(NamesQueryFilter.java:89) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:62) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at org.apache.cassandra.db.CollationController.collectTimeOrderedData(CollationController.java:125) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:59) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1873) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1681) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at org.apache.cassandra.db.Keyspace.getRow(Keyspace.java:341) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at org.apache.cassandra.db.SliceByNamesReadCommand.getRow(SliceByNamesReadCommand.java:55) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1393) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2060) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) ~[na:1.7.0_13] at org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:162) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:103) [apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at java.lang.Thread.run(Thread.java:722) [na:1.7.0_13] {code} I have a three node cluster, which I populate with the following stress command: {code} cassandra-stress write n=6000 -schema replication\(factor\=2\) -key populate=1..6000 -rate threads=42 -mode native prepared cql3 -port native=9043 thrift=9161 -node athena06-a,athena06-b,athena06-c -col n=fixed\(21\) size=exp\(11..42\) {code} Then I run the read stress: {code} cassandra-stress read n=1 -key dist=extr\(1..6,2\) -mode native prepared cql3 -port native=9043 thrift=9161 -node athena06-b,athena06-c,athena06-a -col n=fixed\(21\) -rate threads=64 {code} The above exception occurs semi-frequently (several to ~50 times a minute, but seems to depend on amount of data in cluster - anecdotal evidence only). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7056) Add RAMP transactions
[ https://issues.apache.org/jira/browse/CASSANDRA-7056?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14046914#comment-14046914 ] Benedict commented on CASSANDRA-7056: - I can say that, from the point of view of a prior target consumer, the addition of cross-cluster consistent reads would have been exciting for me. On implementation details, thinking more from the point of view of my prior self, I would love to see this support streamed batches of arbitrary size. By which I mean I would have liked to start a write transaction, stream arbitrary amounts of data, and have it commit with complete isolation or not. To this end, I'm leaning towards writing the data straight into the memtables, but maintain a separate set of uncommitted transaction ids, which can be filtered out at read time. If a record is overwritten either before or after it is committed, it is moved to the read-buffer. I doubt this will be dramatically more complex, but the approach to implementation is fundamentally different. It seems to me supporting transactions of arbitrary size is an equally powerful win to consistent transactions. Add RAMP transactions - Key: CASSANDRA-7056 URL: https://issues.apache.org/jira/browse/CASSANDRA-7056 Project: Cassandra Issue Type: Wish Components: Core Reporter: Tupshin Harper Priority: Minor We should take a look at [RAMP|http://www.bailis.org/blog/scalable-atomic-visibility-with-ramp-transactions/] transactions, and figure out if they can be used to provide more efficient LWT (or LWT-like) operations. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7056) Add RAMP transactions
[ https://issues.apache.org/jira/browse/CASSANDRA-7056?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14046919#comment-14046919 ] Benedict commented on CASSANDRA-7056: - Another separate point to consider, as a follow up: RAMP transactions may also permit us to provide consistent reads with less than QUORUM nodes involved. If we are performing a consistent read with a known transaction id, we only need to ensure the node has seen the totality of that transaction (i.e. any bulk insert has completed its first round, but not necessarily its second (commit) round) to be certain we have all of the data we need to answer the query correctly. So we can potentially answer QUORUM queries at the coordinator only. Note this only works if the coordinator has seen _exactly_ this transaction id, though some similar optimisations are likely possible to expand that. I can envisage answering multiple queries with the following scheme: 1) start transaction, by asking for the latest transaction_id from a given coordinator for the data we are interested in; 2) query all coordinators directly for the regions they own, providing them with the transaction_id All of those that were updated with the given transaction_id have the potential to be answered with only the coordinator's involvement Further, to outline a sketch client-side API, I would suggest something like: Txn txn = client.begin() FutureResultSet rsf1 = txn.execute(stmt1); FutureResultSet rsf2 = txn.execute(stmt2); ... txn.execute(); ResultSet rs1 = rsf1.get(); ... Add RAMP transactions - Key: CASSANDRA-7056 URL: https://issues.apache.org/jira/browse/CASSANDRA-7056 Project: Cassandra Issue Type: Wish Components: Core Reporter: Tupshin Harper Priority: Minor We should take a look at [RAMP|http://www.bailis.org/blog/scalable-atomic-visibility-with-ramp-transactions/] transactions, and figure out if they can be used to provide more efficient LWT (or LWT-like) operations. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7468) Add time-based execution to cassandra-stress
[ https://issues.apache.org/jira/browse/CASSANDRA-7468?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14047559#comment-14047559 ] Benedict commented on CASSANDRA-7468: - This is already pretty easy to specify: If you specify [command] n30 n30, it will run for exactly 30s (well, 30 measurements, which may be slightly longer if there are many cross-cluster pauses). We could add a specific command, but I'm not sure it adds a great deal when it's already easy to do. Add time-based execution to cassandra-stress Key: CASSANDRA-7468 URL: https://issues.apache.org/jira/browse/CASSANDRA-7468 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Matt Kennedy Priority: Minor Attachments: trunk-7468.patch -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6146) CQL-native stress
[ https://issues.apache.org/jira/browse/CASSANDRA-6146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14047575#comment-14047575 ] Benedict commented on CASSANDRA-6146: - I've pushed a version of these changes [here|https://github.com/belliottsmith/cassandra/tree/6146-cqlstress] I wanted to integrate the changes a bit more tightly with the old stress, so we didn't seem to simply have two different stresses only nominally related. At the same time I wanted to address a few things I felt were important to setup so that future improvements are easy to introduce: # We now generate partitions predictably, so when we perform queries we can be sure we're using data that is relevant to the partition we're operating over # We explicitly generate multi-row partitions, with configurable distirbution of clustering components # We can support multiple queries / inserts simultaneously in the new path # The new path is executed with a more standard syntax (it's execute with stress user, instead of stress write/read; can perform e.g. inserts/queries with stress user ops(insert=1,query=10) for 90/10 read/write workload) # I've switched configs to all support the range of distributions we could previously (including for size, etc.) # All old paths use the same partition generators as the new paths to keep maintenance and extension simpler # I've moved a few more config parameters into the yaml Some other implications: # To simplify matters and maintenance, I've stripped from the old paths support for super columns, indexes and multi-gets, as we did not typically seem to exercise these paths and these are probably best encapsulated with the new ones # The old path now generates a lot more garbage, because the new path has to, so it will be slightly higher overhead than it was previously. We also only generate random data on the old path, so we may again see a decline in performance Some things still to do in near future; all of which reasonably easy but wanted to limit scope of refactor: # Support deletes # Support partial inserts/deletes (currently insert only supports writing the whole partition) # Support query result validation The diff is quite big, but I think a lot of the changes are due to package movements. CQL-native stress - Key: CASSANDRA-6146 URL: https://issues.apache.org/jira/browse/CASSANDRA-6146 Project: Cassandra Issue Type: New Feature Components: Tools Reporter: Jonathan Ellis Assignee: T Jake Luciani Fix For: 2.1.1 Attachments: 6146-v2.txt, 6146.txt, 6164-v3.txt The existing CQL support in stress is not worth discussing. We need to start over, and we might as well kill two birds with one stone and move to the native protocol while we're at it. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Comment Edited] (CASSANDRA-6146) CQL-native stress
[ https://issues.apache.org/jira/browse/CASSANDRA-6146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14047575#comment-14047575 ] Benedict edited comment on CASSANDRA-6146 at 6/30/14 11:48 AM: --- I've pushed a version of these changes [here|https://github.com/belliottsmith/cassandra/tree/6146-cqlstress] I wanted to integrate the changes a bit more tightly with the old stress, so we didn't seem to simply have two different stresses only nominally related. At the same time I wanted to address a few things I felt were important to setup so that future improvements are easy to introduce: # We now generate partitions predictably, so when we perform queries we can be sure we're using data that is relevant to the partition we're operating over # We explicitly generate multi-row partitions, with configurable distirbution of clustering components # We can support multiple queries / inserts simultaneously in the new path # The new path is executed with a more standard syntax (it's execute with stress user, instead of stress write/read; can perform e.g. inserts/queries with stress user ops(insert=1,query=10) for 90/10 read/write workload) # I've switched configs to all support the range of distributions we could previously (including for size, etc.) # All old paths use the same partition generators as the new paths to keep maintenance and extension simpler # I've moved a few more config parameters into the yaml # We report partition and row statistics now Some other implications: # To simplify matters and maintenance, I've stripped from the old paths support for super columns, indexes and multi-gets, as we did not typically seem to exercise these paths and these are probably best encapsulated with the new ones # The old path now generates a lot more garbage, because the new path has to, so it will be slightly higher overhead than it was previously. We also only generate random data on the old path, so we may again see a decline in performance Some things still to do in near future; all of which reasonably easy but wanted to limit scope of refactor: # Support deletes # Support partial inserts/deletes (currently insert only supports writing the whole partition) # Support query result validation The diff is quite big, but I think a lot of the changes are due to package movements. The basic functionality of your patch is left intact, so hopefully it shouldn't be too tricky to figure out what's happening now. was (Author: benedict): I've pushed a version of these changes [here|https://github.com/belliottsmith/cassandra/tree/6146-cqlstress] I wanted to integrate the changes a bit more tightly with the old stress, so we didn't seem to simply have two different stresses only nominally related. At the same time I wanted to address a few things I felt were important to setup so that future improvements are easy to introduce: # We now generate partitions predictably, so when we perform queries we can be sure we're using data that is relevant to the partition we're operating over # We explicitly generate multi-row partitions, with configurable distirbution of clustering components # We can support multiple queries / inserts simultaneously in the new path # The new path is executed with a more standard syntax (it's execute with stress user, instead of stress write/read; can perform e.g. inserts/queries with stress user ops(insert=1,query=10) for 90/10 read/write workload) # I've switched configs to all support the range of distributions we could previously (including for size, etc.) # All old paths use the same partition generators as the new paths to keep maintenance and extension simpler # I've moved a few more config parameters into the yaml Some other implications: # To simplify matters and maintenance, I've stripped from the old paths support for super columns, indexes and multi-gets, as we did not typically seem to exercise these paths and these are probably best encapsulated with the new ones # The old path now generates a lot more garbage, because the new path has to, so it will be slightly higher overhead than it was previously. We also only generate random data on the old path, so we may again see a decline in performance Some things still to do in near future; all of which reasonably easy but wanted to limit scope of refactor: # Support deletes # Support partial inserts/deletes (currently insert only supports writing the whole partition) # Support query result validation The diff is quite big, but I think a lot of the changes are due to package movements. CQL-native stress - Key: CASSANDRA-6146 URL: https://issues.apache.org/jira/browse/CASSANDRA-6146 Project: Cassandra Issue Type: New Feature Components: Tools Reporter: Jonathan Ellis Assignee: T
[jira] [Resolved] (CASSANDRA-6810) SSTable and Index Layout Improvements/Modifications
[ https://issues.apache.org/jira/browse/CASSANDRA-6810?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benedict resolved CASSANDRA-6810. - Resolution: Duplicate SSTable and Index Layout Improvements/Modifications --- Key: CASSANDRA-6810 URL: https://issues.apache.org/jira/browse/CASSANDRA-6810 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Benedict Labels: performance Fix For: 3.0 Right now SSTables are somewhat inefficient in their storage of composite keys. I propose resolving this by merging (some of) the index functionality with the storage of keys, through introducing a composite btree/trie structure (e.g. string b-tree) to represent the key, and for this structure to index into the cell position in the file. This structure can then serve as both an efficient index and the key data itself. If we then offer the option of (possibly automatically decided for you at flush) storing this either packed into the same file directly prepending the data, or in a separate key file (with small pages), with an uncompressed page cache we can get good performance for wide rows by storing it separately and relying on the page cache for CQL row index lookups, whereas storing it inline will allow very efficient lookups of small rows where index lookups aren't particularly helpful. This removal of extra data from the index file, however, will allow CASSANDRA-6709 to massively scale up the efficiency of the key cache, whilst also reducing the total disk footprint of sstables and (most likely) offering better indexing capability in similar space -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Resolved] (CASSANDRA-6830) Changes to SSTable Index file
[ https://issues.apache.org/jira/browse/CASSANDRA-6830?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benedict resolved CASSANDRA-6830. - Resolution: Duplicate Changes to SSTable Index file - Key: CASSANDRA-6830 URL: https://issues.apache.org/jira/browse/CASSANDRA-6830 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Benedict Priority: Minor Labels: performance Fix For: 3.0 Building on the ideas introduced in CASSANDRA-6709, and _possibly_ obseleting them before they are introduced: Once we have CASSANDRA-6810, we could make the following change to the (current) index file: instead of producing a sorted decoratedkey file, we could instead generate a near\-optimal hash table of murmurhash\-of\-key \- position in data/(6810\-)index file. This index might permit multiple locations for each hash, in which case all locations would need to be checked, but a hash table could be built that minimises this (whilst also maximising compact representation on disk) This then might completely obviate the need for a separate key cache, as we simply rely on whatever buffer cache we have to map in/out the pages we need for our query in any index. We should be able to guarantee we only ever need to look at one page for any query. Once we bring page-caching in process, the size of the pages we actually choose to cache could be configurable which would bring behaviour to near same as key cache currently stands, except more compact, and also effectively auto-sizing itself to optimally reduce reads (by using more buffer cache space if it is helpful, and yielding it to other reads otherwise). The obvious disadvantage is that partition key ranges become a little more expensive, but (the?/)an index summary should reduce the problem here, so that binary search for a start point can be targeted to a few or single (6810\-)index page. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Resolved] (CASSANDRA-7260) Stress overflows past 2^31 keys
[ https://issues.apache.org/jira/browse/CASSANDRA-7260?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benedict resolved CASSANDRA-7260. - Resolution: Duplicate Stress overflows past 2^31 keys --- Key: CASSANDRA-7260 URL: https://issues.apache.org/jira/browse/CASSANDRA-7260 Project: Cassandra Issue Type: Bug Components: Tools Reporter: Ryan McGuire Assignee: Benedict Priority: Minor Writing more than 2^31 keys with stress causes an overflow making stress start to count negative key numbers. It doesn't stop the write, but it does foul up the aggregate statistics: {code} 2147395249, 69404, 69404,11.7, 5.1,35.4, 135.7, 169.9, 250.8,27093.0, 0.00211 -2147481870, 89925, 89925, 9.0, 4.8,27.1, 110.3, 120.3, 128.5,27094.0, 0.00211 . -1894967296,7318,7318, 0.7, 0.6, 0.8, 1.9,60.4, 60.4,30546.6, 0.00197 Results: real op rate : -62035 adjusted op rate stderr : 0 key rate : -62035 latency mean : -13.0 latency median: 3.9 latency 95th percentile : 16.2 latency 99th percentile : 35.1 latency 99.9th percentile : 170.1 latency max : 4376.7 Total operation time : 08:29:06 END {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Resolved] (CASSANDRA-6806) In AtomicBTreeColumns, construct list of unwinds after a race lazily
[ https://issues.apache.org/jira/browse/CASSANDRA-6806?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benedict resolved CASSANDRA-6806. - Resolution: Won't Fix This is a minor improvement, and it is likely we will deliver a new memtable data structure after CASSANDRA-7447, so wont' fixing In AtomicBTreeColumns, construct list of unwinds after a race lazily Key: CASSANDRA-6806 URL: https://issues.apache.org/jira/browse/CASSANDRA-6806 Project: Cassandra Issue Type: Improvement Reporter: Benedict Priority: Minor Fix For: 3.0 Currently we store these in a List, but this is wasteful. We can construct them lazily from a diff between the original and partially constructed replacement BTree. The UpdaterFunction could define a method to be passed such a collection in the event of an early abort. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Resolved] (CASSANDRA-6804) Consolidate on-disk and NativeCell layouts so that reads from disk require less memory
[ https://issues.apache.org/jira/browse/CASSANDRA-6804?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benedict resolved CASSANDRA-6804. - Resolution: Won't Fix CASSANDRA-7447 probably makes this impossible, or at least undesirable, since we will not store all necessary data for a single cell contiguously. Consolidate on-disk and NativeCell layouts so that reads from disk require less memory -- Key: CASSANDRA-6804 URL: https://issues.apache.org/jira/browse/CASSANDRA-6804 Project: Cassandra Issue Type: Improvement Reporter: Benedict Fix For: 3.0 If the on-disk Cell representation were the same as we use for NativeCell, we could easily allocate a NativeCell instead of a BufferCell, immediately reducing the amount of garbage generated on reads. With further work we may also be able to reach a zero-copy allocation as well, reducing further the read costs. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7465) DecoratedKey assertion error on reads
[ https://issues.apache.org/jira/browse/CASSANDRA-7465?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14047879#comment-14047879 ] Benedict commented on CASSANDRA-7465: - Fixing this bug is trivial; being certain we haven't still got any others is evidently less so. DecoratedKey assertion error on reads - Key: CASSANDRA-7465 URL: https://issues.apache.org/jira/browse/CASSANDRA-7465 Project: Cassandra Issue Type: Bug Components: Core Environment: 3 nodes Oracle Linux Server 6.3 kernel ver 2.6.39 Reporter: Jason Brown Priority: Blocker Fix For: 2.1.0 Getting the following exception when running read stress: {code}WARN [SharedPool-Worker-31] 2014-06-27 21:25:51,391 AbstractTracingAwareExecutorService.java:166 - Uncaught exception on thread Thread[SharedPool-Worker-31,10,main]: {} java.lang.AssertionError: DecoratedKey(-5397116645141815707, 30303031393143364639) != DecoratedKey(-5397116645141815707, 30303031343439443233) in /u/sdd/cassandra-jasobrown/data/Keyspace1/Standard1-6ab9bd90fe3b11e385edff96c2ef2fd6/Keyspace1-Standard1-ka-73-Data.db at org.apache.cassandra.db.columniterator.SSTableNamesIterator.read(SSTableNamesIterator.java:111) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at org.apache.cassandra.db.columniterator.SSTableNamesIterator.init(SSTableNamesIterator.java:59) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at org.apache.cassandra.db.filter.NamesQueryFilter.getSSTableColumnIterator(NamesQueryFilter.java:89) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:62) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at org.apache.cassandra.db.CollationController.collectTimeOrderedData(CollationController.java:125) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:59) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1873) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1681) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at org.apache.cassandra.db.Keyspace.getRow(Keyspace.java:341) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at org.apache.cassandra.db.SliceByNamesReadCommand.getRow(SliceByNamesReadCommand.java:55) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1393) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2060) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) ~[na:1.7.0_13] at org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:162) ~[apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:103) [apache-cassandra-2.1.0-rc2-SNAPSHOT.jar:2.1.0-rc2-SNAPSHOT] at java.lang.Thread.run(Thread.java:722) [na:1.7.0_13] {code} I have a three node cluster, which I populate with the following stress command: {code} cassandra-stress write n=6000 -schema replication\(factor\=2\) -key populate=1..6000 -rate threads=42 -mode native prepared cql3 -port native=9043 thrift=9161 -node athena06-a,athena06-b,athena06-c -col n=fixed\(21\) size=exp\(11..42\) {code} Then I run the read stress: {code} cassandra-stress read n=1 -key dist=extr\(1..6,2\) -mode native prepared cql3 -port native=9043 thrift=9161 -node athena06-b,athena06-c,athena06-a -col n=fixed\(21\) -rate threads=64 {code} The above exception occurs semi-frequently (several to ~50 times a minute, but seems to depend on amount of data in cluster - anecdotal evidence only). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7468) Add time-based execution to cassandra-stress
[ https://issues.apache.org/jira/browse/CASSANDRA-7468?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14047913#comment-14047913 ] Benedict commented on CASSANDRA-7468: - Well, -d isn't much clearer to me. We could change the n n to s(amples)? s(amples)?, and offer a parallel s(amples)?= also, so that we have a standardised syntax. Add time-based execution to cassandra-stress Key: CASSANDRA-7468 URL: https://issues.apache.org/jira/browse/CASSANDRA-7468 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Matt Kennedy Priority: Minor Attachments: trunk-7468.patch -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6146) CQL-native stress
[ https://issues.apache.org/jira/browse/CASSANDRA-6146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14048601#comment-14048601 ] Benedict commented on CASSANDRA-6146: - It looks like I heavily overestimated how much of the changes were down to package moves, or under estimated how much I overhauled stress. I've rebased into 4 commits: jake's patch, package movements, deletion of old functionality, then the guts of the refactor. The last step is still a pretty significant chunk of changes (~2.5k +/-), and primarily revolves around the introduction of the concept of PartitionGenerator and SeedGenerator (and removal of the old RowGen/KeyGen), which subtly changes program flow pretty much everywhere. There's also the parallel introduction of OpDistribution which requires some annoying changes in the settings hierarchy, but simplifies the changes necessary outside to support mixed operations of both the old and new kind. * ~0.8k +/- goes on in generate, which are pretty trivial changes; * ~0.6k are refactoring the old operations to use the new generators, and is mostly straightforward; primarily it involves the introduction of a new PredefinedOperation class, and rewiring the old classes to use its slightly different methods * ~.2k are refactoring the new insert/read statements to share the same common superclass, and use the new partition generator; * ~.7k are in the settings classes, and are probably the most annoying changes to review, but also not super important * the remainder are in the base classes Operation, StressAction and StressProfile If this is too painful, I'll see what can be done to split the patch out further. CQL-native stress - Key: CASSANDRA-6146 URL: https://issues.apache.org/jira/browse/CASSANDRA-6146 Project: Cassandra Issue Type: New Feature Components: Tools Reporter: Jonathan Ellis Assignee: T Jake Luciani Fix For: 2.1.1 Attachments: 6146-v2.txt, 6146.txt, 6164-v3.txt The existing CQL support in stress is not worth discussing. We need to start over, and we might as well kill two birds with one stone and move to the native protocol while we're at it. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Comment Edited] (CASSANDRA-6146) CQL-native stress
[ https://issues.apache.org/jira/browse/CASSANDRA-6146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14048601#comment-14048601 ] Benedict edited comment on CASSANDRA-6146 at 7/1/14 7:43 AM: - It looks like I heavily overestimated how much of the changes were down to package moves, or under estimated how much I overhauled stress. I've rebased into 4 commits: jake's patch, package movements, deletion of old functionality, then the guts of the refactor. The last step is still a pretty significant chunk of changes (~2.5k +/-), and primarily revolves around the introduction of the concept of PartitionGenerator and SeedGenerator (and removal of the old RowGen/KeyGen), which subtly changes program flow pretty much everywhere. There's also the parallel introduction of OpDistribution which requires some annoying changes in the settings hierarchy, but simplifies the changes necessary outside to support mixed operations of both the old and new kind. * ~0.8k +/- goes on in generate, which are pretty trivial changes; * ~0.6k are refactoring the old operations to use the new generators, and is mostly straightforward; primarily it involves the introduction of a new PredefinedOperation class, and rewiring the old classes to use its slightly different methods * ~.2k are refactoring the new insert/read statements to share the same common superclass, and use the new partition generator; * ~.7k are in the settings classes, and are probably the most annoying changes to review, but also not super important * the remainder are in the base classes Operation, StressAction and StressProfile If this is too painful, I'll see what can be done to split the patch out further. Branch can be found [here|https://github.com/belliottsmith/cassandra/commits/6146-cqlstress-inc] was (Author: benedict): It looks like I heavily overestimated how much of the changes were down to package moves, or under estimated how much I overhauled stress. I've rebased into 4 commits: jake's patch, package movements, deletion of old functionality, then the guts of the refactor. The last step is still a pretty significant chunk of changes (~2.5k +/-), and primarily revolves around the introduction of the concept of PartitionGenerator and SeedGenerator (and removal of the old RowGen/KeyGen), which subtly changes program flow pretty much everywhere. There's also the parallel introduction of OpDistribution which requires some annoying changes in the settings hierarchy, but simplifies the changes necessary outside to support mixed operations of both the old and new kind. * ~0.8k +/- goes on in generate, which are pretty trivial changes; * ~0.6k are refactoring the old operations to use the new generators, and is mostly straightforward; primarily it involves the introduction of a new PredefinedOperation class, and rewiring the old classes to use its slightly different methods * ~.2k are refactoring the new insert/read statements to share the same common superclass, and use the new partition generator; * ~.7k are in the settings classes, and are probably the most annoying changes to review, but also not super important * the remainder are in the base classes Operation, StressAction and StressProfile If this is too painful, I'll see what can be done to split the patch out further. CQL-native stress - Key: CASSANDRA-6146 URL: https://issues.apache.org/jira/browse/CASSANDRA-6146 Project: Cassandra Issue Type: New Feature Components: Tools Reporter: Jonathan Ellis Assignee: T Jake Luciani Fix For: 2.1.1 Attachments: 6146-v2.txt, 6146.txt, 6164-v3.txt The existing CQL support in stress is not worth discussing. We need to start over, and we might as well kill two birds with one stone and move to the native protocol while we're at it. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6146) CQL-native stress
[ https://issues.apache.org/jira/browse/CASSANDRA-6146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14048619#comment-14048619 ] Benedict commented on CASSANDRA-6146: - I've commented on the final github commit with brief explanations of what is happening with each block of changes. It's a lot of lines of code, but the actual complexity isn't that high. CQL-native stress - Key: CASSANDRA-6146 URL: https://issues.apache.org/jira/browse/CASSANDRA-6146 Project: Cassandra Issue Type: New Feature Components: Tools Reporter: Jonathan Ellis Assignee: T Jake Luciani Fix For: 2.1.1 Attachments: 6146-v2.txt, 6146.txt, 6164-v3.txt The existing CQL support in stress is not worth discussing. We need to start over, and we might as well kill two birds with one stone and move to the native protocol while we're at it. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7125) Fail to start by default if Commit Log fails to validate any messages
[ https://issues.apache.org/jira/browse/CASSANDRA-7125?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14048921#comment-14048921 ] Benedict commented on CASSANDRA-7125: - Pushed a patch [here|https://github.com/belliottsmith/cassandra/tree/7125-failcl] Fail to start by default if Commit Log fails to validate any messages - Key: CASSANDRA-7125 URL: https://issues.apache.org/jira/browse/CASSANDRA-7125 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Benedict Assignee: Benedict Priority: Minor Labels: correctness Fix For: 2.1.1 Current behaviour can be pretty dangerous, and also has a tendency to mask bugs during development. We should change the behaviour to default to failure if anything unexpected happens, and introduce a cassandra.yaml option that permits overriding the default behaviour. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6146) CQL-native stress
[ https://issues.apache.org/jira/browse/CASSANDRA-6146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14048994#comment-14048994 ] Benedict commented on CASSANDRA-6146: - bq. It sounds like writing to an entire partition at once is a step backwards from the original patch, since you can't test writing incrementally to a wide row. all clustering columns are written at once (unless I'm misunderstanding). Previously the population distribution of a column was not within a partition so you could make it very large. The problem with the prior approach was that you could not control the size of partition you created, nor whether or not you were actually querying any data for the non-insert operations. The only control you had was the size of your population for each field, so the only way to perform incremental inserts to a partition was to constrain your partition key domain to a fraction of the domain of the clustering columns. This did not give you much capacity to control or reason about how much data was being inserted to a given partition, nor how this was distributed, nor, importantly, how many distinct partitions were updated for a single batch statement, and it meant that we would likely benchmark queries that returned (and even operated over) no data, with no way of knowing if this was correct or not. The new approach lets us validate the data we get back, be certain we are operating over data that should exist (so does real work), and even knows how much data it's operating over to report accurate statistics. It also lets us control how many cql rows we insert into a single partition in one batch. Modifying the current approach to write/generate only a portion of a partition at a time is relatively trivial; we can even support an extra batch option that supports splitting an insert for a single partition into multiple distinct batch statements, I only left it out to put some kind of cap on the number of changes introduced in this ticket, but don't mind including it this round. bq. I'm not sure how I feel about putting the batchsize and batchtype into the yaml. Those feel like command line args to me. The problem with a command line option is it applies to all operations; whilst we don't currently support batching for anything other than inserts, it's quite likely we'll want to for, e.g., deletes and potentially also for queries with IN statements. But I'm not dead set against moving this out onto the command line. bq. I think we should change the term identity to population as it seems clearer to me for the columnspec. and in the code identityDistribution to populationDistribution Sure. We should comment that this is a unique seed population, and not the actual population, however. bq. I'm trying to run with one of the yaml files and getting an error: Whoops. Obviously I broke something in a final tweak somewhere :/ CQL-native stress - Key: CASSANDRA-6146 URL: https://issues.apache.org/jira/browse/CASSANDRA-6146 Project: Cassandra Issue Type: New Feature Components: Tools Reporter: Jonathan Ellis Assignee: T Jake Luciani Fix For: 2.1.1 Attachments: 6146-v2.txt, 6146.txt, 6164-v3.txt The existing CQL support in stress is not worth discussing. We need to start over, and we might as well kill two birds with one stone and move to the native protocol while we're at it. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Comment Edited] (CASSANDRA-6146) CQL-native stress
[ https://issues.apache.org/jira/browse/CASSANDRA-6146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14048994#comment-14048994 ] Benedict edited comment on CASSANDRA-6146 at 7/1/14 4:03 PM: - bq. It sounds like writing to an entire partition at once is a step backwards from the original patch, since you can't test writing incrementally to a wide row. all clustering columns are written at once (unless I'm misunderstanding). Previously the population distribution of a column was not within a partition so you could make it very large. The problem with the prior approach was that you could not control the size of partition you created, nor whether or not you were actually querying any data for the non-insert operations. The only control you had was the size of your population for each field, so the only way to perform incremental inserts to a partition was to constrain your partition key domain to a fraction of the domain of the clustering columns. This did not give you much capacity to control or reason about how much data was being inserted to a given partition, nor how this was distributed, nor, importantly, how many distinct partitions were updated for a single batch statement, and it meant that we would likely benchmark queries that returned (and even operated over) no data, with no way of knowing if this was correct or not. The new approach lets us validate the data we get back, be certain we are operating over data that should exist (so does real work), and even knows how much data it's operating over to report accurate statistics. It also lets us control how many cql rows we insert into a single partition in one batch. Modifying the current approach to write/generate only a portion of a partition at a time is relatively trivial; we can even support an extra batch option that supports splitting an insert for a single partition into multiple distinct batch statements so we can control very specifically how incrementally the data is written. I only left it out to put some kind of cap on the number of changes introduced in this ticket, but don't mind including it this round. bq. I'm not sure how I feel about putting the batchsize and batchtype into the yaml. Those feel like command line args to me. The problem with a command line option is it applies to all operations; whilst we don't currently support batching for anything other than inserts, it's quite likely we'll want to for, e.g., deletes and potentially also for queries with IN statements. But I'm not dead set against moving this out onto the command line. bq. I think we should change the term identity to population as it seems clearer to me for the columnspec. and in the code identityDistribution to populationDistribution Sure. We should comment that this is a unique seed population, and not the actual population, however. bq. I'm trying to run with one of the yaml files and getting an error: Whoops. Obviously I broke something in a final tweak somewhere :/ was (Author: benedict): bq. It sounds like writing to an entire partition at once is a step backwards from the original patch, since you can't test writing incrementally to a wide row. all clustering columns are written at once (unless I'm misunderstanding). Previously the population distribution of a column was not within a partition so you could make it very large. The problem with the prior approach was that you could not control the size of partition you created, nor whether or not you were actually querying any data for the non-insert operations. The only control you had was the size of your population for each field, so the only way to perform incremental inserts to a partition was to constrain your partition key domain to a fraction of the domain of the clustering columns. This did not give you much capacity to control or reason about how much data was being inserted to a given partition, nor how this was distributed, nor, importantly, how many distinct partitions were updated for a single batch statement, and it meant that we would likely benchmark queries that returned (and even operated over) no data, with no way of knowing if this was correct or not. The new approach lets us validate the data we get back, be certain we are operating over data that should exist (so does real work), and even knows how much data it's operating over to report accurate statistics. It also lets us control how many cql rows we insert into a single partition in one batch. Modifying the current approach to write/generate only a portion of a partition at a time is relatively trivial; we can even support an extra batch option that supports splitting an insert for a single partition into multiple distinct batch statements, I only left it out to put some kind of cap on the number of changes introduced in this ticket, but don't mind
[jira] [Commented] (CASSANDRA-7486) Compare CMS and G1 pause times
[ https://issues.apache.org/jira/browse/CASSANDRA-7486?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14049217#comment-14049217 ] Benedict commented on CASSANDRA-7486: - We need to make sure we test over an extended period with a variety of operations being exercised against the cluster. This is probably a good opportunity to try and define a real world burn in test as well, and what parameters should be included. Some things to consider: # Range of data distributions, including (esp. for this) large partitions and very large cells. Possibly run two or three parallel stress profiles with very different data profiles to really give GC a headache dealing with different velocities / lifetimes. # Incremental and full repairs # Hint accumulation / node death # Tombstones / Range Tombstones # Secondary indexes? I'd suggest ignoring some variables, and e.g. stick with just netty, so we can define a single complex workload and run it for an extended period and get a good result. While our client buffers behave quite differently with each, I'm happy tuning defaults for native now it's faster. It might also be useful, for this test only, to see for a single node how well the two degrade as heap pressure increases, by artificially consuming large portions of the heap for the duration of a more simple stress test. Compare CMS and G1 pause times -- Key: CASSANDRA-7486 URL: https://issues.apache.org/jira/browse/CASSANDRA-7486 Project: Cassandra Issue Type: Test Components: Config Reporter: Jonathan Ellis Assignee: Ryan McGuire Fix For: 2.1.0 See http://www.slideshare.net/MonicaBeckwith/garbage-first-garbage-collector-g1-gc-migration-to-expectations-and-advanced-tuning and https://twitter.com/rbranson/status/482113561431265281 May want to default 2.1 to G1. 2.1 is a different animal from 2.0 after moving most of memtables off heap. Suspect this will help G1 even more than CMS. (NB this is off by default but needs to be part of the test.) -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7437) Drop Keyspace and ColumnFamily can raise assertions inside of forceRecycleAll still
[ https://issues.apache.org/jira/browse/CASSANDRA-7437?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14050582#comment-14050582 ] Benedict commented on CASSANDRA-7437: - Except I have no way to prevent those actions from writing data that would hit the CL afterwards, and we need to be certain this does not happen (so that we can recycle all of the segments) Drop Keyspace and ColumnFamily can raise assertions inside of forceRecycleAll still --- Key: CASSANDRA-7437 URL: https://issues.apache.org/jira/browse/CASSANDRA-7437 Project: Cassandra Issue Type: Bug Components: Core Reporter: Benedict Assignee: Benedict Priority: Minor Fix For: 2.1.0 I've noticed on unit test output that there are still assertions being raised here, so I've taken a torch to the code path to make damned certain it cannot happen in future # We now wait for all running reads on a column family or writes on the keyspace during a dropCf call # We wait for all appends to the prior commit log segments before recycling them # We pass the list of dropped Cfs into the CL.forceRecycle call so that they can be markedClean definitely after they have been marked finished # Finally, to prevent any possibility of this still happening causing any negative consequences, I've suppressed the assertion in favour of an error log message, as the assertion would break correct program flow for the drop and potentially result in undefined behaviour -(in actuality there is the slightest possibility still of a race condition on read of a secondary index that causes a repair driven write, but this is a really tiny race window, as I force wait for all reads after unlinking the CF, so it would have to be a read that grabbed the CFS reference before it was dropped, but hadn't quite started its read op yet).- In fact this is also safe, as these modifications all grab a write op from the Keyspace, which has to happen before they get the CFS, and also because we drop the data before waiting for reads to finish on the CFS. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Comment Edited] (CASSANDRA-7437) Drop Keyspace and ColumnFamily can raise assertions inside of forceRecycleAll still
[ https://issues.apache.org/jira/browse/CASSANDRA-7437?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14050582#comment-14050582 ] Benedict edited comment on CASSANDRA-7437 at 7/2/14 7:14 PM: - Except I have no way to prevent those actions from writing data that would hit the CL afterwards, and we need to be certain this does not happen (so that we can recycle all of the segments). The only thing we can safely do is wait for them to finish (unless you have a suggestion :)) was (Author: benedict): Except I have no way to prevent those actions from writing data that would hit the CL afterwards, and we need to be certain this does not happen (so that we can recycle all of the segments) Drop Keyspace and ColumnFamily can raise assertions inside of forceRecycleAll still --- Key: CASSANDRA-7437 URL: https://issues.apache.org/jira/browse/CASSANDRA-7437 Project: Cassandra Issue Type: Bug Components: Core Reporter: Benedict Assignee: Benedict Priority: Minor Fix For: 2.1.0 I've noticed on unit test output that there are still assertions being raised here, so I've taken a torch to the code path to make damned certain it cannot happen in future # We now wait for all running reads on a column family or writes on the keyspace during a dropCf call # We wait for all appends to the prior commit log segments before recycling them # We pass the list of dropped Cfs into the CL.forceRecycle call so that they can be markedClean definitely after they have been marked finished # Finally, to prevent any possibility of this still happening causing any negative consequences, I've suppressed the assertion in favour of an error log message, as the assertion would break correct program flow for the drop and potentially result in undefined behaviour -(in actuality there is the slightest possibility still of a race condition on read of a secondary index that causes a repair driven write, but this is a really tiny race window, as I force wait for all reads after unlinking the CF, so it would have to be a read that grabbed the CFS reference before it was dropped, but hadn't quite started its read op yet).- In fact this is also safe, as these modifications all grab a write op from the Keyspace, which has to happen before they get the CFS, and also because we drop the data before waiting for reads to finish on the CFS. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7437) Drop Keyspace and ColumnFamily can raise assertions inside of forceRecycleAll still
[ https://issues.apache.org/jira/browse/CASSANDRA-7437?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14050587#comment-14050587 ] Benedict commented on CASSANDRA-7437: - It's worth noting the drop has basically happened by then; it's just tidying up Drop Keyspace and ColumnFamily can raise assertions inside of forceRecycleAll still --- Key: CASSANDRA-7437 URL: https://issues.apache.org/jira/browse/CASSANDRA-7437 Project: Cassandra Issue Type: Bug Components: Core Reporter: Benedict Assignee: Benedict Priority: Minor Fix For: 2.1.0 I've noticed on unit test output that there are still assertions being raised here, so I've taken a torch to the code path to make damned certain it cannot happen in future # We now wait for all running reads on a column family or writes on the keyspace during a dropCf call # We wait for all appends to the prior commit log segments before recycling them # We pass the list of dropped Cfs into the CL.forceRecycle call so that they can be markedClean definitely after they have been marked finished # Finally, to prevent any possibility of this still happening causing any negative consequences, I've suppressed the assertion in favour of an error log message, as the assertion would break correct program flow for the drop and potentially result in undefined behaviour -(in actuality there is the slightest possibility still of a race condition on read of a secondary index that causes a repair driven write, but this is a really tiny race window, as I force wait for all reads after unlinking the CF, so it would have to be a read that grabbed the CFS reference before it was dropped, but hadn't quite started its read op yet).- In fact this is also safe, as these modifications all grab a write op from the Keyspace, which has to happen before they get the CFS, and also because we drop the data before waiting for reads to finish on the CFS. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7437) Drop Keyspace and ColumnFamily can raise assertions inside of forceRecycleAll still
[ https://issues.apache.org/jira/browse/CASSANDRA-7437?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14050592#comment-14050592 ] Benedict commented on CASSANDRA-7437: - But they shouldn't find it, as it's been removed from the set of known CFs - it's only those _already with a reference_ we're waiting for Drop Keyspace and ColumnFamily can raise assertions inside of forceRecycleAll still --- Key: CASSANDRA-7437 URL: https://issues.apache.org/jira/browse/CASSANDRA-7437 Project: Cassandra Issue Type: Bug Components: Core Reporter: Benedict Assignee: Benedict Priority: Minor Fix For: 2.1.0 I've noticed on unit test output that there are still assertions being raised here, so I've taken a torch to the code path to make damned certain it cannot happen in future # We now wait for all running reads on a column family or writes on the keyspace during a dropCf call # We wait for all appends to the prior commit log segments before recycling them # We pass the list of dropped Cfs into the CL.forceRecycle call so that they can be markedClean definitely after they have been marked finished # Finally, to prevent any possibility of this still happening causing any negative consequences, I've suppressed the assertion in favour of an error log message, as the assertion would break correct program flow for the drop and potentially result in undefined behaviour -(in actuality there is the slightest possibility still of a race condition on read of a secondary index that causes a repair driven write, but this is a really tiny race window, as I force wait for all reads after unlinking the CF, so it would have to be a read that grabbed the CFS reference before it was dropped, but hadn't quite started its read op yet).- In fact this is also safe, as these modifications all grab a write op from the Keyspace, which has to happen before they get the CFS, and also because we drop the data before waiting for reads to finish on the CFS. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6146) CQL-native stress
[ https://issues.apache.org/jira/browse/CASSANDRA-6146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14050760#comment-14050760 ] Benedict commented on CASSANDRA-6146: - Ok, I've pushed another update to the repository. This makes a few minor change as well as a major one: # Default is now to use -mode cql3 native prepared # Thrift support is fixed for the new schema mode # Partitions are now generated _completely procedurally_, i.e. if there are C clustering components each with N items _each time they occur_ (i.e. at each depth, or with clustering: fixed(N)), then we require O(CN) memory to generate a partition, not O(N^C) # We can choose to generate a random ratio of the total rows of a partition for a single update, and we can choose to batch these in different ways as well (e.g. we can choose to generate only 2% of a potentially huge partition, and then choose to submit that in batches of between 10% and 50% of the generated subset (i.e. between 0.2% and 1% of the total possible partition)). We should probably expand this to support a hard maximum for batches as well, to ensure the client can maintain predictable memory utilisation with extreme distributions, but this is probably not super important right now. At some point it would be really nice to make the particular row selection for a given pass predictable, so we can know the exact state of the row on the server, so we can procedurally generate inserts/deletes and still always perform queries/deletes on data that we know exists. But I think that's a hard problem with maintaining minimal client state. CQL-native stress - Key: CASSANDRA-6146 URL: https://issues.apache.org/jira/browse/CASSANDRA-6146 Project: Cassandra Issue Type: New Feature Components: Tools Reporter: Jonathan Ellis Assignee: T Jake Luciani Fix For: 2.1.1 Attachments: 6146-v2.txt, 6146.txt, 6164-v3.txt The existing CQL support in stress is not worth discussing. We need to start over, and we might as well kill two birds with one stone and move to the native protocol while we're at it. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7437) Ensure writes have completed after dropping a table, before recycling commit log segments (CASSANDRA-7437)
[ https://issues.apache.org/jira/browse/CASSANDRA-7437?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benedict updated CASSANDRA-7437: Summary: Ensure writes have completed after dropping a table, before recycling commit log segments (CASSANDRA-7437) (was: Drop Keyspace and ColumnFamily can raise assertions inside of forceRecycleAll still) Ensure writes have completed after dropping a table, before recycling commit log segments (CASSANDRA-7437) --- Key: CASSANDRA-7437 URL: https://issues.apache.org/jira/browse/CASSANDRA-7437 Project: Cassandra Issue Type: Bug Components: Core Reporter: Benedict Assignee: Benedict Priority: Minor Fix For: 2.1.0 I've noticed on unit test output that there are still assertions being raised here, so I've taken a torch to the code path to make damned certain it cannot happen in future # We now wait for all running reads on a column family or writes on the keyspace during a dropCf call # We wait for all appends to the prior commit log segments before recycling them # We pass the list of dropped Cfs into the CL.forceRecycle call so that they can be markedClean definitely after they have been marked finished # Finally, to prevent any possibility of this still happening causing any negative consequences, I've suppressed the assertion in favour of an error log message, as the assertion would break correct program flow for the drop and potentially result in undefined behaviour -(in actuality there is the slightest possibility still of a race condition on read of a secondary index that causes a repair driven write, but this is a really tiny race window, as I force wait for all reads after unlinking the CF, so it would have to be a read that grabbed the CFS reference before it was dropped, but hadn't quite started its read op yet).- In fact this is also safe, as these modifications all grab a write op from the Keyspace, which has to happen before they get the CFS, and also because we drop the data before waiting for reads to finish on the CFS. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-7489) Track lower bound necessary for a repair, live, without actually repairing
Benedict created CASSANDRA-7489: --- Summary: Track lower bound necessary for a repair, live, without actually repairing Key: CASSANDRA-7489 URL: https://issues.apache.org/jira/browse/CASSANDRA-7489 Project: Cassandra Issue Type: Improvement Reporter: Benedict We will need a few things in place to get this right, but it should be possible to track live what the current health of a single range is across the cluster. If we force an owning node to be the coordinator for an update (so if a non-smart client sends a mutation to a non-owning node, it just proxies it on to an owning node to coordinate the update; this should tend to minimal overhead as smart clients become the norm, and smart clients scale up to cope with huge clusters), then each owner can maintain the oldest known timestamp it has coordinated an update for that was not acknowledged by every owning node it propagated it to. The minimum of all of these for a region is the lower bound from which we need to either repair, or retain tombstones. There are some subtleties with this, but it means tombstones can be cleared potentially only minutes after they are generated, instead of days or weeks. It also means even repairs can be even more incremental, only operating over ranges and time periods we know to be potentially out of sync. It will most likely need RAMP transactions in place, so that atomic batch mutations are not serialized on non-owning nodes. Having owning nodes coordinate updates is to ensure robustness in case of a single node failure - in this case all ranges owned by the node are considered to have a lower bound of -Inf. Without this a single node being down would result in the entire cluster being considered out of sync. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7489) Track lower bound necessary for a repair, live, without actually repairing
[ https://issues.apache.org/jira/browse/CASSANDRA-7489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14050844#comment-14050844 ] Benedict commented on CASSANDRA-7489: - cc [~rbranson] Track lower bound necessary for a repair, live, without actually repairing -- Key: CASSANDRA-7489 URL: https://issues.apache.org/jira/browse/CASSANDRA-7489 Project: Cassandra Issue Type: Improvement Reporter: Benedict Labels: performance, repair We will need a few things in place to get this right, but it should be possible to track live what the current health of a single range is across the cluster. If we force an owning node to be the coordinator for an update (so if a non-smart client sends a mutation to a non-owning node, it just proxies it on to an owning node to coordinate the update; this should tend to minimal overhead as smart clients become the norm, and smart clients scale up to cope with huge clusters), then each owner can maintain the oldest known timestamp it has coordinated an update for that was not acknowledged by every owning node it propagated it to. The minimum of all of these for a region is the lower bound from which we need to either repair, or retain tombstones. With vnode file segregation we can mark an entire vnode range as repaired up to the most recently determined healthy lower bound. There are some subtleties with this, but it means tombstones can be cleared potentially only minutes after they are generated, instead of days or weeks. It also means even repairs can be even more incremental, only operating over ranges and time periods we know to be potentially out of sync. It will most likely need RAMP transactions in place, so that atomic batch mutations are not serialized on non-owning nodes. Having owning nodes coordinate updates is to ensure robustness in case of a single node failure - in this case all ranges owned by the node are considered to have a lower bound of -Inf. Without this a single node being down would result in the entire cluster being considered out of sync. We will still need a short grace period for clients to send timestamps, and we would have to outright reject any updates that arrived with a timestamp near to that window expiring. But that window could safely be just minutes. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7489) Track lower bound necessary for a repair, live, without actually repairing
[ https://issues.apache.org/jira/browse/CASSANDRA-7489?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benedict updated CASSANDRA-7489: Description: We will need a few things in place to get this right, but it should be possible to track live what the current health of a single range is across the cluster. If we force an owning node to be the coordinator for an update (so if a non-smart client sends a mutation to a non-owning node, it just proxies it on to an owning node to coordinate the update; this should tend to minimal overhead as smart clients become the norm, and smart clients scale up to cope with huge clusters), then each owner can maintain the oldest known timestamp it has coordinated an update for that was not acknowledged by every owning node it propagated it to. The minimum of all of these for a region is the lower bound from which we need to either repair, or retain tombstones. With vnode file segregation we can mark an entire vnode range as repaired up to the most recently determined healthy lower bound. There are some subtleties with this, but it means tombstones can be cleared potentially only minutes after they are generated, instead of days or weeks. It also means even repairs can be even more incremental, only operating over ranges and time periods we know to be potentially out of sync. It will most likely need RAMP transactions in place, so that atomic batch mutations are not serialized on non-owning nodes. Having owning nodes coordinate updates is to ensure robustness in case of a single node failure - in this case all ranges owned by the node are considered to have a lower bound of -Inf. Without this a single node being down would result in the entire cluster being considered out of sync. We will still need a short grace period for clients to send timestamps, and we would have to outright reject any updates that arrived with a timestamp near to that window expiring. But that window could safely be just minutes. was: We will need a few things in place to get this right, but it should be possible to track live what the current health of a single range is across the cluster. If we force an owning node to be the coordinator for an update (so if a non-smart client sends a mutation to a non-owning node, it just proxies it on to an owning node to coordinate the update; this should tend to minimal overhead as smart clients become the norm, and smart clients scale up to cope with huge clusters), then each owner can maintain the oldest known timestamp it has coordinated an update for that was not acknowledged by every owning node it propagated it to. The minimum of all of these for a region is the lower bound from which we need to either repair, or retain tombstones. There are some subtleties with this, but it means tombstones can be cleared potentially only minutes after they are generated, instead of days or weeks. It also means even repairs can be even more incremental, only operating over ranges and time periods we know to be potentially out of sync. It will most likely need RAMP transactions in place, so that atomic batch mutations are not serialized on non-owning nodes. Having owning nodes coordinate updates is to ensure robustness in case of a single node failure - in this case all ranges owned by the node are considered to have a lower bound of -Inf. Without this a single node being down would result in the entire cluster being considered out of sync. Track lower bound necessary for a repair, live, without actually repairing -- Key: CASSANDRA-7489 URL: https://issues.apache.org/jira/browse/CASSANDRA-7489 Project: Cassandra Issue Type: Improvement Reporter: Benedict Labels: performance, repair We will need a few things in place to get this right, but it should be possible to track live what the current health of a single range is across the cluster. If we force an owning node to be the coordinator for an update (so if a non-smart client sends a mutation to a non-owning node, it just proxies it on to an owning node to coordinate the update; this should tend to minimal overhead as smart clients become the norm, and smart clients scale up to cope with huge clusters), then each owner can maintain the oldest known timestamp it has coordinated an update for that was not acknowledged by every owning node it propagated it to. The minimum of all of these for a region is the lower bound from which we need to either repair, or retain tombstones. With vnode file segregation we can mark an entire vnode range as repaired up to the most recently determined healthy lower bound. There are some subtleties with this, but it means tombstones can be cleared potentially only minutes after they are generated,
[jira] [Commented] (CASSANDRA-7125) Fail to start by default if Commit Log fails to validate any messages
[ https://issues.apache.org/jira/browse/CASSANDRA-7125?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14051143#comment-14051143 ] Benedict commented on CASSANDRA-7125: - Thanks for taking a look [~vijay2...@yahoo.com] bq. is the changes to CommitLogSegment.recycle needed? You're right, that changes made it into 2.0 in another CL related ticket, and was missed when merging into 2.1; I wanted to sneak it in here while we were changing the CL anyway. I can remove it if you'd prefer and file separately. bq. CommitLogDescriptor changes is not needed too (as the scoping was right earlier). It's necessary for testing. I'll add an \@VisibleForTesting annotation bq. CassandraDaemon changes might also be not needed since we print the warn statement and exit out with a stack trace. I wanted to give users some immediate info about how they can correct the problem, which a stack trace doesn't do. We could encapsulate this inside of CommitLogReplayer though, which might be better. Fail to start by default if Commit Log fails to validate any messages - Key: CASSANDRA-7125 URL: https://issues.apache.org/jira/browse/CASSANDRA-7125 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Benedict Assignee: Benedict Priority: Minor Labels: correctness Fix For: 2.1.1 Current behaviour can be pretty dangerous, and also has a tendency to mask bugs during development. We should change the behaviour to default to failure if anything unexpected happens, and introduce a cassandra.yaml option that permits overriding the default behaviour. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7489) Track lower bound necessary for a repair, live, without actually repairing
[ https://issues.apache.org/jira/browse/CASSANDRA-7489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14051629#comment-14051629 ] Benedict commented on CASSANDRA-7489: - I don't see how hints are a problem; they're explicitly part of the design. As soon as you start hinting, you log the earliest hint as the (exclusive) upper bound on correctness that you're aware of. If and when hints clear, we bump our (local) bound to account for the hints clearing. Periodically an owner queries all other owners to get the cross-cluster upper bound on known correctness, and can mark that as 'repaired'. If they cannot contact all owners, they don't do anything. Track lower bound necessary for a repair, live, without actually repairing -- Key: CASSANDRA-7489 URL: https://issues.apache.org/jira/browse/CASSANDRA-7489 Project: Cassandra Issue Type: Improvement Reporter: Benedict Labels: performance, repair We will need a few things in place to get this right, but it should be possible to track live what the current health of a single range is across the cluster. If we force an owning node to be the coordinator for an update (so if a non-smart client sends a mutation to a non-owning node, it just proxies it on to an owning node to coordinate the update; this should tend to minimal overhead as smart clients become the norm, and smart clients scale up to cope with huge clusters), then each owner can maintain the oldest known timestamp it has coordinated an update for that was not acknowledged by every owning node it propagated it to. The minimum of all of these for a region is the lower bound from which we need to either repair, or retain tombstones. With vnode file segregation we can mark an entire vnode range as repaired up to the most recently determined healthy lower bound. There are some subtleties with this, but it means tombstones can be cleared potentially only minutes after they are generated, instead of days or weeks. It also means even repairs can be even more incremental, only operating over ranges and time periods we know to be potentially out of sync. It will most likely need RAMP transactions in place, so that atomic batch mutations are not serialized on non-owning nodes. Having owning nodes coordinate updates is to ensure robustness in case of a single node failure - in this case all ranges owned by the node are considered to have a lower bound of -Inf. Without this a single node being down would result in the entire cluster being considered out of sync. We will still need a short grace period for clients to send timestamps, and we would have to outright reject any updates that arrived with a timestamp near to that window expiring. But that window could safely be just minutes. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7489) Track lower bound necessary for a repair, live, without actually repairing
[ https://issues.apache.org/jira/browse/CASSANDRA-7489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14051740#comment-14051740 ] Benedict commented on CASSANDRA-7489: - AFAICT this new scheme suffers none of the problems mentioned in CASSANDRA-3620. That's not to say this is definitely foolproof, but I think it is worth exploring. Track lower bound necessary for a repair, live, without actually repairing -- Key: CASSANDRA-7489 URL: https://issues.apache.org/jira/browse/CASSANDRA-7489 Project: Cassandra Issue Type: Improvement Reporter: Benedict Labels: performance, repair We will need a few things in place to get this right, but it should be possible to track live what the current health of a single range is across the cluster. If we force an owning node to be the coordinator for an update (so if a non-smart client sends a mutation to a non-owning node, it just proxies it on to an owning node to coordinate the update; this should tend to minimal overhead as smart clients become the norm, and smart clients scale up to cope with huge clusters), then each owner can maintain the oldest known timestamp it has coordinated an update for that was not acknowledged by every owning node it propagated it to. The minimum of all of these for a region is the lower bound from which we need to either repair, or retain tombstones. With vnode file segregation we can mark an entire vnode range as repaired up to the most recently determined healthy lower bound. There are some subtleties with this, but it means tombstones can be cleared potentially only minutes after they are generated, instead of days or weeks. It also means even repairs can be even more incremental, only operating over ranges and time periods we know to be potentially out of sync. It will most likely need RAMP transactions in place, so that atomic batch mutations are not serialized on non-owning nodes. Having owning nodes coordinate updates is to ensure robustness in case of a single node failure - in this case all ranges owned by the node are considered to have a lower bound of -Inf. Without this a single node being down would result in the entire cluster being considered out of sync. We will still need a short grace period for clients to send timestamps, and we would have to outright reject any updates that arrived with a timestamp near to that window expiring. But that window could safely be just minutes. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7489) Track lower bound necessary for a repair, live, without actually repairing
[ https://issues.apache.org/jira/browse/CASSANDRA-7489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14051801#comment-14051801 ] Benedict commented on CASSANDRA-7489: - I am definitely not suggesting this will be easy or without subtlety. However, to respond to these specific points: bq. forcing the coordinator to be a member of the replica set effectively kills CL.ANY (which is a stupid CL that nobody should be using, but people are, and people will, and we aren't at that stage anymore where we can just take away stuff like this, lightly) We could simply convert CL.ANY to CL.ONE, or reject completely, on any cluster with this feature enabled; if you want real CL.ANY you have to switch the feature off. (I'm comfortable with just always translating ANY into ONE though, as it's a safe swap) bq. people do use non-current-time-in-micros timestamps. Some do it w/ nano-timestamps, some with milli-timestamps, some don't use time whatsoever. You can't force people to not do this. We are already looking at overhauling timestamps to be e.g. 128-bit. At this time we could potentially force users to switch to a set of approved timestamp sources, and this may in fact be necessary anyway for RAMP. Note that the timestamps generated only have to be no _earlier_ than the window; future timestamps are still perfectly acceptable. bq. assuming a TokenAware client is extremely optimistic. Most drivers out there do not support it. Add (regular) batches to the picture, and proxying every request twice does suddenly become expensive. (with async writes and TokenAware strategy available you don't need, and shouldn't be using, multiple-partition-key batches, but, again, people are doing it, and people will be doing it) I'm not assuming it. However I am assuming that users for whom this is a pain point will be using token aware, and the extra hop on write (only) is unlikely to be a major impact, and may well be offset with potential for reduced read costs as optimisations permitting consistent reads with fewer nodes (or at least fewer reads across involved nodes) potentially become possible. Multiple partitions are tricky, however, and are the whole point of RAMP. I would suggest that for each token being written to, the coordinator can nominate an owning 'correctness' coordinator, to whom each of the owners being written to ACK their completion of the _first round_ of the RAMP transaction (asynchronously), at the same time as ACKing it to the actual coordinator; their ACK is also ACKed by this correctness coordinator. If either side receives no ACK, they mark the range inconsistent from that transaction onwards. Potentially these ACKs could be hinted and dealt with by the same mechanism, so that they can recover without repair, but that is an optimisation. This scheme has no latency impact, only a slightly increase message burden, however these can be batched, are of fixed size, and are proportional to the number of messages already being sent for a RAMP transaction. bq. following that, there is commit log replay, there is custom backup solutions using bulk-loading, and just uses of bulk-loading that make outright reject any updates that arrived with a timestamp near to that window expiring. But that window could safely be just minutes. an unrealistic restriction Batch loading is relatively easily special cased. Batch loading in general could have a switch to support inconsistent loads, since they are typically performed directly to all coordinators. In these cases it's acceptable to require the user take responsibility - _or_ simply require that all ranges for which we're bulk loading get marked 'in need of repair'. Track lower bound necessary for a repair, live, without actually repairing -- Key: CASSANDRA-7489 URL: https://issues.apache.org/jira/browse/CASSANDRA-7489 Project: Cassandra Issue Type: Improvement Reporter: Benedict Labels: performance, repair We will need a few things in place to get this right, but it should be possible to track live what the current health of a single range is across the cluster. If we force an owning node to be the coordinator for an update (so if a non-smart client sends a mutation to a non-owning node, it just proxies it on to an owning node to coordinate the update; this should tend to minimal overhead as smart clients become the norm, and smart clients scale up to cope with huge clusters), then each owner can maintain the oldest known timestamp it has coordinated an update for that was not acknowledged by every owning node it propagated it to. The minimum of all of these for a region is the lower bound from which we need to either repair, or retain tombstones. With vnode file
[jira] [Commented] (CASSANDRA-7489) Track lower bound necessary for a repair, live, without actually repairing
[ https://issues.apache.org/jira/browse/CASSANDRA-7489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14051857#comment-14051857 ] Benedict commented on CASSANDRA-7489: - bq. and it really doesn't give us all that much over hourly incremental repair Well, except that they can potentially be filtered out at write time. However cleaning up tombstones is only a small part of any benefit. Repairing once every X hours means 2*RF IO for all writes in the interval, and any read needs ~RF IO to be serviced. This optimisation permits halving the amount of IO necessary for writes (i.e. we do not repeat the work for repair) and permits avoiding (1-RF)/RF IO on read. Potentially multiplying cluster throughput by RF for reads. Track lower bound necessary for a repair, live, without actually repairing -- Key: CASSANDRA-7489 URL: https://issues.apache.org/jira/browse/CASSANDRA-7489 Project: Cassandra Issue Type: Improvement Reporter: Benedict Labels: performance, repair We will need a few things in place to get this right, but it should be possible to track live what the current health of a single range is across the cluster. If we force an owning node to be the coordinator for an update (so if a non-smart client sends a mutation to a non-owning node, it just proxies it on to an owning node to coordinate the update; this should tend to minimal overhead as smart clients become the norm, and smart clients scale up to cope with huge clusters), then each owner can maintain the oldest known timestamp it has coordinated an update for that was not acknowledged by every owning node it propagated it to. The minimum of all of these for a region is the lower bound from which we need to either repair, or retain tombstones. With vnode file segregation we can mark an entire vnode range as repaired up to the most recently determined healthy lower bound. There are some subtleties with this, but it means tombstones can be cleared potentially only minutes after they are generated, instead of days or weeks. It also means even repairs can be even more incremental, only operating over ranges and time periods we know to be potentially out of sync. It will most likely need RAMP transactions in place, so that atomic batch mutations are not serialized on non-owning nodes. Having owning nodes coordinate updates is to ensure robustness in case of a single node failure - in this case all ranges owned by the node are considered to have a lower bound of -Inf. Without this a single node being down would result in the entire cluster being considered out of sync. We will still need a short grace period for clients to send timestamps, and we would have to outright reject any updates that arrived with a timestamp near to that window expiring. But that window could safely be just minutes. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6146) CQL-native stress
[ https://issues.apache.org/jira/browse/CASSANDRA-6146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14051902#comment-14051902 ] Benedict commented on CASSANDRA-6146: - bq. The value component generator uses the seed of the last clustering component so it always gets the same value for all rows in a partition, since the seeds are cached. The seed is different for each row, though? So the seed at each bq. You can reproduce by changing the default clustering distribution to uniform(1..1024) Well, since there are 6 clustering components, a uniform(1..1024) default distribution would yield 512^6 (=(2^9)^6 = 2^54) _average_ number of rows per partition. Not surprisingly this causes an overflow in calculations. Probably worth spotting and letting people know this is an absurdly large size if it happens, and also worth using double instead of float everywhere we calculate a probability. bq. no_warmup option doesn't work Good spot. I didn't wire it up. bq. The value component generator uses the seed of the last clustering component so it always gets the same value for all rows in a partition, since the seeds are cached. Ah, you mean all _leaf_ rows (i.e. those sharing the second-lowest level clustering component) are the same? Well spotted, this is an off-by-1 bug, and I wasn't using a clustering1 for the leaf. It' shouldn't be the case that they are the same for the whole partition. bq. I'm concerned we won't be able to explain how to use this to joe user but perhaps if we come up with better terminology it and some visual examples it will make more sense. For example the clustering distribution is used to define the possible values in a single partition? if you have a population of uniform(1..1000) and clustering of fixed(1) you only see one value per partition We may need to bikeshed the nomenclature. I don't think clustering is that tough though: it is the number of instances of that component for each instance of its parent (i.e. for C components with average N clustering, there will be N^C rows). The only complex bit IMO is the updateratio and useratio; perhaps we could relabel these to 'rowspervisit' and 'rowsperbatch' and indicate in the description that they are ratios. CQL-native stress - Key: CASSANDRA-6146 URL: https://issues.apache.org/jira/browse/CASSANDRA-6146 Project: Cassandra Issue Type: New Feature Components: Tools Reporter: Jonathan Ellis Assignee: T Jake Luciani Fix For: 2.1.1 Attachments: 6146-v2.txt, 6146.txt, 6164-v3.txt The existing CQL support in stress is not worth discussing. We need to start over, and we might as well kill two birds with one stone and move to the native protocol while we're at it. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Comment Edited] (CASSANDRA-6146) CQL-native stress
[ https://issues.apache.org/jira/browse/CASSANDRA-6146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14051902#comment-14051902 ] Benedict edited comment on CASSANDRA-6146 at 7/3/14 8:58 PM: - bq. You can reproduce by changing the default clustering distribution to uniform(1..1024) Well, since there are 6 clustering components, a uniform(1..1024) default distribution would yield 512^6 (=(2^9)^6 = 2^54) _average_ number of rows per partition. Not surprisingly this causes an overflow in calculations. Probably worth spotting and letting people know this is an absurdly large size if it happens, and also worth using double instead of float everywhere we calculate a probability. bq. no_warmup option doesn't work Good spot. I didn't wire it up. bq. The value component generator uses the seed of the last clustering component so it always gets the same value for all rows in a partition, since the seeds are cached. Ah, you mean all _leaf_ rows (i.e. those sharing the second-lowest level clustering component) are the same? Well spotted, this is an off-by-1 bug, and I wasn't using a clustering1 for the leaf. It' shouldn't be the case that they are the same for the whole partition. bq. I'm concerned we won't be able to explain how to use this to joe user but perhaps if we come up with better terminology it and some visual examples it will make more sense. For example the clustering distribution is used to define the possible values in a single partition? if you have a population of uniform(1..1000) and clustering of fixed(1) you only see one value per partition We may need to bikeshed the nomenclature. I don't think clustering is that tough though: it is the number of instances of that component for each instance of its parent (i.e. for C components with average N clustering, there will be N^C rows). The only complex bit IMO is the updateratio and useratio; perhaps we could relabel these to 'rowspervisit' and 'rowsperbatch' and indicate in the description that they are ratios. was (Author: benedict): bq. The value component generator uses the seed of the last clustering component so it always gets the same value for all rows in a partition, since the seeds are cached. The seed is different for each row, though? So the seed at each bq. You can reproduce by changing the default clustering distribution to uniform(1..1024) Well, since there are 6 clustering components, a uniform(1..1024) default distribution would yield 512^6 (=(2^9)^6 = 2^54) _average_ number of rows per partition. Not surprisingly this causes an overflow in calculations. Probably worth spotting and letting people know this is an absurdly large size if it happens, and also worth using double instead of float everywhere we calculate a probability. bq. no_warmup option doesn't work Good spot. I didn't wire it up. bq. The value component generator uses the seed of the last clustering component so it always gets the same value for all rows in a partition, since the seeds are cached. Ah, you mean all _leaf_ rows (i.e. those sharing the second-lowest level clustering component) are the same? Well spotted, this is an off-by-1 bug, and I wasn't using a clustering1 for the leaf. It' shouldn't be the case that they are the same for the whole partition. bq. I'm concerned we won't be able to explain how to use this to joe user but perhaps if we come up with better terminology it and some visual examples it will make more sense. For example the clustering distribution is used to define the possible values in a single partition? if you have a population of uniform(1..1000) and clustering of fixed(1) you only see one value per partition We may need to bikeshed the nomenclature. I don't think clustering is that tough though: it is the number of instances of that component for each instance of its parent (i.e. for C components with average N clustering, there will be N^C rows). The only complex bit IMO is the updateratio and useratio; perhaps we could relabel these to 'rowspervisit' and 'rowsperbatch' and indicate in the description that they are ratios. CQL-native stress - Key: CASSANDRA-6146 URL: https://issues.apache.org/jira/browse/CASSANDRA-6146 Project: Cassandra Issue Type: New Feature Components: Tools Reporter: Jonathan Ellis Assignee: T Jake Luciani Fix For: 2.1.1 Attachments: 6146-v2.txt, 6146.txt, 6164-v3.txt The existing CQL support in stress is not worth discussing. We need to start over, and we might as well kill two birds with one stone and move to the native protocol while we're at it. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Comment Edited] (CASSANDRA-6146) CQL-native stress
[ https://issues.apache.org/jira/browse/CASSANDRA-6146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14051902#comment-14051902 ] Benedict edited comment on CASSANDRA-6146 at 7/3/14 9:09 PM: - bq. You can reproduce by changing the default clustering distribution to uniform(1..1024) Well, since there are 6 clustering components, a uniform(1..1024) default distribution would yield 512^6 (=(2^9)^6 = 2^54) _average_ number of rows per partition. Not surprisingly this causes an overflow in calculations. Probably worth spotting and letting people know this is an absurdly large size if it happens, and also worth using double instead of float everywhere we calculate a probability. bq. no_warmup option doesn't work Good spot. I didn't wire it up. bq. The value component generator uses the seed of the last clustering component so it always gets the same value for all rows in a partition, since the seeds are cached. -Ah, you mean all _leaf_ rows (i.e. those sharing the second-lowest level clustering component) are the same? Well spotted, this is an off-by-1 bug, and I wasn't using a clustering1 for the leaf. It' shouldn't be the case that they are the same for the whole partition.- Ah, nuts, the off-by-1 would cause it to always generate the same seeds. Whoops bq. I'm concerned we won't be able to explain how to use this to joe user but perhaps if we come up with better terminology it and some visual examples it will make more sense. For example the clustering distribution is used to define the possible values in a single partition? if you have a population of uniform(1..1000) and clustering of fixed(1) you only see one value per partition We may need to bikeshed the nomenclature. I don't think clustering is that tough though: it is the number of instances of that component for each instance of its parent (i.e. for C components with average N clustering, there will be N^C rows). The only complex bit IMO is the updateratio and useratio; perhaps we could relabel these to 'rowspervisit' and 'rowsperbatch' and indicate in the description that they are ratios. was (Author: benedict): bq. You can reproduce by changing the default clustering distribution to uniform(1..1024) Well, since there are 6 clustering components, a uniform(1..1024) default distribution would yield 512^6 (=(2^9)^6 = 2^54) _average_ number of rows per partition. Not surprisingly this causes an overflow in calculations. Probably worth spotting and letting people know this is an absurdly large size if it happens, and also worth using double instead of float everywhere we calculate a probability. bq. no_warmup option doesn't work Good spot. I didn't wire it up. bq. The value component generator uses the seed of the last clustering component so it always gets the same value for all rows in a partition, since the seeds are cached. Ah, you mean all _leaf_ rows (i.e. those sharing the second-lowest level clustering component) are the same? Well spotted, this is an off-by-1 bug, and I wasn't using a clustering1 for the leaf. It' shouldn't be the case that they are the same for the whole partition. bq. I'm concerned we won't be able to explain how to use this to joe user but perhaps if we come up with better terminology it and some visual examples it will make more sense. For example the clustering distribution is used to define the possible values in a single partition? if you have a population of uniform(1..1000) and clustering of fixed(1) you only see one value per partition We may need to bikeshed the nomenclature. I don't think clustering is that tough though: it is the number of instances of that component for each instance of its parent (i.e. for C components with average N clustering, there will be N^C rows). The only complex bit IMO is the updateratio and useratio; perhaps we could relabel these to 'rowspervisit' and 'rowsperbatch' and indicate in the description that they are ratios. CQL-native stress - Key: CASSANDRA-6146 URL: https://issues.apache.org/jira/browse/CASSANDRA-6146 Project: Cassandra Issue Type: New Feature Components: Tools Reporter: Jonathan Ellis Assignee: T Jake Luciani Fix For: 2.1.1 Attachments: 6146-v2.txt, 6146.txt, 6164-v3.txt The existing CQL support in stress is not worth discussing. We need to start over, and we might as well kill two birds with one stone and move to the native protocol while we're at it. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6146) CQL-native stress
[ https://issues.apache.org/jira/browse/CASSANDRA-6146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14051992#comment-14051992 ] Benedict commented on CASSANDRA-6146: - Ok, I've pushed a version that should: # Wire up no-warmup # Handle more gracefully absurdly large partitions # Handle more gracefully absurdly large batches (or many small batches over absurdly large partitions) # Warn in the case of either of the above, as it's likely something bad will happen anyway # Fixes the seed generation # Also fixes continueChance in the batch iterator; it should be 1 - ratio^visitCount; not simply ratio (i.e. it should be the chance necessary to on average yield a sequence of ratio*expectedCount unbroken chain of successes) CQL-native stress - Key: CASSANDRA-6146 URL: https://issues.apache.org/jira/browse/CASSANDRA-6146 Project: Cassandra Issue Type: New Feature Components: Tools Reporter: Jonathan Ellis Assignee: T Jake Luciani Fix For: 2.1.1 Attachments: 6146-v2.txt, 6146.txt, 6164-v3.txt The existing CQL support in stress is not worth discussing. We need to start over, and we might as well kill two birds with one stone and move to the native protocol while we're at it. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7403) Reconciliation doesn't consider fields specific to expiring cells
[ https://issues.apache.org/jira/browse/CASSANDRA-7403?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14052279#comment-14052279 ] Benedict commented on CASSANDRA-7403: - Patch available [here|https://github.com/belliottsmith/cassandra/tree/7403-ttlreconcile] Reconciliation doesn't consider fields specific to expiring cells -- Key: CASSANDRA-7403 URL: https://issues.apache.org/jira/browse/CASSANDRA-7403 Project: Cassandra Issue Type: Bug Components: Core Reporter: Sam Tunnicliffe Assignee: Benedict Reconciling 2 ExpiringColumns which are equal in every way except for the localExpirationTime field will always favour the instance on which reconcile is called as fields specific to expiration are not considered. This is actually beneficial in pre-2.1 versions as in AtomicSortedColumns.Holder.addColumn we call reconcile on the new column, which 'wins' the reconcilliation and so the localExpirationTime is effectively extended. From 2.1 onwards, reconcile is actually called on the existing value (in BTreeSortedColumns.ColumnUpdater) and so it wins the reconcilliation and the ttl doesn't get extended. The same thing happens in the iterator returned from MergeIterator.Reducer.getReducer() so we see the same behaviour when merging cells from the multiple SSTables and/or memtables. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7403) Reconciliation doesn't consider fields specific to expiring cells
[ https://issues.apache.org/jira/browse/CASSANDRA-7403?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Benedict updated CASSANDRA-7403: Reproduced In: 2.1 rc1, 2.0.8, 1.2.16 (was: 1.2.16, 2.0.8, 2.1 rc1) Fix Version/s: 2.1.0 Reconciliation doesn't consider fields specific to expiring cells -- Key: CASSANDRA-7403 URL: https://issues.apache.org/jira/browse/CASSANDRA-7403 Project: Cassandra Issue Type: Bug Components: Core Reporter: Sam Tunnicliffe Assignee: Benedict Fix For: 2.1.0 Reconciling 2 ExpiringColumns which are equal in every way except for the localExpirationTime field will always favour the instance on which reconcile is called as fields specific to expiration are not considered. This is actually beneficial in pre-2.1 versions as in AtomicSortedColumns.Holder.addColumn we call reconcile on the new column, which 'wins' the reconcilliation and so the localExpirationTime is effectively extended. From 2.1 onwards, reconcile is actually called on the existing value (in BTreeSortedColumns.ColumnUpdater) and so it wins the reconcilliation and the ttl doesn't get extended. The same thing happens in the iterator returned from MergeIterator.Reducer.getReducer() so we see the same behaviour when merging cells from the multiple SSTables and/or memtables. -- This message was sent by Atlassian JIRA (v6.2#6252)