Re: Practical node size limits

2012-09-06 Thread Dustin Wenz
This is actually another problem that we've encountered with Cassandra; the 
range of platforms it can be deployed on is fairly limited. If you want to run 
with Oracle's JRE (which is apparently recommended), you are pretty much stuck 
with Linux on x86/64 (I haven't tried the new JDK on ARM yet, but it sounds 
promising). You could probably do ok on Solaris, too, with a custom Snappy jar 
and some JNA concessions.

- .Dustin

On Sep 5, 2012, at 10:36 PM, Rob Coli  wrote:

> On Sun, Jul 29, 2012 at 7:40 PM, Dustin Wenz  wrote:
>> We've just set up a new 7-node cluster with Cassandra 1.1.2 running under 
>> OpenJDK6.
> 
> It's worth noting that Cassandra project recommends Sun JRE. Without
> the Sun JRE, you might not be able to use JAMM to determine the live
> ratio. Very few people use OpenJDK in production, so using it also
> increases the likelihood that you might be the first to encounter a
> given issue. FWIW!
> 
> =Rob
> 
> -- 
> =Robert Coli
> AIM>ALK - rc...@palominodb.com
> YAHOO - rcoli.palominob
> SKYPE - rcoli_palominodb



Re: Practical node size limits

2012-09-04 Thread Dustin Wenz
I'm following up on this issue, which I've been monitoring for the last several 
weeks. I thought people might find my observations interesting.

Ever since increasing the heap size to 64GB, we've had no OOM conditions that 
resulted in a JVM termination. Our nodes have around 2.5TB of data each, and 
the replication factor is four. IO on the cluster seems to be fine, though I 
haven't been paying particular attention to any GC hangs.

The bottleneck now seems to be the repair time. If any node becomes too 
inconsistent, or needs to be replaced, the rebuilt time is over a week. That 
issue alone makes this cluster configuration unsuitable for production use.

- .Dustin

On Jul 30, 2012, at 2:04 PM, Dustin Wenz  wrote:

> Thanks for the pointer! It sounds likely that's what I'm seeing. CFStats 
> reports that the bloom filter size is currently several gigabytes. Is there 
> any way to estimate how much heap space a repair would require? Is it a 
> function of simply adding up the filter file sizes, plus some fraction of 
> neighboring nodes?
> 
> I'm still curious about the largest heap sizes that people are running with 
> on their deployments. I'm considering increasing ours to 64GB (with 96GB 
> physical memory) to see where that gets us. Would it be necessary to keep the 
> young-gen size small to avoid long GC pauses? I also suspect that I may need 
> to keep my memtable sizes small to avoid long flushes; maybe in the 1-2GB 
> range.
> 
>   - .Dustin
> 
> On Jul 29, 2012, at 10:45 PM, Edward Capriolo  wrote:
> 
>> Yikes. You should read:
>> 
>> http://wiki.apache.org/cassandra/LargeDataSetConsiderations
>> 
>> Essentially what it sounds like your are now running into is this:
>> 
>> The BloomFilters for each SSTable must exist in main memory. Repair
>> tends to create some extra data which normally gets compacted away
>> later.
>> 
>> Your best bet is to temporarily raise the Xmx heap and adjust the
>> index sampling size. If you need to save the data (if it is just test
>> data you may want to give up and start fresh)
>> 
>> Generally the issue with the large disk configurations it is hard to
>> keep a good ram/disk ratio. Then most reads turn into disk seeks and
>> the throughput is low. I get the vibe people believe large stripes are
>> going to help Cassandra. The issue is that stripes generally only
>> increase sequential throughput, but Cassandra is a random read system.
>> 
>> How much ram/disk you need is case dependent but 1/5 ratio of RAM to
>> disk is where I think most people want to be, unless their system is
>> carrying SSD disks.
>> 
>> Again you have to keep your bloom filters in java heap memory so and
>> design that tries to create a quatrillion small rows is going to have
>> memory issues as well.
>> 
>> On Sun, Jul 29, 2012 at 10:40 PM, Dustin Wenz  wrote:
>>> I'm trying to determine if there are any practical limits on the amount of 
>>> data that a single node can handle efficiently, and if so, whether I've hit 
>>> that limit or not.
>>> 
>>> We've just set up a new 7-node cluster with Cassandra 1.1.2 running under 
>>> OpenJDK6. Each node is 12-core Xeon with 24GB of RAM and is connected to a 
>>> stripe of 10 3TB disk mirrors (a total of 20 spindles each) and connected 
>>> via dual SATA-3 interconnects. I can read and write around 900MB/s 
>>> sequentially on the arrays. I started out with Cassandra tuned with 
>>> all-default values, with the exception of the compaction throughput which 
>>> was increased from 16MB/s to 100MB/s. These defaults will set the heap size 
>>> to 6GB.
>>> 
>>> Our schema is pretty simple; only 4 column families and each has one 
>>> secondary index. The replication factor was set to four, and compression 
>>> disabled. Our access patterns are intended to be about equal numbers of 
>>> inserts and selects, with no updates, and the occasional delete.
>>> 
>>> The first thing we did was begin to load data into the cluster. We could 
>>> perform about 3000 inserts per second, which stayed mostly flat. Things 
>>> started to go wrong around the time the nodes exceeded 800GB. Cassandra 
>>> began to generate a lot of "mutations messages dropped" warnings, and was 
>>> complaining that the heap was over 75% capacity.
>>> 
>>> At that point, we stopped all activity on the cluster and attempted a 
>>> repair. We did this so we could be sure that the data was fully consistent 
>>> before continuing. Our mistake was pr

