[jira] [Commented] (CASSANDRA-7486) Compare CMS and G1 pause times

2015-04-28 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-7486?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14518707#comment-14518707
 ] 

Rick Branson commented on CASSANDRA-7486:
-

I think it definitely makes sense as a default. My guess is that it'll result 
in fewer headaches for most people.

 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: Shawn Kumar
 Fix For: 2.1.5


 See 
 http://www.slideshare.net/MonicaBeckwith/garbage-first-garbage-collector-g1-7486gc-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.3.4#6332)


[jira] [Created] (CASSANDRA-9258) Range movement causes CPU performance impact

2015-04-28 Thread Rick Branson (JIRA)
Rick Branson created CASSANDRA-9258:
---

 Summary: Range movement causes CPU  performance impact
 Key: CASSANDRA-9258
 URL: https://issues.apache.org/jira/browse/CASSANDRA-9258
 Project: Cassandra
  Issue Type: Bug
 Environment: Cassandra 2.1.4
Reporter: Rick Branson


Observing big CPU  latency regressions when doing range movements on clusters 
with many tens of thousands of vnodes. See CPU usage increase by ~80% when a 
single node is being replaced.

Top methods are:

1) Ljava/math/BigInteger;.compareTo in 
Lorg/apache/cassandra/dht/ComparableObjectToken;.compareTo 
2) Lcom/google/common/collect/AbstractMapBasedMultimap;.wrapCollection in 
Lcom/google/common/collect/AbstractMapBasedMultimap$AsMap$AsMapIterator;.next
3) Lorg/apache/cassandra/db/DecoratedKey;.compareTo in 
Lorg/apache/cassandra/dht/Range;.contains

Here's a sample stack from a thread dump:

Thrift:50673 daemon prio=10 tid=0x7f2f20164800 nid=0x3a04af runnable 
[0x7f2d878d]
   java.lang.Thread.State: RUNNABLE
  at org.apache.cassandra.dht.Range.isWrapAround(Range.java:260)
  at org.apache.cassandra.dht.Range.contains(Range.java:51)
  at org.apache.cassandra.dht.Range.contains(Range.java:110)
  at 
org.apache.cassandra.locator.TokenMetadata.pendingEndpointsFor(TokenMetadata.java:916)
  at 
org.apache.cassandra.service.StorageProxy.performWrite(StorageProxy.java:775)
  at org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:541)
  at 
org.apache.cassandra.service.StorageProxy.mutateWithTriggers(StorageProxy.java:616)
  at 
org.apache.cassandra.thrift.CassandraServer.doInsert(CassandraServer.java:1101)
  at 
org.apache.cassandra.thrift.CassandraServer.doInsert(CassandraServer.java:1083)
  at 
org.apache.cassandra.thrift.CassandraServer.batch_mutate(CassandraServer.java:976)
  at 
org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.getResult(Cassandra.java:3996)
  at 
org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.getResult(Cassandra.java:3980)
  at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39)
  at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39)
  at 
org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:205)
  at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
  at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
  at java.lang.Thread.run(Thread.java:745)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-9258) Range movement causes CPU performance impact

2015-04-28 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-9258?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14518182#comment-14518182
 ] 

Rick Branson commented on CASSANDRA-9258:
-

This is also an issue for large 2.0 clusters as well. The TokenMetadata path is 
identical between 2.1 and 2.0.

 Range movement causes CPU  performance impact
 --

 Key: CASSANDRA-9258
 URL: https://issues.apache.org/jira/browse/CASSANDRA-9258
 Project: Cassandra
  Issue Type: Bug
 Environment: Cassandra 2.1.4
Reporter: Rick Branson
 Fix For: 2.1.6


 Observing big CPU  latency regressions when doing range movements on 
 clusters with many tens of thousands of vnodes. See CPU usage increase by 
 ~80% when a single node is being replaced.
 Top methods are:
 1) Ljava/math/BigInteger;.compareTo in 
 Lorg/apache/cassandra/dht/ComparableObjectToken;.compareTo 
 2) Lcom/google/common/collect/AbstractMapBasedMultimap;.wrapCollection in 
 Lcom/google/common/collect/AbstractMapBasedMultimap$AsMap$AsMapIterator;.next
 3) Lorg/apache/cassandra/db/DecoratedKey;.compareTo in 
 Lorg/apache/cassandra/dht/Range;.contains
 Here's a sample stack from a thread dump:
 {code}
 Thrift:50673 daemon prio=10 tid=0x7f2f20164800 nid=0x3a04af runnable 
 [0x7f2d878d]
java.lang.Thread.State: RUNNABLE
   at org.apache.cassandra.dht.Range.isWrapAround(Range.java:260)
   at org.apache.cassandra.dht.Range.contains(Range.java:51)
   at org.apache.cassandra.dht.Range.contains(Range.java:110)
   at 
 org.apache.cassandra.locator.TokenMetadata.pendingEndpointsFor(TokenMetadata.java:916)
   at 
 org.apache.cassandra.service.StorageProxy.performWrite(StorageProxy.java:775)
   at 
 org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:541)
   at 
 org.apache.cassandra.service.StorageProxy.mutateWithTriggers(StorageProxy.java:616)
   at 
 org.apache.cassandra.thrift.CassandraServer.doInsert(CassandraServer.java:1101)
   at 
 org.apache.cassandra.thrift.CassandraServer.doInsert(CassandraServer.java:1083)
   at 
 org.apache.cassandra.thrift.CassandraServer.batch_mutate(CassandraServer.java:976)
   at 
 org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.getResult(Cassandra.java:3996)
   at 
 org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.getResult(Cassandra.java:3980)
   at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39)
   at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39)
   at 
 org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:205)
   at 
 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
   at java.lang.Thread.run(Thread.java:745){code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-9244) replace_address is not topology-aware

2015-04-25 Thread Rick Branson (JIRA)
Rick Branson created CASSANDRA-9244:
---

 Summary: replace_address is not topology-aware
 Key: CASSANDRA-9244
 URL: https://issues.apache.org/jira/browse/CASSANDRA-9244
 Project: Cassandra
  Issue Type: Bug
 Environment: 2.0.12
Reporter: Rick Branson


Replaced a node with one in another rack (using replace_address) and it caused 
improper distribution after the bootstrap was finished. It looks like the 
ranges for the streams are not created in a way that is topology-aware. This 
should probably either be prevented, or ideally, would work properly. The use 
case is migrating several nodes from one rack to another.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-9244) replace_address is not topology-aware

2015-04-25 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-9244?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14512756#comment-14512756
 ] 

Rick Branson commented on CASSANDRA-9244:
-

GossipingPropertyFileSnitch. The scenario is say you have NTS with RF=3, 10 
racks A-J, and range R1, which is spread across nodes A1, C1, and E1 in racks 
A, C, and E respectively. If you replace_address C1 with node E2, it will cause 
range R1 to shift to A1, E2, and some node in neither A nor E racks. A similar 
range change is achieved if you were to bootstrap with tokens-1, but the shifts 
will work properly in that case.

 replace_address is not topology-aware
 -

 Key: CASSANDRA-9244
 URL: https://issues.apache.org/jira/browse/CASSANDRA-9244
 Project: Cassandra
  Issue Type: Bug
 Environment: 2.0.12
Reporter: Rick Branson

 Replaced a node with one in another rack (using replace_address) and it 
 caused improper distribution after the bootstrap was finished. It looks like 
 the ranges for the streams are not created in a way that is topology-aware. 
 This should probably either be prevented, or ideally, would work properly. 
 The use case is migrating several nodes from one rack to another.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Resolved] (CASSANDRA-9227) Support num_tokens: 0

2015-04-23 Thread Rick Branson (JIRA)

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

Rick Branson resolved CASSANDRA-9227.
-
Resolution: Not A Problem

We tested join_ring=false after 2.0.7 came out and didn't have any luck. It 
would join the ring but end up looping. I tested it again and it works 
perfectly now. Thanks and sorry for confusion :) Will probably post a short 
GIST when we try this in prod.

 Support num_tokens: 0
 -

 Key: CASSANDRA-9227
 URL: https://issues.apache.org/jira/browse/CASSANDRA-9227
 Project: Cassandra
  Issue Type: Improvement
Reporter: Rick Branson

 num_tokens: 0 would allow serving proxy traffic on non-storage machines. 
 Token-aware routing does improve the cost of proxying a bit, but it only 
 works in cases where CL=1 for reads, and doesn't really improve the write 
 situation. For some installations, token-aware routing is also impractical, 
 such as when there are a very large number of clients.
 For use cases that are throughput-bound rather than storage-bound, this would 
 allow running CPU-intensive proxy work on cheaper diskless machines. 
 Proxies could also be installed on the same machine as the client or on 
 separate edge machines that are closer to the client than the storage 
 nodes. In my observation, anywhere between 25-50% of a node's CPU is spent on 
 proxy work, depending on the workload.
 I say diskless because they can't be truly diskless as they would have to 
 store hints. Hint storage can use cheap spinning disks or small solid-state 
 drives instead of expensive big disk arrays or big SSDs.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-9227) Support num_tokens: 0

2015-04-22 Thread Rick Branson (JIRA)

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

Rick Branson updated CASSANDRA-9227:

Description: 
num_tokens: 0 would allow serving proxy traffic on non-storage machines. 
Token-aware routing does improve the cost of proxying a bit, but it only works 
in cases where CL=1 for reads, and doesn't really improve the write situation. 
For some installations, token-aware routing is also impractical, such as when 
there are a very large number of clients.

For use cases that are throughput-bound rather than storage-bound, this would 
allow running CPU-intensive proxy work on cheaper diskless machines. Proxies 
could also be installed on the same machine as the client or on separate edge 
machines that are closer to the client than the storage nodes. In my 
observation, anywhere between 25-50% of a node's CPU is spent on proxy work, 
depending on the workload.

I say diskless because they can't be truly diskless as they would have to 
store hints. Hint storage can use cheap spinning disks or small solid-state 
drives instead of expensive big disk arrays or big SSDs.

  was:
num_tokens: 0 would allow serving proxy traffic on non-storage machines. 
Token-aware routing does improve the cost of proxying a bit, but it only works 
in cases where CL1 for reads, and doesn't really improve the write situation. 
For some installations, token-aware routing is also impractical, such as when 
there are a very large number of clients.

For use cases that are throughput-bound rather than storage-bound, this would 
allow running CPU-intensive proxy work on cheaper diskless machines. Proxies 
could also be installed on the same machine as the client or on separate edge 
machines that are closer to the client than the storage nodes. In my 
observation, anywhere between 25-50% of a node's CPU is spent on proxy work, 
depending on the workload.

I say diskless because they can't be truly diskless as they would have to 
store hints. Hint storage can use cheap spinning disks or small solid-state 
drives instead of expensive big disk arrays or big SSDs.


 Support num_tokens: 0
 -

 Key: CASSANDRA-9227
 URL: https://issues.apache.org/jira/browse/CASSANDRA-9227
 Project: Cassandra
  Issue Type: Improvement
Reporter: Rick Branson

 num_tokens: 0 would allow serving proxy traffic on non-storage machines. 
 Token-aware routing does improve the cost of proxying a bit, but it only 
 works in cases where CL=1 for reads, and doesn't really improve the write 
 situation. For some installations, token-aware routing is also impractical, 
 such as when there are a very large number of clients.
 For use cases that are throughput-bound rather than storage-bound, this would 
 allow running CPU-intensive proxy work on cheaper diskless machines. 
 Proxies could also be installed on the same machine as the client or on 
 separate edge machines that are closer to the client than the storage 
 nodes. In my observation, anywhere between 25-50% of a node's CPU is spent on 
 proxy work, depending on the workload.
 I say diskless because they can't be truly diskless as they would have to 
 store hints. Hint storage can use cheap spinning disks or small solid-state 
 drives instead of expensive big disk arrays or big SSDs.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-9227) Support num_tokens: 0

2015-04-22 Thread Rick Branson (JIRA)
Rick Branson created CASSANDRA-9227:
---

 Summary: Support num_tokens: 0
 Key: CASSANDRA-9227
 URL: https://issues.apache.org/jira/browse/CASSANDRA-9227
 Project: Cassandra
  Issue Type: Improvement
Reporter: Rick Branson


num_tokens: 0 would allow serving proxy traffic on non-storage machines. 
Token-aware routing does improve the cost of proxying a bit, but it only works 
in cases where CL1 for reads, and doesn't really improve the write situation. 
For some installations, token-aware routing is also impractical, such as when 
there are a very large number of clients.

For use cases that are throughput-bound rather than storage-bound, this would 
allow running CPU-intensive proxy work on cheaper diskless machines. Proxies 
could also be installed on the same machine as the client or on separate edge 
machines that are closer to the client than the storage nodes. In my 
observation, anywhere between 25-50% of a node's CPU is spent on proxy work, 
depending on the workload.

I say diskless because they can't be truly diskless as they would have to 
store hints. Hint storage can use cheap spinning disks or small solid-state 
drives instead of expensive big disk arrays or big SSDs.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-8835) 100% CPU spikes from disruptor thrift server

2015-02-19 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-8835?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14327993#comment-14327993
 ] 

Rick Branson commented on CASSANDRA-8835:
-

also seeing this pop to the top simultaneously on some nodes (above 
selectorIterationComplete):
Ljava/util/HashMap$HashIterator;.remove in 
Lcom/thinkaurelius/thrift/TDisruptorServer$AbstractSelectorThread;.select   
 

 100% CPU spikes from disruptor thrift server
 

 Key: CASSANDRA-8835
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8835
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: 2.0.11, hsha, JDK7u65, ~6,000 connections per machine
Reporter: Rick Branson

 Seeing big CPU spikes (to 100%) inside of the TDisruptorServer when running 
 hsha. perf top loaded with JVM symbols shows this one floating to the top for 
 anywhere between 15 and 90 seconds and then falling down:
 Lcom/thinkaurelius/thrift/TDisruptorServer$SelectorThread;.selectorIterationComplete
  in Lcom/thinkaurelius/thrift/TDisruptorServer$AbstractSelectorThread;.select



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-8835) 100% CPU spikes from disruptor thrift server

2015-02-19 Thread Rick Branson (JIRA)
Rick Branson created CASSANDRA-8835:
---

 Summary: 100% CPU spikes from disruptor thrift server
 Key: CASSANDRA-8835
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8835
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: 2.0.11, hsha, JDK7u65, ~6,000 connections per machine
Reporter: Rick Branson


Seeing big CPU spikes (to 100%) inside of the TDisruptorServer when running 
hsha. perf top loaded with JVM symbols shows this one floating to the top for 
anywhere between 15 and 90 seconds and then falling down:

Lcom/thinkaurelius/thrift/TDisruptorServer$SelectorThread;.selectorIterationComplete
 in Lcom/thinkaurelius/thrift/TDisruptorServer$AbstractSelectorThread;.select



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-8463) Constant compaction under LCS

2014-12-17 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-8463?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14251110#comment-14251110
 ] 

Rick Branson commented on CASSANDRA-8463:
-

Code LGTM.

 Constant compaction under LCS
 -

 Key: CASSANDRA-8463
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8463
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: Hardware is recent 2-socket, 16-core (x2 Hyperthreaded), 
 144G RAM, solid-state storage.
 Platform is Linux 3.2.51, Oracle JDK 64-bit 1.7.0_65.
 Heap is 32G total, 4G newsize.
 8G/8G on-heap/off-heap memtables, offheap_buffer allocator, 0.5 
 memtable_cleanup_threshold
 concurrent_compactors: 20
Reporter: Rick Branson
Assignee: Marcus Eriksson
 Fix For: 2.1.3

 Attachments: 0001-better-logging.patch, 
 0001-make-sure-we-set-lastCompactedKey-properly.patch, log-for-8463.txt


 It appears that tables configured with LCS will completely re-compact 
 themselves over some period of time after upgrading from 2.0 to 2.1 (2.0.11 
 - 2.1.2, specifically). It starts out with 10 pending tasks for an hour or 
 so, then starts building up, now with 50-100 tasks pending across the cluster 
 after 12 hours. These nodes are under heavy write load, but were easily able 
 to keep up in 2.0 (they rarely had 5 pending compaction tasks), so I don't 
 think it's LCS in 2.1 actually being worse, just perhaps some different LCS 
 behavior that causes the layout of tables from 2.0 to prompt the compactor to 
 reorganize them?
 The nodes flushed ~11MB SSTables under 2.0. They're currently flushing ~36MB 
 SSTables due to the improved memtable setup in 2.1. Before I upgraded the 
 entire cluster to 2.1, I noticed the problem and tried several variations on 
 the flush size, thinking perhaps the larger tables in L0 were causing some 
 kind of cascading compactions. Even if they're sized roughly like the 2.0 
 flushes were, same behavior occurs. I also tried both enabling  disabling 
 STCS in L0 with no real change other than L0 began to back up faster, so I 
 left the STCS in L0 enabled.
 Tables are configured with 32MB sstable_size_in_mb, which was found to be an 
 improvement on the 160MB table size for compaction performance. Maybe this is 
 wrong now? Otherwise, the tables are configured with defaults. Compaction has 
 been unthrottled to help them catch-up. The compaction threads stay very 
 busy, with the cluster-wide CPU at 45% nice time. No nodes have completely 
 caught up yet. I'll update JIRA with status about their progress if anything 
 interesting happens.
 From a node around 12 hours ago, around an hour after the upgrade, with 19 
 pending compaction tasks:
 SSTables in each level: [6/4, 10, 105/100, 268, 0, 0, 0, 0, 0]
 SSTables in each level: [6/4, 10, 106/100, 271, 0, 0, 0, 0, 0]
 SSTables in each level: [1, 16/10, 105/100, 269, 0, 0, 0, 0, 0]
 SSTables in each level: [5/4, 10, 103/100, 272, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 11/10, 105/100, 270, 0, 0, 0, 0, 0]
 SSTables in each level: [1, 12/10, 105/100, 271, 0, 0, 0, 0, 0]
 SSTables in each level: [1, 14/10, 104/100, 267, 0, 0, 0, 0, 0]
 SSTables in each level: [9/4, 10, 103/100, 265, 0, 0, 0, 0, 0]
 Recently, with 41 pending compaction tasks:
 SSTables in each level: [4, 13/10, 106/100, 269, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 12/10, 106/100, 273, 0, 0, 0, 0, 0]
 SSTables in each level: [5/4, 11/10, 106/100, 271, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 12/10, 103/100, 275, 0, 0, 0, 0, 0]
 SSTables in each level: [2, 13/10, 106/100, 273, 0, 0, 0, 0, 0]
 SSTables in each level: [3, 10, 104/100, 275, 0, 0, 0, 0, 0]
 SSTables in each level: [6/4, 11/10, 103/100, 269, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 16/10, 105/100, 264, 0, 0, 0, 0, 0]
 More information about the use case: writes are roughly uniform across these 
 tables. The data is sharded across these 8 tables by key to improve 
 compaction parallelism. Each node receives up to 75,000 writes/sec sustained 
 at peak, and a small number of reads. This is a pre-production cluster that's 
 being warmed up with new data, so the low volume of reads (~100/sec per node) 
 is just from automatic sampled data checks, otherwise we'd just use STCS :)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-8463) Constant compaction under LCS

2014-12-16 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-8463?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14248518#comment-14248518
 ] 

Rick Branson commented on CASSANDRA-8463:
-

I applied that patch to one node. What should I be looking for? Is there any 
persisted state for that data? Shortly after restarting the daemon it did 
another one of those single table L1 compactions into L2. After finishing that, 
it immediately compacted that single new L2 file into 452 L3 tables. That 
pattern is what's causing the problem.

 Constant compaction under LCS
 -

 Key: CASSANDRA-8463
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8463
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: Hardware is recent 2-socket, 16-core (x2 Hyperthreaded), 
 144G RAM, solid-state storage.
 Platform is Linux 3.2.51, Oracle JDK 64-bit 1.7.0_65.
 Heap is 32G total, 4G newsize.
 8G/8G on-heap/off-heap memtables, offheap_buffer allocator, 0.5 
 memtable_cleanup_threshold
 concurrent_compactors: 20
Reporter: Rick Branson
Assignee: Marcus Eriksson
 Fix For: 2.1.3

 Attachments: 0001-better-logging.patch, 
 0001-make-sure-we-set-lastCompactedKey-properly.patch, log-for-8463.txt


 It appears that tables configured with LCS will completely re-compact 
 themselves over some period of time after upgrading from 2.0 to 2.1 (2.0.11 
 - 2.1.2, specifically). It starts out with 10 pending tasks for an hour or 
 so, then starts building up, now with 50-100 tasks pending across the cluster 
 after 12 hours. These nodes are under heavy write load, but were easily able 
 to keep up in 2.0 (they rarely had 5 pending compaction tasks), so I don't 
 think it's LCS in 2.1 actually being worse, just perhaps some different LCS 
 behavior that causes the layout of tables from 2.0 to prompt the compactor to 
 reorganize them?
 The nodes flushed ~11MB SSTables under 2.0. They're currently flushing ~36MB 
 SSTables due to the improved memtable setup in 2.1. Before I upgraded the 
 entire cluster to 2.1, I noticed the problem and tried several variations on 
 the flush size, thinking perhaps the larger tables in L0 were causing some 
 kind of cascading compactions. Even if they're sized roughly like the 2.0 
 flushes were, same behavior occurs. I also tried both enabling  disabling 
 STCS in L0 with no real change other than L0 began to back up faster, so I 
 left the STCS in L0 enabled.
 Tables are configured with 32MB sstable_size_in_mb, which was found to be an 
 improvement on the 160MB table size for compaction performance. Maybe this is 
 wrong now? Otherwise, the tables are configured with defaults. Compaction has 
 been unthrottled to help them catch-up. The compaction threads stay very 
 busy, with the cluster-wide CPU at 45% nice time. No nodes have completely 
 caught up yet. I'll update JIRA with status about their progress if anything 
 interesting happens.
 From a node around 12 hours ago, around an hour after the upgrade, with 19 
 pending compaction tasks:
 SSTables in each level: [6/4, 10, 105/100, 268, 0, 0, 0, 0, 0]
 SSTables in each level: [6/4, 10, 106/100, 271, 0, 0, 0, 0, 0]
 SSTables in each level: [1, 16/10, 105/100, 269, 0, 0, 0, 0, 0]
 SSTables in each level: [5/4, 10, 103/100, 272, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 11/10, 105/100, 270, 0, 0, 0, 0, 0]
 SSTables in each level: [1, 12/10, 105/100, 271, 0, 0, 0, 0, 0]
 SSTables in each level: [1, 14/10, 104/100, 267, 0, 0, 0, 0, 0]
 SSTables in each level: [9/4, 10, 103/100, 265, 0, 0, 0, 0, 0]
 Recently, with 41 pending compaction tasks:
 SSTables in each level: [4, 13/10, 106/100, 269, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 12/10, 106/100, 273, 0, 0, 0, 0, 0]
 SSTables in each level: [5/4, 11/10, 106/100, 271, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 12/10, 103/100, 275, 0, 0, 0, 0, 0]
 SSTables in each level: [2, 13/10, 106/100, 273, 0, 0, 0, 0, 0]
 SSTables in each level: [3, 10, 104/100, 275, 0, 0, 0, 0, 0]
 SSTables in each level: [6/4, 11/10, 103/100, 269, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 16/10, 105/100, 264, 0, 0, 0, 0, 0]
 More information about the use case: writes are roughly uniform across these 
 tables. The data is sharded across these 8 tables by key to improve 
 compaction parallelism. Each node receives up to 75,000 writes/sec sustained 
 at peak, and a small number of reads. This is a pre-production cluster that's 
 being warmed up with new data, so the low volume of reads (~100/sec per node) 
 is just from automatic sampled data checks, otherwise we'd just use STCS :)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-8463) Constant compaction under LCS

2014-12-16 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-8463?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14249047#comment-14249047
 ] 

Rick Branson commented on CASSANDRA-8463:
-

It did take some time to rebalance the tables within the levels, but nodes are 
able to clear their compaction log with the patch applied. +1

 Constant compaction under LCS
 -

 Key: CASSANDRA-8463
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8463
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: Hardware is recent 2-socket, 16-core (x2 Hyperthreaded), 
 144G RAM, solid-state storage.
 Platform is Linux 3.2.51, Oracle JDK 64-bit 1.7.0_65.
 Heap is 32G total, 4G newsize.
 8G/8G on-heap/off-heap memtables, offheap_buffer allocator, 0.5 
 memtable_cleanup_threshold
 concurrent_compactors: 20
Reporter: Rick Branson
Assignee: Marcus Eriksson
 Fix For: 2.1.3

 Attachments: 0001-better-logging.patch, 
 0001-make-sure-we-set-lastCompactedKey-properly.patch, log-for-8463.txt


 It appears that tables configured with LCS will completely re-compact 
 themselves over some period of time after upgrading from 2.0 to 2.1 (2.0.11 
 - 2.1.2, specifically). It starts out with 10 pending tasks for an hour or 
 so, then starts building up, now with 50-100 tasks pending across the cluster 
 after 12 hours. These nodes are under heavy write load, but were easily able 
 to keep up in 2.0 (they rarely had 5 pending compaction tasks), so I don't 
 think it's LCS in 2.1 actually being worse, just perhaps some different LCS 
 behavior that causes the layout of tables from 2.0 to prompt the compactor to 
 reorganize them?
 The nodes flushed ~11MB SSTables under 2.0. They're currently flushing ~36MB 
 SSTables due to the improved memtable setup in 2.1. Before I upgraded the 
 entire cluster to 2.1, I noticed the problem and tried several variations on 
 the flush size, thinking perhaps the larger tables in L0 were causing some 
 kind of cascading compactions. Even if they're sized roughly like the 2.0 
 flushes were, same behavior occurs. I also tried both enabling  disabling 
 STCS in L0 with no real change other than L0 began to back up faster, so I 
 left the STCS in L0 enabled.
 Tables are configured with 32MB sstable_size_in_mb, which was found to be an 
 improvement on the 160MB table size for compaction performance. Maybe this is 
 wrong now? Otherwise, the tables are configured with defaults. Compaction has 
 been unthrottled to help them catch-up. The compaction threads stay very 
 busy, with the cluster-wide CPU at 45% nice time. No nodes have completely 
 caught up yet. I'll update JIRA with status about their progress if anything 
 interesting happens.
 From a node around 12 hours ago, around an hour after the upgrade, with 19 
 pending compaction tasks:
 SSTables in each level: [6/4, 10, 105/100, 268, 0, 0, 0, 0, 0]
 SSTables in each level: [6/4, 10, 106/100, 271, 0, 0, 0, 0, 0]
 SSTables in each level: [1, 16/10, 105/100, 269, 0, 0, 0, 0, 0]
 SSTables in each level: [5/4, 10, 103/100, 272, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 11/10, 105/100, 270, 0, 0, 0, 0, 0]
 SSTables in each level: [1, 12/10, 105/100, 271, 0, 0, 0, 0, 0]
 SSTables in each level: [1, 14/10, 104/100, 267, 0, 0, 0, 0, 0]
 SSTables in each level: [9/4, 10, 103/100, 265, 0, 0, 0, 0, 0]
 Recently, with 41 pending compaction tasks:
 SSTables in each level: [4, 13/10, 106/100, 269, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 12/10, 106/100, 273, 0, 0, 0, 0, 0]
 SSTables in each level: [5/4, 11/10, 106/100, 271, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 12/10, 103/100, 275, 0, 0, 0, 0, 0]
 SSTables in each level: [2, 13/10, 106/100, 273, 0, 0, 0, 0, 0]
 SSTables in each level: [3, 10, 104/100, 275, 0, 0, 0, 0, 0]
 SSTables in each level: [6/4, 11/10, 103/100, 269, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 16/10, 105/100, 264, 0, 0, 0, 0, 0]
 More information about the use case: writes are roughly uniform across these 
 tables. The data is sharded across these 8 tables by key to improve 
 compaction parallelism. Each node receives up to 75,000 writes/sec sustained 
 at peak, and a small number of reads. This is a pre-production cluster that's 
 being warmed up with new data, so the low volume of reads (~100/sec per node) 
 is just from automatic sampled data checks, otherwise we'd just use STCS :)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-8463) Constant compaction under LCS

