You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by Russ Lavoie <us...@yahoo.com> on 2014/03/28 19:15:33 UTC

Cassandra Snapshots giving me corrupted SSTables in the logs

We are using cassandra 1.2.10 (With JNA installed) on ubuntu 12.04.3 and are running our instances in Amazon Web Services.

What I am trying to do.

Our cassandra systems data is on an EBS volume so we can take snapshots of the data and create volumes based on those snapshots and restore them where we want to.

The snapshot process 

Step 1
Login to  the cassandra node.

Step 2
Run nodetool clearsnapshot

Step 3
Run nodetool snapshot

Step 4
Take EBS snapshot

The above steps are performed only after the previous command returns.

Restore Process

Step 1
Remove data/system, commit_log and the saved_caches data/<keyspace>/* (excluding the snapshot directory)

Step 2
Move all snapshot files into their respective KS/CF locations

Step 3
Start Cassandra

Step 4 
Create the schema

Step 5
Look at the log.  This is where I find a corrupted sstable in our keyspace (not system).

Trouble shooting

I suspected a race condition so I did the following:

I inserted a sleep for 60 seconds after issuing “nodetool clearsnapshot” 
I inserted a sleep for 60 seconds after issuing “nodetool snapshot”

Took the snapshot
Restored the snapshot as stated above following those same steps.
It worked with no problem at all.

So my assumption is that Cassandra is doing a few more things after the “nodetool snapshot” returns.

Now that you know what is going on, I have my question.

How can I tell when a snapshot is fully complete so I do not have corrupted SSTables?

I can reproduce this 100% of the time.

Thanks for your help

Re: Cassandra Snapshots giving me corrupted SSTables in the logs

Posted by "Laing, Michael" <mi...@nytimes.com>.
In your step 4, be sure you create a consistent EBS snapshot. You may have
pieces of your sstables that have not actually been flushed all the way to
EBS.

See https://github.com/alestic/ec2-consistent-snapshot

ml


On Fri, Mar 28, 2014 at 3:21 PM, Russ Lavoie <us...@yahoo.com> wrote:

> Thank you for your quick response.
>
> Is there a way to tell when a snapshot is completely done?
>
>
>   On Friday, March 28, 2014 1:30 PM, Robert Coli <rc...@eventbrite.com>
> wrote:
>  On Fri, Mar 28, 2014 at 11:15 AM, Russ Lavoie <us...@yahoo.com>wrote:
>
> We are using cassandra 1.2.10 (With JNA installed) on ubuntu 12.04.3 and
> are running our instances in Amazon Web Services.
>
>
>
>  Our cassandra systems data is on an EBS volume
>
>
> Best practice for Cassandra on AWS is to run on ephemeral stripe, not EBS.
>
>
>  so we can take snapshots of the data and create volumes based on those
> snapshots and restore them where we want to.
>
>
> https://github.com/synack/tablesnap
>
>
> ?
>
>
>  How can I tell when a snapshot is fully complete so I do not have
> corrupted SSTables?
>
>
> SStables are immutable after they are created. I'm not sure how you're
> getting a snapshot that has corrupted SSTables in it. If you can repro
> reliably, file a JIRA on issues.apache.org.
>
> =Rob
>
>
>
>

Re: Cassandra Snapshots giving me corrupted SSTables in the logs

Posted by Jonathan Haddad <jo...@jonhaddad.com>.
Another thing to keep in mind is that if you are hitting the issue I
described, waiting 60 seconds will not absolutely solve your problem, it
will only make it less likely to occur.  If a memtable has been partially
flushed at the 60 second mark you will end up with the same corrupt sstable.


On Fri, Mar 28, 2014 at 1:32 PM, Laing, Michael
<mi...@nytimes.com>wrote:

> +1 for tablesnap
>
>
> On Fri, Mar 28, 2014 at 4:28 PM, Jonathan Haddad <jo...@jonhaddad.com>wrote:
>
>> I will +1 the recommendation on using tablesnap over EBS.  S3 is at least
>> predictable.
>>
>> Additionally, from a practical standpoint, you may want to back up your
>> sstables somewhere.  If you use S3, it's easy to pull just the new tables
>> out via aws-cli tools (s3 sync), to your remote, non-aws server, and not
>> incur the overhead of routinely backing up the entire dataset.  For a non
>> trivial database, this matters quite a bit.
>>
>>
>> On Fri, Mar 28, 2014 at 1:21 PM, Laing, Michael <
>> michael.laing@nytimes.com> wrote:
>>
>>> As I tried to say, EBS snapshots require much care or you get corruption
>>> such as you have encountered.
>>>
>>> Does Cassandra quiesce the file system after a snapshot using fsfreeze
>>> or xfs_freeze? Somehow I doubt it...
>>>
>>>
>>> On Fri, Mar 28, 2014 at 4:17 PM, Jonathan Haddad <jo...@jonhaddad.com>wrote:
>>>
>>>> I have a nagging memory of reading about issues with virtualization and
>>>> not actually having durable versions of your data even after an fsync
>>>> (within the VM).  Googling around lead me to this post:
>>>> http://petercai.com/virtualization-is-bad-for-database-integrity/
>>>>
>>>> It's possible you're hitting this issue, with with the virtualization
>>>> layer, or with EBS itself.  Just a shot in the dark though, other people
>>>> would likely know much more than I.
>>>>
>>>>
>>>>
>>>> On Fri, Mar 28, 2014 at 12:50 PM, Russ Lavoie <us...@yahoo.com>wrote:
>>>>
>>>>> Robert,
>>>>>
>>>>> That is what I thought as well.  But apparently something is
>>>>> happening.  The only way I can get away with doing this is adding a sleep
>>>>> 60 right after the nodetool snapshot is executed.  I can reproduce this
>>>>> 100% of the time by not issuing a sleep after nodetool snapshot.
>>>>>
>>>>> This is the error.
>>>>>
>>>>> ERROR [SSTableBatchOpen:1] 2014-03-28 17:08:14,290
>>>>> CassandraDaemon.java (line 191) Exception in thread
>>>>> Thread[SSTableBatchOpen:1,5,main]
>>>>> org.apache.cassandra.io.sstable.CorruptSSTableException:
>>>>> java.io.EOFException
>>>>> at
>>>>> org.apache.cassandra.io.compress.CompressionMetadata.<init>(CompressionMetadata.java:108)
>>>>> at
>>>>> org.apache.cassandra.io.compress.CompressionMetadata.create(CompressionMetadata.java:63)
>>>>>  at
>>>>> org.apache.cassandra.io.util.CompressedPoolingSegmentedFile$Builder.complete(CompressedPoolingSegmentedFile.java:42)
>>>>> at
>>>>> org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:407)
>>>>>  at
>>>>> org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:198)
>>>>> at
>>>>> org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:157)
>>>>> at
>>>>> org.apache.cassandra.io.sstable.SSTableReader$1.run(SSTableReader.java:262)
>>>>> 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)
>>>>> Caused by: java.io.EOFException
>>>>> at java.io.DataInputStream.readUnsignedShort(DataInputStream.java:340)
>>>>> at java.io.DataInputStream.readUTF(DataInputStream.java:589)
>>>>> at java.io.DataInputStream.readUTF(DataInputStream.java:564)
>>>>> at
>>>>> org.apache.cassandra.io.compress.CompressionMetadata.<init>(CompressionMetadata.java:83)
>>>>>  ... 11 more
>>>>>
>>>>>
>>>>>   On Friday, March 28, 2014 2:38 PM, Robert Coli <rc...@eventbrite.com>
>>>>> wrote:
>>>>>  On Fri, Mar 28, 2014 at 12:21 PM, Russ Lavoie <us...@yahoo.com>wrote:
>>>>>
>>>>> Thank you for your quick response.
>>>>>
>>>>> Is there a way to tell when a snapshot is completely done?
>>>>>
>>>>>
>>>>> IIRC, the JMX call blocks until the snapshot completes. It should be
>>>>> done when nodetool returns.
>>>>>
>>>>> =Rob
>>>>>
>>>>>
>>>>>
>>>>
>>>>
>>>> --
>>>> Jon Haddad
>>>> http://www.rustyrazorblade.com
>>>> skype: rustyrazorblade
>>>>
>>>
>>>
>>
>>
>> --
>> Jon Haddad
>> http://www.rustyrazorblade.com
>> skype: rustyrazorblade
>>
>
>


-- 
Jon Haddad
http://www.rustyrazorblade.com
skype: rustyrazorblade

Re: Cassandra Snapshots giving me corrupted SSTables in the logs

Posted by "Laing, Michael" <mi...@nytimes.com>.
+1 for tablesnap


On Fri, Mar 28, 2014 at 4:28 PM, Jonathan Haddad <jo...@jonhaddad.com> wrote:

> I will +1 the recommendation on using tablesnap over EBS.  S3 is at least
> predictable.
>
> Additionally, from a practical standpoint, you may want to back up your
> sstables somewhere.  If you use S3, it's easy to pull just the new tables
> out via aws-cli tools (s3 sync), to your remote, non-aws server, and not
> incur the overhead of routinely backing up the entire dataset.  For a non
> trivial database, this matters quite a bit.
>
>
> On Fri, Mar 28, 2014 at 1:21 PM, Laing, Michael <michael.laing@nytimes.com
> > wrote:
>
>> As I tried to say, EBS snapshots require much care or you get corruption
>> such as you have encountered.
>>
>> Does Cassandra quiesce the file system after a snapshot using fsfreeze or
>> xfs_freeze? Somehow I doubt it...
>>
>>
>> On Fri, Mar 28, 2014 at 4:17 PM, Jonathan Haddad <jo...@jonhaddad.com>wrote:
>>
>>> I have a nagging memory of reading about issues with virtualization and
>>> not actually having durable versions of your data even after an fsync
>>> (within the VM).  Googling around lead me to this post:
>>> http://petercai.com/virtualization-is-bad-for-database-integrity/
>>>
>>> It's possible you're hitting this issue, with with the virtualization
>>> layer, or with EBS itself.  Just a shot in the dark though, other people
>>> would likely know much more than I.
>>>
>>>
>>>
>>> On Fri, Mar 28, 2014 at 12:50 PM, Russ Lavoie <us...@yahoo.com>wrote:
>>>
>>>> Robert,
>>>>
>>>> That is what I thought as well.  But apparently something is happening.
>>>>  The only way I can get away with doing this is adding a sleep 60 right
>>>> after the nodetool snapshot is executed.  I can reproduce this 100% of the
>>>> time by not issuing a sleep after nodetool snapshot.
>>>>
>>>> This is the error.
>>>>
>>>> ERROR [SSTableBatchOpen:1] 2014-03-28 17:08:14,290 CassandraDaemon.java
>>>> (line 191) Exception in thread Thread[SSTableBatchOpen:1,5,main]
>>>> org.apache.cassandra.io.sstable.CorruptSSTableException:
>>>> java.io.EOFException
>>>> at
>>>> org.apache.cassandra.io.compress.CompressionMetadata.<init>(CompressionMetadata.java:108)
>>>> at
>>>> org.apache.cassandra.io.compress.CompressionMetadata.create(CompressionMetadata.java:63)
>>>>  at
>>>> org.apache.cassandra.io.util.CompressedPoolingSegmentedFile$Builder.complete(CompressedPoolingSegmentedFile.java:42)
>>>> at
>>>> org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:407)
>>>>  at
>>>> org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:198)
>>>> at
>>>> org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:157)
>>>> at
>>>> org.apache.cassandra.io.sstable.SSTableReader$1.run(SSTableReader.java:262)
>>>> 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)
>>>> Caused by: java.io.EOFException
>>>> at java.io.DataInputStream.readUnsignedShort(DataInputStream.java:340)
>>>> at java.io.DataInputStream.readUTF(DataInputStream.java:589)
>>>> at java.io.DataInputStream.readUTF(DataInputStream.java:564)
>>>> at
>>>> org.apache.cassandra.io.compress.CompressionMetadata.<init>(CompressionMetadata.java:83)
>>>>  ... 11 more
>>>>
>>>>
>>>>   On Friday, March 28, 2014 2:38 PM, Robert Coli <rc...@eventbrite.com>
>>>> wrote:
>>>>  On Fri, Mar 28, 2014 at 12:21 PM, Russ Lavoie <us...@yahoo.com>wrote:
>>>>
>>>> Thank you for your quick response.
>>>>
>>>> Is there a way to tell when a snapshot is completely done?
>>>>
>>>>
>>>> IIRC, the JMX call blocks until the snapshot completes. It should be
>>>> done when nodetool returns.
>>>>
>>>> =Rob
>>>>
>>>>
>>>>
>>>
>>>
>>> --
>>> Jon Haddad
>>> http://www.rustyrazorblade.com
>>> skype: rustyrazorblade
>>>
>>
>>
>
>
> --
> Jon Haddad
> http://www.rustyrazorblade.com
> skype: rustyrazorblade
>

Re: Cassandra Snapshots giving me corrupted SSTables in the logs

Posted by Jonathan Haddad <jo...@jonhaddad.com>.
I will +1 the recommendation on using tablesnap over EBS.  S3 is at least
predictable.

Additionally, from a practical standpoint, you may want to back up your
sstables somewhere.  If you use S3, it's easy to pull just the new tables
out via aws-cli tools (s3 sync), to your remote, non-aws server, and not
incur the overhead of routinely backing up the entire dataset.  For a non
trivial database, this matters quite a bit.


On Fri, Mar 28, 2014 at 1:21 PM, Laing, Michael
<mi...@nytimes.com>wrote:

> As I tried to say, EBS snapshots require much care or you get corruption
> such as you have encountered.
>
> Does Cassandra quiesce the file system after a snapshot using fsfreeze or
> xfs_freeze? Somehow I doubt it...
>
>
> On Fri, Mar 28, 2014 at 4:17 PM, Jonathan Haddad <jo...@jonhaddad.com>wrote:
>
>> I have a nagging memory of reading about issues with virtualization and
>> not actually having durable versions of your data even after an fsync
>> (within the VM).  Googling around lead me to this post:
>> http://petercai.com/virtualization-is-bad-for-database-integrity/
>>
>> It's possible you're hitting this issue, with with the virtualization
>> layer, or with EBS itself.  Just a shot in the dark though, other people
>> would likely know much more than I.
>>
>>
>>
>> On Fri, Mar 28, 2014 at 12:50 PM, Russ Lavoie <us...@yahoo.com>wrote:
>>
>>> Robert,
>>>
>>> That is what I thought as well.  But apparently something is happening.
>>>  The only way I can get away with doing this is adding a sleep 60 right
>>> after the nodetool snapshot is executed.  I can reproduce this 100% of the
>>> time by not issuing a sleep after nodetool snapshot.
>>>
>>> This is the error.
>>>
>>> ERROR [SSTableBatchOpen:1] 2014-03-28 17:08:14,290 CassandraDaemon.java
>>> (line 191) Exception in thread Thread[SSTableBatchOpen:1,5,main]
>>> org.apache.cassandra.io.sstable.CorruptSSTableException:
>>> java.io.EOFException
>>> at
>>> org.apache.cassandra.io.compress.CompressionMetadata.<init>(CompressionMetadata.java:108)
>>> at
>>> org.apache.cassandra.io.compress.CompressionMetadata.create(CompressionMetadata.java:63)
>>>  at
>>> org.apache.cassandra.io.util.CompressedPoolingSegmentedFile$Builder.complete(CompressedPoolingSegmentedFile.java:42)
>>> at
>>> org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:407)
>>>  at
>>> org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:198)
>>> at
>>> org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:157)
>>> at
>>> org.apache.cassandra.io.sstable.SSTableReader$1.run(SSTableReader.java:262)
>>> 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)
>>> Caused by: java.io.EOFException
>>> at java.io.DataInputStream.readUnsignedShort(DataInputStream.java:340)
>>> at java.io.DataInputStream.readUTF(DataInputStream.java:589)
>>> at java.io.DataInputStream.readUTF(DataInputStream.java:564)
>>> at
>>> org.apache.cassandra.io.compress.CompressionMetadata.<init>(CompressionMetadata.java:83)
>>>  ... 11 more
>>>
>>>
>>>   On Friday, March 28, 2014 2:38 PM, Robert Coli <rc...@eventbrite.com>
>>> wrote:
>>>  On Fri, Mar 28, 2014 at 12:21 PM, Russ Lavoie <us...@yahoo.com>wrote:
>>>
>>> Thank you for your quick response.
>>>
>>> Is there a way to tell when a snapshot is completely done?
>>>
>>>
>>> IIRC, the JMX call blocks until the snapshot completes. It should be
>>> done when nodetool returns.
>>>
>>> =Rob
>>>
>>>
>>>
>>
>>
>> --
>> Jon Haddad
>> http://www.rustyrazorblade.com
>> skype: rustyrazorblade
>>
>
>


-- 
Jon Haddad
http://www.rustyrazorblade.com
skype: rustyrazorblade

Re: Cassandra Snapshots giving me corrupted SSTables in the logs

Posted by "Laing, Michael" <mi...@nytimes.com>.
As I tried to say, EBS snapshots require much care or you get corruption
such as you have encountered.

Does Cassandra quiesce the file system after a snapshot using fsfreeze or
xfs_freeze? Somehow I doubt it...


On Fri, Mar 28, 2014 at 4:17 PM, Jonathan Haddad <jo...@jonhaddad.com> wrote:

> I have a nagging memory of reading about issues with virtualization and
> not actually having durable versions of your data even after an fsync
> (within the VM).  Googling around lead me to this post:
> http://petercai.com/virtualization-is-bad-for-database-integrity/
>
> It's possible you're hitting this issue, with with the virtualization
> layer, or with EBS itself.  Just a shot in the dark though, other people
> would likely know much more than I.
>
>
>
> On Fri, Mar 28, 2014 at 12:50 PM, Russ Lavoie <us...@yahoo.com> wrote:
>
>> Robert,
>>
>> That is what I thought as well.  But apparently something is happening.
>>  The only way I can get away with doing this is adding a sleep 60 right
>> after the nodetool snapshot is executed.  I can reproduce this 100% of the
>> time by not issuing a sleep after nodetool snapshot.
>>
>> This is the error.
>>
>> ERROR [SSTableBatchOpen:1] 2014-03-28 17:08:14,290 CassandraDaemon.java
>> (line 191) Exception in thread Thread[SSTableBatchOpen:1,5,main]
>> org.apache.cassandra.io.sstable.CorruptSSTableException:
>> java.io.EOFException
>> at
>> org.apache.cassandra.io.compress.CompressionMetadata.<init>(CompressionMetadata.java:108)
>> at
>> org.apache.cassandra.io.compress.CompressionMetadata.create(CompressionMetadata.java:63)
>>  at
>> org.apache.cassandra.io.util.CompressedPoolingSegmentedFile$Builder.complete(CompressedPoolingSegmentedFile.java:42)
>> at
>> org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:407)
>>  at
>> org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:198)
>> at
>> org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:157)
>> at
>> org.apache.cassandra.io.sstable.SSTableReader$1.run(SSTableReader.java:262)
>> 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)
>> Caused by: java.io.EOFException
>> at java.io.DataInputStream.readUnsignedShort(DataInputStream.java:340)
>> at java.io.DataInputStream.readUTF(DataInputStream.java:589)
>> at java.io.DataInputStream.readUTF(DataInputStream.java:564)
>> at
>> org.apache.cassandra.io.compress.CompressionMetadata.<init>(CompressionMetadata.java:83)
>>  ... 11 more
>>
>>
>>   On Friday, March 28, 2014 2:38 PM, Robert Coli <rc...@eventbrite.com>
>> wrote:
>>  On Fri, Mar 28, 2014 at 12:21 PM, Russ Lavoie <us...@yahoo.com>wrote:
>>
>> Thank you for your quick response.
>>
>> Is there a way to tell when a snapshot is completely done?
>>
>>
>> IIRC, the JMX call blocks until the snapshot completes. It should be done
>> when nodetool returns.
>>
>> =Rob
>>
>>
>>
>
>
> --
> Jon Haddad
> http://www.rustyrazorblade.com
> skype: rustyrazorblade
>

Re: Cassandra Snapshots giving me corrupted SSTables in the logs

Posted by Jonathan Haddad <jo...@jonhaddad.com>.
I have a nagging memory of reading about issues with virtualization and not
actually having durable versions of your data even after an fsync (within
the VM).  Googling around lead me to this post:
http://petercai.com/virtualization-is-bad-for-database-integrity/

It's possible you're hitting this issue, with with the virtualization
layer, or with EBS itself.  Just a shot in the dark though, other people
would likely know much more than I.



On Fri, Mar 28, 2014 at 12:50 PM, Russ Lavoie <us...@yahoo.com> wrote:

> Robert,
>
> That is what I thought as well.  But apparently something is happening.
>  The only way I can get away with doing this is adding a sleep 60 right
> after the nodetool snapshot is executed.  I can reproduce this 100% of the
> time by not issuing a sleep after nodetool snapshot.
>
> This is the error.
>
> ERROR [SSTableBatchOpen:1] 2014-03-28 17:08:14,290 CassandraDaemon.java
> (line 191) Exception in thread Thread[SSTableBatchOpen:1,5,main]
> org.apache.cassandra.io.sstable.CorruptSSTableException:
> java.io.EOFException
> at
> org.apache.cassandra.io.compress.CompressionMetadata.<init>(CompressionMetadata.java:108)
> at
> org.apache.cassandra.io.compress.CompressionMetadata.create(CompressionMetadata.java:63)
> at
> org.apache.cassandra.io.util.CompressedPoolingSegmentedFile$Builder.complete(CompressedPoolingSegmentedFile.java:42)
> at
> org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:407)
> at
> org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:198)
> at
> org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:157)
> at
> org.apache.cassandra.io.sstable.SSTableReader$1.run(SSTableReader.java:262)
> 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)
> Caused by: java.io.EOFException
> at java.io.DataInputStream.readUnsignedShort(DataInputStream.java:340)
> at java.io.DataInputStream.readUTF(DataInputStream.java:589)
> at java.io.DataInputStream.readUTF(DataInputStream.java:564)
> at
> org.apache.cassandra.io.compress.CompressionMetadata.<init>(CompressionMetadata.java:83)
>  ... 11 more
>
>
>   On Friday, March 28, 2014 2:38 PM, Robert Coli <rc...@eventbrite.com>
> wrote:
>  On Fri, Mar 28, 2014 at 12:21 PM, Russ Lavoie <us...@yahoo.com>wrote:
>
> Thank you for your quick response.
>
> Is there a way to tell when a snapshot is completely done?
>
>
> IIRC, the JMX call blocks until the snapshot completes. It should be done
> when nodetool returns.
>
> =Rob
>
>
>


-- 
Jon Haddad
http://www.rustyrazorblade.com
skype: rustyrazorblade

Re: Cassandra Snapshots giving me corrupted SSTables in the logs

Posted by Russ Lavoie <us...@yahoo.com>.
Robert,

That is what I thought as well.  But apparently something is happening.  The only way I can get away with doing this is adding a sleep 60 right after the nodetool snapshot is executed.  I can reproduce this 100% of the time by not issuing a sleep after nodetool snapshot.

This is the error.

ERROR [SSTableBatchOpen:1] 2014-03-28 17:08:14,290 CassandraDaemon.java (line 191) Exception in thread Thread[SSTableBatchOpen:1,5,main]
org.apache.cassandra.io.sstable.CorruptSSTableException: java.io.EOFException
at org.apache.cassandra.io.compress.CompressionMetadata.<init>(CompressionMetadata.java:108)
at org.apache.cassandra.io.compress.CompressionMetadata.create(CompressionMetadata.java:63)
at org.apache.cassandra.io.util.CompressedPoolingSegmentedFile$Builder.complete(CompressedPoolingSegmentedFile.java:42)
at org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:407)
at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:198)
at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:157)
at org.apache.cassandra.io.sstable.SSTableReader$1.run(SSTableReader.java:262)
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)
Caused by: java.io.EOFException
at java.io.DataInputStream.readUnsignedShort(DataInputStream.java:340)
at java.io.DataInputStream.readUTF(DataInputStream.java:589)
at java.io.DataInputStream.readUTF(DataInputStream.java:564)
at org.apache.cassandra.io.compress.CompressionMetadata.<init>(CompressionMetadata.java:83)
... 11 more



On Friday, March 28, 2014 2:38 PM, Robert Coli <rc...@eventbrite.com> wrote:
 
On Fri, Mar 28, 2014 at 12:21 PM, Russ Lavoie <us...@yahoo.com> wrote:

Thank you for your quick response.
>
>
>Is there a way to tell when a snapshot is completely done?

IIRC, the JMX call blocks until the snapshot completes. It should be done when nodetool returns.


=Rob

Re: Cassandra Snapshots giving me corrupted SSTables in the logs

Posted by Robert Coli <rc...@eventbrite.com>.
On Fri, Mar 28, 2014 at 12:21 PM, Russ Lavoie <us...@yahoo.com> wrote:

> Thank you for your quick response.
>
> Is there a way to tell when a snapshot is completely done?
>

IIRC, the JMX call blocks until the snapshot completes. It should be done
when nodetool returns.

=Rob

Re: Cassandra Snapshots giving me corrupted SSTables in the logs

Posted by Russ Lavoie <us...@yahoo.com>.
Thank you for your quick response.

Is there a way to tell when a snapshot is completely done?



On Friday, March 28, 2014 1:30 PM, Robert Coli <rc...@eventbrite.com> wrote:
 
On Fri, Mar 28, 2014 at 11:15 AM, Russ Lavoie <us...@yahoo.com> wrote:

We are using cassandra 1.2.10 (With JNA installed) on ubuntu 12.04.3 and are running our instances in Amazon Web Services.
 
Our cassandra systems data is on an EBS volume

Best practice for Cassandra on AWS is to run on ephemeral stripe, not EBS.
 
so we can take snapshots of the data and create volumes based on those snapshots and restore them where we want to.

https://github.com/synack/tablesnap



?
 
How can I tell when a snapshot is fully complete so I do not have corrupted SSTables?

SStables are immutable after they are created. I'm not sure how you're getting a snapshot that has corrupted SSTables in it. If you can repro reliably, file a JIRA on issues.apache.org.

=Rob

Re: Cassandra Snapshots giving me corrupted SSTables in the logs

Posted by Robert Coli <rc...@eventbrite.com>.
On Fri, Mar 28, 2014 at 11:15 AM, Russ Lavoie <us...@yahoo.com> wrote:

> We are using cassandra 1.2.10 (With JNA installed) on ubuntu 12.04.3 and
> are running our instances in Amazon Web Services.
>


> Our cassandra systems data is on an EBS volume
>

Best practice for Cassandra on AWS is to run on ephemeral stripe, not EBS.


> so we can take snapshots of the data and create volumes based on those
> snapshots and restore them where we want to.
>

https://github.com/synack/tablesnap

?


> How can I tell when a snapshot is fully complete so I do not have
> corrupted SSTables?
>

SStables are immutable after they are created. I'm not sure how you're
getting a snapshot that has corrupted SSTables in it. If you can repro
reliably, file a JIRA on issues.apache.org.

=Rob