Re: Practical node size limits

2012-07-30 Thread Dustin Wenz
Thanks for the pointer! It sounds likely that's what I'm seeing. CFStats 
reports that the bloom filter size is currently several gigabytes. Is there any 
way to estimate how much heap space a repair would require? Is it a function of 
simply adding up the filter file sizes, plus some fraction of neighboring nodes?

I'm still curious about the largest heap sizes that people are running with on 
their deployments. I'm considering increasing ours to 64GB (with 96GB physical 
memory) to see where that gets us. Would it be necessary to keep the young-gen 
size small to avoid long GC pauses? I also suspect that I may need to keep my 
memtable sizes small to avoid long flushes; maybe in the 1-2GB range.

- .Dustin

On Jul 29, 2012, at 10:45 PM, Edward Capriolo  wrote:

> Yikes. You should read:
> 
> http://wiki.apache.org/cassandra/LargeDataSetConsiderations
> 
> Essentially what it sounds like your are now running into is this:
> 
> The BloomFilters for each SSTable must exist in main memory. Repair
> tends to create some extra data which normally gets compacted away
> later.
> 
> Your best bet is to temporarily raise the Xmx heap and adjust the
> index sampling size. If you need to save the data (if it is just test
> data you may want to give up and start fresh)
> 
> Generally the issue with the large disk configurations it is hard to
> keep a good ram/disk ratio. Then most reads turn into disk seeks and
> the throughput is low. I get the vibe people believe large stripes are
> going to help Cassandra. The issue is that stripes generally only
> increase sequential throughput, but Cassandra is a random read system.
> 
> How much ram/disk you need is case dependent but 1/5 ratio of RAM to
> disk is where I think most people want to be, unless their system is
> carrying SSD disks.
> 
> Again you have to keep your bloom filters in java heap memory so and
> design that tries to create a quatrillion small rows is going to have
> memory issues as well.
> 
> On Sun, Jul 29, 2012 at 10:40 PM, Dustin Wenz  wrote:
>> I'm trying to determine if there are any practical limits on the amount of 
>> data that a single node can handle efficiently, and if so, whether I've hit 
>> that limit or not.
>> 
>> We've just set up a new 7-node cluster with Cassandra 1.1.2 running under 
>> OpenJDK6. Each node is 12-core Xeon with 24GB of RAM and is connected to a 
>> stripe of 10 3TB disk mirrors (a total of 20 spindles each) and connected 
>> via dual SATA-3 interconnects. I can read and write around 900MB/s 
>> sequentially on the arrays. I started out with Cassandra tuned with 
>> all-default values, with the exception of the compaction throughput which 
>> was increased from 16MB/s to 100MB/s. These defaults will set the heap size 
>> to 6GB.
>> 
>> Our schema is pretty simple; only 4 column families and each has one 
>> secondary index. The replication factor was set to four, and compression 
>> disabled. Our access patterns are intended to be about equal numbers of 
>> inserts and selects, with no updates, and the occasional delete.
>> 
>> The first thing we did was begin to load data into the cluster. We could 
>> perform about 3000 inserts per second, which stayed mostly flat. Things 
>> started to go wrong around the time the nodes exceeded 800GB. Cassandra 
>> began to generate a lot of "mutations messages dropped" warnings, and was 
>> complaining that the heap was over 75% capacity.
>> 
>> At that point, we stopped all activity on the cluster and attempted a 
>> repair. We did this so we could be sure that the data was fully consistent 
>> before continuing. Our mistake was probably trying to repair all of the 
>> nodes simultaneously - within an hour, Java terminated on one of the nodes 
>> with a heap out-of-memory message. I then increased all of the heap sizes to 
>> 8GB, and reduced the heap_newsize to 800MB. All of the nodes were restarted, 
>> and there was no no outside activity on the cluster. I then began a repair 
>> on a single node. Within a few hours, it OOMed again and exited. I then 
>> increased the heap to 12GB, and attempted the same thing. This time, the 
>> repair ran for about 7 hours before exiting from an OOM condition.
>> 
>> By now, the repair had increased the amount of data on some of the nodes to 
>> over 1.2TB. There is no going back to a 6GB heap size - Cassandra now exits 
>> with an OOM during startup unless the heap is set higher. It's at 16GB now, 
>> and a single node has been repairing for a couple of days. Though I have no 
>> personal experience with this, I've been told that Java's garbage col