2014-12-15 Thread Rick Branson (JIRA)

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

Rick Branson updated CASSANDRA-8463:

Summary: Constant compaction under LCS  (was: Upgrading 2.0 to 2.1 causes 
LCS to recompact all files)

 Constant compaction under LCS
 -

 Key: CASSANDRA-8463
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8463
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: Hardware is recent 2-socket, 16-core (x2 Hyperthreaded), 
 144G RAM, solid-state storage.
 Platform is Linux 3.2.51, Oracle JDK 64-bit 1.7.0_65.
 Heap is 32G total, 4G newsize.
 8G/8G on-heap/off-heap memtables, offheap_buffer allocator, 0.5 
 memtable_cleanup_threshold
 concurrent_compactors: 20
Reporter: Rick Branson
Assignee: Marcus Eriksson
 Fix For: 2.1.3

 Attachments: 0001-better-logging.patch, log-for-8463.txt


 It appears that tables configured with LCS will completely re-compact 
 themselves over some period of time after upgrading from 2.0 to 2.1 (2.0.11 
 - 2.1.2, specifically). It starts out with 10 pending tasks for an hour or 
 so, then starts building up, now with 50-100 tasks pending across the cluster 
 after 12 hours. These nodes are under heavy write load, but were easily able 
 to keep up in 2.0 (they rarely had 5 pending compaction tasks), so I don't 
 think it's LCS in 2.1 actually being worse, just perhaps some different LCS 
 behavior that causes the layout of tables from 2.0 to prompt the compactor to 
 reorganize them?
 The nodes flushed ~11MB SSTables under 2.0. They're currently flushing ~36MB 
 SSTables due to the improved memtable setup in 2.1. Before I upgraded the 
 entire cluster to 2.1, I noticed the problem and tried several variations on 
 the flush size, thinking perhaps the larger tables in L0 were causing some 
 kind of cascading compactions. Even if they're sized roughly like the 2.0 
 flushes were, same behavior occurs. I also tried both enabling  disabling 
 STCS in L0 with no real change other than L0 began to back up faster, so I 
 left the STCS in L0 enabled.
 Tables are configured with 32MB sstable_size_in_mb, which was found to be an 
 improvement on the 160MB table size for compaction performance. Maybe this is 
 wrong now? Otherwise, the tables are configured with defaults. Compaction has 
 been unthrottled to help them catch-up. The compaction threads stay very 
 busy, with the cluster-wide CPU at 45% nice time. No nodes have completely 
 caught up yet. I'll update JIRA with status about their progress if anything 
 interesting happens.
 From a node around 12 hours ago, around an hour after the upgrade, with 19 
 pending compaction tasks:
 SSTables in each level: [6/4, 10, 105/100, 268, 0, 0, 0, 0, 0]
 SSTables in each level: [6/4, 10, 106/100, 271, 0, 0, 0, 0, 0]
 SSTables in each level: [1, 16/10, 105/100, 269, 0, 0, 0, 0, 0]
 SSTables in each level: [5/4, 10, 103/100, 272, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 11/10, 105/100, 270, 0, 0, 0, 0, 0]
 SSTables in each level: [1, 12/10, 105/100, 271, 0, 0, 0, 0, 0]
 SSTables in each level: [1, 14/10, 104/100, 267, 0, 0, 0, 0, 0]
 SSTables in each level: [9/4, 10, 103/100, 265, 0, 0, 0, 0, 0]
 Recently, with 41 pending compaction tasks:
 SSTables in each level: [4, 13/10, 106/100, 269, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 12/10, 106/100, 273, 0, 0, 0, 0, 0]
 SSTables in each level: [5/4, 11/10, 106/100, 271, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 12/10, 103/100, 275, 0, 0, 0, 0, 0]
 SSTables in each level: [2, 13/10, 106/100, 273, 0, 0, 0, 0, 0]
 SSTables in each level: [3, 10, 104/100, 275, 0, 0, 0, 0, 0]
 SSTables in each level: [6/4, 11/10, 103/100, 269, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 16/10, 105/100, 264, 0, 0, 0, 0, 0]
 More information about the use case: writes are roughly uniform across these 
 tables. The data is sharded across these 8 tables by key to improve 
 compaction parallelism. Each node receives up to 75,000 writes/sec sustained 
 at peak, and a small number of reads. This is a pre-production cluster that's 
 being warmed up with new data, so the low volume of reads (~100/sec per node) 
 is just from automatic sampled data checks, otherwise we'd just use STCS :)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-8463) Constant compaction under LCS

2014-12-15 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-8463?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14246905#comment-14246905
 ] 

Rick Branson commented on CASSANDRA-8463:
-

It looks like this is just a regression with 2.1, not just an issue upgrading 
from 2.0 to 2.1. I emptied a node's SSTables and after ~12 hours it was back 
into the backlogged compaction state. I tried pushing the memtable size up very 
high, to where it was flushing ~100MB SSTables. This allowed the backlog to 
clear while there was only L0-L2 tables, but as soon as L3 tables appeared, it 
began to constantly compact again. After examining diffs between the 2.0 and 
2.1 code, I also tried setting the tombstone compaction threshold to 1.0 to no 
avail. I'll see if I can get you some better logs with that patch, Marcus.

 Constant compaction under LCS
 -

 Key: CASSANDRA-8463
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8463
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: Hardware is recent 2-socket, 16-core (x2 Hyperthreaded), 
 144G RAM, solid-state storage.
 Platform is Linux 3.2.51, Oracle JDK 64-bit 1.7.0_65.
 Heap is 32G total, 4G newsize.
 8G/8G on-heap/off-heap memtables, offheap_buffer allocator, 0.5 
 memtable_cleanup_threshold
 concurrent_compactors: 20
Reporter: Rick Branson
Assignee: Marcus Eriksson
 Fix For: 2.1.3

 Attachments: 0001-better-logging.patch, log-for-8463.txt


 It appears that tables configured with LCS will completely re-compact 
 themselves over some period of time after upgrading from 2.0 to 2.1 (2.0.11 
 - 2.1.2, specifically). It starts out with 10 pending tasks for an hour or 
 so, then starts building up, now with 50-100 tasks pending across the cluster 
 after 12 hours. These nodes are under heavy write load, but were easily able 
 to keep up in 2.0 (they rarely had 5 pending compaction tasks), so I don't 
 think it's LCS in 2.1 actually being worse, just perhaps some different LCS 
 behavior that causes the layout of tables from 2.0 to prompt the compactor to 
 reorganize them?
 The nodes flushed ~11MB SSTables under 2.0. They're currently flushing ~36MB 
 SSTables due to the improved memtable setup in 2.1. Before I upgraded the 
 entire cluster to 2.1, I noticed the problem and tried several variations on 
 the flush size, thinking perhaps the larger tables in L0 were causing some 
 kind of cascading compactions. Even if they're sized roughly like the 2.0 
 flushes were, same behavior occurs. I also tried both enabling  disabling 
 STCS in L0 with no real change other than L0 began to back up faster, so I 
 left the STCS in L0 enabled.
 Tables are configured with 32MB sstable_size_in_mb, which was found to be an 
 improvement on the 160MB table size for compaction performance. Maybe this is 
 wrong now? Otherwise, the tables are configured with defaults. Compaction has 
 been unthrottled to help them catch-up. The compaction threads stay very 
 busy, with the cluster-wide CPU at 45% nice time. No nodes have completely 
 caught up yet. I'll update JIRA with status about their progress if anything 
 interesting happens.
 From a node around 12 hours ago, around an hour after the upgrade, with 19 
 pending compaction tasks:
 SSTables in each level: [6/4, 10, 105/100, 268, 0, 0, 0, 0, 0]
 SSTables in each level: [6/4, 10, 106/100, 271, 0, 0, 0, 0, 0]
 SSTables in each level: [1, 16/10, 105/100, 269, 0, 0, 0, 0, 0]
 SSTables in each level: [5/4, 10, 103/100, 272, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 11/10, 105/100, 270, 0, 0, 0, 0, 0]
 SSTables in each level: [1, 12/10, 105/100, 271, 0, 0, 0, 0, 0]
 SSTables in each level: [1, 14/10, 104/100, 267, 0, 0, 0, 0, 0]
 SSTables in each level: [9/4, 10, 103/100, 265, 0, 0, 0, 0, 0]
 Recently, with 41 pending compaction tasks:
 SSTables in each level: [4, 13/10, 106/100, 269, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 12/10, 106/100, 273, 0, 0, 0, 0, 0]
 SSTables in each level: [5/4, 11/10, 106/100, 271, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 12/10, 103/100, 275, 0, 0, 0, 0, 0]
 SSTables in each level: [2, 13/10, 106/100, 273, 0, 0, 0, 0, 0]
 SSTables in each level: [3, 10, 104/100, 275, 0, 0, 0, 0, 0]
 SSTables in each level: [6/4, 11/10, 103/100, 269, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 16/10, 105/100, 264, 0, 0, 0, 0, 0]
 More information about the use case: writes are roughly uniform across these 
 tables. The data is sharded across these 8 tables by key to improve 
 compaction parallelism. Each node receives up to 75,000 writes/sec sustained 
 at peak, and a small number of reads. This is a pre-production cluster that's 
 being warmed up with new data, so the low volume of reads (~100/sec per node) 
 is just from automatic sampled data checks, otherwise we'd just use STCS :)




[jira] [Commented] (CASSANDRA-8463) Constant compaction under LCS

2014-12-15 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-8463?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14247061#comment-14247061
 ] 

Rick Branson commented on CASSANDRA-8463:
-

Writes had to be turned off to this cluster to avoid some errors we began to 
see. I started to see some SSTables appear in L0 w/o any flush activity. Could 
this be a clue?

 Constant compaction under LCS
 -

 Key: CASSANDRA-8463
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8463
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: Hardware is recent 2-socket, 16-core (x2 Hyperthreaded), 
 144G RAM, solid-state storage.
 Platform is Linux 3.2.51, Oracle JDK 64-bit 1.7.0_65.
 Heap is 32G total, 4G newsize.
 8G/8G on-heap/off-heap memtables, offheap_buffer allocator, 0.5 
 memtable_cleanup_threshold
 concurrent_compactors: 20
Reporter: Rick Branson
Assignee: Marcus Eriksson
 Fix For: 2.1.3

 Attachments: 0001-better-logging.patch, log-for-8463.txt


 It appears that tables configured with LCS will completely re-compact 
 themselves over some period of time after upgrading from 2.0 to 2.1 (2.0.11 
 - 2.1.2, specifically). It starts out with 10 pending tasks for an hour or 
 so, then starts building up, now with 50-100 tasks pending across the cluster 
 after 12 hours. These nodes are under heavy write load, but were easily able 
 to keep up in 2.0 (they rarely had 5 pending compaction tasks), so I don't 
 think it's LCS in 2.1 actually being worse, just perhaps some different LCS 
 behavior that causes the layout of tables from 2.0 to prompt the compactor to 
 reorganize them?
 The nodes flushed ~11MB SSTables under 2.0. They're currently flushing ~36MB 
 SSTables due to the improved memtable setup in 2.1. Before I upgraded the 
 entire cluster to 2.1, I noticed the problem and tried several variations on 
 the flush size, thinking perhaps the larger tables in L0 were causing some 
 kind of cascading compactions. Even if they're sized roughly like the 2.0 
 flushes were, same behavior occurs. I also tried both enabling  disabling 
 STCS in L0 with no real change other than L0 began to back up faster, so I 
 left the STCS in L0 enabled.
 Tables are configured with 32MB sstable_size_in_mb, which was found to be an 
 improvement on the 160MB table size for compaction performance. Maybe this is 
 wrong now? Otherwise, the tables are configured with defaults. Compaction has 
 been unthrottled to help them catch-up. The compaction threads stay very 
 busy, with the cluster-wide CPU at 45% nice time. No nodes have completely 
 caught up yet. I'll update JIRA with status about their progress if anything 
 interesting happens.
 From a node around 12 hours ago, around an hour after the upgrade, with 19 
 pending compaction tasks:
 SSTables in each level: [6/4, 10, 105/100, 268, 0, 0, 0, 0, 0]
 SSTables in each level: [6/4, 10, 106/100, 271, 0, 0, 0, 0, 0]
 SSTables in each level: [1, 16/10, 105/100, 269, 0, 0, 0, 0, 0]
 SSTables in each level: [5/4, 10, 103/100, 272, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 11/10, 105/100, 270, 0, 0, 0, 0, 0]
 SSTables in each level: [1, 12/10, 105/100, 271, 0, 0, 0, 0, 0]
 SSTables in each level: [1, 14/10, 104/100, 267, 0, 0, 0, 0, 0]
 SSTables in each level: [9/4, 10, 103/100, 265, 0, 0, 0, 0, 0]
 Recently, with 41 pending compaction tasks:
 SSTables in each level: [4, 13/10, 106/100, 269, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 12/10, 106/100, 273, 0, 0, 0, 0, 0]
 SSTables in each level: [5/4, 11/10, 106/100, 271, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 12/10, 103/100, 275, 0, 0, 0, 0, 0]
 SSTables in each level: [2, 13/10, 106/100, 273, 0, 0, 0, 0, 0]
 SSTables in each level: [3, 10, 104/100, 275, 0, 0, 0, 0, 0]
 SSTables in each level: [6/4, 11/10, 103/100, 269, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 16/10, 105/100, 264, 0, 0, 0, 0, 0]
 More information about the use case: writes are roughly uniform across these 
 tables. The data is sharded across these 8 tables by key to improve 
 compaction parallelism. Each node receives up to 75,000 writes/sec sustained 
 at peak, and a small number of reads. This is a pre-production cluster that's 
 being warmed up with new data, so the low volume of reads (~100/sec per node) 
 is just from automatic sampled data checks, otherwise we'd just use STCS :)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-8463) Constant compaction under LCS

2014-12-15 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-8463?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14247293#comment-14247293
 ] 

Rick Branson commented on CASSANDRA-8463:
-

I applied the logging patch on one node and quickly noticed a suspect 
compaction. This contains 130 sstables from L3 and only 1 from L2.

2014-12-15_21:50:10.24752 INFO  21:50:10 Compacting 
(5750bed0-84a4-11e4-8177-4571ae47521c) 
[SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35274-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35305-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35330-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35409-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35512-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35286-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35323-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35366-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35407-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35558-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35206-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35570-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35247-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35361-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35335-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35261-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35588-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35368-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35545-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35311-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35231-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35416-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35567-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35265-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35500-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35516-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35304-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35358-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35256-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35550-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35393-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35533-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35494-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35538-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35239-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35544-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35401-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35502-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35495-Data.db,
 level=3), 
SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_005/ks1-DataByUserID_005-ka-35553-Data.db,
 level=3), 

[jira] [Commented] (CASSANDRA-8463) Constant compaction under LCS

2014-12-15 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-8463?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14247337#comment-14247337
 ] 

Rick Branson commented on CASSANDRA-8463:
-

This also seems suspect for creating overlap (single-table L1 compaction output 
placed into L2??):

2014-12-15_22:17:28.58953 INFO  22:17:28 Compacting 
(27d85a60-84a8-11e4-8b9d-79ee1dbb2761) 
[SSTableReader(path=/data/cassandra/data/ks1/DataByUserID_001/ks1-DataByUserID_001-ka-42169-Data.db,
 level=1), ]
2014-12-15_22:17:57.84703 INFO  22:17:57 Compacted 
(27d85a60-84a8-11e4-8b9d-79ee1dbb2761) 1 sstables to 
[/data/cassandra/data/ks1/DataByUserID_001/ks1-DataByUserID_001-ka-42835,] to 
level=2.  33,581,350 bytes to 33,581,350 (~100% of original) in 29,256ms = 
1.094670MB/s.  80,322 total partitions merged to 80,322.  Partition merge 
counts were {1:80322, }

 Constant compaction under LCS
 -

 Key: CASSANDRA-8463
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8463
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: Hardware is recent 2-socket, 16-core (x2 Hyperthreaded), 
 144G RAM, solid-state storage.
 Platform is Linux 3.2.51, Oracle JDK 64-bit 1.7.0_65.
 Heap is 32G total, 4G newsize.
 8G/8G on-heap/off-heap memtables, offheap_buffer allocator, 0.5 
 memtable_cleanup_threshold
 concurrent_compactors: 20
Reporter: Rick Branson
Assignee: Marcus Eriksson
 Fix For: 2.1.3

 Attachments: 0001-better-logging.patch, log-for-8463.txt


 It appears that tables configured with LCS will completely re-compact 
 themselves over some period of time after upgrading from 2.0 to 2.1 (2.0.11 
 - 2.1.2, specifically). It starts out with 10 pending tasks for an hour or 
 so, then starts building up, now with 50-100 tasks pending across the cluster 
 after 12 hours. These nodes are under heavy write load, but were easily able 
 to keep up in 2.0 (they rarely had 5 pending compaction tasks), so I don't 
 think it's LCS in 2.1 actually being worse, just perhaps some different LCS 
 behavior that causes the layout of tables from 2.0 to prompt the compactor to 
 reorganize them?
 The nodes flushed ~11MB SSTables under 2.0. They're currently flushing ~36MB 
 SSTables due to the improved memtable setup in 2.1. Before I upgraded the 
 entire cluster to 2.1, I noticed the problem and tried several variations on 
 the flush size, thinking perhaps the larger tables in L0 were causing some 
 kind of cascading compactions. Even if they're sized roughly like the 2.0 
 flushes were, same behavior occurs. I also tried both enabling  disabling 
 STCS in L0 with no real change other than L0 began to back up faster, so I 
 left the STCS in L0 enabled.
 Tables are configured with 32MB sstable_size_in_mb, which was found to be an 
 improvement on the 160MB table size for compaction performance. Maybe this is 
 wrong now? Otherwise, the tables are configured with defaults. Compaction has 
 been unthrottled to help them catch-up. The compaction threads stay very 
 busy, with the cluster-wide CPU at 45% nice time. No nodes have completely 
 caught up yet. I'll update JIRA with status about their progress if anything 
 interesting happens.
 From a node around 12 hours ago, around an hour after the upgrade, with 19 
 pending compaction tasks:
 SSTables in each level: [6/4, 10, 105/100, 268, 0, 0, 0, 0, 0]
 SSTables in each level: [6/4, 10, 106/100, 271, 0, 0, 0, 0, 0]
 SSTables in each level: [1, 16/10, 105/100, 269, 0, 0, 0, 0, 0]
 SSTables in each level: [5/4, 10, 103/100, 272, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 11/10, 105/100, 270, 0, 0, 0, 0, 0]
 SSTables in each level: [1, 12/10, 105/100, 271, 0, 0, 0, 0, 0]
 SSTables in each level: [1, 14/10, 104/100, 267, 0, 0, 0, 0, 0]
 SSTables in each level: [9/4, 10, 103/100, 265, 0, 0, 0, 0, 0]
 Recently, with 41 pending compaction tasks:
 SSTables in each level: [4, 13/10, 106/100, 269, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 12/10, 106/100, 273, 0, 0, 0, 0, 0]
 SSTables in each level: [5/4, 11/10, 106/100, 271, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 12/10, 103/100, 275, 0, 0, 0, 0, 0]
 SSTables in each level: [2, 13/10, 106/100, 273, 0, 0, 0, 0, 0]
 SSTables in each level: [3, 10, 104/100, 275, 0, 0, 0, 0, 0]
 SSTables in each level: [6/4, 11/10, 103/100, 269, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 16/10, 105/100, 264, 0, 0, 0, 0, 0]
 More information about the use case: writes are roughly uniform across these 
 tables. The data is sharded across these 8 tables by key to improve 
 compaction parallelism. Each node receives up to 75,000 writes/sec sustained 
 at peak, and a small number of reads. This is a pre-production cluster that's 
 being warmed up with new data, so the low volume of reads (~100/sec per node) 
 is just from automatic sampled data 

[jira] [Commented] (CASSANDRA-8462) Upgrading a 2.0 to 2.1 breaks CFMetaData on 2.0 nodes

2014-12-15 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-8462?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14247499#comment-14247499
 ] 

Rick Branson commented on CASSANDRA-8462:
-

[~iamaleksey] this happened first time I restarted a 2.0 node after adding a 
2.1 node to the cluster. It happened every time I restarted a 2.0 node, until I 
upgraded it to 2.1.

 Upgrading a 2.0 to 2.1 breaks CFMetaData on 2.0 nodes
 -

 Key: CASSANDRA-8462
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8462
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Rick Branson
Assignee: Aleksey Yeschenko

 Added a 2.1.2 node to a cluster running 2.0.11. Didn't make any schema 
 changes. When I tried to reboot one of the 2.0 nodes, it failed to boot with 
 this exception. Besides an obvious fix, any workarounds for this?
 {noformat}
 java.lang.IllegalArgumentException: No enum constant 
 org.apache.cassandra.config.CFMetaData.Caching.{keys:ALL, 
 rows_per_partition:NONE}
 at java.lang.Enum.valueOf(Enum.java:236)
 at 
 org.apache.cassandra.config.CFMetaData$Caching.valueOf(CFMetaData.java:286)
 at 
 org.apache.cassandra.config.CFMetaData.fromSchemaNoColumnsNoTriggers(CFMetaData.java:1713)
 at 
 org.apache.cassandra.config.CFMetaData.fromSchema(CFMetaData.java:1793)
 at 
 org.apache.cassandra.config.KSMetaData.deserializeColumnFamilies(KSMetaData.java:307)
 at 
 org.apache.cassandra.config.KSMetaData.fromSchema(KSMetaData.java:288)
 at 
 org.apache.cassandra.db.DefsTables.loadFromKeyspace(DefsTables.java:131)
 at 
 org.apache.cassandra.config.DatabaseDescriptor.loadSchemas(DatabaseDescriptor.java:529)
 at 
 org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:270)
 at 
 org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:496)
 at 
 org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:585)
 {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-8463) Upgrading 2.0 to 2.1 causes LCS to recompact all files

2014-12-12 Thread Rick Branson (JIRA)

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

Rick Branson updated CASSANDRA-8463:

Attachment: log-for-8463.txt

 Upgrading 2.0 to 2.1 causes LCS to recompact all files
 --

 Key: CASSANDRA-8463
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8463
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: Hardware is recent 2-socket, 16-core (x2 Hyperthreaded), 
 144G RAM, solid-state storage.
 Platform is Linux 3.2.51, Oracle JDK 64-bit 1.7.0_65.
 Heap is 32G total, 4G newsize.
 8G/8G on-heap/off-heap memtables, offheap_buffer allocator, 0.5 
 memtable_cleanup_threshold
 concurrent_compactors: 20
Reporter: Rick Branson
Assignee: Marcus Eriksson
 Fix For: 2.1.3

 Attachments: log-for-8463.txt


 It appears that tables configured with LCS will completely re-compact 
 themselves over some period of time after upgrading from 2.0 to 2.1 (2.0.11 
 - 2.1.2, specifically). It starts out with 10 pending tasks for an hour or 
 so, then starts building up, now with 50-100 tasks pending across the cluster 
 after 12 hours. These nodes are under heavy write load, but were easily able 
 to keep up in 2.0 (they rarely had 5 pending compaction tasks), so I don't 
 think it's LCS in 2.1 actually being worse, just perhaps some different LCS 
 behavior that causes the layout of tables from 2.0 to prompt the compactor to 
 reorganize them?
 The nodes flushed ~11MB SSTables under 2.0. They're currently flushing ~36MB 
 SSTables due to the improved memtable setup in 2.1. Before I upgraded the 
 entire cluster to 2.1, I noticed the problem and tried several variations on 
 the flush size, thinking perhaps the larger tables in L0 were causing some 
 kind of cascading compactions. Even if they're sized roughly like the 2.0 
 flushes were, same behavior occurs. I also tried both enabling  disabling 
 STCS in L0 with no real change other than L0 began to back up faster, so I 
 left the STCS in L0 enabled.
 Tables are configured with 32MB sstable_size_in_mb, which was found to be an 
 improvement on the 160MB table size for compaction performance. Maybe this is 
 wrong now? Otherwise, the tables are configured with defaults. Compaction has 
 been unthrottled to help them catch-up. The compaction threads stay very 
 busy, with the cluster-wide CPU at 45% nice time. No nodes have completely 
 caught up yet. I'll update JIRA with status about their progress if anything 
 interesting happens.
 From a node around 12 hours ago, around an hour after the upgrade, with 19 
 pending compaction tasks:
 SSTables in each level: [6/4, 10, 105/100, 268, 0, 0, 0, 0, 0]
 SSTables in each level: [6/4, 10, 106/100, 271, 0, 0, 0, 0, 0]
 SSTables in each level: [1, 16/10, 105/100, 269, 0, 0, 0, 0, 0]
 SSTables in each level: [5/4, 10, 103/100, 272, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 11/10, 105/100, 270, 0, 0, 0, 0, 0]
 SSTables in each level: [1, 12/10, 105/100, 271, 0, 0, 0, 0, 0]
 SSTables in each level: [1, 14/10, 104/100, 267, 0, 0, 0, 0, 0]
 SSTables in each level: [9/4, 10, 103/100, 265, 0, 0, 0, 0, 0]
 Recently, with 41 pending compaction tasks:
 SSTables in each level: [4, 13/10, 106/100, 269, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 12/10, 106/100, 273, 0, 0, 0, 0, 0]
 SSTables in each level: [5/4, 11/10, 106/100, 271, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 12/10, 103/100, 275, 0, 0, 0, 0, 0]
 SSTables in each level: [2, 13/10, 106/100, 273, 0, 0, 0, 0, 0]
 SSTables in each level: [3, 10, 104/100, 275, 0, 0, 0, 0, 0]
 SSTables in each level: [6/4, 11/10, 103/100, 269, 0, 0, 0, 0, 0]
 SSTables in each level: [4, 16/10, 105/100, 264, 0, 0, 0, 0, 0]
 More information about the use case: writes are roughly uniform across these 
 tables. The data is sharded across these 8 tables by key to improve 
 compaction parallelism. Each node receives up to 75,000 writes/sec sustained 
 at peak, and a small number of reads. This is a pre-production cluster that's 
 being warmed up with new data, so the low volume of reads (~100/sec per node) 
 is just from automatic sampled data checks, otherwise we'd just use STCS :)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-8475) Altering Table's tombstone_threshold stalls compaction until restart

2014-12-12 Thread Rick Branson (JIRA)
Rick Branson created CASSANDRA-8475:
---

 Summary: Altering Table's tombstone_threshold stalls compaction 