Practical node size limits

2012-07-29 Thread Dustin Wenz
I'm trying to determine if there are any practical limits on the amount of data 
that a single node can handle efficiently, and if so, whether I've hit that 
limit or not.

We've just set up a new 7-node cluster with Cassandra 1.1.2 running under 
OpenJDK6. Each node is 12-core Xeon with 24GB of RAM and is connected to a 
stripe of 10 3TB disk mirrors (a total of 20 spindles each) and connected via 
dual SATA-3 interconnects. I can read and write around 900MB/s sequentially on 
the arrays. I started out with Cassandra tuned with all-default values, with 
the exception of the compaction throughput which was increased from 16MB/s to 
100MB/s. These defaults will set the heap size to 6GB.

Our schema is pretty simple; only 4 column families and each has one secondary 
index. The replication factor was set to four, and compression disabled. Our 
access patterns are intended to be about equal numbers of inserts and selects, 
with no updates, and the occasional delete.

The first thing we did was begin to load data into the cluster. We could 
perform about 3000 inserts per second, which stayed mostly flat. Things started 
to go wrong around the time the nodes exceeded 800GB. Cassandra began to 
generate a lot of "mutations messages dropped" warnings, and was complaining 
that the heap was over 75% capacity.

At that point, we stopped all activity on the cluster and attempted a repair. 
We did this so we could be sure that the data was fully consistent before 
continuing. Our mistake was probably trying to repair all of the nodes 
simultaneously - within an hour, Java terminated on one of the nodes with a 
heap out-of-memory message. I then increased all of the heap sizes to 8GB, and 
reduced the heap_newsize to 800MB. All of the nodes were restarted, and there 
was no no outside activity on the cluster. I then began a repair on a single 
node. Within a few hours, it OOMed again and exited. I then increased the heap 
to 12GB, and attempted the same thing. This time, the repair ran for about 7 
hours before exiting from an OOM condition.

By now, the repair had increased the amount of data on some of the nodes to 
over 1.2TB. There is no going back to a 6GB heap size - Cassandra now exits 
with an OOM during startup unless the heap is set higher. It's at 16GB now, and 
a single node has been repairing for a couple of days. Though I have no 
personal experience with this, I've been told that Java's garbage collector 
doesn't perform well with heaps above 8GB. I'm wary of setting it higher, but I 
can add up to 192GB of RAM to each node if necessary.

How much heap does cassandra need for this amount of data with only four CFs? 
Am I scaling this cluster in completely the wrong direction? Is there a magic 
garbage collection setting that I need to add in cassandra-env that isn't there 
by default?

Thanks,

  - .Dustin


Re: Increased replication factor not evident in CLI

2012-07-13 Thread Dustin Wenz
I was able to apply the patch in the cited bug report to the public source for 
version 1.1.2. It seemed pretty straightforward; six lines in 
MigrationManager.java were switched from System.currentTimeMillis() to 
FBUtilities.timestampMicros(). I then re-built the project by running 'ant 
artifacts' in the cassandra root.

After I was up and running with the new version, I attempted to increase the 
replication factor, and then the compressions options.

Unfortunately, new patch did not seem to help in my case. Neither of the schema 
attributes would change. Running a "describe cluster" shows that all node 
schemas are consistent.

Are there any other ways that I could potentially force Cassandra to accept 
these changes?

- .Dustin

On Jul 13, 2012, at 10:02 AM, Dustin Wenz wrote:

> It sounds plausible that is what we are running into. All of our nodes report 
> a replication factor of 2 (both using describe, and show schema), even though 
> the cluster reported that all schemas agree after I issued the change to 4.
> 
> If this is related to the bug that you filed, it might also explain why I've 
> had difficulty changing the compression options on this same cluster. I issue 
> an update command, schemas agree, but yet the change is not evident.
> 
>   - .Dustin
> 
> On Jul 12, 2012, at 7:56 PM, Michael Theroux wrote:
> 
>> Sounds a lot like a bug that I hit that was filed and fixed recently:
>> 
>> https://issues.apache.org/jira/browse/CASSANDRA-4432
>> 
>> -Mike
>> 
>> On Jul 12, 2012, at 8:16 PM, Edward Capriolo wrote:
>> 
>>> Possibly the bug with nanotime causing cassandra to think the change 
>>> happened in the past. Talked about onlist in past few days.
>>> On Thursday, July 12, 2012, aaron morton  wrote:
>>>> Do multiple nodes say the RF is 2 ? Can you show the output from the CLI ? 
>>>> Do show schema and show keyspace say the same thing ?
>>>> Cheers
>>>> 
>>>> 
>>>> -
>>>> Aaron Morton
>>>> Freelance Developer
>>>> @aaronmorton
>>>> http://www.thelastpickle.com
>>>> On 13/07/2012, at 7:39 AM, Dustin Wenz wrote:
>>>> 
>>>> We recently increased the replication factor of a keyspace in our 
>>>> cassandra 1.1.1 cluster from 2 to 4. This was done by setting the 
>>>> replication factor to 4 in cassandra-cli, and then running a repair on 
>>>> each node.
>>>> 
>>>> Everything seems to have worked; the commands completed successfully and 
>>>> disk usage increased significantly. However, if I perform a describe on 
>>>> the keyspace, it still shows replication_factor:2. So, it appears that the 
>>>> replication factor might be 4, but it reports as 2. I'm not entirely sure 
>>>> how to confirm one or the other.
>>>> 
>>>> Since then, I've stopped and restarted the cluster, and even ran an 
>>>> upgradesstables on each node. The replication factor still doesn't report 
>>>> as I would expect. Am I missing something here?
>>>> 
>>>> - .Dustin
>>>> 
>>>> 
>>>> 
>> 
> 



Re: Increased replication factor not evident in CLI

2012-07-13 Thread Dustin Wenz
It sounds plausible that is what we are running into. All of our nodes report a 
replication factor of 2 (both using describe, and show schema), even though the 
cluster reported that all schemas agree after I issued the change to 4.

If this is related to the bug that you filed, it might also explain why I've 
had difficulty changing the compression options on this same cluster. I issue 
an update command, schemas agree, but yet the change is not evident.

- .Dustin

On Jul 12, 2012, at 7:56 PM, Michael Theroux wrote:

> Sounds a lot like a bug that I hit that was filed and fixed recently:
> 
> https://issues.apache.org/jira/browse/CASSANDRA-4432
> 
> -Mike
> 
> On Jul 12, 2012, at 8:16 PM, Edward Capriolo wrote:
> 
>> Possibly the bug with nanotime causing cassandra to think the change 
>> happened in the past. Talked about onlist in past few days.
>> On Thursday, July 12, 2012, aaron morton  wrote:
>> > Do multiple nodes say the RF is 2 ? Can you show the output from the CLI ? 
>> > Do show schema and show keyspace say the same thing ?
>> > Cheers
>> >
>> >
>> > -
>> > Aaron Morton
>> > Freelance Developer
>> > @aaronmorton
>> > http://www.thelastpickle.com
>> > On 13/07/2012, at 7:39 AM, Dustin Wenz wrote:
>> >
>> > We recently increased the replication factor of a keyspace in our 
>> > cassandra 1.1.1 cluster from 2 to 4. This was done by setting the 
>> > replication factor to 4 in cassandra-cli, and then running a repair on 
>> > each node.
>> >
>> > Everything seems to have worked; the commands completed successfully and 
>> > disk usage increased significantly. However, if I perform a describe on 
>> > the keyspace, it still shows replication_factor:2. So, it appears that the 
>> > replication factor might be 4, but it reports as 2. I'm not entirely sure 
>> > how to confirm one or the other.
>> >
>> > Since then, I've stopped and restarted the cluster, and even ran an 
>> > upgradesstables on each node. The replication factor still doesn't report 
>> > as I would expect. Am I missing something here?
>> >
>> > - .Dustin
>> >
>> >
>> >
> 