until restart
 Key: CASSANDRA-8475
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8475
 Project: Cassandra
  Issue Type: Bug
Reporter: Rick Branson


Compaction won't move forward on the table until a restart takes place and the 
temp table is ignored. My hunch is that running CompactionTasks are killed and 
there are still pre-opened temp files ref'd but they get deleted with the 
CompactionTask dies?

Exception:
2014-12-12_22:03:19.84572 ERROR 22:03:19 Exception in thread 
Thread[CompactionExecutor:671,1,main]
2014-12-12_22:03:19.84576 java.lang.RuntimeException: 
java.io.FileNotFoundException: 
/data/cassandra/data/ks1/DataByUserID_007/ks1-DataByUserID_007-tmplink-ka-21801-Data.db
 (No such file or directory)
2014-12-12_22:03:19.84576   at 
org.apache.cassandra.io.compress.CompressedThrottledReader.open(CompressedThrottledReader.java:52)
 ~[apache-cassandra-2.1.2.jar:2.1.2]
2014-12-12_22:03:19.84577   at 
org.apache.cassandra.io.sstable.SSTableReader.openDataReader(SSTableReader.java:1895)
 ~[apache-cassandra-2.1.2.jar:2.1.2]
2014-12-12_22:03:19.84578   at 
org.apache.cassandra.io.sstable.SSTableScanner.init(SSTableScanner.java:67) 
~[apache-cassandra-2.1.2.jar:2.1.2]
2014-12-12_22:03:19.84579   at 
org.apache.cassandra.io.sstable.SSTableReader.getScanner(SSTableReader.java:1681)
 ~[apache-cassandra-2.1.2.jar:2.1.2]
2014-12-12_22:03:19.84579   at 
org.apache.cassandra.io.sstable.SSTableReader.getScanner(SSTableReader.java:1693)
 ~[apache-cassandra-2.1.2.jar:2.1.2]
2014-12-12_22:03:19.84580   at 
org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getScanners(LeveledCompactionStrategy.java:181)
 ~[apache-cassandra-2.1.2.jar:2.1.2]
2014-12-12_22:03:19.84581   at 
org.apache.cassandra.db.compaction.WrappingCompactionStrategy.getScanners(WrappingCompactionStrategy.java:320)
 ~[apache-cassandra-2.1.2.jar:2.1.2]
2014-12-12_22:03:19.84581   at 
org.apache.cassandra.db.compaction.AbstractCompactionStrategy.getScanners(AbstractCompactionStrategy.java:340)
 ~[apache-cassandra-2.1.2.jar:2.1.2]
2014-12-12_22:03:19.84582   at 
org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:151)
 ~[apache-cassandra-2.1.2.jar:2.1.2]
2014-12-12_22:03:19.84583   at 
org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
 ~[apache-cassandra-2.1.2.jar:2.1.2]
2014-12-12_22:03:19.84583   at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) 
~[apache-cassandra-2.1.2.jar:2.1.2]
2014-12-12_22:03:19.84583   at 
org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:75)
 ~[apache-cassandra-2.1.2.jar:2.1.2]
2014-12-12_22:03:19.84584   at 
org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
 ~[apache-cassandra-2.1.2.jar:2.1.2]
2014-12-12_22:03:19.84584   at 
org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:232)
 ~[apache-cassandra-2.1.2.jar:2.1.2]
2014-12-12_22:03:19.84585   at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
~[na:1.7.0_65]
2014-12-12_22:03:19.84586   at 
java.util.concurrent.FutureTask.run(FutureTask.java:262) ~[na:1.7.0_65]
2014-12-12_22:03:19.84586   at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
~[na:1.7.0_65]
2014-12-12_22:03:19.84587   at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
[na:1.7.0_65]
2014-12-12_22:03:19.84587   at java.lang.Thread.run(Thread.java:745) 
[na:1.7.0_65]
2014-12-12_22:03:19.84587 Caused by: java.io.FileNotFoundException: 
/data/cassandra/data/ks1/DataByUserID_007/ks1-DataByUserID_007-tmplink-ka-21801-Data.db
 (No such file or directory)
2014-12-12_22:03:19.84588   at java.io.RandomAccessFile.open(Native Method) 
~[na:1.7.0_65]
2014-12-12_22:03:19.84588   at 
java.io.RandomAccessFile.init(RandomAccessFile.java:241) ~[na:1.7.0_65]
2014-12-12_22:03:19.84589   at 
org.apache.cassandra.io.util.RandomAccessReader.init(RandomAccessReader.java:58)
 ~[apache-cassandra-2.1.2.jar:2.1.2]
2014-12-12_22:03:19.84590   at 
org.apache.cassandra.io.compress.CompressedRandomAccessReader.init(CompressedRandomAccessReader.java:77)
 ~[apache-cassandra-2.1.2.jar:2.1.2]
2014-12-12_22:03:19.84590   at 
org.apache.cassandra.io.compress.CompressedThrottledReader.init(CompressedThrottledReader.java:34)
 ~[apache-cassandra-2.1.2.jar:2.1.2]
2014-12-12_22:03:19.84591   at 
org.apache.cassandra.io.compress.CompressedThrottledReader.open(CompressedThrottledReader.java:48)
 ~[apache-cassandra-2.1.2.jar:2.1.2]
2014-12-12_22:03:19.84591   ... 18 common frames omitted



--
This message was 

[jira] [Commented] (CASSANDRA-8475) Altering Table's tombstone_threshold stalls compaction until restart

2014-12-12 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-8475?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14244887#comment-14244887
 ] 

Rick Branson commented on CASSANDRA-8475:
-

Workaround for this is to disable  stop all compactions before making the 
schema change, then re-enable compactions after the schema change is made.

 Altering Table's tombstone_threshold stalls compaction until restart
 

 Key: CASSANDRA-8475
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8475
 Project: Cassandra
  Issue Type: Bug
Reporter: Rick Branson

 Compaction won't move forward on the table until a restart takes place and 
 the temp table is ignored. My hunch is that running CompactionTasks are 
 killed and there are still pre-opened temp files ref'd but they get deleted 
 with the CompactionTask dies?
 Exception:
 2014-12-12_22:03:19.84572 ERROR 22:03:19 Exception in thread 
 Thread[CompactionExecutor:671,1,main]
 2014-12-12_22:03:19.84576 java.lang.RuntimeException: 
 java.io.FileNotFoundException: 
 /data/cassandra/data/ks1/DataByUserID_007/ks1-DataByUserID_007-tmplink-ka-21801-Data.db
  (No such file or directory)
 2014-12-12_22:03:19.84576   at 
 org.apache.cassandra.io.compress.CompressedThrottledReader.open(CompressedThrottledReader.java:52)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 2014-12-12_22:03:19.84577   at 
 org.apache.cassandra.io.sstable.SSTableReader.openDataReader(SSTableReader.java:1895)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 2014-12-12_22:03:19.84578   at 
 org.apache.cassandra.io.sstable.SSTableScanner.init(SSTableScanner.java:67) 
 ~[apache-cassandra-2.1.2.jar:2.1.2]
 2014-12-12_22:03:19.84579   at 
 org.apache.cassandra.io.sstable.SSTableReader.getScanner(SSTableReader.java:1681)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 2014-12-12_22:03:19.84579   at 
 org.apache.cassandra.io.sstable.SSTableReader.getScanner(SSTableReader.java:1693)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 2014-12-12_22:03:19.84580   at 
 org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getScanners(LeveledCompactionStrategy.java:181)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 2014-12-12_22:03:19.84581   at 
 org.apache.cassandra.db.compaction.WrappingCompactionStrategy.getScanners(WrappingCompactionStrategy.java:320)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 2014-12-12_22:03:19.84581   at 
 org.apache.cassandra.db.compaction.AbstractCompactionStrategy.getScanners(AbstractCompactionStrategy.java:340)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 2014-12-12_22:03:19.84582   at 
 org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:151)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 2014-12-12_22:03:19.84583   at 
 org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 2014-12-12_22:03:19.84583   at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) 
 ~[apache-cassandra-2.1.2.jar:2.1.2]
 2014-12-12_22:03:19.84583   at 
 org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:75)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 2014-12-12_22:03:19.84584   at 
 org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 2014-12-12_22:03:19.84584   at 
 org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:232)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 2014-12-12_22:03:19.84585   at 
 java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
 ~[na:1.7.0_65]
 2014-12-12_22:03:19.84586   at 
 java.util.concurrent.FutureTask.run(FutureTask.java:262) ~[na:1.7.0_65]
 2014-12-12_22:03:19.84586   at 
 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
  ~[na:1.7.0_65]
 2014-12-12_22:03:19.84587   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
  [na:1.7.0_65]
 2014-12-12_22:03:19.84587   at java.lang.Thread.run(Thread.java:745) 
 [na:1.7.0_65]
 2014-12-12_22:03:19.84587 Caused by: java.io.FileNotFoundException: 
 /data/cassandra/data/ks1/DataByUserID_007/ks1-DataByUserID_007-tmplink-ka-21801-Data.db
  (No such file or directory)
 2014-12-12_22:03:19.84588   at java.io.RandomAccessFile.open(Native 
 Method) ~[na:1.7.0_65]
 2014-12-12_22:03:19.84588   at 
 java.io.RandomAccessFile.init(RandomAccessFile.java:241) ~[na:1.7.0_65]
 2014-12-12_22:03:19.84589   at 
 org.apache.cassandra.io.util.RandomAccessReader.init(RandomAccessReader.java:58)
  ~[apache-cassandra-2.1.2.jar:2.1.2]
 2014-12-12_22:03:19.84590   at 
 org.apache.cassandra.io.compress.CompressedRandomAccessReader.init(CompressedRandomAccessReader.java:77)
  

[jira] [Created] (CASSANDRA-8462) Upgrading a 2.0 to 2.1 breaks CFMetaData on 2.0 nodes

2014-12-11 Thread Rick Branson (JIRA)
Rick Branson created CASSANDRA-8462:
---

 Summary: Upgrading a 2.0 to 2.1 breaks CFMetaData on 2.0 nodes
 Key: CASSANDRA-8462
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8462
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Rick Branson


Added a 2.1.2 node to a cluster running 2.0.11. Didn't make any schema changes. 
When I tried to reboot one of the 2.0 nodes, it failed to boot with this 
exception. Besides an obvious fix, any workarounds for this?

{code}
java.lang.IllegalArgumentException: No enum constant 
org.apache.cassandra.config.CFMetaData.Caching.{keys:ALL, 
rows_per_partition:NONE}
at java.lang.Enum.valueOf(Enum.java:236)
at 
org.apache.cassandra.config.CFMetaData$Caching.valueOf(CFMetaData.java:286)
at 
org.apache.cassandra.config.CFMetaData.fromSchemaNoColumnsNoTriggers(CFMetaData.java:1713)
at 
org.apache.cassandra.config.CFMetaData.fromSchema(CFMetaData.java:1793)
at 
org.apache.cassandra.config.KSMetaData.deserializeColumnFamilies(KSMetaData.java:307)
at 
org.apache.cassandra.config.KSMetaData.fromSchema(KSMetaData.java:288)
at 
org.apache.cassandra.db.DefsTables.loadFromKeyspace(DefsTables.java:131)
at 
org.apache.cassandra.config.DatabaseDescriptor.loadSchemas(DatabaseDescriptor.java:529)
at 
org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:270)
at 
org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:496)
at 
org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:585)
{/code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-8463) Upgrading 2.0 to 2.1 causes LCS to recompact all files

2014-12-11 Thread Rick Branson (JIRA)
Rick Branson created CASSANDRA-8463:
---

 Summary: Upgrading 2.0 to 2.1 causes LCS to recompact all files
 Key: CASSANDRA-8463
 URL: https://issues.apache.org/jira/browse/CASSANDRA-8463
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: Hardware is recent 2-socket, 16-core (x2 Hyperthreaded), 
144G RAM, solid-state storage.
Platform is Linux 3.2.51, Oracle JDK 64-bit 1.7.0_65.
Heap is 32G total, 4G newsize.
8G/8G on-heap/off-heap memtables, offheap_buffer allocator, 0.5 
memtable_cleanup_threshold
concurrent_compactors: 20
Reporter: Rick Branson


It appears that tables configured with LCS will completely re-compact 
themselves over some period of time after upgrading from 2.0 to 2.1 (2.0.11 - 
2.1.2, specifically). It starts out with 10 pending tasks for an hour or so, 
then starts building up, now with 50-100 tasks pending across the cluster after 
12 hours. These nodes are under heavy write load, but were easily able to keep 
up in 2.0 (they rarely had 5 pending compaction tasks), so I don't think it's 
LCS in 2.1 actually being worse, just perhaps some different LCS behavior that 
causes the layout of tables from 2.0 to prompt the compactor to reorganize them?

The nodes flushed ~11MB SSTables under 2.0. They're currently flushing ~36MB 
SSTables due to the improved memtable setup in 2.1. Before I upgraded the 
entire cluster to 2.1, I noticed the problem and tried several variations on 
the flush size, thinking perhaps the larger tables in L0 were causing some kind 
of cascading compactions. Even if they're sized roughly like the 2.0 flushes 
were, same behavior occurs. I also tried both enabling  disabling STCS in L0 
with no real change other than L0 began to back up faster, so I left the STCS 
in L0 enabled.

Tables are configured with 32MB sstable_size_in_mb, which was found to be an 
improvement on the 160MB table size for compaction performance. Maybe this is 
wrong now? Otherwise, the tables are configured with defaults. Compaction has 
been unthrottled to help them catch-up. The compaction threads stay very busy, 
with the cluster-wide CPU at 45% nice time. No nodes have completely caught 
up yet. I'll update JIRA with status about their progress if anything 
interesting happens.

From a node around 12 hours ago, around an hour after the upgrade, with 19 
pending compaction tasks:
SSTables in each level: [6/4, 10, 105/100, 268, 0, 0, 0, 0, 0]
SSTables in each level: [6/4, 10, 106/100, 271, 0, 0, 0, 0, 0]
SSTables in each level: [1, 16/10, 105/100, 269, 0, 0, 0, 0, 0]
SSTables in each level: [5/4, 10, 103/100, 272, 0, 0, 0, 0, 0]
SSTables in each level: [4, 11/10, 105/100, 270, 0, 0, 0, 0, 0]
SSTables in each level: [1, 12/10, 105/100, 271, 0, 0, 0, 0, 0]
SSTables in each level: [1, 14/10, 104/100, 267, 0, 0, 0, 0, 0]
SSTables in each level: [9/4, 10, 103/100, 265, 0, 0, 0, 0, 0]

Recently, with 41 pending compaction tasks:
SSTables in each level: [4, 13/10, 106/100, 269, 0, 0, 0, 0, 0]
SSTables in each level: [4, 12/10, 106/100, 273, 0, 0, 0, 0, 0]
SSTables in each level: [5/4, 11/10, 106/100, 271, 0, 0, 0, 0, 0]
SSTables in each level: [4, 12/10, 103/100, 275, 0, 0, 0, 0, 0]
SSTables in each level: [2, 13/10, 106/100, 273, 0, 0, 0, 0, 0]
SSTables in each level: [3, 10, 104/100, 275, 0, 0, 0, 0, 0]
SSTables in each level: [6/4, 11/10, 103/100, 269, 0, 0, 0, 0, 0]
SSTables in each level: [4, 16/10, 105/100, 264, 0, 0, 0, 0, 0]

More information about the use case: writes are roughly uniform across these 
tables. The data is sharded across these 8 tables by key to improve 
compaction parallelism. Each node receives up to 75,000 writes/sec sustained at 
peak, and a small number of reads. This is a pre-production cluster that's 
being warmed up with new data, so the low volume of reads (~100/sec per node) 
is just from automatic sampled data checks, otherwise we'd just use STCS :)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-7764) RFC: Range movements will wake up previously invisible data

2014-08-13 Thread Rick Branson (JIRA)
Rick Branson created CASSANDRA-7764:
---

 Summary: RFC: Range movements will wake up previously invisible 
data
 Key: CASSANDRA-7764
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7764
 Project: Cassandra
  Issue Type: Bug
Reporter: Rick Branson


Presumably this has been going on as long as Cassandra has existed, but wanted 
to capture it here since it came up in an IRC discussion. This issue will 
probably show up on any cluster eventually.

Scenario:

1) Start with a 3-node cluster, RF=1
2) A 4th node is added to the cluster
3) Data is deleted on ranges belonging to 4th node
4) Wait for GC to clean up some tombstones on 4th node
4) 4th node removed from cluster
5) Deleted data will reappear since it was dormant on the original 3 nodes

This could definitely happen in many other situations where dormant data could 
exist such as inconsistencies that aren't resolved before range movement, but 
the case above seemed the most reasonable to propose as a real-world problem.

The cleanup operation can be used to get rid of the dormant data, but from my 
experience people don't run cleanup unless they're low on disk. It's definitely 
not a best practice for data integrity.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-7246) Gossip Null Pointer Exception when a cassandra instance in ring is restarted

2014-08-13 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-7246?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14095965#comment-14095965
 ] 

Rick Branson commented on CASSANDRA-7246:
-

Got some similar NPEs on restart on a node. This resulted in this node having a 
broken gossip table in memory (ugh) that didn't get fixed until a restart. This 
host was running 1.2.18. Presumably this is getEndpointStateForEndpoint 
returning null inside of getHostId.

{code}
java.lang.NullPointerException
at org.apache.cassandra.gms.Gossiper.getHostId(Gossiper.java:698)
at 
org.apache.cassandra.service.StorageService.handleStateNormal(StorageService.java:1521)
at 
org.apache.cassandra.service.StorageService.onChange(StorageService.java:1341)
at org.apache.cassandra.gms.Gossiper.doNotifications(Gossiper.java:975)
at org.apache.cassandra.gms.Gossiper.applyNewStates(Gossiper.java:966)
at 
org.apache.cassandra.gms.Gossiper.applyStateLocally(Gossiper.java:924)
at 
org.apache.cassandra.gms.GossipDigestAck2VerbHandler.doVerb(GossipDigestAck2VerbHandler.java:50)
at 
org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:56)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
{code}

{code}
java.lang.NullPointerException
at org.apache.cassandra.gms.Gossiper.getHostId(Gossiper.java:698)
at 
org.apache.cassandra.service.StorageService.handleStateNormal(StorageService.java:1521)
at 
org.apache.cassandra.service.StorageService.onChange(StorageService.java:1341)
at 
org.apache.cassandra.service.StorageService.onJoin(StorageService.java:2033)
at 
org.apache.cassandra.gms.Gossiper.handleMajorStateChange(Gossiper.java:863)
at 
org.apache.cassandra.gms.Gossiper.applyStateLocally(Gossiper.java:914)
at 
org.apache.cassandra.gms.GossipDigestAck2VerbHandler.doVerb(GossipDigestAck2VerbHandler.java:50)
at 
org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:56)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
{code}

 Gossip Null Pointer Exception when a cassandra instance in ring is restarted
 

 Key: CASSANDRA-7246
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7246
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: 12 node ring of 1.2.x.
 11 of 12 are 1.2.15.
 1 is 1.2.16.
Reporter: Peter Haggerty
Assignee: Brandon Williams
Priority: Minor
  Labels: gossip, nullpointerexception
 Attachments: 7246.txt


 12 Cassandra instances, one per node.
 11 of the Cassandra instances are 1.2.15.
 1 of the Cassandra instances is 1.2.16.
 One of the eleven 1.2.15 Cassandra instances is restarted (disable thrift, 
 gossip, then flush, drain, stop, start).
 The 1.2.16 Cassandra instance noted this by throwing a Null Pointer 
 Exception. None of the 1.2.15 instances threw an exception and this is new 
 behavior that hasn't been observed before.
 ERROR 02:18:06,009 Exception in thread Thread[GossipStage:1,5,main]
 java.lang.NullPointerException
 at org.apache.cassandra.gms.Gossiper.convict(Gossiper.java:264)
 at 
 org.apache.cassandra.gms.FailureDetector.forceConviction(FailureDetector.java:246)
 at 
 org.apache.cassandra.gms.GossipShutdownVerbHandler.doVerb(GossipShutdownVerbHandler.java:37)
 at 
 org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:56)
 at 
 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
 at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
 at java.lang.Thread.run(Thread.java:724)
  INFO 02:18:23,402 Node /10.x.y.x is now part of the cluster
  INFO 02:18:23,403 InetAddress /10.x.y.z is now UP
  INFO 02:18:53,494 FatClient /10.x.y.z has been silent for 3ms, removing 
 from gossip
  INFO 02:19:00,031 Handshaking version with /10.x.y.z



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Created] (CASSANDRA-7758) Some gossip messages are very slow to process on vnode clusters

2014-08-12 Thread Rick Branson (JIRA)
Rick Branson created CASSANDRA-7758:
---

 Summary: Some gossip messages are very slow to process on vnode 
clusters
 Key: CASSANDRA-7758
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7758
 Project: Cassandra
  Issue Type: Bug
Reporter: Rick Branson
Assignee: Rick Branson


Both StorageService.getChangedRangesForLeaving and 
PendingRangeCalculatorService.calculatePendingRanges clone the TokenMetadata 
map once per range that a node covers. This is a very expensive operation on a 
cluster with vnodes since the TokenMetadata map can easily be tens of thousands 
of entries, even for a small cluster, and this runs thousands of times for each 
changing node. This can cause big cluster-wide slowdowns  problems in the 
GossipStage and PendingRangeCalculator thread pools. Processing just one 
message can generate enough garbage to pause the entire cluster simultaneously 
when one of these is being propagated. Patches forthcoming.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Updated] (CASSANDRA-7758) Some gossip messages are very slow to process on vnode clusters

2014-08-12 Thread Rick Branson (JIRA)

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

Rick Branson updated CASSANDRA-7758:


Attachment: 7758-1-v1.2.txt

 Some gossip messages are very slow to process on vnode clusters
 ---

 Key: CASSANDRA-7758
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7758
 Project: Cassandra
  Issue Type: Bug
Reporter: Rick Branson
Assignee: Rick Branson
 Attachments: 7758-1-v1.2.txt


 Both StorageService.getChangedRangesForLeaving and 
 PendingRangeCalculatorService.calculatePendingRanges clone the TokenMetadata 
 map once per range that a node covers. This is a very expensive operation on 
 a cluster with vnodes since the TokenMetadata map can easily be tens of 
 thousands of entries, even for a small cluster, and this runs thousands of 
 times for each changing node. This can cause big cluster-wide slowdowns  
 problems in the GossipStage and PendingRangeCalculator thread pools. 
 Processing just one message can generate enough garbage to pause the entire 
 cluster simultaneously when one of these is being propagated. Patches 
 forthcoming.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Updated] (CASSANDRA-7758) Some gossip messages are very slow to process on vnode clusters

2014-08-12 Thread Rick Branson (JIRA)

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

Rick Branson updated CASSANDRA-7758:


Attachment: 7758-1-v2.0.txt

 Some gossip messages are very slow to process on vnode clusters
 ---

 Key: CASSANDRA-7758
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7758
 Project: Cassandra
  Issue Type: Bug
Reporter: Rick Branson
Assignee: Rick Branson
 Attachments: 7758-1-v1.2.txt, 7758-1-v2.0.txt


 Both StorageService.getChangedRangesForLeaving and 
 PendingRangeCalculatorService.calculatePendingRanges clone the TokenMetadata 
 map once per range that a node covers. This is a very expensive operation on 
 a cluster with vnodes since the TokenMetadata map can easily be tens of 
 thousands of entries, even for a small cluster, and this runs thousands of 
 times for each changing node. This can cause big cluster-wide slowdowns  
 problems in the GossipStage and PendingRangeCalculator thread pools. 
 Processing just one message can generate enough garbage to pause the entire 
 cluster simultaneously when one of these is being propagated. Patches 
 forthcoming.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Updated] (CASSANDRA-7704) FileNotFoundException during STREAM-OUT triggers 100% CPU usage

2014-08-08 Thread Rick Branson (JIRA)

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

Rick Branson updated CASSANDRA-7704:


Attachment: other-errors.txt

There wasn't anything in the logs that indicated *why* the failure happened. Th 
I attached anything suspect. The IndexOutOfBoundsException occurred on the 
bootstrapping node *after* the stream failure occurred on the node that was 
streaming out.

There was a CompactionTask that ran at 2014-08-05 18:00:25,804 (4 minutes 
before the StreamOut task) that tried to compact that SSTable that referenced 
in the FileNotFoundException. No other log messages related to that file though.

 FileNotFoundException during STREAM-OUT triggers 100% CPU usage
 ---

 Key: CASSANDRA-7704
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7704
 Project: Cassandra
  Issue Type: Bug
Reporter: Rick Branson
Assignee: Benedict
 Attachments: 7704.txt, backtrace.txt, other-errors.txt


 See attached backtrace which was what triggered this. This stream failed and 
 then ~12 seconds later it emitted that exception. At that point, all CPUs 
 went to 100%. A thread dump shows all the ReadStage threads stuck inside 
 IntervalTree.searchInternal inside of CFS.markReferenced().



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Updated] (CASSANDRA-7664) IndexOutOfBoundsException thrown during repair

2014-08-08 Thread Rick Branson (JIRA)

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

Rick Branson updated CASSANDRA-7664:


Attachment: other-errors.txt

Got this on a bootstrapping node shortly after a stream failed. See 
other-errors.txt file for the relevant log information.

 IndexOutOfBoundsException thrown during repair
 --

 Key: CASSANDRA-7664
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7664
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: RHEL 6.1
 Casandra 1.2.3 - 1.2.18
Reporter: ZhongYu
 Attachments: $O[TOX~GGUZRW~IHPYPEG{0.jpg, other-errors.txt


 I was running repair command with moderate read and write load at the same 
 time. And I found tens of IndexOutOfBoundsException in system log as follows:
 {quote}
 ERROR [Thread-6056] 2013-05-22 14:47:59,416 CassandraDaemon.java (line132) 
 Exception in thread Thread[Thread-6056,5,main]
 java.lang.IndexOutOfBoundsException
 at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:75)
 at 
 org.apache.cassandra.streaming.compress.CompressedInputStream$Reader.runMayThrow(CompressedInputStream.java:151)
 at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
 at java.lang.Thread.run(Thread.java:662)
 {quote}
 I read the source code of CompressedInputStream.java and found there surely 
 will throw IndexOutOfBoundsException in the following situation:
 {code:title=CompressedInputStream.java|borderStyle=solid}
 // Part of CompressedInputStream.java start from Line 139
 protected void runMayThrow() throws Exception
 {
 byte[] compressedWithCRC;
 while (chunks.hasNext())
 {
 CompressionMetadata.Chunk chunk = chunks.next();
 int readLength = chunk.length + 4; // read with CRC
 compressedWithCRC = new byte[readLength];
 int bufferRead = 0;
 while (bufferRead  readLength)
 bufferRead += source.read(compressedWithCRC, bufferRead, 
 readLength - bufferRead);
 dataBuffer.put(compressedWithCRC);
 }
 }
 {code}
 If read function read nothing because the end of the stream has been reached, 
 it will return -1, thus bufferRead can be negetive. In the next circle, read 
 function will throw IndexOutOfBoundsException because bufferRead is negetive. 



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-6335) Hints broken for nodes that change broadcast address

2014-08-08 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6335?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14091142#comment-14091142
 ] 