Increased replication factor not evident in CLI

2012-07-12 Thread Dustin Wenz
We recently increased the replication factor of a keyspace in our cassandra 
1.1.1 cluster from 2 to 4. This was done by setting the replication factor to 4 
in cassandra-cli, and then running a repair on each node.

Everything seems to have worked; the commands completed successfully and disk 
usage increased significantly. However, if I perform a describe on the 
keyspace, it still shows replication_factor:2. So, it appears that the 
replication factor might be 4, but it reports as 2. I'm not entirely sure how 
to confirm one or the other.

Since then, I've stopped and restarted the cluster, and even ran an 
upgradesstables on each node. The replication factor still doesn't report as I 
would expect. Am I missing something here?

- .Dustin



Re: Amazingly bad compaction performance

2012-06-28 Thread Dustin Wenz
My maximum and initial heap sizes are set to 6GB. Actual memory usage for the 
VM is around 11-12GB. The machine has 24GB of physical memory, so there isn't 
any paging going in.

I don't see any GC events logged that are longer than a few hundred 
milliseconds. Is it possible that GC is taking significant time without it 
being reported?

- .Dustin

On Jun 27, 2012, at 1:31 AM, Igor wrote:

> Hello
> 
> Too much GC? Check JVM heap settings and real usage.
> 
> On 06/27/2012 01:37 AM, Dustin Wenz wrote:
>> We occasionally see fairly poor compaction performance on random nodes in 
>> our 7-node cluster, and I have no idea why. This is one example from the log:
>> 
>>  [CompactionExecutor:45] 2012-06-26 13:40:18,721 CompactionTask.java 
>> (line 221) Compacted to 
>> [/raid00/cassandra_data/main/basic/main-basic.basic_id_index-hd-160-Data.db,].
>>   26,632,210 to 26,679,667 (~100% of original) bytes for 2 keys at 
>> 0.006250MB/s.  Time: 4,071,163ms.
>> 
>> That particular event took over an hour to compact only 25 megabytes. During 
>> that time, there was very little disk IO, and the java process (OpenJDK 7) 
>> was pegged at 200% CPU. The node was also completely unresponsive to network 
>> requests until the compaction was finished. Most compactions run just over 
>> 7MB/s. This is an extreme outlier, but users definitely notice the hit when 
>> it occurs.
>> 
>> I grabbed a sample of the process using jstack, and this was the only thread 
>> in CompactionExecutor:
>> 
>>  "CompactionExecutor:54" daemon prio=1 tid=41247522816 nid=0x99a5ff740 
>> runnable [140737253617664]
>> java.lang.Thread.State: RUNNABLE
>>  at org.xerial.snappy.SnappyNative.rawCompress(Native Method)
>>  at org.xerial.snappy.Snappy.rawCompress(Snappy.java:358)
>>  at 
>> org.apache.cassandra.io.compress.SnappyCompressor.compress(SnappyCompressor.java:80)
>>  at 
>> org.apache.cassandra.io.compress.CompressedSequentialWriter.flushData(CompressedSequentialWriter.java:89)
>>  at 
>> org.apache.cassandra.io.util.SequentialWriter.flushInternal(SequentialWriter.java:196)
>>  at 
>> org.apache.cassandra.io.util.SequentialWriter.reBuffer(SequentialWriter.java:260)
>>  at 
>> org.apache.cassandra.io.util.SequentialWriter.writeAtMost(SequentialWriter.java:128)
>>  at 
>> org.apache.cassandra.io.util.SequentialWriter.write(SequentialWriter.java:112)
>>  at java.io.DataOutputStream.write(DataOutputStream.java:107)
>>  - locked <36527862064> (a java.io.DataOutputStream)
>>  at 
>> org.apache.cassandra.db.compaction.PrecompactedRow.write(PrecompactedRow.java:142)
>>  at 
>> org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:156)
>>  at 
>> org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:159)
>>  at 
>> org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(CompactionManager.java:150)
>>  at 
>> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
>>  at 
>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
>>  at 
>> java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
>>  at java.util.concurrent.FutureTask.run(FutureTask.java:166)
>>  at 
>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
>>  at 
>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
>>  at java.lang.Thread.run(Thread.java:722)
>> 
>> Is it possible that there is an issue with snappy compression? Based on the 
>> lousy compression ratio, I think we could get by without it just fine. Can 
>> compression be changed or disabled on-the-fly with cassandra?
>> 
>>  - .Dustin
> 
> 



Amazingly bad compaction performance

2012-06-26 Thread Dustin Wenz
We occasionally see fairly poor compaction performance on random nodes in our 
7-node cluster, and I have no idea why. This is one example from the log:

[CompactionExecutor:45] 2012-06-26 13:40:18,721 CompactionTask.java 
(line 221) Compacted to 
[/raid00/cassandra_data/main/basic/main-basic.basic_id_index-hd-160-Data.db,].  
26,632,210 to 26,679,667 (~100% of original) bytes for 2 keys at 0.006250MB/s.  
Time: 4,071,163ms.

That particular event took over an hour to compact only 25 megabytes. During 
that time, there was very little disk IO, and the java process (OpenJDK 7) was 
pegged at 200% CPU. The node was also completely unresponsive to network 
requests until the compaction was finished. Most compactions run just over 
7MB/s. This is an extreme outlier, but users definitely notice the hit when it 
occurs.

I grabbed a sample of the process using jstack, and this was the only thread in 
CompactionExecutor:

"CompactionExecutor:54" daemon prio=1 tid=41247522816 nid=0x99a5ff740 
runnable [140737253617664]
   java.lang.Thread.State: RUNNABLE
at org.xerial.snappy.SnappyNative.rawCompress(Native Method)
at org.xerial.snappy.Snappy.rawCompress(Snappy.java:358)
at 
org.apache.cassandra.io.compress.SnappyCompressor.compress(SnappyCompressor.java:80)
at 
org.apache.cassandra.io.compress.CompressedSequentialWriter.flushData(CompressedSequentialWriter.java:89)
at 
org.apache.cassandra.io.util.SequentialWriter.flushInternal(SequentialWriter.java:196)
at 
org.apache.cassandra.io.util.SequentialWriter.reBuffer(SequentialWriter.java:260)
at 
org.apache.cassandra.io.util.SequentialWriter.writeAtMost(SequentialWriter.java:128)
at 
org.apache.cassandra.io.util.SequentialWriter.write(SequentialWriter.java:112)
at java.io.DataOutputStream.write(DataOutputStream.java:107)
- locked <36527862064> (a java.io.DataOutputStream)
at 
org.apache.cassandra.db.compaction.PrecompactedRow.write(PrecompactedRow.java:142)
at 
org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:156)
at 
org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:159)
at 
org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(CompactionManager.java:150)
at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
at 
java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
at java.util.concurrent.FutureTask.run(FutureTask.java:166)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
at java.lang.Thread.run(Thread.java:722)

Is it possible that there is an issue with snappy compression? Based on the 
lousy compression ratio, I think we could get by without it just fine. Can 
compression be changed or disabled on-the-fly with cassandra?

- .Dustin

Cassandra 1.1.1 crash during compaction

2012-06-07 Thread Dustin Wenz
We observed a JRE crash on one node in a seven node cluster about a half hour 
after upgrading to version 1.1.1 yesterday. Immediately after the upgrade, 
everything seemed to be working fine. The last item in the cassandra log was a 
info-level notification that compaction had started on a data file. Four 
minutes later, the process crashed.

The host OS is FreeBSD 8.2, built for the amd64 architecture. Most of the 
cluster settings are left to their defaults and the replication factor is set 
to 2 for our keyspace. We are using the RandomPartitioner and 
RackInferringSnitch. JNA is enabled, but cannot use mlockall since the process 
runs as a non-privileged user. It was also necessary to build our own Snappy 
compressor jar file, since the required architecture was not built-in to the 
public distribution.

Cassandra is a fairly new software deployment for us, and I was hoping someone 
could give me some pointers on interpreting the crash report below.

Thanks,

- .Dustin