Rick Branson commented on CASSANDRA-6335:
-

Easiest way to repro might be to terminate a bootstrap before it finishes:

{code}
INFO [GossipTasks:1] 2014-08-05 16:56:41,331 Gossiper.java (line 658) FatClient 
/10.210.165.35 has been silent for 3ms, removing from gossip
ERROR [MutationStage:1] 2014-08-05 16:56:41,341 CassandraDaemon.java (line 199) 
Exception in thread Thread[MutationStage:1,5,main]
java.lang.AssertionError: Missing host ID for 10.210.165.35
at 
org.apache.cassandra.service.StorageProxy.writeHintForMutation(StorageProxy.java:898)
at 
org.apache.cassandra.service.StorageProxy$6.runMayThrow(StorageProxy.java:869)
at 
org.apache.cassandra.service.StorageProxy$HintRunnable.run(StorageProxy.java:1987)
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
at java.util.concurrent.FutureTask.run(FutureTask.java:262)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
{code}

 Hints broken for nodes that change broadcast address
 

 Key: CASSANDRA-6335
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6335
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Rick Branson

 When a node changes it's broadcast address, the transition process works 
 properly, but hints that are destined for it can't be delivered because of 
 the address change. It produces an exception:
 java.lang.AssertionError: Missing host ID for 10.1.60.22
 at 
 org.apache.cassandra.service.StorageProxy.writeHintForMutation(StorageProxy.java:598)
 at 
 org.apache.cassandra.service.StorageProxy$5.runMayThrow(StorageProxy.java:567)
 at 
 org.apache.cassandra.service.StorageProxy$HintRunnable.run(StorageProxy.java:1679)
 at 
 java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
 at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
 at java.util.concurrent.FutureTask.run(FutureTask.java:138)
 at 
 java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
 at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Created] (CASSANDRA-7704) FileNotFoundException during STREAM-OUT triggers 100% CPU usage

2014-08-05 Thread Rick Branson (JIRA)
Rick Branson created CASSANDRA-7704:
---

 Summary: FileNotFoundException during STREAM-OUT triggers 100% CPU 
usage
 Key: CASSANDRA-7704
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7704
 Project: Cassandra
  Issue Type: Bug
Reporter: Rick Branson
 Attachments: backtrace.txt

See attached backtrace which was what triggered this. This stream failed and 
then ~12 seconds later it emitted that exception. At that point, all CPUs went 
to 100%. A thread dump shows all the ReadStage threads stuck inside 
IntervalTree.searchInternal inside of CFS.markReferenced().



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-7432) Add new CMS GC flags to cassandra_env.sh for JVM later than 1.7.0_60

2014-08-04 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-7432?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14084927#comment-14084927
 ] 

Rick Branson commented on CASSANDRA-7432:
-

Very happy with this in production. Enabling -XX:+CMSParallelInitialMarkEnabled 
and -XX:+CMSEdenChunksRecordAlways has *drastically* improved the STW portions 
of CMS for our nodes with 4GB+ young gen. Not seeing any downsides to enabling 
these options either.

We're also running a few more options that have trade-offs which could be 
considered:

-XX:+CMSScavengeBeforeRemark which has helped reduce the pause times of STW 
portions of CMS remark with large eden spaces, since it's dependent on how much 
live garbage is in the eden space. If CMS is ran infrequently (every few 
minutes or so) I can't see how this option could hurt people even with small 
eden spaces.

-XX:CMSWaitDuration=1. This is how long the CMS will wait for a young gen 
collection before running the initial mark. It waits for a collection because 
the stop-the-world initial mark time depends on how many live objects are in 
eden space. The default is 2000ms, which seems low to me. The trade-off is that 
if it waits too long, CMS could run out of time and a promotion failure could 
occur. This seems unlikely though at 10 seconds in any even remotely healthy 
scenario.

 Add new CMS GC flags to cassandra_env.sh for JVM later than 1.7.0_60
 

 Key: CASSANDRA-7432
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7432
 Project: Cassandra
  Issue Type: Improvement
  Components: Packaging
Reporter: graham sanderson
Assignee: Brandon Williams
 Fix For: 1.2.19

 Attachments: 7432.txt


 The new flags in question are as follows:
 {code}
 -XX:+CMSParallelInitialMarkEnabled
 -XX:+CMSEdenChunksRecordAlways
 {code}
 Given we already have
 {code}
 JVM_OPTS=$JVM_OPTS -XX:+UseParNewGC 
 JVM_OPTS=$JVM_OPTS -XX:+UseConcMarkSweepGC 
 JVM_OPTS=$JVM_OPTS -XX:+CMSParallelRemarkEnabled 
 JVM_OPTS=$JVM_OPTS -XX:+UseTLAB
 if [ $JVM_ARCH = 64-Bit ] ; then
 JVM_OPTS=$JVM_OPTS -XX:+UseCondCardMark
 fi
 {code}
 The assumption would be that people are at least running on large number CPU 
 cores/threads
 I would therefore recommend defaulting these flags if available - the only 
 two possible downsides for {{+CMSEdenChunksRecordAlways}}:
 1) There is a new very short (probably un-contended) lock in the slow (non 
 TLAB) eden allocation path with {{+CMSEdenChunksRecordAlways}}. I haven't 
 detected this timing wise - this is the slow path after all
 2) If you are running with {{-XX:-UseCMSCompactAtFullCollection}} (not the 
 default) *and* you call {{System.gc()}} then  {{+CMSEdenChunksRecordAlways}} 
 will expose you to a possible seg fault: (see
 [http://bugs.java.com/bugdatabase/view_bug.do?bug_id=8021809])



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Created] (CASSANDRA-7594) Disruptor Thrift server worker thread pool not adjustable

2014-07-23 Thread Rick Branson (JIRA)
Rick Branson created CASSANDRA-7594:
---

 Summary: Disruptor Thrift server worker thread pool not adjustable
 Key: CASSANDRA-7594
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7594
 Project: Cassandra
  Issue Type: Bug
Reporter: Rick Branson


For the THsHaDisruptorServer, there may not be enough threads to run blocking 
StorageProxy methods. The current number of worker threads is hardcoded at 2 
per selector, so 2 * numAvailableProcessors(), or 64 threads on a 16-core 
hyperthreaded machine. StorageProxy methods block these threads, so this puts 
an upper bound on the throughput if hsha is enabled. If operations take 10ms on 
average, the node can only handle a maximum of 6,400 operations per second. 
This is a regression from hsha on 1.2.x, where the thread pool was tunable 
using rpc_min_threads and rpc_max_threads.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-7594) Disruptor Thrift server worker thread pool not adjustable

2014-07-23 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-7594?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14071975#comment-14071975
 ] 

Rick Branson commented on CASSANDRA-7594:
-

The ideal situation would be if rpc_min_threads and rpc_max_threads still 
worked, at least roughly. The problem with implementing this is that the 
TDisruptorServer only creates fixed-size thread pools specified by 
numWorkersPerSelector. The variable-size thread pools are useful because they 
allow working around temporary congestion problems by using a large number of 
executor threads, but fall back to a smaller thread pool when unneeded (i.e. 
128 - 2048). An example of this is when a two replicas for the same range are 
simultaneously slow or unresponsive using QUORUM on RF=3. The RPC thread pool 
is *rapidly* consumed, which can cause fallout to other, unrelated requests.

 Disruptor Thrift server worker thread pool not adjustable
 -

 Key: CASSANDRA-7594
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7594
 Project: Cassandra
  Issue Type: Bug
Reporter: Rick Branson
Assignee: Pavel Yaskevich

 For the THsHaDisruptorServer, there may not be enough threads to run blocking 
 StorageProxy methods. The current number of worker threads is hardcoded at 2 
 per selector, so 2 * numAvailableProcessors(), or 64 threads on a 16-core 
 hyperthreaded machine. StorageProxy methods block these threads, so this puts 
 an upper bound on the throughput if hsha is enabled. If operations take 10ms 
 on average, the node can only handle a maximum of 6,400 operations per 
 second. This is a regression from hsha on 1.2.x, where the thread pool was 
 tunable using rpc_min_threads and rpc_max_threads.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-5239) Asynchronous (non-blocking) StorageProxy

2014-07-23 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5239?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14072002#comment-14072002
 ] 

Rick Branson commented on CASSANDRA-5239:
-

That is an epic patch [~slebresne]. Throwing my +1 on this one. It would have 
profound reliability  performance impact. Not having to monopolize a thread 
for every running operation in a largely network and I/O bound system is big. 
It would really help smooth out the impact from proxy calls for temporary 
slowdowns on replicas. Unless I'm misunderstanding, this is the remaining 
synchronous piece in the network pipeline.

 Asynchronous (non-blocking) StorageProxy
 

 Key: CASSANDRA-5239
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5239
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Affects Versions: 2.0 beta 1
Reporter: Vijay
Assignee: Sylvain Lebresne
  Labels: performance
 Fix For: 3.0


 Problem Statement: 
 Currently we have rpc_min_threads, rpc_max_threads/ 
 native_transport_min_threads/native_transport_max_threads all of the 
 threads in the TPE are blocking and takes resources, the threads are mostly 
 sleeping. Increasing the Context switch costs.
 Details: 
 We should change StorageProxy methods to provide a callback which contains 
 the location where the results has to be written. When the response arrive 
 StorageProxy callback can write the results directly into the connection. 
 Timeouts can be handled in the same way.
 Fixing Netty should be trivial with some refactor in the storage proxy 
 (currently it is one method call for sending the request and waiting) we need 
 callback.
 Fixing Thrift may be harder because thrift calls the method and expects a 
 return value. We might need to write a custom Codec on Netty for thrift 
 support, which can potentially do callbacks (A Custom codec may be similar to 
 http://engineering.twitter.com/2011/04/twitter-search-is-now-3x-faster_1656.html
  but we dont know details about it). Another option is to update thrift to 
 have a callback.
 FYI, The motivation for this ticket is from another project which i am 
 working on with similar Proxy (blocking Netty transport) and making it Async 
 gave us 2x throughput improvement.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-5239) Asynchronous (non-blocking) StorageProxy

2014-07-23 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5239?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14072265#comment-14072265
 ] 

Rick Branson commented on CASSANDRA-5239:
-

[~benedict]: Not sure I follow how those are related? This ticket is important 
for operations that need to coordinate with other nodes over the network. It 
might be worth just focusing this on the native protocol since Thrift is 
basically deprecated. The CQL3 execution process is totally synchronous 
starting at the Netty worker thread. I know there's been some talk about 
turning into a pipeline where results stream through incrementally. This is 
obviously non-trivial, but async execution seems like it would fit naturally 
into a refactor of CQL3 to stream results.

 Asynchronous (non-blocking) StorageProxy
 

 Key: CASSANDRA-5239
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5239
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Affects Versions: 2.0 beta 1
Reporter: Vijay
Assignee: Sylvain Lebresne
  Labels: performance
 Fix For: 3.0


 Problem Statement: 
 Currently we have rpc_min_threads, rpc_max_threads/ 
 native_transport_min_threads/native_transport_max_threads all of the 
 threads in the TPE are blocking and takes resources, the threads are mostly 
 sleeping. Increasing the Context switch costs.
 Details: 
 We should change StorageProxy methods to provide a callback which contains 
 the location where the results has to be written. When the response arrive 
 StorageProxy callback can write the results directly into the connection. 
 Timeouts can be handled in the same way.
 Fixing Netty should be trivial with some refactor in the storage proxy 
 (currently it is one method call for sending the request and waiting) we need 
 callback.
 Fixing Thrift may be harder because thrift calls the method and expects a 
 return value. We might need to write a custom Codec on Netty for thrift 
 support, which can potentially do callbacks (A Custom codec may be similar to 
 http://engineering.twitter.com/2011/04/twitter-search-is-now-3x-faster_1656.html
  but we dont know details about it). Another option is to update thrift to 
 have a callback.
 FYI, The motivation for this ticket is from another project which i am 
 working on with similar Proxy (blocking Netty transport) and making it Async 
 gave us 2x throughput improvement.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-7402) limit the on heap memory available to requests

2014-07-08 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-7402?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14055168#comment-14055168
 ] 

Rick Branson commented on CASSANDRA-7402:
-

The more practical thing here is to make sure nothing on the read/write paths 
can potentially allocate big chunks of heap. If one looks at the read/write 
paths for something like PostgreSQL, they have made sure that the only 
potentially large allocations (sorts, hash-joins, etc) occur in a very 
controlled manner and have hard limits attached to them that can be configured. 
This seems like a reasonable thing to ask for at this stage in Cassandra's life.

 limit the on heap memory available to requests
 --

 Key: CASSANDRA-7402
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7402
 Project: Cassandra
  Issue Type: Improvement
Reporter: T Jake Luciani
 Fix For: 3.0


 When running a production cluster one common operational issue is quantifying 
 GC pauses caused by ongoing requests.
 Since different queries return varying amount of data you can easily get your 
 self into a situation where you Stop the world from a couple of bad actors in 
 the system.  Or more likely the aggregate garbage generated on a single node 
 across all in flight requests causes a GC.
 We should be able to set a limit on the max heap we can allocate to all 
 outstanding requests and track the garbage per requests to stop this from 
 happening.  It should increase a single nodes availability substantially.
 In the yaml this would be
 {code}
 total_request_memory_space_mb: 400
 {code}
 It would also be nice to have either a log of queries which generate the most 
 garbage so operators can track this.  Also a histogram.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-7486) Compare CMS and G1 pause times

2014-07-02 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-7486?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14050677#comment-14050677
 ] 

Rick Branson commented on CASSANDRA-7486:
-

Mad anecdotes:

We ran with G1 enabled for around 4 days in a 33-node cluster running 1.2.17 on 
JDK7u45 that has around a 1:5 read:write ratio. We tried a few different 
configurations with short durations, but most of the time we ran it with the 
out-of-the-box G1 configuration on a 20G heap and 32 parallel GC threads (16 
core, 32 hyperthreaded). There were some somewhat scary bugs fixed in 7u60 that 
ultimately caused me to roll back to the CMS collector after the experiment.

* The experiment pointed out that our young gen was basically too small and was 
pulling latency up significantly. When we returned back to CMS, I doubled new 
size from 800M - 1600M. We had moved to new hardware and hadn't taken the time 
to sit down and play with GC settings. This cut our mean latency dramatically 
as perceived from the client, ~50% for writes and ~30% for reads, similar to 
what we saw with G1. I was quite thrilled with this result.
* I tried both 100ms and 150ms pause times targets with 12G, 16G, and 20G 
heaps, and while these resulted in slightly lower mean latency (~5-10%), Mixed 
GC activity caused P99s to suffer greatly. There's compelling evidence that the 
200ms default is nearly ideal for the way the G1 algorithm works in its current 
incarnation.
* We basically needed a 20G heap to make G1 work well for us, since by default 
G1 will use up to half of the max heap for eden space and Cassandra needs quite 
a large old gen to stay happy. G1 appears to need a much larger eden space to 
work efficiently, sizes that would make ParNew die in a fire. GCs of the eden 
space were impressively fast, with a ~10G eden space taking ~120ms on average 
to collect.
* G1's huge eden space was helpful working around some issues with compaction 
on hints CF which had dozens of very wide partitions, hundreds of thousands of 
cells each.
* Overall, at the default 200ms pause time target, we didn't see much of an 
increase in CPU usage over CMS.

In the end, my tests basically told us that G1 requires a larger heap to get 
the same results with *far* less tuning. If there are GC issues, it seems like 
in the vast majority of cases G1 can either eliminate them or G1 makes it easy 
to just workaround them by cranking up the heap size. Someone should probably 
test G1 with a variable-sized heap since it's designed to give back RAM when it 
thinks it doesn't need it. That might or might not actually work. While we 
didn't test this, a configuration of G1 + heap size min of 1/8 RAM and max of 
1/2 RAM might make a really nice default for Cassandra at some point.

 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-6621) STCS fallback is not optimal when bootstrapping

2014-06-25 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6621?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14043921#comment-14043921
 ] 

Rick Branson commented on CASSANDRA-6621:
-

How about we add a switch to enable/disable this and disable it by default in 
2.0 then? This behavior is a net negative for people using LCS *unless* they 
happen to have long bursts of *very* high write volume.

 STCS fallback is not optimal when bootstrapping
 ---

 Key: CASSANDRA-6621
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6621
 Project: Cassandra
  Issue Type: Improvement
Reporter: Bartłomiej Romański
Assignee: Marcus Eriksson
Priority: Minor
  Labels: compaction, streaming
 Fix For: 2.0.9

 Attachments: 0001-wip-keep-sstable-level-when-bootstrapping.patch


 The initial discussion started in (closed) CASSANDRA-5371. I've rewritten my 
 last comment here...
 After streaming (e.g. during boostrap) Cassandra places all sstables at L0. 
 At the end of the process we end up with huge number of sstables at the 
 lowest level. 
 Currently, Cassandra falls back to STCS until the number of sstables at L0 
 reaches the reasonable level (32 or something).
 I'm not sure if falling back to STCS is the best way to handle this 
 particular situation. I've read the comment in the code and I'm aware why it 
 is a good thing to do if we have to many sstables at L0 as a result of too 
 many random inserts. We have a lot of sstables, each of them covers the whole 
 ring, there's simply no better option.
 However, after the bootstrap situation looks a bit different. The loaded 
 sstables already have very small ranges! We just have to tidy up a bit and 
 everything should be OK. STCS ignores that completely and after a while we 
 have a bit less sstables but each of them covers the whole ring instead of 
 just a small part. I believe that in that case letting LCS do the job is a 
 better option that allowing STCS mix everything up before.
 Is there a way to disable STCS fallback? I'd like to test that scenario in 
 practice during our next bootstrap...
 Does Cassandra really have to put streamed sstables at L0? The only thing we 
 have to assure is that sstables at any given level do not overlap. If we 
 stream different regions from different nodes how can we get any overlaps?



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-6621) STCS fallback is not optimal when bootstrapping

2014-06-25 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6621?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14043976#comment-14043976
 ] 

Rick Branson commented on CASSANDRA-6621:
-

As I said above, it causes bootstraps to require 2x disk space. One of the 
stated goals of LCS is to avoid the 2x disk space requirement, which the 
fallback violates.

 STCS fallback is not optimal when bootstrapping
 ---

 Key: CASSANDRA-6621
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6621
 Project: Cassandra
  Issue Type: Improvement
Reporter: Bartłomiej Romański
Assignee: Marcus Eriksson
Priority: Minor
  Labels: compaction, streaming
 Fix For: 2.0.9

 Attachments: 0001-wip-keep-sstable-level-when-bootstrapping.patch


 The initial discussion started in (closed) CASSANDRA-5371. I've rewritten my 
 last comment here...
 After streaming (e.g. during boostrap) Cassandra places all sstables at L0. 
 At the end of the process we end up with huge number of sstables at the 
 lowest level. 
 Currently, Cassandra falls back to STCS until the number of sstables at L0 
 reaches the reasonable level (32 or something).
 I'm not sure if falling back to STCS is the best way to handle this 
 particular situation. I've read the comment in the code and I'm aware why it 
 is a good thing to do if we have to many sstables at L0 as a result of too 
 many random inserts. We have a lot of sstables, each of them covers the whole 
 ring, there's simply no better option.
 However, after the bootstrap situation looks a bit different. The loaded 
 sstables already have very small ranges! We just have to tidy up a bit and 
 everything should be OK. STCS ignores that completely and after a while we 
 have a bit less sstables but each of them covers the whole ring instead of 
 just a small part. I believe that in that case letting LCS do the job is a 
 better option that allowing STCS mix everything up before.
 Is there a way to disable STCS fallback? I'd like to test that scenario in 
 practice during our next bootstrap...
 Does Cassandra really have to put streamed sstables at L0? The only thing we 
 have to assure is that sstables at any given level do not overlap. If we 
 stream different regions from different nodes how can we get any overlaps?



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-6621) STCS fallback is not optimal when bootstrapping

2014-06-10 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6621?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14026916#comment-14026916
 ] 

Rick Branson commented on CASSANDRA-6621:
-

This is definitely sub-optimal for us as well, just in terms of time spent 
compacting after a bootstrap/rebuild. We never get behind in L0 during normal 
operation. In addition, it causes one of the nice things about LCS to be 
invalidated as well, which is that you never have to worry about having double 
the disk space to compact. Bootstraps cause large compactions (~50% of the size 
of the CF), which means we need a ton of extra disk on bootstrap just to build 
nodes.

 STCS fallback is not optimal when bootstrapping
 ---

 Key: CASSANDRA-6621
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6621
 Project: Cassandra
  Issue Type: Improvement
Reporter: Bartłomiej Romański
Priority: Minor

 The initial discussion started in (closed) CASSANDRA-5371. I've rewritten my 
 last comment here...
 After streaming (e.g. during boostrap) Cassandra places all sstables at L0. 
 At the end of the process we end up with huge number of sstables at the 
 lowest level. 
 Currently, Cassandra falls back to STCS until the number of sstables at L0 
 reaches the reasonable level (32 or something).
 I'm not sure if falling back to STCS is the best way to handle this 
 particular situation. I've read the comment in the code and I'm aware why it 
 is a good thing to do if we have to many sstables at L0 as a result of too 
 many random inserts. We have a lot of sstables, each of them covers the whole 
 ring, there's simply no better option.
 However, after the bootstrap situation looks a bit different. The loaded 
 sstables already have very small ranges! We just have to tidy up a bit and 
 everything should be OK. STCS ignores that completely and after a while we 
 have a bit less sstables but each of them covers the whole ring instead of 
 just a small part. I believe that in that case letting LCS do the job is a 
 better option that allowing STCS mix everything up before.
 Is there a way to disable STCS fallback? I'd like to test that scenario in 
 practice during our next bootstrap...
 Does Cassandra really have to put streamed sstables at L0? The only thing we 
 have to assure is that sstables at any given level do not overlap. If we 
 stream different regions from different nodes how can we get any overlaps?



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-7262) During streaming: java.lang.AssertionError: Reference counter -1

2014-05-29 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-7262?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14012902#comment-14012902
 ] 

Rick Branson commented on CASSANDRA-7262:
-

Also seeing this NPE:

at 
org.apache.cassandra.streaming.StreamTransferTask.createMessageForRetry(StreamTransferTask.java:106)
at 
org.apache.cassandra.streaming.StreamSession.retry(StreamSession.java:525)
at 
org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:401)
at 
org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:289)
at java.lang.Thread.run(Thread.java:744)

Same thing or should I open another ticket? Seems like it's in the same vein.


 During streaming: java.lang.AssertionError: Reference counter -1
 

 Key: CASSANDRA-7262
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7262
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: Cassandra 2.07, x86-64 Ubuntu 12.04.4, Oracle java 
 1.7.0_45
Reporter: Duncan Sands
Assignee: Joshua McKenzie
Priority: Minor
 Fix For: 2.0.9

 Attachments: 7262_v1.txt, 7262_v2.txt, system.log.gz


 Got this assertion failure this weekend during repair:
 ERROR [STREAM-IN-/192.168.21.14] 2014-05-17 01:17:52,332 StreamSession.java 
 (line 420) [Stream #3a3ac8a2-dd50-11e3-b3c1-6bf6dccd6457] Streaming error 
 occurred
 java.lang.RuntimeException: Outgoing stream handler has been closed
 at 
 org.apache.cassandra.streaming.ConnectionHandler.sendMessage(ConnectionHandler.java:170)
 at 
 org.apache.cassandra.streaming.StreamSession.receive(StreamSession.java:483)
 at 
 org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:372)
 at 
 org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:289)
 at java.lang.Thread.run(Thread.java:744)
 ERROR [STREAM-IN-/192.168.21.14] 2014-05-17 01:17:52,350 CassandraDaemon.java 
 (line 198) Exception in thread Thread[STREAM-IN-/192.168.21.14,5,RMI Runtime]
 java.lang.AssertionError: Reference counter -1 for 
 /mnt/ssd1/cassandra/data/ldn_production/historical_accounts/ldn_production-historical_accounts-jb-79827-Data.db
 at 
 org.apache.cassandra.io.sstable.SSTableReader.releaseReference(SSTableReader.java:1107)
 at 
 org.apache.cassandra.streaming.StreamTransferTask.abort(StreamTransferTask.java:80)
 at 
 org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:322)
 at 
 org.apache.cassandra.streaming.StreamSession.onError(StreamSession.java:425)
 at 
 org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:300)
 at java.lang.Thread.run(Thread.java:744)
 followed by a few more (the reference counter got down to -3).  Got the same 
 kind of assertion failure on one other node (in a different data centre; 
 there are 21 nodes altogether distributed over 4 data centres).
 I've attached the relevant part of the log.  It starts quite a bit before the 
 assertion failure at the first exception on this node (Cannot proceed on 
 repair because a neighbor ... is dead), and finishes a few hours afterwards 
 when the node was restarted.
 Edit: The following Reference counter assertion failures followed the 1st on 
 a different file and have a different stack trace:
 ERROR [CompactionExecutor:382] 2014-05-17 01:17:53,157 CassandraDaemon.java 
 (line 198) Exception in thread Thread[CompactionExecutor:382,1,main]
 java.lang.AssertionError: Reference counter -1 for 
 /mnt/ssd1/cassandra/data/ldn_production/historical_accounts/ldn_production-historical_accounts-jb-83888-Data.db
at 
 org.apache.cassandra.io.sstable.SSTableReader.releaseReference(SSTableReader.java:1107)
at 
 org.apache.cassandra.io.sstable.SSTableReader.releaseReferences(SSTableReader.java:1429)
at 
 org.apache.cassandra.db.compaction.CompactionController.close(CompactionController.java:207)
at 
 org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:220)
at 
 org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
at 
 org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60)
at 
 org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
at 
 org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:197)
...



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-7262) During streaming: java.lang.AssertionError: Reference counter -1

2014-05-28 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-7262?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14012015#comment-14012015
 ] 

Rick Branson commented on CASSANDRA-7262:
-

Would this cause all of the ReadStage threads to spin on decrementAndGet in 
org.apache.cassandra.io.sstable.SSTableReader.releaseReference?

 During streaming: java.lang.AssertionError: Reference counter -1
 

 Key: CASSANDRA-7262
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7262
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: Cassandra 2.07, x86-64 Ubuntu 12.04.4, Oracle java 
 1.7.0_45
Reporter: Duncan Sands
Assignee: Joshua McKenzie
Priority: Minor
 Fix For: 2.0.9

 Attachments: 7262_v1.txt, 7262_v2.txt, system.log.gz


 Got this assertion failure this weekend during repair:
 ERROR [STREAM-IN-/192.168.21.14] 2014-05-17 01:17:52,332 StreamSession.java 
 (line 420) [Stream #3a3ac8a2-dd50-11e3-b3c1-6bf6dccd6457] Streaming error 
 occurred
 java.lang.RuntimeException: Outgoing stream handler has been closed
 at 
 org.apache.cassandra.streaming.ConnectionHandler.sendMessage(ConnectionHandler.java:170)
 at 
 org.apache.cassandra.streaming.StreamSession.receive(StreamSession.java:483)
 at 
 org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:372)
 at 
 org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:289)
 at java.lang.Thread.run(Thread.java:744)
 ERROR [STREAM-IN-/192.168.21.14] 2014-05-17 01:17:52,350 CassandraDaemon.java 
 (line 198) Exception in thread Thread[STREAM-IN-/192.168.21.14,5,RMI Runtime]
 java.lang.AssertionError: Reference counter -1 for 
 /mnt/ssd1/cassandra/data/ldn_production/historical_accounts/ldn_production-historical_accounts-jb-79827-Data.db
 at 
 org.apache.cassandra.io.sstable.SSTableReader.releaseReference(SSTableReader.java:1107)
 at 
 org.apache.cassandra.streaming.StreamTransferTask.abort(StreamTransferTask.java:80)
 at 
 org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:322)
 at 
 org.apache.cassandra.streaming.StreamSession.onError(StreamSession.java:425)
 at 
 org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:300)
 at java.lang.Thread.run(Thread.java:744)
 followed by a few more (the reference counter got down to -3).  Got the same 
 kind of assertion failure on one other node (in a different data centre; 
 there are 21 nodes altogether distributed over 4 data centres).
 I've attached the relevant part of the log.  It starts quite a bit before the 
 assertion failure at the first exception on this node (Cannot proceed on 
 repair because a neighbor ... is dead), and finishes a few hours afterwards 
 when the node was restarted.
 Edit: The following Reference counter assertion failures followed the 1st on 
 a different file and have a different stack trace:
 ERROR [CompactionExecutor:382] 2014-05-17 01:17:53,157 CassandraDaemon.java 
 (line 198) Exception in thread Thread[CompactionExecutor:382,1,main]
 java.lang.AssertionError: Reference counter -1 for 
 /mnt/ssd1/cassandra/data/ldn_production/historical_accounts/ldn_production-historical_accounts-jb-83888-Data.db
at 
 org.apache.cassandra.io.sstable.SSTableReader.releaseReference(SSTableReader.java:1107)
at 
 org.apache.cassandra.io.sstable.SSTableReader.releaseReferences(SSTableReader.java:1429)
at 
 org.apache.cassandra.db.compaction.CompactionController.close(CompactionController.java:207)
at 
 org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:220)
at 
 org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
at 
 org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60)
at 
 org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
at 
 org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:197)
...



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Comment Edited] (CASSANDRA-7262) During streaming: java.lang.AssertionError: Reference counter -1

2014-05-28 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-7262?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14012015#comment-14012015
 ] 

Rick Branson edited comment on CASSANDRA-7262 at 5/29/14 3:28 AM:
--

Would this cause all of the ReadStage threads to spin on decrementAndGet in 
org.apache.cassandra.io.sstable.SSTableReader.releaseReference?

   at 
java.util.concurrent.atomic.AtomicInteger.decrementAndGet(AtomicInteger.java:222)
   at 
org.apache.cassandra.io.sstable.SSTableReader.releaseReference(SSTableReader.java:1093)
   at 
org.apache.cassandra.io.sstable.SSTableReader.acquireReferences(SSTableReader.java:1420)
   at 
org.apache.cassandra.db.ColumnFamilyStore.markReferenced(ColumnFamilyStore.java:1458)
   at 
org.apache.cassandra.db.ColumnFamilyStore.markReferenced(ColumnFamilyStore.java:1473)
   at 
org.apache.cassandra.db.CollationController.collectTimeOrderedData(CollationController.java:69)


was (Author: rbranson):
Would this cause all of the ReadStage threads to spin on decrementAndGet in 
org.apache.cassandra.io.sstable.SSTableReader.releaseReference?

 During streaming: java.lang.AssertionError: Reference counter -1
 

 Key: CASSANDRA-7262
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7262
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: Cassandra 2.07, x86-64 Ubuntu 12.04.4, Oracle java 
 1.7.0_45
Reporter: Duncan Sands
Assignee: Joshua McKenzie
Priority: Minor
 Fix For: 2.0.9

 Attachments: 7262_v1.txt, 7262_v2.txt, system.log.gz


 Got this assertion failure this weekend during repair:
 ERROR [STREAM-IN-/192.168.21.14] 2014-05-17 01:17:52,332 StreamSession.java 
 (line 420) [Stream #3a3ac8a2-dd50-11e3-b3c1-6bf6dccd6457] Streaming error 
 occurred
 java.lang.RuntimeException: Outgoing stream handler has been closed
 at 
 org.apache.cassandra.streaming.ConnectionHandler.sendMessage(ConnectionHandler.java:170)
 at 
 org.apache.cassandra.streaming.StreamSession.receive(StreamSession.java:483)
 at 
 org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:372)
 at 
 org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:289)
 at java.lang.Thread.run(Thread.java:744)
 ERROR [STREAM-IN-/192.168.21.14] 2014-05-17 01:17:52,350 CassandraDaemon.java 
 (line 198) Exception in thread Thread[STREAM-IN-/192.168.21.14,5,RMI Runtime]
 java.lang.AssertionError: Reference counter -1 for 
 /mnt/ssd1/cassandra/data/ldn_production/historical_accounts/ldn_production-historical_accounts-jb-79827-Data.db
 at 
 org.apache.cassandra.io.sstable.SSTableReader.releaseReference(SSTableReader.java:1107)
 at 
 org.apache.cassandra.streaming.StreamTransferTask.abort(StreamTransferTask.java:80)
 at 
 org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:322)
 at 
 org.apache.cassandra.streaming.StreamSession.onError(StreamSession.java:425)
 at 
 org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:300)
 at java.lang.Thread.run(Thread.java:744)
 followed by a few more (the reference counter got down to -3).  Got the same 
 kind of assertion failure on one other node (in a different data centre; 
 there are 21 nodes altogether distributed over 4 data centres).
 I've attached the relevant part of the log.  It starts quite a bit before the 
 assertion failure at the first exception on this node (Cannot proceed on 
 repair because a neighbor ... is dead), and finishes a few hours afterwards 
 when the node was restarted.
 Edit: The following Reference counter assertion failures followed the 1st on 
 a different file and have a different stack trace:
 ERROR [CompactionExecutor:382] 2014-05-17 01:17:53,157 CassandraDaemon.java 
 (line 198) Exception in thread Thread[CompactionExecutor:382,1,main]
 java.lang.AssertionError: Reference counter -1 for 
 /mnt/ssd1/cassandra/data/ldn_production/historical_accounts/ldn_production-historical_accounts-jb-83888-Data.db
at 
 org.apache.cassandra.io.sstable.SSTableReader.releaseReference(SSTableReader.java:1107)
at 
 org.apache.cassandra.io.sstable.SSTableReader.releaseReferences(SSTableReader.java:1429)
at 
 org.apache.cassandra.db.compaction.CompactionController.close(CompactionController.java:207)
at 
 org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:220)
at 
 org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
at 
 

[jira] [Created] (CASSANDRA-7309) Explicit limit for maximum number of Thrift connections

2014-05-27 Thread Rick Branson (JIRA)
Rick Branson created CASSANDRA-7309:
---

 Summary: Explicit limit for maximum number of Thrift connections
 Key: CASSANDRA-7309
 URL: https://issues.apache.org/jira/browse/CASSANDRA-7309
 Project: Cassandra
  Issue Type: Improvement
Reporter: Rick Branson


The 'sync' Thrift server gets this by way of rpc_max_threads in the YAML 
file, but the 'hsha' server misses out here. Not having a limit here is 
dangerous because it allows badly behaving clients to use up all the file 
handles and prevents Cassandra from opening any files, leaving the daemon dead 
in the water.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-05-19 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6285?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14002142#comment-14002142
 ] 

Rick Branson commented on CASSANDRA-6285:
-

Haven't been able to repro in over 5 days. We're considering the 
enable_reallocate_buffers.txt patch fixed and production-ready.

 2.0 HSHA server introduces corrupt data
 ---

 Key: CASSANDRA-6285
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6285
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: 4 nodes, shortly updated from 1.2.11 to 2.0.2
Reporter: David Sauer
Assignee: Pavel Yaskevich
Priority: Critical
 Fix For: 2.0.8

 Attachments: 6285_testnotes1.txt, 
 CASSANDRA-6285-disruptor-heap.patch, cassandra-attack-src.zip, 
 compaction_test.py, disruptor-high-cpu.patch, 
 disruptor-memory-corruption.patch, enable_reallocate_buffers.txt


 After altering everything to LCS the table OpsCenter.rollups60 amd one other 
 none OpsCenter-Table got stuck with everything hanging around in L0.
 The compaction started and ran until the logs showed this:
 ERROR [CompactionExecutor:111] 2013-11-01 19:14:53,865 CassandraDaemon.java 
 (line 187) Exception in thread Thread[CompactionExecutor:111,1,RMI Runtime]
 java.lang.RuntimeException: Last written key 
 DecoratedKey(1326283851463420237, 
 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574426c6f6f6d46696c746572537061636555736564)
  = current key DecoratedKey(954210699457429663, 
 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574546f74616c4469736b5370616365557365640b0f)
  writing into 
 /var/lib/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-jb-58656-Data.db
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:141)
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:164)
   at 
 org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160)
   at 
 org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
   at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
   at 
 org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60)
   at 
 org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
   at 
 org.apache.cassandra.db.compaction.CompactionManager$6.runMayThrow(CompactionManager.java:296)
   at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
   at 
 java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
   at java.util.concurrent.FutureTask.run(FutureTask.java:262)
   at 
 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
   at java.lang.Thread.run(Thread.java:724)
 Moving back to STC worked to keep the compactions running.
 Especialy my own Table i would like to move to LCS.
 After a major compaction with STC the move to LCS fails with the same 
 Exception.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Reopened] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-05-14 Thread Rick Branson (JIRA)

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

Rick Branson reopened CASSANDRA-6285:
-

Reproduced In: 2.0.6  (was: 2.0.2)

This is not fixed. Still seeing the same exception running 2.0.6.

ERROR [CompactionExecutor:7] 2014-05-09 17:59:58,640 CassandraDaemon.java (line 
196) Exception in thread Thread[CompactionExecutor:7,1,main]
java.lang.RuntimeException: Last written key 
DecoratedKey(132126721345628486111245439753727165857, 0f3b67f2) = current key 
DecoratedKey(37424530135488872684523334498941679307, 196b70ab) writing into 
/data/cassandra/data/redacted/Redacted/redacted-Redacted-tmp-jb-156533-Data.db
at 
org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:142)
at 
org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:165)
at 
org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160)
at 
org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
at 
org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60)
at 
org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
at 
org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:197)
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
at java.util.concurrent.FutureTask.run(FutureTask.java:262)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:744)

 2.0 HSHA server introduces corrupt data
 ---

 Key: CASSANDRA-6285
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6285
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: 4 nodes, shortly updated from 1.2.11 to 2.0.2
Reporter: David Sauer
Assignee: Pavel Yaskevich
Priority: Critical
 Fix For: 2.0.8

 Attachments: 6285_testnotes1.txt, 
 CASSANDRA-6285-disruptor-heap.patch, cassandra-attack-src.zip, 
 compaction_test.py, disruptor-high-cpu.patch, 
 disruptor-memory-corruption.patch, enable_reallocate_buffers.txt


 After altering everything to LCS the table OpsCenter.rollups60 amd one other 
 none OpsCenter-Table got stuck with everything hanging around in L0.
 The compaction started and ran until the logs showed this:
 ERROR [CompactionExecutor:111] 2013-11-01 19:14:53,865 CassandraDaemon.java 
 (line 187) Exception in thread Thread[CompactionExecutor:111,1,RMI Runtime]
 java.lang.RuntimeException: Last written key 
 DecoratedKey(1326283851463420237, 
 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574426c6f6f6d46696c746572537061636555736564)
  = current key DecoratedKey(954210699457429663, 
 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574546f74616c4469736b5370616365557365640b0f)
  writing into 
 /var/lib/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-jb-58656-Data.db
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:141)
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:164)
   at 
 org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160)
   at 
 org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
   at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
   at 
 org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60)
   at 
 org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
   at 
 org.apache.cassandra.db.compaction.CompactionManager$6.runMayThrow(CompactionManager.java:296)
   at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
   at 
 java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
   at java.util.concurrent.FutureTask.run(FutureTask.java:262)
   at 
 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
   at java.lang.Thread.run(Thread.java:724)
 Moving back to STC worked to keep the compactions running.
 Especialy my own Table i would like to move to LCS.
 After a major compaction with STC the move to LCS 

[jira] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-05-13 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6285?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13996712#comment-13996712
 ] 

Rick Branson commented on CASSANDRA-6285:
-

I think what might help this specific quality issue out is just moving to the 
new HSHA implementation entirely in a later version and removing the choice. 
The new HSHA supposedly eliminates the performance issues that made it not a 
good default choice, so it appears as if there's no advantage to having the 
other choices.

 2.0 HSHA server introduces corrupt data
 ---

 Key: CASSANDRA-6285
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6285
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: 4 nodes, shortly updated from 1.2.11 to 2.0.2
Reporter: David Sauer
Assignee: Pavel Yaskevich
Priority: Critical
 Fix For: 2.0.8

 Attachments: 6285_testnotes1.txt, 
 CASSANDRA-6285-disruptor-heap.patch, cassandra-attack-src.zip, 
 compaction_test.py, disruptor-high-cpu.patch, 
 disruptor-memory-corruption.patch, enable_reallocate_buffers.txt


 After altering everything to LCS the table OpsCenter.rollups60 amd one other 
 none OpsCenter-Table got stuck with everything hanging around in L0.
 The compaction started and ran until the logs showed this:
 ERROR [CompactionExecutor:111] 2013-11-01 19:14:53,865 CassandraDaemon.java 
 (line 187) Exception in thread Thread[CompactionExecutor:111,1,RMI Runtime]
 java.lang.RuntimeException: Last written key 
 DecoratedKey(1326283851463420237, 
 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574426c6f6f6d46696c746572537061636555736564)
  = current key DecoratedKey(954210699457429663, 
 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574546f74616c4469736b5370616365557365640b0f)
  writing into 
 /var/lib/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-jb-58656-Data.db
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:141)
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:164)
   at 
 org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160)
   at 
 org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
   at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
   at 
 org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60)
   at 
 org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
   at 
 org.apache.cassandra.db.compaction.CompactionManager$6.runMayThrow(CompactionManager.java:296)
   at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
   at 
 java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
   at java.util.concurrent.FutureTask.run(FutureTask.java:262)
   at 
 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
   at java.lang.Thread.run(Thread.java:724)
 Moving back to STC worked to keep the compactions running.
 Especialy my own Table i would like to move to LCS.
 After a major compaction with STC the move to LCS fails with the same 
 Exception.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-05-13 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6285?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13996899#comment-13996899
 ] 

Rick Branson commented on CASSANDRA-6285:
-

I did some more digging around on our cluster that was running 2.0.6 when it 
saw the corruption: it took anywhere from a few hours to 48 hours for the first 
compaction with the out of order key exception to throw. These nodes are 
receiving thousands of writes per second, so it's not going to be trivially 
reproducible. We've been running one of the nodes with 2.0.8-tenative + 
enable_reallocate_buffers.txt and will report back once we've reached 72 hours 
and are comfortable rolling this out wide to our own clusters.

 2.0 HSHA server introduces corrupt data
 ---

 Key: CASSANDRA-6285
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6285
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: 4 nodes, shortly updated from 1.2.11 to 2.0.2
Reporter: David Sauer
Assignee: Pavel Yaskevich
Priority: Critical
 Fix For: 2.0.8

 Attachments: 6285_testnotes1.txt, 
 CASSANDRA-6285-disruptor-heap.patch, cassandra-attack-src.zip, 
 compaction_test.py, disruptor-high-cpu.patch, 
 disruptor-memory-corruption.patch, enable_reallocate_buffers.txt


 After altering everything to LCS the table OpsCenter.rollups60 amd one other 
 none OpsCenter-Table got stuck with everything hanging around in L0.
 The compaction started and ran until the logs showed this:
 ERROR [CompactionExecutor:111] 2013-11-01 19:14:53,865 CassandraDaemon.java 
 (line 187) Exception in thread Thread[CompactionExecutor:111,1,RMI Runtime]
 java.lang.RuntimeException: Last written key 
 DecoratedKey(1326283851463420237, 
 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574426c6f6f6d46696c746572537061636555736564)
  = current key DecoratedKey(954210699457429663, 
 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574546f74616c4469736b5370616365557365640b0f)
  writing into 
 /var/lib/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-jb-58656-Data.db
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:141)
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:164)
   at 
 org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160)
   at 
 org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
   at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
   at 
 org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60)
   at 
 org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
   at 
 org.apache.cassandra.db.compaction.CompactionManager$6.runMayThrow(CompactionManager.java:296)
   at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
   at 
 java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
   at java.util.concurrent.FutureTask.run(FutureTask.java:262)
   at 
 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
   at java.lang.Thread.run(Thread.java:724)
 Moving back to STC worked to keep the compactions running.
 Especialy my own Table i would like to move to LCS.
 After a major compaction with STC the move to LCS fails with the same 
 Exception.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-6815) Decided if we want to bring back thrift HSHA in 2.0.7

2014-05-11 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6815?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13993814#comment-13993814
 ] 

Rick Branson commented on CASSANDRA-6815:
-

FYI I reopened CASSANDRA-6285 because the fix doesn't fix the issue at all.

 Decided if we want to bring back thrift HSHA in 2.0.7
 -

 Key: CASSANDRA-6815
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6815
 Project: Cassandra
  Issue Type: Improvement
Reporter: Sylvain Lebresne
Assignee: Pavel Yaskevich

 This is the followup of CASSANDRA-6285, to decide what we want to do 
 regarding thrift servers moving forward. My reading of CASSANDRA-6285 
 suggests that the possible options includes:
 # bring back the old HSHA implementation from 1.2 as hsha and make the 
 disruptor implementation be disruptor_hsha.
 # use the new TThreadedSelectorServer from thrift as hsha, making the 
 disruptor implementation disruptor_hsha as above
 # just wait for Pavel to fix the disruptor implementation for off-heap 
 buffers to switch back to that, keeping on-heap buffer until then.
 # keep on-heap buffer for the disruptor implementation and do nothing 
 particular.
 I could be missing some options and we can probably do some mix of those. I 
 don't have a particular opinion to offer on the matter.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-6996) Setting severity via JMX broken

2014-04-22 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6996?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13977068#comment-13977068
 ] 

Rick Branson commented on CASSANDRA-6996:
-

LGTM, works on a test box.

 Setting severity via JMX broken
 ---

 Key: CASSANDRA-6996
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6996
 Project: Cassandra
  Issue Type: Bug
  Components: Tools
Reporter: Rick Branson
Assignee: Vijay
Priority: Minor
 Fix For: 2.0.8

 Attachments: 0001-CASSANDRA-6996.patch


 Looks like setting the Severity attribute in the DynamicEndpointSnitch via 
 JMX is a no-op.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-6961) nodes should go into hibernate when join_ring is false

2014-04-09 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6961?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13964626#comment-13964626
 ] 

Rick Branson commented on CASSANDRA-6961:
-

Very happy about this. It will make a ton of things easier from an operations 
perspective (bringing up new DCs, bringing up hosts after long-ish 
maintenance), but also interested in using this to potentially have dedicated 
coordinator nodes that are separate from storage. We find ourselves CPU bound 
on more capacity-constrained and expensive storage-class hardware. Most of this 
CPU time is spent on request coordination. Moving this work to cheap diskless 
application-class hardware is much more ideal and will allow us to maximize the 
capacity of our storage nodes.

 nodes should go into hibernate when join_ring is false
 --

 Key: CASSANDRA-6961
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6961
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Brandon Williams
Assignee: Brandon Williams
 Fix For: 2.0.7, 2.1 beta2

 Attachments: 6961.txt


 The impetus here is this: a node that was down for some period and comes back 
 can serve stale information.  We know from CASSANDRA-768 that we can't just 
 wait for hints, and know that tangentially related CASSANDRA-3569 prevents us 
 from having the node in a down (from the FD's POV) state handle streaming.
 We can *almost* set join_ring to false, then repair, and then join the ring 
 to narrow the window (actually, you can do this and everything succeeds 
 because the node doesn't know it's a member yet, which is probably a bit of a 
 bug.)  If instead we modified this to put the node in hibernate, like 
 replace_address does, it could work almost like replace, except you could run 
 a repair (manually) while in the hibernate state, and then flip to normal 
 when it's done.
 This won't prevent the staleness 100%, but it will greatly reduce the chance 
 if the node has been down a significant amount of time.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-6757) SliceQueryFilter can accept multiple ColumnSlice but not exposed to thrift

2014-04-09 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6757?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13964961#comment-13964961
 ] 

Rick Branson commented on CASSANDRA-6757:
-

*begs for 2.1 version of this*

 SliceQueryFilter can accept multiple ColumnSlice but not exposed to thrift
 --

 Key: CASSANDRA-6757
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6757
 Project: Cassandra
  Issue Type: New Feature
Reporter: Edward Capriolo
Assignee: Edward Capriolo
 Fix For: 3.0

 Attachments: Cassandra-6757.1.diff


 Thrift has getSlice() which takes a key and a predicate. SliceQueryFilter can 
 accept multiple ColumnSlice(s). Users should have access to this capability 
 already found in Cassandra.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Created] (CASSANDRA-6992) Bootstrap on vnodes clusters can cause stampeding/storm behavior

2014-04-07 Thread Rick Branson (JIRA)
Rick Branson created CASSANDRA-6992:
---

 Summary: Bootstrap on vnodes clusters can cause stampeding/storm 
behavior
 Key: CASSANDRA-6992
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6992
 Project: Cassandra
  Issue Type: Bug
 Environment: Various vnodes-enabled clusters in EC2, m1.xlarge and 
hi1.4xlarge, ~3000-8000 tokens.
Reporter: Rick Branson


Assuming this is an issue with vnodes clusters because 
SSTableReader#getPositionsForRanges is more expensive to compute with 256x the 
ranges, but could be wrong. On even well-provisioned hosts, this can cause a 
severe spike in network throughput  CPU utilization from a storm of flushes, 
which impacts long-tail times pretty badly. On weaker hosts (like m1.xlarge 
with ~500GB of data), it can result in minutes of churn while the node gets 
through StreamOut#createPendingFiles. This *might* be better in 2.0, but it's 
probably still reproducible because the bootstrapping node sends out all of 
it's streaming requests at once. 

I'm thinking that this could be staggered at the bootstrapping node to avoid 
the simultaneous spike across the whole cluster. Not sure on how to stagger it 
besides something very naive like one-at-a-time with a pause. Maybe this should 
also be throttled in StreamOut#createPendingFiles on the out-streaming host? 
Any thoughts?

From the stack dump of one of our weaker nodes that was struggling for a few 
minutes just starting the StreamOut:

MiscStage:1 daemon prio=10 tid=0x0292f000 nid=0x688 runnable 
[0x7f7b03df6000]
   java.lang.Thread.State: RUNNABLE
at 
org.apache.cassandra.utils.ByteBufferUtil.readShortLength(ByteBufferUtil.java:361)
at 
org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:371)
at 
org.apache.cassandra.io.sstable.IndexHelper$IndexInfo.deserialize(IndexHelper.java:187)
at 
org.apache.cassandra.db.RowIndexEntry$Serializer.deserialize(RowIndexEntry.java:125)
at 
org.apache.cassandra.io.sstable.SSTableReader.getPosition(SSTableReader.java:889)
at 
org.apache.cassandra.io.sstable.SSTableReader.getPosition(SSTableReader.java:790)
at 
org.apache.cassandra.io.sstable.SSTableReader.getPositionsForRanges(SSTableReader.java:730)
at 
org.apache.cassandra.streaming.StreamOut.createPendingFiles(StreamOut.java:172)
at 
org.apache.cassandra.streaming.StreamOut.transferSSTables(StreamOut.java:157)
at 
org.apache.cassandra.streaming.StreamOut.transferRanges(StreamOut.java:148)
at 
org.apache.cassandra.streaming.StreamOut.transferRanges(StreamOut.java:116)
at 
org.apache.cassandra.streaming.StreamRequestVerbHandler.doVerb(StreamRequestVerbHandler.java:44)
at 
org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:56)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:895)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918)
at java.lang.Thread.run(Thread.java:662)



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Created] (CASSANDRA-6996) Setting severity via JMX broken

2014-04-07 Thread Rick Branson (JIRA)
Rick Branson created CASSANDRA-6996:
---

 Summary: Setting severity via JMX broken
 Key: CASSANDRA-6996
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6996
 Project: Cassandra
  Issue Type: Bug
Reporter: Rick Branson
Priority: Minor


Looks like setting the Severity attribute in the DynamicEndpointSnitch via JMX 
is a no-op.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-6685) Nodes never bootstrap if schema is empty

2014-03-22 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6685?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13944211#comment-13944211
 ] 

Rick Branson commented on CASSANDRA-6685:
-

Ran into this today trying to test out some new machines. It's not good that 
we've just been sitting on this for 6 weeks now.

 Nodes never bootstrap if schema is empty
 

 Key: CASSANDRA-6685
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6685
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Richard Low
Assignee: Brandon Williams
 Fix For: 1.2.16


 Since 1.2.15, bootstrap never completes if the schema is empty. The 
 bootstrapping node endlessly prints:
 bq. {{INFO 12:37:44,863 JOINING: waiting for schema information to complete}}
 until you add something to the schema (i.e. create a keyspace).
 The problem looks to be caused by CASSANDRA-6648, where 
 MigrationManager.isReadForBootstrap() was changed to:
 bq. {{return Schema.instance.getVersion() != null  
 !Schema.emptyVersion.equals(Schema.instance.getVersion());}}
 This is wrong since 
 {{Schema.emptyVersion.equals(Schema.instance.getVersion())}} is always true 
 if there is no schema.
 We need some different logic for determining when the schema is propagated.
 I haven't tested, but I expect this issue appears in 2.0.5 too.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-06 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6285?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13923245#comment-13923245
 ] 

Rick Branson commented on CASSANDRA-6285:
-

Unfortunately this hit us during our 2.0.5 upgrade and 'sync' is not an option 
for the # of connections we have per node (tried this). We've been running 
Marcus' patch in prod and limping along on it, but it looks like the 
requestInvoke() override is causing the requests to get executed on the 
selector pool (which is limited by CPU #) instead of the executor service so 
our response times are pretty bad. The lack of anything showing up in the JMX 
for the executor service definitely points towards this. 

 2.0 HSHA server introduces corrupt data
 ---

 Key: CASSANDRA-6285
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6285
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: 4 nodes, shortly updated from 1.2.11 to 2.0.2
Reporter: David Sauer
Assignee: Pavel Yaskevich
Priority: Critical
 Fix For: 2.0.6

 Attachments: 6285_testnotes1.txt, 
 CASSANDRA-6285-disruptor-heap.patch, compaction_test.py


 After altering everything to LCS the table OpsCenter.rollups60 amd one other 
 none OpsCenter-Table got stuck with everything hanging around in L0.
 The compaction started and ran until the logs showed this:
 ERROR [CompactionExecutor:111] 2013-11-01 19:14:53,865 CassandraDaemon.java 
 (line 187) Exception in thread Thread[CompactionExecutor:111,1,RMI Runtime]
 java.lang.RuntimeException: Last written key 
 DecoratedKey(1326283851463420237, 
 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574426c6f6f6d46696c746572537061636555736564)
  = current key DecoratedKey(954210699457429663, 
 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574546f74616c4469736b5370616365557365640b0f)
  writing into 
 /var/lib/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-jb-58656-Data.db
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:141)
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:164)
   at 
 org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160)
   at 
 org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
   at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
   at 
 org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60)
   at 
 org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
   at 
 org.apache.cassandra.db.compaction.CompactionManager$6.runMayThrow(CompactionManager.java:296)
   at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
   at 
 java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
   at java.util.concurrent.FutureTask.run(FutureTask.java:262)
   at 
 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
   at java.lang.Thread.run(Thread.java:724)
 Moving back to STC worked to keep the compactions running.
 Especialy my own Table i would like to move to LCS.
 After a major compaction with STC the move to LCS fails with the same 
 Exception.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-06 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6285?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13923441#comment-13923441
 ] 