#
# An unexpected error has been detected by Java Runtime Environment:
#
#  SIGBUS (0xa) at pc=0x000801199140, pid=44897, tid=0x8d1fdc80
#
# Java VM: Diablo Java HotSpot(TM) 64-Bit Server VM (10.0-b23 mixed mode 
bsd-amd64)
# Problematic frame:
# V  [libjvm.so+0x599140]
#
# Please submit bug reports to freebsd-j...@freebsd.org
#

---  T H R E A D  ---

Current thread (0x000aa9d56000):  JavaThread "CompactionExecutor:30" daemon 
[_thread_in_vm, id=-1927291776, stack(0x7898e000,0x78a8e000)]

siginfo:si_signo=SIGBUS: si_errno=0, si_code=3 (BUS_OBJERR), 
si_addr=0x000801199140

Registers:
RAX=0x000aa95f8fe8, RBX=0x000aa95fc2c0, RCX=0x000987a334f0, 
RDX=0x0009274e9888
RSP=0x78a8d630, RBP=0x78a8d640, RSI=0x0009274e9888, 
RDI=0xc90009274e99
R8 =0x00098c203bd8, R9 =0x0008809ff4b8, R10=0x000801488580, 
R11=0x0001
R12=0x000aa9d56000, R13=0x000aa95f8c00, R14=0x78a8d818, 
R15=0x000aa95f8c10
RIP=0x000801199140, EFL=0x003b003b0001, ERR=0x
  TRAPNO=0x001b00130009

Top of Stack: (sp=0x78a8d630)
0x78a8d630:   000aa95fc2c0 000aa9d56000
0x78a8d640:   78a8d660 00080119917e
0x78a8d650:   00080345b690 000aa95fc2c0
0x78a8d660:   78a8d6a0 000800f3a1fd
0x78a8d670:   000aa95f8fe8 0009274e9869
0x78a8d680:   000986f7acb0 000986f7e591
0x78a8d690:   78a8d818 000aa9d56000
0x78a8d6a0:   78a8d700 00080346556f
0x78a8d6b0:   0009274e9888 00080346553b
0x78a8d6c0:   78a8d6c0 000986f7e591
0x78a8d6d0:   78a8d818 000987a30198
0x78a8d6e0:   000987a334f0 000986f7e918
0x78a8d6f0:    78a8d810
0x78a8d700:   78a8d870 00080345c04e
0x78a8d710:    
0x78a8d720:    
0x78a8d730:    
0x78a8d740:   0009274aa428 00080817df80
0x78a8d750:   0008081714c0 
0x78a8d760:   00080817df30 
0x78a8d770:   00091b5ed0a0 
0x78a8d780:    
0x78a8d790:    
0x78a8d7a0:    
0x78a8d7b0:    
0x78a8d7c0:    
0x78a8d7d0:    0017974b
0x78a8d7e0:   deaddeaf 0137c3fdd558
0x78a8d7f0:   deaddeaf 
0x78a8d800:    00091b638910
0x78a8d810:    
0x78a8d820:   000882d0d2e8 0008406a3de8 

Instructions: (pc=0x000801199140)
0x000801199130:   55 48 89 e5 48 89 5d f0 4c 89 65 f8 48 83 ec 10
0x000801199140:   0f b7 47 10 48 89 fb 44 8d 60 01 49 63 fc e8 4d 

Stack: [0x7898e000,0x78a8e000],  sp=0x78a8d630,  free 
space=1021k
Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native code)
V  [libjvm.so+0x599140]
V  [libjvm.so+0x59917e]
V  [libjvm.so+0x33a1fd]
v  ~BufferBlob::Interpreter
v  ~BufferBlob::Interpreter
v  ~BufferBlob::Interpreter
v  ~BufferBlob::Interpreter
v  ~BufferBlob::Interpreter
v  ~BufferBlob::Interpreter

Java frames: (J=compiled Java code, j=interpreted, Vv=VM code)
v  ~BufferBlob::Interpreter
v  ~BufferBlob::Interpreter
v  ~BufferBlob::Interpreter
v  ~BufferBlob::Interpreter
v  ~BufferBlob::Interpreter
v  ~BufferBlob::Interpreter
J  java.util.concurrent.ThreadPoolExecutor$Worker.run()V
v  ~BufferBlob::Interpreter
v  ~BufferBlob::StubRoutines (1)

---  P R O C E S S  ---

Java Threads: ( => current thread )
=>0x000aa9