Rick Branson commented on CASSANDRA-6285:
-

We put in the TThreadedSelectorServer patch from Marcus. On top of that, to get 
our response times down from the 10x what they should be, I rolled out a larger 
hard-coded selector thread pool size of 256 (instead of the # of processors -- 
a measly 16). This is shaping up nicely.

 2.0 HSHA server introduces corrupt data
 ---

 Key: CASSANDRA-6285
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6285
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: 4 nodes, shortly updated from 1.2.11 to 2.0.2
Reporter: David Sauer
Assignee: Pavel Yaskevich
Priority: Critical
 Fix For: 2.0.6

 Attachments: 6285_testnotes1.txt, 
 CASSANDRA-6285-disruptor-heap.patch, compaction_test.py


 After altering everything to LCS the table OpsCenter.rollups60 amd one other 
 none OpsCenter-Table got stuck with everything hanging around in L0.
 The compaction started and ran until the logs showed this:
 ERROR [CompactionExecutor:111] 2013-11-01 19:14:53,865 CassandraDaemon.java 
 (line 187) Exception in thread Thread[CompactionExecutor:111,1,RMI Runtime]
 java.lang.RuntimeException: Last written key 
 DecoratedKey(1326283851463420237, 
 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574426c6f6f6d46696c746572537061636555736564)
  = current key DecoratedKey(954210699457429663, 
 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574546f74616c4469736b5370616365557365640b0f)
  writing into 
 /var/lib/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-jb-58656-Data.db
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:141)
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:164)
   at 
 org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160)
   at 
 org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
   at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
   at 
 org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60)
   at 
 org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
   at 
 org.apache.cassandra.db.compaction.CompactionManager$6.runMayThrow(CompactionManager.java:296)
   at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
   at 
 java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
   at java.util.concurrent.FutureTask.run(FutureTask.java:262)
   at 
 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
   at java.lang.Thread.run(Thread.java:724)
 Moving back to STC worked to keep the compactions running.
 Especialy my own Table i would like to move to LCS.
 After a major compaction with STC the move to LCS fails with the same 
 Exception.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-06 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6285?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13923466#comment-13923466
 ] 

Rick Branson commented on CASSANDRA-6285:
-

[~xedin]: We had perf issues with the disruptor as well (sudden spikes of CPU 
to 100%) + this so I just wanted to get production away from it ASAP.

 2.0 HSHA server introduces corrupt data
 ---

 Key: CASSANDRA-6285
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6285
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: 4 nodes, shortly updated from 1.2.11 to 2.0.2
Reporter: David Sauer
Assignee: Pavel Yaskevich
Priority: Critical
 Fix For: 2.0.6

 Attachments: 6285_testnotes1.txt, 
 CASSANDRA-6285-disruptor-heap.patch, compaction_test.py


 After altering everything to LCS the table OpsCenter.rollups60 amd one other 
 none OpsCenter-Table got stuck with everything hanging around in L0.
 The compaction started and ran until the logs showed this:
 ERROR [CompactionExecutor:111] 2013-11-01 19:14:53,865 CassandraDaemon.java 
 (line 187) Exception in thread Thread[CompactionExecutor:111,1,RMI Runtime]
 java.lang.RuntimeException: Last written key 
 DecoratedKey(1326283851463420237, 
 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574426c6f6f6d46696c746572537061636555736564)
  = current key DecoratedKey(954210699457429663, 
 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574546f74616c4469736b5370616365557365640b0f)
  writing into 
 /var/lib/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-jb-58656-Data.db
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:141)
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:164)
   at 
 org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160)
   at 
 org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
   at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
   at 
 org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60)
   at 
 org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
   at 
 org.apache.cassandra.db.compaction.CompactionManager$6.runMayThrow(CompactionManager.java:296)
   at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
   at 
 java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
   at java.util.concurrent.FutureTask.run(FutureTask.java:262)
   at 
 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
   at java.lang.Thread.run(Thread.java:724)
 Moving back to STC worked to keep the compactions running.
 Especialy my own Table i would like to move to LCS.
 After a major compaction with STC the move to LCS fails with the same 
 Exception.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-06 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6285?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13923514#comment-13923514
 ] 

Rick Branson commented on CASSANDRA-6285:
-

[~xedin]: off-heap for disruptor.

I think that we should really consider bringing back the old HSHA 
implementation from 1.2 as the hsha and allow switching to the disruptor 
implementation as another rpc_server_type for those that want to try it out.

 2.0 HSHA server introduces corrupt data
 ---

 Key: CASSANDRA-6285
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6285
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: 4 nodes, shortly updated from 1.2.11 to 2.0.2
Reporter: David Sauer
Assignee: Pavel Yaskevich
Priority: Critical
 Fix For: 2.0.6

 Attachments: 6285_testnotes1.txt, 
 CASSANDRA-6285-disruptor-heap.patch, compaction_test.py


 After altering everything to LCS the table OpsCenter.rollups60 amd one other 
 none OpsCenter-Table got stuck with everything hanging around in L0.
 The compaction started and ran until the logs showed this:
 ERROR [CompactionExecutor:111] 2013-11-01 19:14:53,865 CassandraDaemon.java 
 (line 187) Exception in thread Thread[CompactionExecutor:111,1,RMI Runtime]
 java.lang.RuntimeException: Last written key 
 DecoratedKey(1326283851463420237, 
 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574426c6f6f6d46696c746572537061636555736564)
  = current key DecoratedKey(954210699457429663, 
 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574546f74616c4469736b5370616365557365640b0f)
  writing into 
 /var/lib/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-jb-58656-Data.db
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:141)
   at 
 org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:164)
   at 
 org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160)
   at 
 org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
   at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
   at 
 org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60)
   at 
 org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
   at 
 org.apache.cassandra.db.compaction.CompactionManager$6.runMayThrow(CompactionManager.java:296)
   at 
 org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
   at 
 java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
   at java.util.concurrent.FutureTask.run(FutureTask.java:262)
   at 
 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
   at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
   at java.lang.Thread.run(Thread.java:724)
 Moving back to STC worked to keep the compactions running.
 Especialy my own Table i would like to move to LCS.
 After a major compaction with STC the move to LCS fails with the same 
 Exception.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Updated] (CASSANDRA-6592) IllegalArgumentException when Preparing Statements

2014-01-23 Thread Rick Branson (JIRA)

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

Rick Branson updated CASSANDRA-6592:


Priority: Critical  (was: Minor)

 IllegalArgumentException when Preparing Statements
 --

 Key: CASSANDRA-6592
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6592
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Tyler Hobbs
Assignee: Lyuben Todorov
Priority: Critical
 Fix For: 1.2.14, 2.0.5


 When preparing a lot of statements with the python native driver, I 
 occasionally get an error response with an error that corresponds to the 
 following stacktrace in the cassandra logs:
 {noformat}
 ERROR [Native-Transport-Requests:126] 2014-01-11 13:58:05,503 
 ErrorMessage.java (line 210) Unexpected exception during request
 java.lang.IllegalArgumentException
 at 
 com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.checkArgument(ConcurrentLinkedHashMap.java:259)
 at 
 com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$BoundedEntryWeigher.weightOf(ConcurrentLinkedHashMap.java:1448)
 at 
 com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.put(ConcurrentLinkedHashMap.java:764)
 at 
 com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap.put(ConcurrentLinkedHashMap.java:743)
 at 
 org.apache.cassandra.cql3.QueryProcessor.storePreparedStatement(QueryProcessor.java:255)
 at 
 org.apache.cassandra.cql3.QueryProcessor.prepare(QueryProcessor.java:221)
 at 
 org.apache.cassandra.transport.messages.PrepareMessage.execute(PrepareMessage.java:77)
 at 
 org.apache.cassandra.transport.Message$Dispatcher.messageReceived(Message.java:287)
 at 
 org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70)
 at 
 org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564)
 at 
 org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791)
 at 
 org.jboss.netty.handler.execution.ChannelUpstreamEventRunnable.doRun(ChannelUpstreamEventRunnable.java:43)
 at 
 org.jboss.netty.handler.execution.ChannelEventRunnable.run(ChannelEventRunnable.java:67)
 at 
 java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:895)
 at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918)
 at java.lang.Thread.run(Thread.java:662)
 {noformat}
 Looking at the CLHM source, this means we're giving the statement a weight 
 that's less than 1.  I'll also note that these errors frequently happen in 
 clumps of 2 or 3 at a time.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (CASSANDRA-6569) Batchlog replays copy the entire batchlog table into the heap

2014-01-13 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6569?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13869618#comment-13869618
 ] 

Rick Branson commented on CASSANDRA-6569:
-

Looks like deleteBatch got switched off the CQL code-path. This seems 
inconsistent from the rest of this replay logic.

 Batchlog replays copy the entire batchlog table into the heap
 -

 Key: CASSANDRA-6569
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6569
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Rick Branson
Assignee: Aleksey Yeschenko
 Fix For: 1.2.14, 2.0.5

 Attachments: 6569.txt


 The current batchlog replay path will read the entire batchlog table into the 
 heap. This is pretty bad. This was compounded by CASSANDRA-5762, which caused 
 the SELECT statement used by the batchlog replay to bring the entire row into 
 memory instead of just the selected columns.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Commented] (CASSANDRA-6569) Batchlog replays copy the entire batchlog table into the heap

2014-01-13 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6569?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13869620#comment-13869620
 ] 

Rick Branson commented on CASSANDRA-6569:
-

Also, would it be possible to get some INFO-level logging (like HH has) on 
batchlog replay while we're here?

 Batchlog replays copy the entire batchlog table into the heap
 -

 Key: CASSANDRA-6569
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6569
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Rick Branson
Assignee: Aleksey Yeschenko
 Fix For: 1.2.14, 2.0.5

 Attachments: 6569.txt


 The current batchlog replay path will read the entire batchlog table into the 
 heap. This is pretty bad. This was compounded by CASSANDRA-5762, which caused 
 the SELECT statement used by the batchlog replay to bring the entire row into 
 memory instead of just the selected columns.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Created] (CASSANDRA-6569) Batchlog replays copy the entire batchlog table into the heap

2014-01-10 Thread Rick Branson (JIRA)
Rick Branson created CASSANDRA-6569:
---

 Summary: Batchlog replays copy the entire batchlog table into the 
heap
 Key: CASSANDRA-6569
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6569
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Rick Branson
Assignee: Aleksey Yeschenko


The current batchlog replay path will read the entire batchlog table into the 
heap. This is pretty bad. This was compounded by CASSANDRA-5762, which caused 
the SELECT statement used by the batchlog replay to bring the entire row into 
memory instead of just the selected columns.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Created] (CASSANDRA-6551) Rack-aware batchlog replication

2014-01-03 Thread Rick Branson (JIRA)
Rick Branson created CASSANDRA-6551:
---

 Summary: Rack-aware batchlog replication
 Key: CASSANDRA-6551
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6551
 Project: Cassandra
  Issue Type: Improvement
Reporter: Rick Branson
Assignee: Aleksey Yeschenko


Right now the batchlog replication code just randomly picks 2 other nodes in 
the same DC, regardless of rack. Ideally we'd pick 2 replicas in other racks to 
achieve higher fault tolerance.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Updated] (CASSANDRA-6551) Rack-aware batchlog replication

2014-01-03 Thread Rick Branson (JIRA)

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

Rick Branson updated CASSANDRA-6551:


  Component/s: Core
Reproduced In: 2.0.4, 1.2.13
Fix Version/s: 2.0.5
   1.2.14

 Rack-aware batchlog replication
 ---

 Key: CASSANDRA-6551
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6551
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Rick Branson
Assignee: Aleksey Yeschenko
 Fix For: 1.2.14, 2.0.5


 Right now the batchlog replication code just randomly picks 2 other nodes in 
 the same DC, regardless of rack. Ideally we'd pick 2 replicas in other racks 
 to achieve higher fault tolerance.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Updated] (CASSANDRA-6488) Batchlog writes consume unnecessarily large amounts of CPU on vnodes clusters

2013-12-13 Thread Rick Branson (JIRA)

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

Rick Branson updated CASSANDRA-6488:


Attachment: graph (21).png

CPU usage dropping on a production cluster after the attached patch is rolled 
out.

 Batchlog writes consume unnecessarily large amounts of CPU on vnodes clusters
 -

 Key: CASSANDRA-6488
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6488
 Project: Cassandra
  Issue Type: Bug
Reporter: Rick Branson
Assignee: Aleksey Yeschenko
 Attachments: 6488-rbranson-patch.txt, graph (21).png


 The cloneTokenOnlyMap call in StorageProxy.getBatchlogEndpoints causes 
 enormous amounts of CPU to be consumed on clusters with many vnodes. I 
 created a patch to cache this data as a workaround and deployed it to a 
 production cluster with 15,000 tokens. CPU consumption drop to 1/5th. This 
 highlights the overall issues with cloneOnlyTokenMap() calls on vnodes 
 clusters. I'm including the maybe-not-the-best-quality workaround patch to 
 use as a reference, but cloneOnlyTokenMap is a systemic issue and every place 
 it's called should probably be investigated.



--
This message was sent by Atlassian JIRA
(v6.1.4#6159)


[jira] [Created] (CASSANDRA-6488) Batchlog writes consume unnecessarily large amounts of CPU on vnodes clusters

2013-12-13 Thread Rick Branson (JIRA)
Rick Branson created CASSANDRA-6488:
---

 Summary: Batchlog writes consume unnecessarily large amounts of 
CPU on vnodes clusters
 Key: CASSANDRA-6488
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6488
 Project: Cassandra
  Issue Type: Bug
Reporter: Rick Branson
Assignee: Aleksey Yeschenko
 Attachments: 6488-rbranson-patch.txt, graph (21).png

The cloneTokenOnlyMap call in StorageProxy.getBatchlogEndpoints causes enormous 
amounts of CPU to be consumed on clusters with many vnodes. I created a patch 
to cache this data as a workaround and deployed it to a production cluster with 
15,000 tokens. CPU consumption drop to 1/5th. This highlights the overall 
issues with cloneOnlyTokenMap() calls on vnodes clusters. I'm including the 
maybe-not-the-best-quality workaround patch to use as a reference, but 
cloneOnlyTokenMap is a systemic issue and every place it's called should 
probably be investigated.



--
This message was sent by Atlassian JIRA
(v6.1.4#6159)


[jira] [Updated] (CASSANDRA-6488) Batchlog writes consume unnecessarily large amounts of CPU on vnodes clusters

2013-12-13 Thread Rick Branson (JIRA)

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

Rick Branson updated CASSANDRA-6488:


Attachment: 6488-rbranson-patch.txt

 Batchlog writes consume unnecessarily large amounts of CPU on vnodes clusters
 -

 Key: CASSANDRA-6488
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6488
 Project: Cassandra
  Issue Type: Bug
Reporter: Rick Branson
Assignee: Aleksey Yeschenko
 Attachments: 6488-rbranson-patch.txt, graph (21).png


 The cloneTokenOnlyMap call in StorageProxy.getBatchlogEndpoints causes 
 enormous amounts of CPU to be consumed on clusters with many vnodes. I 
 created a patch to cache this data as a workaround and deployed it to a 
 production cluster with 15,000 tokens. CPU consumption drop to 1/5th. This 
 highlights the overall issues with cloneOnlyTokenMap() calls on vnodes 
 clusters. I'm including the maybe-not-the-best-quality workaround patch to 
 use as a reference, but cloneOnlyTokenMap is a systemic issue and every place 
 it's called should probably be investigated.



--
This message was sent by Atlassian JIRA
(v6.1.4#6159)


[jira] [Commented] (CASSANDRA-6485) NPE in calculateNaturalEndpoints

2013-12-13 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6485?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13848118#comment-13848118
 ] 

Rick Branson commented on CASSANDRA-6485:
-

LGTM.

 NPE in calculateNaturalEndpoints
 

 Key: CASSANDRA-6485
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6485
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Russell Alexander Spitzer
Assignee: Jonathan Ellis
 Fix For: 1.2.13

 Attachments: 6485.txt


 I was running a test where I added a new data center to an existing cluster. 
 Test outline:
 Start 25 Node DC1
 Keyspace Setup Replication 3
 Begin insert against DC1 Using Stress
 While the inserts are occuring
 Start up 25 Node DC2
 Alter Keyspace to include Replication in 2nd DC
 Run rebuild on DC2
 Wait for stress to finish
 Run repair on Cluster
 ... Some other operations
 Although there are no issues with smaller clusters or clusters without 
 vnodes, Larger setups with vnodes seem to consistently see the following 
 exception in the logs as well as a write operation failing for each 
 exception. Usually this happens between 1-8 times during an experiment. 
 The exceptions/failures are Occurring when DC2 is brought online but *before* 
 any alteration of the Keyspace. All of the exceptions are happening on DC1 
 nodes. One of the exceptions occurred on a seed node though this doesn't seem 
 to be the case most of the time. 
 While the test was running, nodetool was run every second to get cluster 
 status. At no time did any nodes report themselves as down. 
 {code}
 ystem_logs-107.21.186.208/system.log-ERROR [Thrift:1] 2013-12-13 06:19:52,647 
 CustomTThreadPoolServer.java (line 217) Error occurred during processing of 
 message.
 system_logs-107.21.186.208/system.log:java.lang.NullPointerException
 system_logs-107.21.186.208/system.log-at 
 org.apache.cassandra.locator.AbstractReplicationStrategy.getNaturalEndpoints(AbstractReplicationStrategy.java:128)
 system_logs-107.21.186.208/system.log-at 
 org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:2624)
 system_logs-107.21.186.208/system.log-at 
 org.apache.cassandra.service.StorageProxy.performWrite(StorageProxy.java:375)
 system_logs-107.21.186.208/system.log-at 
 org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:190)
 system_logs-107.21.186.208/system.log-at 
 org.apache.cassandra.thrift.CassandraServer.doInsert(CassandraServer.java:866)
 system_logs-107.21.186.208/system.log-at 
 org.apache.cassandra.thrift.CassandraServer.doInsert(CassandraServer.java:849)
 system_logs-107.21.186.208/system.log-at 
 org.apache.cassandra.thrift.CassandraServer.batch_mutate(CassandraServer.java:749)
 system_logs-107.21.186.208/system.log-at 
 org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.getResult(Cassandra.java:3690)
 system_logs-107.21.186.208/system.log-at 
 org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.getResult(Cassandra.java:3678)
 system_logs-107.21.186.208/system.log-at 
 org.apache.thrift.ProcessFunction.process(ProcessFunction.java:32)
 system_logs-107.21.186.208/system.log-at 
 org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:34)
 system_logs-107.21.186.208/system.log-at 
 org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:199)
 system_logs-107.21.186.208/system.log-at 
 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
 system_logs-107.21.186.208/system.log-at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
 system_logs-107.21.186.208/system.log-at 
 java.lang.Thread.run(Thread.java:724)
 {code}



--
This message was sent by Atlassian JIRA
(v6.1.4#6159)


[jira] [Commented] (CASSANDRA-6345) Endpoint cache invalidation causes CPU spike (on vnode rings?)

2013-11-26 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6345?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13832783#comment-13832783
 ] 

Rick Branson commented on CASSANDRA-6345:
-

Thanks for taking the time to explain the consistency story. It makes perfect 
sense. 

My defensiveness comment suggested bumping the version number (this is 
practically free) each time the TM write lock is released, which would be in 
addition to the existing invalidations. You're probably a much better gauge on 
the usefulness of this, so up to you.

Really nice that the v5 patch is so compact. Two minor comments: the 
endpointsLock declaration is still in there, and not to be all nitpicky but 
there are two typos in the comments (wo we keep and clone got invalidted).

 Endpoint cache invalidation causes CPU spike (on vnode rings?)
 --

 Key: CASSANDRA-6345
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6345
 Project: Cassandra
  Issue Type: Bug
 Environment: 30 nodes total, 2 DCs
 Cassandra 1.2.11
 vnodes enabled (256 per node)
Reporter: Rick Branson
Assignee: Jonathan Ellis
 Fix For: 1.2.13

 Attachments: 6345-rbranson-v2.txt, 6345-rbranson.txt, 6345-v2.txt, 
 6345-v3.txt, 6345-v4.txt, 6345-v5.txt, 6345.txt, 
 half-way-thru-6345-rbranson-patch-applied.png


 We've observed that events which cause invalidation of the endpoint cache 
 (update keyspace, add/remove nodes, etc) in AbstractReplicationStrategy 
 result in several seconds of thundering herd behavior on the entire cluster. 
 A thread dump shows over a hundred threads (I stopped counting at that point) 
 with a backtrace like this:
 at java.net.Inet4Address.getAddress(Inet4Address.java:288)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:106)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:103)
 at java.util.TreeMap.getEntryUsingComparator(TreeMap.java:351)
 at java.util.TreeMap.getEntry(TreeMap.java:322)
 at java.util.TreeMap.get(TreeMap.java:255)
 at 
 com.google.common.collect.AbstractMultimap.put(AbstractMultimap.java:200)
 at 
 com.google.common.collect.AbstractSetMultimap.put(AbstractSetMultimap.java:117)
 at com.google.common.collect.TreeMultimap.put(TreeMultimap.java:74)
 at 
 com.google.common.collect.AbstractMultimap.putAll(AbstractMultimap.java:273)
 at com.google.common.collect.TreeMultimap.putAll(TreeMultimap.java:74)
 at 
 org.apache.cassandra.utils.SortedBiMultiValMap.create(SortedBiMultiValMap.java:60)
 at 
 org.apache.cassandra.locator.TokenMetadata.cloneOnlyTokenMap(TokenMetadata.java:598)
 at 
 org.apache.cassandra.locator.AbstractReplicationStrategy.getNaturalEndpoints(AbstractReplicationStrategy.java:104)
 at 
 org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:2671)
 at 
 org.apache.cassandra.service.StorageProxy.performWrite(StorageProxy.java:375)
 It looks like there's a large amount of cost in the 
 TokenMetadata.cloneOnlyTokenMap that 
 AbstractReplicationStrategy.getNaturalEndpoints is calling each time there is 
 a cache miss for an endpoint. It seems as if this would only impact clusters 
 with large numbers of tokens, so it's probably a vnodes-only issue.
 Proposal: In AbstractReplicationStrategy.getNaturalEndpoints(), cache the 
 cloned TokenMetadata instance returned by TokenMetadata.cloneOnlyTokenMap(), 
 wrapping it with a lock to prevent stampedes, and clearing it in 
 clearEndpointCache(). Thoughts?



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Comment Edited] (CASSANDRA-6345) Endpoint cache invalidation causes CPU spike (on vnode rings?)

2013-11-26 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6345?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13832783#comment-13832783
 ] 

Rick Branson edited comment on CASSANDRA-6345 at 11/26/13 5:34 PM:
---

Thanks for taking the time to explain the consistency story. It makes perfect 
sense. 

My defensiveness comment suggested bumping the version number each time the TM 
write lock is released, which would be in addition to the existing 
invalidations. You're probably a much better gauge on the usefulness of this, 
so up to you.

Really nice that the v5 patch is so compact. Two minor comments: the 
endpointsLock declaration is still in there, and not to be all nitpicky but 
there are two typos in the comments (wo we keep and clone got invalidted).


was (Author: rbranson):
Thanks for taking the time to explain the consistency story. It makes perfect 
sense. 

My defensiveness comment suggested bumping the version number (this is 
practically free) each time the TM write lock is released, which would be in 
addition to the existing invalidations. You're probably a much better gauge on 
the usefulness of this, so up to you.

Really nice that the v5 patch is so compact. Two minor comments: the 
endpointsLock declaration is still in there, and not to be all nitpicky but 
there are two typos in the comments (wo we keep and clone got invalidted).

 Endpoint cache invalidation causes CPU spike (on vnode rings?)
 --

 Key: CASSANDRA-6345
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6345
 Project: Cassandra
  Issue Type: Bug
 Environment: 30 nodes total, 2 DCs
 Cassandra 1.2.11
 vnodes enabled (256 per node)
Reporter: Rick Branson
Assignee: Jonathan Ellis
 Fix For: 1.2.13

 Attachments: 6345-rbranson-v2.txt, 6345-rbranson.txt, 6345-v2.txt, 
 6345-v3.txt, 6345-v4.txt, 6345-v5.txt, 6345.txt, 
 half-way-thru-6345-rbranson-patch-applied.png


 We've observed that events which cause invalidation of the endpoint cache 
 (update keyspace, add/remove nodes, etc) in AbstractReplicationStrategy 
 result in several seconds of thundering herd behavior on the entire cluster. 
 A thread dump shows over a hundred threads (I stopped counting at that point) 
 with a backtrace like this:
 at java.net.Inet4Address.getAddress(Inet4Address.java:288)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:106)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:103)
 at java.util.TreeMap.getEntryUsingComparator(TreeMap.java:351)
 at java.util.TreeMap.getEntry(TreeMap.java:322)
 at java.util.TreeMap.get(TreeMap.java:255)
 at 
 com.google.common.collect.AbstractMultimap.put(AbstractMultimap.java:200)
 at 
 com.google.common.collect.AbstractSetMultimap.put(AbstractSetMultimap.java:117)
 at com.google.common.collect.TreeMultimap.put(TreeMultimap.java:74)
 at 
 com.google.common.collect.AbstractMultimap.putAll(AbstractMultimap.java:273)
 at com.google.common.collect.TreeMultimap.putAll(TreeMultimap.java:74)
 at 
 org.apache.cassandra.utils.SortedBiMultiValMap.create(SortedBiMultiValMap.java:60)
 at 
 org.apache.cassandra.locator.TokenMetadata.cloneOnlyTokenMap(TokenMetadata.java:598)
 at 
 org.apache.cassandra.locator.AbstractReplicationStrategy.getNaturalEndpoints(AbstractReplicationStrategy.java:104)
 at 
 org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:2671)
 at 
 org.apache.cassandra.service.StorageProxy.performWrite(StorageProxy.java:375)
 It looks like there's a large amount of cost in the 
 TokenMetadata.cloneOnlyTokenMap that 
 AbstractReplicationStrategy.getNaturalEndpoints is calling each time there is 
 a cache miss for an endpoint. It seems as if this would only impact clusters 
 with large numbers of tokens, so it's probably a vnodes-only issue.
 Proposal: In AbstractReplicationStrategy.getNaturalEndpoints(), cache the 
 cloned TokenMetadata instance returned by TokenMetadata.cloneOnlyTokenMap(), 
 wrapping it with a lock to prevent stampedes, and clearing it in 
 clearEndpointCache(). Thoughts?



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (CASSANDRA-5357) Query cache

2013-11-26 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5357?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13832901#comment-13832901
 ] 

Rick Branson commented on CASSANDRA-5357:
-

Perhaps an anecdote from a production system might help find a simple, yet 
useful improvement to the row cache. Facebook's TAO distributed storage system 
supports a data model called assocs which are basically just graph edges, and 
nodes assigned to a given assoc ID hold a write-through cache of the state. The 
assoc storage can be roughly considered a more use-case specific CF. For large 
assocs with many thousands of edges, TAO only maintains the tail of the assoc 
in memory, as those tend to be the most interesting portions of data. More of 
the details are discussed in the linked paper[1].

Perhaps instead of a total overhaul, what's really needed to evolve the row 
cache by modifying it to only cache the head of the row and it's bounds. In 
contrast to the complexity of trying to match queries  mutations to a set of 
serialized query filter objects, the cache only needs to maintain one interval 
for each row at most. This would provide a very simple write-through story. 
After reviewing our production wide row use cases, they seem to fall into two 
camps. The first and most read-performance sensitive is vastly skewed towards 
reads on the head of the row (90% of the time) with a fixed limit. The second 
is randomly distributed slice queries which would not seem to provide a very 
good cache hit rate either way.

[1] https://www.usenix.org/conference/atc13/technical-sessions/papers/bronson)

 Query cache
 ---

 Key: CASSANDRA-5357
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5357
 Project: Cassandra
  Issue Type: Bug
Reporter: Jonathan Ellis
Assignee: Vijay

 I think that most people expect the row cache to act like a query cache, 
 because that's a reasonable model.  Caching the entire partition is, in 
 retrospect, not really reasonable, so it's not surprising that it catches 
 people off guard, especially given the confusion we've inflicted on ourselves 
 as to what a row constitutes.
 I propose replacing it with a true query cache.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (CASSANDRA-6345) Endpoint cache invalidation causes CPU spike (on vnode rings?)

2013-11-26 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6345?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13833150#comment-13833150
 ] 

Rick Branson commented on CASSANDRA-6345:
-

LGTM!

 Endpoint cache invalidation causes CPU spike (on vnode rings?)
 --

 Key: CASSANDRA-6345
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6345
 Project: Cassandra
  Issue Type: Bug
 Environment: 30 nodes total, 2 DCs
 Cassandra 1.2.11
 vnodes enabled (256 per node)
Reporter: Rick Branson
Assignee: Jonathan Ellis
 Fix For: 1.2.13, 2.0.4

 Attachments: 6345-rbranson-v2.txt, 6345-rbranson.txt, 6345-v2.txt, 
 6345-v3.txt, 6345-v4.txt, 6345-v5.txt, 6345.txt, 
 half-way-thru-6345-rbranson-patch-applied.png


 We've observed that events which cause invalidation of the endpoint cache 
 (update keyspace, add/remove nodes, etc) in AbstractReplicationStrategy 
 result in several seconds of thundering herd behavior on the entire cluster. 
 A thread dump shows over a hundred threads (I stopped counting at that point) 
 with a backtrace like this:
 at java.net.Inet4Address.getAddress(Inet4Address.java:288)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:106)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:103)
 at java.util.TreeMap.getEntryUsingComparator(TreeMap.java:351)
 at java.util.TreeMap.getEntry(TreeMap.java:322)
 at java.util.TreeMap.get(TreeMap.java:255)
 at 
 com.google.common.collect.AbstractMultimap.put(AbstractMultimap.java:200)
 at 
 com.google.common.collect.AbstractSetMultimap.put(AbstractSetMultimap.java:117)
 at com.google.common.collect.TreeMultimap.put(TreeMultimap.java:74)
 at 
 com.google.common.collect.AbstractMultimap.putAll(AbstractMultimap.java:273)
 at com.google.common.collect.TreeMultimap.putAll(TreeMultimap.java:74)
 at 
 org.apache.cassandra.utils.SortedBiMultiValMap.create(SortedBiMultiValMap.java:60)
 at 
 org.apache.cassandra.locator.TokenMetadata.cloneOnlyTokenMap(TokenMetadata.java:598)
 at 
 org.apache.cassandra.locator.AbstractReplicationStrategy.getNaturalEndpoints(AbstractReplicationStrategy.java:104)
 at 
 org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:2671)
 at 
 org.apache.cassandra.service.StorageProxy.performWrite(StorageProxy.java:375)
 It looks like there's a large amount of cost in the 
 TokenMetadata.cloneOnlyTokenMap that 
 AbstractReplicationStrategy.getNaturalEndpoints is calling each time there is 
 a cache miss for an endpoint. It seems as if this would only impact clusters 
 with large numbers of tokens, so it's probably a vnodes-only issue.
 Proposal: In AbstractReplicationStrategy.getNaturalEndpoints(), cache the 
 cloned TokenMetadata instance returned by TokenMetadata.cloneOnlyTokenMap(), 
 wrapping it with a lock to prevent stampedes, and clearing it in 
 clearEndpointCache(). Thoughts?



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Created] (CASSANDRA-6408) Efficient multi-partition mutations

2013-11-26 Thread Rick Branson (JIRA)
Rick Branson created CASSANDRA-6408:
---

 Summary: Efficient multi-partition mutations
 Key: CASSANDRA-6408
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6408
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Rick Branson


At the SF Summit this year, Sylvain suggested that C* drops a very large amount 
of write throughput on the floor for multi-partition mutations because they are 
broken into RowMutations and executed individually. Stress tests that I've run 
show 10X the throughput for 1-row x 1000-col writes versus 1000-row x 1-col 
writes. We have a core high-write-skew use case which involves fan-out-on-write 
against hundreds or up to thousands of keys at a time currently implemented in 
Redis as it doesn't seem to suffer from the issue. Would love to be able to 
move this to C* at some point.

This is likely a pretty large undertaking as it would require touching a large 
portion of the write path, but I figure I'd put it here for comment and/or 
debate at this point.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (CASSANDRA-6345) Endpoint cache invalidation causes CPU spike (on vnode rings?)

2013-11-20 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6345?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13828097#comment-13828097
 ] 

Rick Branson commented on CASSANDRA-6345:
-

+100 at removing those pub/sub callbacks :)

The concurrency issues I bring up are probably because I'm unfamiliar with the 
guarantees needed by TokenMetadata updates. It looks like the current release 
code is subject to the issue I brought up, where method calls on TokenMetadata 
that change state return successfully before all threads applying mutations 
have seen the update. There will be some mutations in progress that are using 
stale token data to apply writes even after TokenMetadata write methods 
returns as successful. So this does not appear to be a regression, but I'm just 
being overly cautious having been burned by these sort of double-caching 
scenarios before. You bring up the point that over-broad operations are ok, and 
I agree, but I'm more concerned about operations that are too narrow. It seems 
that unless I'm missing something either is possible with the current release 
code, and thus these patches as well (including mine).

TokenMetadata#updateNormalTokens is (implicitly) relying on the 
removeFromMoving call to bump the version, but the tokenToEndpointMap is 
updated afterwards, which means internal data is updated after the version is 
bumped. IMHO to be defensive, any time the write lock is acquired in 
TokenMetadata, the version should be bumped in the finally block before the 
lock is released. I don't think this is exposing a bug in the existing patch 
though, because cloneOnlyTokenMap will be blocked until the write lock is 
released in the finally block.

Is the idea with the striped lock on the endpoint cache in 
AbstractReplicationStrategy to help smooth out the stampede effect when the 
global lock on the cached TM gets released after the fill? How much do you 
think it's worth the extra complexity? FWIW, my v2 patch suffers from this 
issue and it hasn't reared itself in production. The write load for the 
machines in the cluster I've been looking at is comparatively low though 
compared to many others at 6-7k/sec peak on an 8-core box.

 Endpoint cache invalidation causes CPU spike (on vnode rings?)
 --

 Key: CASSANDRA-6345
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6345
 Project: Cassandra
  Issue Type: Bug
 Environment: 30 nodes total, 2 DCs
 Cassandra 1.2.11
 vnodes enabled (256 per node)
Reporter: Rick Branson
Assignee: Jonathan Ellis
 Fix For: 1.2.13

 Attachments: 6345-rbranson-v2.txt, 6345-rbranson.txt, 6345-v2.txt, 
 6345-v3.txt, 6345-v4.txt, 6345.txt, 
 half-way-thru-6345-rbranson-patch-applied.png


 We've observed that events which cause invalidation of the endpoint cache 
 (update keyspace, add/remove nodes, etc) in AbstractReplicationStrategy 
 result in several seconds of thundering herd behavior on the entire cluster. 
 A thread dump shows over a hundred threads (I stopped counting at that point) 
 with a backtrace like this:
 at java.net.Inet4Address.getAddress(Inet4Address.java:288)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:106)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:103)
 at java.util.TreeMap.getEntryUsingComparator(TreeMap.java:351)
 at java.util.TreeMap.getEntry(TreeMap.java:322)
 at java.util.TreeMap.get(TreeMap.java:255)
 at 
 com.google.common.collect.AbstractMultimap.put(AbstractMultimap.java:200)
 at 
 com.google.common.collect.AbstractSetMultimap.put(AbstractSetMultimap.java:117)
 at com.google.common.collect.TreeMultimap.put(TreeMultimap.java:74)
 at 
 com.google.common.collect.AbstractMultimap.putAll(AbstractMultimap.java:273)
 at com.google.common.collect.TreeMultimap.putAll(TreeMultimap.java:74)
 at 
 org.apache.cassandra.utils.SortedBiMultiValMap.create(SortedBiMultiValMap.java:60)
 at 
 org.apache.cassandra.locator.TokenMetadata.cloneOnlyTokenMap(TokenMetadata.java:598)
 at 
 org.apache.cassandra.locator.AbstractReplicationStrategy.getNaturalEndpoints(AbstractReplicationStrategy.java:104)
 at 
 org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:2671)
 at 
 org.apache.cassandra.service.StorageProxy.performWrite(StorageProxy.java:375)
 It looks like there's a large amount of cost in the 
 TokenMetadata.cloneOnlyTokenMap that 
 AbstractReplicationStrategy.getNaturalEndpoints is calling each time there is 
 a cache miss for an endpoint. It seems as if this would only impact clusters 
 with large numbers of tokens, so it's probably a vnodes-only issue.
 Proposal: In 

[jira] [Commented] (CASSANDRA-6345) Endpoint cache invalidation causes CPU spike (on vnode rings?)

2013-11-18 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6345?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13825614#comment-13825614
 ] 

Rick Branson commented on CASSANDRA-6345:
-

I like the simpler approach. I still think the callbacks for invalidation are 
asking for it ;) I also think perhaps the stampede lock should be more explicit 
than a synchronized lock on this to prevent unintended blocking from future 
modifications.

Either way, I think the only material concern I have is the order that 
TokenMetadata changes get applied to the caches in AbstractReplicationStrategy 
instances. Shouldn't the invalidation take place on all threads in all 
instances of AbstractReplicationStrategy before returning from an 
endpoint-mutating write operation in TokenMetadata? It seems as if just setting 
the cache to empty would allow a period of time where TokenMetadata write 
methods had returned but not all threads have seen the mutation yet because 
they are still holding onto the old clone of TM. This might be alright though, 
I'm not sure. Thoughts?

 Endpoint cache invalidation causes CPU spike (on vnode rings?)
 --

 Key: CASSANDRA-6345
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6345
 Project: Cassandra
  Issue Type: Bug
 Environment: 30 nodes total, 2 DCs
 Cassandra 1.2.11
 vnodes enabled (256 per node)
Reporter: Rick Branson
Assignee: Jonathan Ellis
 Fix For: 1.2.12, 2.0.3

 Attachments: 6345-rbranson-v2.txt, 6345-rbranson.txt, 6345-v2.txt, 
 6345-v3.txt, 6345.txt, half-way-thru-6345-rbranson-patch-applied.png


 We've observed that events which cause invalidation of the endpoint cache 
 (update keyspace, add/remove nodes, etc) in AbstractReplicationStrategy 
 result in several seconds of thundering herd behavior on the entire cluster. 
 A thread dump shows over a hundred threads (I stopped counting at that point) 
 with a backtrace like this:
 at java.net.Inet4Address.getAddress(Inet4Address.java:288)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:106)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:103)
 at java.util.TreeMap.getEntryUsingComparator(TreeMap.java:351)
 at java.util.TreeMap.getEntry(TreeMap.java:322)
 at java.util.TreeMap.get(TreeMap.java:255)
 at 
 com.google.common.collect.AbstractMultimap.put(AbstractMultimap.java:200)
 at 
 com.google.common.collect.AbstractSetMultimap.put(AbstractSetMultimap.java:117)
 at com.google.common.collect.TreeMultimap.put(TreeMultimap.java:74)
 at 
 com.google.common.collect.AbstractMultimap.putAll(AbstractMultimap.java:273)
 at com.google.common.collect.TreeMultimap.putAll(TreeMultimap.java:74)
 at 
 org.apache.cassandra.utils.SortedBiMultiValMap.create(SortedBiMultiValMap.java:60)
 at 
 org.apache.cassandra.locator.TokenMetadata.cloneOnlyTokenMap(TokenMetadata.java:598)
 at 
 org.apache.cassandra.locator.AbstractReplicationStrategy.getNaturalEndpoints(AbstractReplicationStrategy.java:104)
 at 
 org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:2671)
 at 
 org.apache.cassandra.service.StorageProxy.performWrite(StorageProxy.java:375)
 It looks like there's a large amount of cost in the 
 TokenMetadata.cloneOnlyTokenMap that 
 AbstractReplicationStrategy.getNaturalEndpoints is calling each time there is 
 a cache miss for an endpoint. It seems as if this would only impact clusters 
 with large numbers of tokens, so it's probably a vnodes-only issue.
 Proposal: In AbstractReplicationStrategy.getNaturalEndpoints(), cache the 
 cloned TokenMetadata instance returned by TokenMetadata.cloneOnlyTokenMap(), 
 wrapping it with a lock to prevent stampedes, and clearing it in 
 clearEndpointCache(). Thoughts?



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Created] (CASSANDRA-6354) No cleanup of excess gossip connections

2013-11-15 Thread Rick Branson (JIRA)
Rick Branson created CASSANDRA-6354:
---

 Summary: No cleanup of excess gossip connections
 Key: CASSANDRA-6354
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6354
 Project: Cassandra
  Issue Type: Bug
Reporter: Rick Branson
Priority: Minor


While trying to cut off communication between two nodes, I noticed a production 
node had 300 connections active established to another node on the storage 
port. It looks like there's no check to keep these limited, so they'll just sit 
around forever.



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (CASSANDRA-6345) Endpoint cache invalidation causes CPU spike (on vnode rings?)

2013-11-14 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6345?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13823274#comment-13823274
 ] 

Rick Branson commented on CASSANDRA-6345:
-

Unfortunately both of the patches suffer from a deadlock, since the 
invalidation and fill are wrapped up in TokenMetadata's locks.

T1 acquires cache read lock
T2 acquires TokenMetadata write lock
T1 acquires cache write lock on miss
T2 is blocked on cache write lock trying to invalidate
T1 is blocked on TokenMetadata read lock trying to cloneOnlyTokenMap to fill 
the cache

Trying to work on a fix.

 Endpoint cache invalidation causes CPU spike (on vnode rings?)
 --

 Key: CASSANDRA-6345
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6345
 Project: Cassandra
  Issue Type: Bug
 Environment: 30 nodes total, 2 DCs
 Cassandra 1.2.11
 vnodes enabled (256 per node)
Reporter: Rick Branson
Assignee: Jonathan Ellis
 Fix For: 1.2.12, 2.0.3

 Attachments: 6345-rbranson.txt, 6345-v2.txt, 6345.txt, 
 half-way-thru-6345-rbranson-patch-applied.png


 We've observed that events which cause invalidation of the endpoint cache 
 (update keyspace, add/remove nodes, etc) in AbstractReplicationStrategy 
 result in several seconds of thundering herd behavior on the entire cluster. 
 A thread dump shows over a hundred threads (I stopped counting at that point) 
 with a backtrace like this:
 at java.net.Inet4Address.getAddress(Inet4Address.java:288)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:106)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:103)
 at java.util.TreeMap.getEntryUsingComparator(TreeMap.java:351)
 at java.util.TreeMap.getEntry(TreeMap.java:322)
 at java.util.TreeMap.get(TreeMap.java:255)
 at 
 com.google.common.collect.AbstractMultimap.put(AbstractMultimap.java:200)
 at 
 com.google.common.collect.AbstractSetMultimap.put(AbstractSetMultimap.java:117)
 at com.google.common.collect.TreeMultimap.put(TreeMultimap.java:74)
 at 
 com.google.common.collect.AbstractMultimap.putAll(AbstractMultimap.java:273)
 at com.google.common.collect.TreeMultimap.putAll(TreeMultimap.java:74)
 at 
 org.apache.cassandra.utils.SortedBiMultiValMap.create(SortedBiMultiValMap.java:60)
 at 
 org.apache.cassandra.locator.TokenMetadata.cloneOnlyTokenMap(TokenMetadata.java:598)
 at 
 org.apache.cassandra.locator.AbstractReplicationStrategy.getNaturalEndpoints(AbstractReplicationStrategy.java:104)
 at 
 org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:2671)
 at 
 org.apache.cassandra.service.StorageProxy.performWrite(StorageProxy.java:375)
 It looks like there's a large amount of cost in the 
 TokenMetadata.cloneOnlyTokenMap that 
 AbstractReplicationStrategy.getNaturalEndpoints is calling each time there is 
 a cache miss for an endpoint. It seems as if this would only impact clusters 
 with large numbers of tokens, so it's probably a vnodes-only issue.
 Proposal: In AbstractReplicationStrategy.getNaturalEndpoints(), cache the 
 cloned TokenMetadata instance returned by TokenMetadata.cloneOnlyTokenMap(), 
 wrapping it with a lock to prevent stampedes, and clearing it in 
 clearEndpointCache(). Thoughts?



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (CASSANDRA-6345) Endpoint cache invalidation causes CPU spike (on vnode rings?)

2013-11-14 Thread Rick Branson (JIRA)

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

Rick Branson updated CASSANDRA-6345:


Attachment: 6345-rbranson-v2.txt

Attached a new patch with the deadlock fixed. We're running this on a 
production cluster.

The primary issue was the callback for invalidation from TokenMetadata to all 
of the registered AbstractReplicationStrategy instances. This was asking for it 
anyway, so in the patch I replaced the push invalidation with simple 
versioning of the TokenMetadata endpoints. TokenMetadata bumps it's version 
number each time the cache would need to be invalidated, and 
AbstractReplicationStrategy checks it's version when it needs to do a read, 
invalidating if necessary. This gets the invalidation out of the gossip threads 
and into the RPC threads, which is probably a good thing. The only thing I'm 
not super crazy about is the extra hot path read lock acquisition on 
TokenMetadata.getEndpointVersion(), which might be avoidable.

 Endpoint cache invalidation causes CPU spike (on vnode rings?)
 --

 Key: CASSANDRA-6345
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6345
 Project: Cassandra
  Issue Type: Bug
 Environment: 30 nodes total, 2 DCs
 Cassandra 1.2.11
 vnodes enabled (256 per node)
Reporter: Rick Branson
Assignee: Jonathan Ellis
 Fix For: 1.2.12, 2.0.3

 Attachments: 6345-rbranson-v2.txt, 6345-rbranson.txt, 6345-v2.txt, 
 6345.txt, half-way-thru-6345-rbranson-patch-applied.png


 We've observed that events which cause invalidation of the endpoint cache 
 (update keyspace, add/remove nodes, etc) in AbstractReplicationStrategy 
 result in several seconds of thundering herd behavior on the entire cluster. 
 A thread dump shows over a hundred threads (I stopped counting at that point) 
 with a backtrace like this:
 at java.net.Inet4Address.getAddress(Inet4Address.java:288)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:106)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:103)
 at java.util.TreeMap.getEntryUsingComparator(TreeMap.java:351)
 at java.util.TreeMap.getEntry(TreeMap.java:322)
 at java.util.TreeMap.get(TreeMap.java:255)
 at 
 com.google.common.collect.AbstractMultimap.put(AbstractMultimap.java:200)
 at 
 com.google.common.collect.AbstractSetMultimap.put(AbstractSetMultimap.java:117)
 at com.google.common.collect.TreeMultimap.put(TreeMultimap.java:74)
 at 
 com.google.common.collect.AbstractMultimap.putAll(AbstractMultimap.java:273)
 at com.google.common.collect.TreeMultimap.putAll(TreeMultimap.java:74)
 at 
 org.apache.cassandra.utils.SortedBiMultiValMap.create(SortedBiMultiValMap.java:60)
 at 
 org.apache.cassandra.locator.TokenMetadata.cloneOnlyTokenMap(TokenMetadata.java:598)
 at 
 org.apache.cassandra.locator.AbstractReplicationStrategy.getNaturalEndpoints(AbstractReplicationStrategy.java:104)
 at 
 org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:2671)
 at 
 org.apache.cassandra.service.StorageProxy.performWrite(StorageProxy.java:375)
 It looks like there's a large amount of cost in the 
 TokenMetadata.cloneOnlyTokenMap that 
 AbstractReplicationStrategy.getNaturalEndpoints is calling each time there is 
 a cache miss for an endpoint. It seems as if this would only impact clusters 
 with large numbers of tokens, so it's probably a vnodes-only issue.
 Proposal: In AbstractReplicationStrategy.getNaturalEndpoints(), cache the 
 cloned TokenMetadata instance returned by TokenMetadata.cloneOnlyTokenMap(), 
 wrapping it with a lock to prevent stampedes, and clearing it in 
 clearEndpointCache(). Thoughts?



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Created] (CASSANDRA-6345) Endpoint cache invalidation causes CPU spike (on vnode rings?)

2013-11-13 Thread Rick Branson (JIRA)
Rick Branson created CASSANDRA-6345:
---

 Summary: Endpoint cache invalidation causes CPU spike (on vnode 
rings?)
 Key: CASSANDRA-6345
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6345
 Project: Cassandra
  Issue Type: Bug
 Environment: 30 nodes total, 2 DCs
Cassandra 1.2.11
vnodes enabled (256 per node)
Reporter: Rick Branson


We've observed that events which cause invalidation of the endpoint cache 
(update keyspace, add/remove nodes, etc) in AbstractReplicationStrategy result 
in several seconds of thundering herd behavior on the entire cluster. 

A thread dump shows over a hundred threads (I stopped counting at that point) 
with a backtrace like this:

at java.net.Inet4Address.getAddress(Inet4Address.java:288)
at 
org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:106)
at 
org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:103)
at java.util.TreeMap.getEntryUsingComparator(TreeMap.java:351)
at java.util.TreeMap.getEntry(TreeMap.java:322)
at java.util.TreeMap.get(TreeMap.java:255)
at 
com.google.common.collect.AbstractMultimap.put(AbstractMultimap.java:200)
at 
com.google.common.collect.AbstractSetMultimap.put(AbstractSetMultimap.java:117)
at com.google.common.collect.TreeMultimap.put(TreeMultimap.java:74)
at 
com.google.common.collect.AbstractMultimap.putAll(AbstractMultimap.java:273)
at com.google.common.collect.TreeMultimap.putAll(TreeMultimap.java:74)
at 
org.apache.cassandra.utils.SortedBiMultiValMap.create(SortedBiMultiValMap.java:60)
at 
org.apache.cassandra.locator.TokenMetadata.cloneOnlyTokenMap(TokenMetadata.java:598)
at 
org.apache.cassandra.locator.AbstractReplicationStrategy.getNaturalEndpoints(AbstractReplicationStrategy.java:104)
at 
org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:2671)
at 
org.apache.cassandra.service.StorageProxy.performWrite(StorageProxy.java:375)

It looks like there's a large amount of cost in the 
TokenMetadata.cloneOnlyTokenMap that 
AbstractReplicationStrategy.getNaturalEndpoints is calling each time there is a 
cache miss for an endpoint. It seems as if this would only impact clusters with 
large numbers of tokens, so it's probably a vnodes-only issue.

Proposal: In AbstractReplicationStrategy.getNaturalEndpoints(), cache the 
cloned TokenMetadata instance returned by TokenMetadata.cloneOnlyTokenMap(), 
wrapping it with a lock to prevent stampedes, and clearing it in 
clearEndpointCache(). Thoughts?



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (CASSANDRA-6345) Endpoint cache invalidation causes CPU spike (on vnode rings?)

2013-11-13 Thread Rick Branson (JIRA)

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

Rick Branson updated CASSANDRA-6345:


Reproduced In: 1.2.11

 Endpoint cache invalidation causes CPU spike (on vnode rings?)
 --

 Key: CASSANDRA-6345
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6345
 Project: Cassandra
  Issue Type: Bug
 Environment: 30 nodes total, 2 DCs
 Cassandra 1.2.11
 vnodes enabled (256 per node)
Reporter: Rick Branson

 We've observed that events which cause invalidation of the endpoint cache 
 (update keyspace, add/remove nodes, etc) in AbstractReplicationStrategy 
 result in several seconds of thundering herd behavior on the entire cluster. 
 A thread dump shows over a hundred threads (I stopped counting at that point) 
 with a backtrace like this:
 at java.net.Inet4Address.getAddress(Inet4Address.java:288)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:106)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:103)
 at java.util.TreeMap.getEntryUsingComparator(TreeMap.java:351)
 at java.util.TreeMap.getEntry(TreeMap.java:322)
 at java.util.TreeMap.get(TreeMap.java:255)
 at 
 com.google.common.collect.AbstractMultimap.put(AbstractMultimap.java:200)
 at 
 com.google.common.collect.AbstractSetMultimap.put(AbstractSetMultimap.java:117)
 at com.google.common.collect.TreeMultimap.put(TreeMultimap.java:74)
 at 
 com.google.common.collect.AbstractMultimap.putAll(AbstractMultimap.java:273)
 at com.google.common.collect.TreeMultimap.putAll(TreeMultimap.java:74)
 at 
 org.apache.cassandra.utils.SortedBiMultiValMap.create(SortedBiMultiValMap.java:60)
 at 
 org.apache.cassandra.locator.TokenMetadata.cloneOnlyTokenMap(TokenMetadata.java:598)
 at 
 org.apache.cassandra.locator.AbstractReplicationStrategy.getNaturalEndpoints(AbstractReplicationStrategy.java:104)
 at 
 org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:2671)
 at 
 org.apache.cassandra.service.StorageProxy.performWrite(StorageProxy.java:375)
 It looks like there's a large amount of cost in the 
 TokenMetadata.cloneOnlyTokenMap that 
 AbstractReplicationStrategy.getNaturalEndpoints is calling each time there is 
 a cache miss for an endpoint. It seems as if this would only impact clusters 
 with large numbers of tokens, so it's probably a vnodes-only issue.
 Proposal: In AbstractReplicationStrategy.getNaturalEndpoints(), cache the 
 cloned TokenMetadata instance returned by TokenMetadata.cloneOnlyTokenMap(), 
 wrapping it with a lock to prevent stampedes, and clearing it in 
 clearEndpointCache(). Thoughts?



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (CASSANDRA-6345) Endpoint cache invalidation causes CPU spike (on vnode rings?)

2013-11-13 Thread Rick Branson (JIRA)

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

Rick Branson updated CASSANDRA-6345:


Attachment: 6345.txt

Attached a patch we deployed to production that fixed the issue.

 Endpoint cache invalidation causes CPU spike (on vnode rings?)
 --

 Key: CASSANDRA-6345
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6345
 Project: Cassandra
  Issue Type: Bug
 Environment: 30 nodes total, 2 DCs
 Cassandra 1.2.11
 vnodes enabled (256 per node)
Reporter: Rick Branson
Assignee: Jonathan Ellis
 Fix For: 1.2.12, 2.0.3

 Attachments: 6345-v2.txt, 6345.txt, 6345.txt


 We've observed that events which cause invalidation of the endpoint cache 
 (update keyspace, add/remove nodes, etc) in AbstractReplicationStrategy 
 result in several seconds of thundering herd behavior on the entire cluster. 
 A thread dump shows over a hundred threads (I stopped counting at that point) 
 with a backtrace like this:
 at java.net.Inet4Address.getAddress(Inet4Address.java:288)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:106)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:103)
 at java.util.TreeMap.getEntryUsingComparator(TreeMap.java:351)
 at java.util.TreeMap.getEntry(TreeMap.java:322)
 at java.util.TreeMap.get(TreeMap.java:255)
 at 
 com.google.common.collect.AbstractMultimap.put(AbstractMultimap.java:200)
 at 
 com.google.common.collect.AbstractSetMultimap.put(AbstractSetMultimap.java:117)
 at com.google.common.collect.TreeMultimap.put(TreeMultimap.java:74)
 at 
 com.google.common.collect.AbstractMultimap.putAll(AbstractMultimap.java:273)
 at com.google.common.collect.TreeMultimap.putAll(TreeMultimap.java:74)
 at 
 org.apache.cassandra.utils.SortedBiMultiValMap.create(SortedBiMultiValMap.java:60)
 at 
 org.apache.cassandra.locator.TokenMetadata.cloneOnlyTokenMap(TokenMetadata.java:598)
 at 
 org.apache.cassandra.locator.AbstractReplicationStrategy.getNaturalEndpoints(AbstractReplicationStrategy.java:104)
 at 
 org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:2671)
 at 
 org.apache.cassandra.service.StorageProxy.performWrite(StorageProxy.java:375)
 It looks like there's a large amount of cost in the 
 TokenMetadata.cloneOnlyTokenMap that 
 AbstractReplicationStrategy.getNaturalEndpoints is calling each time there is 
 a cache miss for an endpoint. It seems as if this would only impact clusters 
 with large numbers of tokens, so it's probably a vnodes-only issue.
 Proposal: In AbstractReplicationStrategy.getNaturalEndpoints(), cache the 
 cloned TokenMetadata instance returned by TokenMetadata.cloneOnlyTokenMap(), 
 wrapping it with a lock to prevent stampedes, and clearing it in 
 clearEndpointCache(). Thoughts?



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (CASSANDRA-6345) Endpoint cache invalidation causes CPU spike (on vnode rings?)

2013-11-13 Thread Rick Branson (JIRA)

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

Rick Branson updated CASSANDRA-6345:


Attachment: (was: 6345.txt)

 Endpoint cache invalidation causes CPU spike (on vnode rings?)
 --

 Key: CASSANDRA-6345
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6345
 Project: Cassandra
  Issue Type: Bug
 Environment: 30 nodes total, 2 DCs
 Cassandra 1.2.11
 vnodes enabled (256 per node)
Reporter: Rick Branson
Assignee: Jonathan Ellis
 Fix For: 1.2.12, 2.0.3

 Attachments: 6345-rbranson.txt, 6345-v2.txt, 6345.txt, 
 half-way-thru-6345-rbranson-patch-applied.png


 We've observed that events which cause invalidation of the endpoint cache 
 (update keyspace, add/remove nodes, etc) in AbstractReplicationStrategy 
 result in several seconds of thundering herd behavior on the entire cluster. 
 A thread dump shows over a hundred threads (I stopped counting at that point) 
 with a backtrace like this:
 at java.net.Inet4Address.getAddress(Inet4Address.java:288)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:106)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:103)
 at java.util.TreeMap.getEntryUsingComparator(TreeMap.java:351)
 at java.util.TreeMap.getEntry(TreeMap.java:322)
 at java.util.TreeMap.get(TreeMap.java:255)
 at 
 com.google.common.collect.AbstractMultimap.put(AbstractMultimap.java:200)
 at 
 com.google.common.collect.AbstractSetMultimap.put(AbstractSetMultimap.java:117)
 at com.google.common.collect.TreeMultimap.put(TreeMultimap.java:74)
 at 
 com.google.common.collect.AbstractMultimap.putAll(AbstractMultimap.java:273)
 at com.google.common.collect.TreeMultimap.putAll(TreeMultimap.java:74)
 at 
 org.apache.cassandra.utils.SortedBiMultiValMap.create(SortedBiMultiValMap.java:60)
 at 
 org.apache.cassandra.locator.TokenMetadata.cloneOnlyTokenMap(TokenMetadata.java:598)
 at 
 org.apache.cassandra.locator.AbstractReplicationStrategy.getNaturalEndpoints(AbstractReplicationStrategy.java:104)
 at 
 org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:2671)
 at 
 org.apache.cassandra.service.StorageProxy.performWrite(StorageProxy.java:375)
 It looks like there's a large amount of cost in the 
 TokenMetadata.cloneOnlyTokenMap that 
 AbstractReplicationStrategy.getNaturalEndpoints is calling each time there is 
 a cache miss for an endpoint. It seems as if this would only impact clusters 
 with large numbers of tokens, so it's probably a vnodes-only issue.
 Proposal: In AbstractReplicationStrategy.getNaturalEndpoints(), cache the 
 cloned TokenMetadata instance returned by TokenMetadata.cloneOnlyTokenMap(), 
 wrapping it with a lock to prevent stampedes, and clearing it in 
 clearEndpointCache(). Thoughts?



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (CASSANDRA-6345) Endpoint cache invalidation causes CPU spike (on vnode rings?)

2013-11-13 Thread Rick Branson (JIRA)

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

Rick Branson updated CASSANDRA-6345:


Attachment: 6345-rbranson.txt

 Endpoint cache invalidation causes CPU spike (on vnode rings?)
 --

 Key: CASSANDRA-6345
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6345
 Project: Cassandra
  Issue Type: Bug
 Environment: 30 nodes total, 2 DCs
 Cassandra 1.2.11
 vnodes enabled (256 per node)
Reporter: Rick Branson
Assignee: Jonathan Ellis
 Fix For: 1.2.12, 2.0.3

 Attachments: 6345-rbranson.txt, 6345-v2.txt, 6345.txt, 
 half-way-thru-6345-rbranson-patch-applied.png


 We've observed that events which cause invalidation of the endpoint cache 
 (update keyspace, add/remove nodes, etc) in AbstractReplicationStrategy 
 result in several seconds of thundering herd behavior on the entire cluster. 
 A thread dump shows over a hundred threads (I stopped counting at that point) 
 with a backtrace like this:
 at java.net.Inet4Address.getAddress(Inet4Address.java:288)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:106)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:103)
 at java.util.TreeMap.getEntryUsingComparator(TreeMap.java:351)
 at java.util.TreeMap.getEntry(TreeMap.java:322)
 at java.util.TreeMap.get(TreeMap.java:255)
 at 
 com.google.common.collect.AbstractMultimap.put(AbstractMultimap.java:200)
 at 
 com.google.common.collect.AbstractSetMultimap.put(AbstractSetMultimap.java:117)
 at com.google.common.collect.TreeMultimap.put(TreeMultimap.java:74)
 at 
 com.google.common.collect.AbstractMultimap.putAll(AbstractMultimap.java:273)
 at com.google.common.collect.TreeMultimap.putAll(TreeMultimap.java:74)
 at 
 org.apache.cassandra.utils.SortedBiMultiValMap.create(SortedBiMultiValMap.java:60)
 at 
 org.apache.cassandra.locator.TokenMetadata.cloneOnlyTokenMap(TokenMetadata.java:598)
 at 
 org.apache.cassandra.locator.AbstractReplicationStrategy.getNaturalEndpoints(AbstractReplicationStrategy.java:104)
 at 
 org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:2671)
 at 
 org.apache.cassandra.service.StorageProxy.performWrite(StorageProxy.java:375)
 It looks like there's a large amount of cost in the 
 TokenMetadata.cloneOnlyTokenMap that 
 AbstractReplicationStrategy.getNaturalEndpoints is calling each time there is 
 a cache miss for an endpoint. It seems as if this would only impact clusters 
 with large numbers of tokens, so it's probably a vnodes-only issue.
 Proposal: In AbstractReplicationStrategy.getNaturalEndpoints(), cache the 
 cloned TokenMetadata instance returned by TokenMetadata.cloneOnlyTokenMap(), 
 wrapping it with a lock to prevent stampedes, and clearing it in 
 clearEndpointCache(). Thoughts?



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Updated] (CASSANDRA-6345) Endpoint cache invalidation causes CPU spike (on vnode rings?)

2013-11-13 Thread Rick Branson (JIRA)

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

Rick Branson updated CASSANDRA-6345:


Attachment: half-way-thru-6345-rbranson-patch-applied.png

CPU user% graph during the rollout of the patch I attached on 1 DC (15 nodes) 
of the cluster. Around ~21:05 the patch starts to roll out and spikes are seen. 
The node in question receives the patch at ~21:30, and afterwards the spikes 
are gone. The rollout finishes at ~21:45.

 Endpoint cache invalidation causes CPU spike (on vnode rings?)
 --

 Key: CASSANDRA-6345
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6345
 Project: Cassandra
  Issue Type: Bug
 Environment: 30 nodes total, 2 DCs
 Cassandra 1.2.11
 vnodes enabled (256 per node)
Reporter: Rick Branson
Assignee: Jonathan Ellis
 Fix For: 1.2.12, 2.0.3

 Attachments: 6345-v2.txt, 6345.txt, 6345.txt, 
 half-way-thru-6345-rbranson-patch-applied.png


 We've observed that events which cause invalidation of the endpoint cache 
 (update keyspace, add/remove nodes, etc) in AbstractReplicationStrategy 
 result in several seconds of thundering herd behavior on the entire cluster. 
 A thread dump shows over a hundred threads (I stopped counting at that point) 
 with a backtrace like this:
 at java.net.Inet4Address.getAddress(Inet4Address.java:288)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:106)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:103)
 at java.util.TreeMap.getEntryUsingComparator(TreeMap.java:351)
 at java.util.TreeMap.getEntry(TreeMap.java:322)
 at java.util.TreeMap.get(TreeMap.java:255)
 at 
 com.google.common.collect.AbstractMultimap.put(AbstractMultimap.java:200)
 at 
 com.google.common.collect.AbstractSetMultimap.put(AbstractSetMultimap.java:117)
 at com.google.common.collect.TreeMultimap.put(TreeMultimap.java:74)
 at 
 com.google.common.collect.AbstractMultimap.putAll(AbstractMultimap.java:273)
 at com.google.common.collect.TreeMultimap.putAll(TreeMultimap.java:74)
 at 
 org.apache.cassandra.utils.SortedBiMultiValMap.create(SortedBiMultiValMap.java:60)
 at 
 org.apache.cassandra.locator.TokenMetadata.cloneOnlyTokenMap(TokenMetadata.java:598)
 at 
 org.apache.cassandra.locator.AbstractReplicationStrategy.getNaturalEndpoints(AbstractReplicationStrategy.java:104)
 at 
 org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:2671)
 at 
 org.apache.cassandra.service.StorageProxy.performWrite(StorageProxy.java:375)
 It looks like there's a large amount of cost in the 
 TokenMetadata.cloneOnlyTokenMap that 
 AbstractReplicationStrategy.getNaturalEndpoints is calling each time there is 
 a cache miss for an endpoint. It seems as if this would only impact clusters 
 with large numbers of tokens, so it's probably a vnodes-only issue.
 Proposal: In AbstractReplicationStrategy.getNaturalEndpoints(), cache the 
 cloned TokenMetadata instance returned by TokenMetadata.cloneOnlyTokenMap(), 
 wrapping it with a lock to prevent stampedes, and clearing it in 
 clearEndpointCache(). Thoughts?



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (CASSANDRA-6345) Endpoint cache invalidation causes CPU spike (on vnode rings?)

2013-11-13 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6345?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13822027#comment-13822027
 ] 

Rick Branson commented on CASSANDRA-6345:
-

Well, I started writing the patch this morning and I don't write multi-threaded 
Java code every day, so I'm overly careful ;) The only theoretical advantage to 
my patch is that it allows concurrent readers.

 Endpoint cache invalidation causes CPU spike (on vnode rings?)
 --

 Key: CASSANDRA-6345
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6345
 Project: Cassandra
  Issue Type: Bug
 Environment: 30 nodes total, 2 DCs
 Cassandra 1.2.11
 vnodes enabled (256 per node)
Reporter: Rick Branson
Assignee: Jonathan Ellis
 Fix For: 1.2.12, 2.0.3

 Attachments: 6345-rbranson.txt, 6345-v2.txt, 6345.txt, 
 half-way-thru-6345-rbranson-patch-applied.png


 We've observed that events which cause invalidation of the endpoint cache 
 (update keyspace, add/remove nodes, etc) in AbstractReplicationStrategy 
 result in several seconds of thundering herd behavior on the entire cluster. 
 A thread dump shows over a hundred threads (I stopped counting at that point) 
 with a backtrace like this:
 at java.net.Inet4Address.getAddress(Inet4Address.java:288)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:106)
 at 
 org.apache.cassandra.locator.TokenMetadata$1.compare(TokenMetadata.java:103)
 at java.util.TreeMap.getEntryUsingComparator(TreeMap.java:351)
 at java.util.TreeMap.getEntry(TreeMap.java:322)
 at java.util.TreeMap.get(TreeMap.java:255)
 at 
 com.google.common.collect.AbstractMultimap.put(AbstractMultimap.java:200)
 at 
 com.google.common.collect.AbstractSetMultimap.put(AbstractSetMultimap.java:117)
 at com.google.common.collect.TreeMultimap.put(TreeMultimap.java:74)
 at 
 com.google.common.collect.AbstractMultimap.putAll(AbstractMultimap.java:273)
 at com.google.common.collect.TreeMultimap.putAll(TreeMultimap.java:74)
 at 
 org.apache.cassandra.utils.SortedBiMultiValMap.create(SortedBiMultiValMap.java:60)
 at 
 org.apache.cassandra.locator.TokenMetadata.cloneOnlyTokenMap(TokenMetadata.java:598)
 at 
 org.apache.cassandra.locator.AbstractReplicationStrategy.getNaturalEndpoints(AbstractReplicationStrategy.java:104)
 at 
 org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:2671)
 at 
 org.apache.cassandra.service.StorageProxy.performWrite(StorageProxy.java:375)
 It looks like there's a large amount of cost in the 
 TokenMetadata.cloneOnlyTokenMap that 
 AbstractReplicationStrategy.getNaturalEndpoints is calling each time there is 
 a cache miss for an endpoint. It seems as if this would only impact clusters 
 with large numbers of tokens, so it's probably a vnodes-only issue.
 Proposal: In AbstractReplicationStrategy.getNaturalEndpoints(), cache the 
 cloned TokenMetadata instance returned by TokenMetadata.cloneOnlyTokenMap(), 
 wrapping it with a lock to prevent stampedes, and clearing it in 
 clearEndpointCache(). Thoughts?



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Created] (CASSANDRA-6334) Option to not listen on Thrift/CQL if cluster is N nodes

2013-11-12 Thread Rick Branson (JIRA)
Rick Branson created CASSANDRA-6334:
---

 Summary: Option to not listen on Thrift/CQL if cluster is N nodes
 Key: CASSANDRA-6334
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6334
 Project: Cassandra
  Issue Type: Improvement
  Components: Core
Reporter: Rick Branson
Priority: Minor


Sometimes nodes get isolated because of bugs or misconfiguration on either the 
server or the client side. A useful configuration option found in other systems 
is to not listen for client requests until the cluster reaches a minimum number 
of nodes. This would prevent perfectly valid reads  writes from taking place 
on these isolated nodes. What do we think about this?



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Created] (CASSANDRA-6335) Hints broken for nodes that change broadcast address

2013-11-12 Thread Rick Branson (JIRA)
Rick Branson created CASSANDRA-6335:
---

 Summary: Hints broken for nodes that change broadcast address
 Key: CASSANDRA-6335
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6335
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Rick Branson


When a node changes it's broadcast address, the transition process works 
properly, but hints that are destined for it can't be delivered because of the 
address change. It produces an exception:

java.lang.AssertionError: Missing host ID for 10.1.60.22
at 
org.apache.cassandra.service.StorageProxy.writeHintForMutation(StorageProxy.java:598)
at 
org.apache.cassandra.service.StorageProxy$5.runMayThrow(StorageProxy.java:567)
at 
org.apache.cassandra.service.StorageProxy$HintRunnable.run(StorageProxy.java:1679)
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
at java.util.concurrent.FutureTask.run(FutureTask.java:138)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (CASSANDRA-6335) Hints broken for nodes that change broadcast address

2013-11-12 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6335?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13820415#comment-13820415
 ] 

Rick Branson commented on CASSANDRA-6335:
-

It looks like it stalls the hint delivery thread pool indefinitely. Restarting 
the node frees it up and it starts delivering hints again. I'm rolling this 
cluster to workaround the issue.

 Hints broken for nodes that change broadcast address
 

 Key: CASSANDRA-6335
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6335
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Rick Branson

 When a node changes it's broadcast address, the transition process works 
 properly, but hints that are destined for it can't be delivered because of 
 the address change. It produces an exception:
 java.lang.AssertionError: Missing host ID for 10.1.60.22
 at 
 org.apache.cassandra.service.StorageProxy.writeHintForMutation(StorageProxy.java:598)
 at 
 org.apache.cassandra.service.StorageProxy$5.runMayThrow(StorageProxy.java:567)
 at 
 org.apache.cassandra.service.StorageProxy$HintRunnable.run(StorageProxy.java:1679)
 at 
 java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
 at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
 at java.util.concurrent.FutureTask.run(FutureTask.java:138)
 at 
 java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
 at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Comment Edited] (CASSANDRA-6335) Hints broken for nodes that change broadcast address

2013-11-12 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6335?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13820415#comment-13820415
 ] 

Rick Branson edited comment on CASSANDRA-6335 at 11/12/13 8:41 PM:
---

It looks like it stalls the hint delivery thread pool indefinitely. Restarting 
the node frees it up and it starts delivering hints again. I'm rolling this 
cluster to workaround the issue.

EDIT: disregard the stalling issue... there were just a large # of hints that 
needed to be delivered to a specific node (because it was down for a bit for 
some maintenance) clogging up the pool and confusing the issue.


was (Author: rbranson):
It looks like it stalls the hint delivery thread pool indefinitely. Restarting 
the node frees it up and it starts delivering hints again. I'm rolling this 
cluster to workaround the issue.

 Hints broken for nodes that change broadcast address
 

 Key: CASSANDRA-6335
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6335
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Reporter: Rick Branson

 When a node changes it's broadcast address, the transition process works 
 properly, but hints that are destined for it can't be delivered because of 
 the address change. It produces an exception:
 java.lang.AssertionError: Missing host ID for 10.1.60.22
 at 
 org.apache.cassandra.service.StorageProxy.writeHintForMutation(StorageProxy.java:598)
 at 
 org.apache.cassandra.service.StorageProxy$5.runMayThrow(StorageProxy.java:567)
 at 
 org.apache.cassandra.service.StorageProxy$HintRunnable.run(StorageProxy.java:1679)
 at 
 java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
 at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
 at java.util.concurrent.FutureTask.run(FutureTask.java:138)
 at 
 java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
 at 
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)



--
This message was sent by Atlassian JIRA
(v6.1#6144)


[jira] [Commented] (CASSANDRA-6106) QueryState.getTimestamp() FBUtilities.timestampMicros() reads current timestamp with System.currentTimeMillis() * 1000 instead of System.nanoTime() / 1000

2013-09-26 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6106?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13779564#comment-13779564
 ] 

Rick Branson commented on CASSANDRA-6106:
-

Could this re-sample the base wallclock at an interval so that it doesn't drift 
too far? (say, every 1s or 100ms)

 QueryState.getTimestamp()  FBUtilities.timestampMicros() reads current 
 timestamp with System.currentTimeMillis() * 1000 instead of System.nanoTime() 
 / 1000
 

 Key: CASSANDRA-6106
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6106
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: DSE Cassandra 3.1, but also HEAD
Reporter: Christopher Smith
Priority: Minor
  Labels: collision, conflict, timestamp
 Attachments: microtimstamp.patch


 I noticed this blog post: http://aphyr.com/posts/294-call-me-maybe-cassandra 
 mentioned issues with millisecond rounding in timestamps and was able to 
 reproduce the issue. If I specify a timestamp in a mutating query, I get 
 microsecond precision, but if I don't, I get timestamps rounded to the 
 nearest millisecond, at least for my first query on a given connection, which 
 substantially increases the possibilities of collision.
 I believe I found the offending code, though I am by no means sure this is 
 comprehensive. I think we probably need a fairly comprehensive replacement of 
 all uses of System.currentTimeMillis() with System.nanoTime().

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-6106) QueryState.getTimestamp() FBUtilities.timestampMicros() reads current timestamp with System.currentTimeMillis() * 1000 instead of System.nanoTime() / 1000

2013-09-26 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6106?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13779573#comment-13779573
 ] 

Rick Branson commented on CASSANDRA-6106:
-

I think you'd actually have to use the discrepancy between currentTimeMillis() 
now-start delta vs nanoTime() now-start delta to detect wall-clock drift/change 
and correct the nanoTime().

 QueryState.getTimestamp()  FBUtilities.timestampMicros() reads current 
 timestamp with System.currentTimeMillis() * 1000 instead of System.nanoTime() 
 / 1000
 

 Key: CASSANDRA-6106
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6106
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: DSE Cassandra 3.1, but also HEAD
Reporter: Christopher Smith
Priority: Minor
  Labels: collision, conflict, timestamp
 Attachments: microtimstamp.patch


 I noticed this blog post: http://aphyr.com/posts/294-call-me-maybe-cassandra 
 mentioned issues with millisecond rounding in timestamps and was able to 
 reproduce the issue. If I specify a timestamp in a mutating query, I get 
 microsecond precision, but if I don't, I get timestamps rounded to the 
 nearest millisecond, at least for my first query on a given connection, which 
 substantially increases the possibilities of collision.
 I believe I found the offending code, though I am by no means sure this is 
 comprehensive. I think we probably need a fairly comprehensive replacement of 
 all uses of System.currentTimeMillis() with System.nanoTime().

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-6106) QueryState.getTimestamp() FBUtilities.timestampMicros() reads current timestamp with System.currentTimeMillis() * 1000 instead of System.nanoTime() / 1000

2013-09-26 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-6106?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13779586#comment-13779586
 ] 

Rick Branson commented on CASSANDRA-6106:
-

Yeah you're right. The thread will probably occasionally get interrupted by the 
kernel between the currentTimeMillis() and the nanoTime() call. It'd probably 
take a bunch of iterations to properly calibrate the base clock. This is 
getting pretty far into the extremely-fragile territory.

 QueryState.getTimestamp()  FBUtilities.timestampMicros() reads current 
 timestamp with System.currentTimeMillis() * 1000 instead of System.nanoTime() 
 / 1000
 

 Key: CASSANDRA-6106
 URL: https://issues.apache.org/jira/browse/CASSANDRA-6106
 Project: Cassandra
  Issue Type: Bug
  Components: Core
 Environment: DSE Cassandra 3.1, but also HEAD
Reporter: Christopher Smith
Priority: Minor
  Labels: collision, conflict, timestamp
 Attachments: microtimstamp.patch


 I noticed this blog post: http://aphyr.com/posts/294-call-me-maybe-cassandra 
 mentioned issues with millisecond rounding in timestamps and was able to 
 reproduce the issue. If I specify a timestamp in a mutating query, I get 
 microsecond precision, but if I don't, I get timestamps rounded to the 
 nearest millisecond, at least for my first query on a given connection, which 
 substantially increases the possibilities of collision.
 I believe I found the offending code, though I am by no means sure this is 
 comprehensive. I think we probably need a fairly comprehensive replacement of 
 all uses of System.currentTimeMillis() with System.nanoTime().

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5948) StreamOut doesn't correctly handle wrapped ranges

2013-09-04 Thread Rick Branson (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-5948?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13758245#comment-13758245
 ] 

Rick Branson commented on CASSANDRA-5948:
-

Curious what the impact of this would have been? Has streaming effectively been 
broken since 1.2.6, since every cluster has wrapping ranges?

 StreamOut doesn't correctly handle wrapped ranges
 -

 Key: CASSANDRA-5948
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5948
 Project: Cassandra
  Issue Type: Bug
Reporter: Sergio Bossa
 Fix For: 1.2.10, 2.0.1

 Attachments: 5948-0001.patch


 StreamOut doesn't normalize ranges, causing AbstractViewSSTableFinder to miss 
 sstables when the requested range is wrapped, and hence breaking node 
 bootstrapping/unbootstrapping on such ranges.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Created] (CASSANDRA-5966) Average name query performance much worse for wide rows

2013-08-31 Thread Rick Branson (JIRA)
Rick Branson created CASSANDRA-5966:
---

 Summary: Average name query performance much worse for wide rows
 Key: CASSANDRA-5966
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5966
 Project: Cassandra
  Issue Type: Bug
Reporter: Rick Branson
Assignee: Rick Branson


The average performance for by-name queries degrades heavily on wide rows. This 
is because in the wide row case SSTableNamesIterator deserializes every column 
in the entire row chunk (64KB by default), where-as in the narrow row case, it 
stops deserializing as soon as it's found the columns it's looking for.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (CASSANDRA-5966) Average name query performance much worse for wide rows

2013-08-31 Thread Rick Branson (JIRA)

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

Rick Branson updated CASSANDRA-5966:


Attachment: 5966.txt

Attached patch modifies SSTableNamesIterator to stop deserializing columns from 
the SSTable as soon as it's reached the maximal column name for a given query 
or indexed range. This is for both the narrow and wide row case.

 Average name query performance much worse for wide rows
 ---

 Key: CASSANDRA-5966
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5966
 Project: Cassandra
  Issue Type: Bug
Reporter: Rick Branson
Assignee: Rick Branson
 Attachments: 5966.txt


 The average performance for by-name queries degrades heavily on wide rows. 
 This is because in the wide row case SSTableNamesIterator deserializes every 
 column in the entire row chunk (64KB by default), where-as in the narrow row 
 case, it stops deserializing as soon as it's found the columns it's looking 
 for.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (CASSANDRA-5966) Average name query performance much worse for wide rows

2013-08-31 Thread Rick Branson (JIRA)

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

Rick Branson updated CASSANDRA-5966:


Attachment: 5966-v2.txt

 Average name query performance much worse for wide rows
 ---

 Key: CASSANDRA-5966
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5966
 Project: Cassandra
  Issue Type: Bug
Reporter: Rick Branson
Assignee: Rick Branson
 Attachments: 5966.txt, 5966-v2.txt


 The average performance for by-name queries degrades heavily on wide rows. 
 This is because in the wide row case SSTableNamesIterator deserializes every 
 column in the entire row chunk (64KB by default), where-as in the narrow row 
 case, it stops deserializing as soon as it's found the columns it's looking 
 for.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


  1   2   >