Lanny,

We will try that.

Thanks a lot


On Thu, Apr 11, 2013 at 11:13 PM, Lanny Ripple <la...@spotright.com> wrote:

> Saw this in earlier versions. Our workaround was disable; drain; snap;
> shutdown; delete; link from snap; restart;
>
>   -ljr
>
> On Apr 11, 2013, at 9:45, <moshe.kr...@barclays.com> wrote:
>
> I have formulated the following theory regarding C* 1.2.2 which may be
> relevant: Whenever there is a disk error during compaction of an SS table
> (e.g., bad block, out of disk space), that SStable’s files stick around
> forever after, and do not subsequently get deleted by normal compaction
> (minor or major), long after all its records have been deleted. This causes
> disk usage to rise dramatically. The only way to make the SStable files
> disappear is to run “nodetool cleanup” (which takes hours to run).****
>
> ** **
>
> Just a theory so far….****
>
> ** **
>
> *From:* Alexis Rodríguez 
> [mailto:arodrig...@inconcertcc.com<arodrig...@inconcertcc.com>]
>
> *Sent:* Thursday, April 11, 2013 5:31 PM
> *To:* user@cassandra.apache.org
> *Subject:* Re: CorruptedBlockException****
>
> ** **
>
> Aaron,****
>
> ** **
>
> It seems that we are in the same situation as Nury, we are storing a lot
> of files of ~5MB in a CF.****
>
> ** **
>
> This happens in a test cluster, with one node using cassandra 1.1.5, we
> have commitlog in a different partition than the data directory. Normally
> our tests use nearly 13 GB in data, but when the exception on compaction
> appears our disk space ramp up to:****
>
> ** **
>
> # df -h****
>
> Filesystem            Size  Used Avail Use% Mounted on****
>
> /dev/sda1             440G  330G   89G  79% /****
>
> tmpfs                 7.9G     0  7.9G   0% /lib/init/rw****
>
> udev                  7.9G  160K  7.9G   1% /dev****
>
> tmpfs                 7.9G     0  7.9G   0% /dev/shm****
>
> /dev/sdb1             459G  257G  179G  59% /cassandra****
>
> ** **
>
> # cd /cassandra/data/Repository/****
>
> ** **
>
> # ls Files/*tmp* | wc -l****
>
> 1671****
>
> ** **
>
> # du -ch Files | tail -1****
>
> 257G    total****
>
> ** **
>
> # du -ch Files/*tmp* | tail -1****
>
> 34G     total****
>
> ** **
>
> We are using cassandra 1.1.5 with one node, our schema for that keyspace
> is:****
>
> ** **
>
> [default@unknown] use Repository;****
>
> Authenticated to keyspace: Repository****
>
> [default@Repository] show schema;****
>
> create keyspace Repository****
>
>   with placement_strategy = 'NetworkTopologyStrategy'****
>
>   and strategy_options = {datacenter1 : 1}****
>
>   and durable_writes = true;****
>
> ** **
>
> use Repository;****
>
> ** **
>
> create column family Files****
>
>   with column_type = 'Standard'****
>
>   and comparator = 'UTF8Type'****
>
>   and default_validation_class = 'BytesType'****
>
>   and key_validation_class = 'BytesType'****
>
>   and read_repair_chance = 0.1****
>
>   and dclocal_read_repair_chance = 0.0****
>
>   and gc_grace = 864000****
>
>   and min_compaction_threshold = 4****
>
>   and max_compaction_threshold = 32****
>
>   and replicate_on_write = true****
>
>   and compaction_strategy =
> 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy'****
>
>   and caching = 'KEYS_ONLY'****
>
>   and compaction_strategy_options = {'sstable_size_in_mb' : '120'}****
>
>   and compression_options = {'sstable_compression' :
> 'org.apache.cassandra.io.compress.SnappyCompressor'};****
>
> ** **
>
> In our logs:****
>
> ** **
>
> ERROR [CompactionExecutor:1831] 2013-04-11 09:12:41,725
> AbstractCassandraDaemon.java (line 135) Exception in thread
> Thread[CompactionExecutor:1831,1,main]****
>
> java.io.IOError: org.apache.cassandra.io.compress.CorruptedBlockException:
> (/cassandra/data/Repository/Files/Repository-Files-he-4533-Data.db):
> corruption detected, chunk at 43325354 of length 65545.****
>
>         at
> org.apache.cassandra.db.compaction.PrecompactedRow.merge(PrecompactedRow.java:116)
> ****
>
>         at
> org.apache.cassandra.db.compaction.PrecompactedRow.<init>(PrecompactedRow.java:99)
> ****
>
>         at
> org.apache.cassandra.db.compaction.CompactionController.getCompactedRow(CompactionController.java:176)
> ****
>
>         at
> org.apache.cassandra.db.compaction.CompactionIterable$Reducer.getReduced(CompactionIterable.java:83)
> ****
>
>         at
> org.apache.cassandra.db.compaction.CompactionIterable$Reducer.getReduced(CompactionIterable.java:68)
> ****
>
>         at
> org.apache.cassandra.utils.MergeIterator$ManyToOne.consume(MergeIterator.java:118)
> ****
>
>         at
> org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:101)
> ****
>
>         at
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:140)
> ****
>
>         at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:135)
> ****
>
>         at
> com.google.common.collect.Iterators$7.computeNext(Iterators.java:614)****
>
>         at
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:140)
> ****
>
>         at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:135)
> ****
>
>         at
> org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:173)
> ****
>
>         at
> org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)
> ****
>
>         at
> org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(CompactionManager.java:154)
> ****
>
>         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:636)****
>
> ** **
>
> ** **
>
>  ****
>
> ** **
>
> On Thu, Jul 5, 2012 at 7:42 PM, aaron morton <aa...@thelastpickle.com>
> wrote:****
>
> > But I don't understand, how was all the available space taken away.****
>
> Take a look on disk at /var/lib/cassandra/data/<your_keyspace> and
> /var/lib/cassandra/commitlog to see what is taking up a lot of space.
>
> Cassandra stores the column names as well as the values, so that can take
> up some space.****
>
>
> >  it says that while compaction a CorruptedBlockException has occured.***
> *
>
> Are you able to reproduce this error ?
>
> Thanks
>
>
> -----------------
> Aaron Morton
> Freelance Developer
> @aaronmorton
> http://www.thelastpickle.com****
>
>
> On 6/07/2012, at 12:04 AM, Nury Redjepow wrote:
>
> > Hello to all,
> >
> >  I have cassandra instance I'm trying to use to store millions of file
> with size ~ 3MB. Data structure is simple, 1 row for 1 file, with row key
> being the id of file.
> > I'm loaded 1GB of data, and total available space is 10GB. And after a
> few hour, all the available space was taken. In log, it says that while
> compaction a CorruptedBlockException has occured. But I don't understand,
> how was all the available space taken away.
> >
> > Data structure
> > CREATE KEYSPACE largeobjectsWITH placement_strategy = 'SimpleStrategy'
> > AND strategy_options={replication_factor:1};
> >
> > create column family content
> >   with column_type = 'Standard'
> >   and comparator = 'UTF8Type'
> >   and default_validation_class = 'BytesType'
> >   and key_validation_class = 'TimeUUIDType'
> >   and read_repair_chance = 0.1
> >   and dclocal_read_repair_chance = 0.0
> >   and gc_grace = 864000
> >   and min_compaction_threshold = 4
> >   and max_compaction_threshold = 32
> >   and replicate_on_write = true
> >   and compaction_strategy = 'SizeTieredCompactionStrategy'
> >   and caching = 'keys_only';
> >
> >
> > Log messages
> >
> > INFO [FlushWriter:9] 2012-07-04 19:56:00,783 Memtable.java (line 266)
> Writing Memtable-content@240294142(3955135/49439187 serialized/live
> bytes, 91 ops)
> > INFO [FlushWriter:9] 2012-07-04 19:56:00,814 Memtable.java (line 307)
> Completed flushing
> /var/lib/cassandra/data/largeobjects/content/largeobjects-content-h
> > d-1608-Data.db (1991862 bytes) for commitlog position
> ReplayPosition(segmentId=24245436475633, position=78253718)
> > INFO [OptionalTasks:1] 2012-07-04 19:56:02,784 MeteredFlusher.java (line
> 62) flushing high-traffic column family CFS(Keyspace='largeobjects',
> ColumnFamily='
> > content') (estimated 46971537 bytes)
> > INFO [OptionalTasks:1] 2012-07-04 19:56:02,784 ColumnFamilyStore.java
> (line 633) Enqueuing flush of Memtable-content@1755783901(3757723/46971537
> serialized/
> > live bytes, 121 ops)
> > INFO [FlushWriter:9] 2012-07-04 19:56:02,785 Memtable.java (line 266)
> Writing Memtable-content@1755783901(3757723/46971537 serialized/live
> bytes, 121 ops)
> > INFO [FlushWriter:9] 2012-07-04 19:56:02,835 Memtable.java (line 307)
> Completed flushing
> /var/lib/cassandra/data/largeobjects/content/largeobjects-content-h
> > d-1609-Data.db (1894897 bytes) for commitlog position
> ReplayPosition(segmentId=24245436475633, position=82028986)
> > INFO [OptionalTasks:1] 2012-07-04 19:56:04,785 MeteredFlusher.java (line
> 62) flushing high-traffic column family CFS(Keyspace='largeobjects',
> ColumnFamily='
> > content') (estimated 56971025 bytes)
> > INFO [OptionalTasks:1] 2012-07-04 19:56:04,785 ColumnFamilyStore.java
> (line 633) Enqueuing flush of Memtable-content@1441175031(4557682/56971025
> serialized/
> > live bytes, 124 ops)
> > INFO [FlushWriter:9] 2012-07-04 19:56:04,786 Memtable.java (line 266)
> Writing Memtable-content@1441175031(4557682/56971025 serialized/live
> bytes, 124 ops)
> > INFO [FlushWriter:9] 2012-07-04 19:56:04,814 Memtable.java (line 307)
> Completed flushing
> /var/lib/cassandra/data/largeobjects/content/largeobjects-content-h
> > d-1610-Data.db (2287280 bytes) for commitlog position
> ReplayPosition(segmentId=24245436475633, position=86604648)
> > INFO [CompactionExecutor:39] 2012-07-04 19:56:04,815 CompactionTask.java
> (line 109) Compacting
> [SSTableReader(path='/var/lib/cassandra/data/largeobjects/con
> > tent/largeobjects-content-hd-1610-Data.db'),
> SSTableReader(path='/var/lib/cassandra/data/largeobjects/content/largeobjects-content-hd-1608-Data.db'),
> SSTable
> >
> Reader(path='/var/lib/cassandra/data/largeobjects/content/largeobjects-content-hd-1609-Data.db'),
> SSTableReader(path='/var/lib/cassandra/data/largeobjects/co
> > ntent/largeobjects-content-hd-1607-Data.db')]
> > INFO [OptionalTasks:1] 2012-07-04 19:56:05,786 MeteredFlusher.java (line
> 62) flushing high-traffic column family CFS(Keyspace='largeobjects',
> ColumnFamily='
> > content') (estimated 28300225 bytes)
> > INFO [OptionalTasks:1] 2012-07-04 19:56:05,786 ColumnFamilyStore.java
> (line 633) Enqueuing flush of Memtable-content@1828084851(2264018/28300225
> serialized/
> > live bytes, 38 ops)
> > INFO [FlushWriter:9] 2012-07-04 19:56:05,787 Memtable.java (line 266)
> Writing Memtable-content@1828084851(2264018/28300225 serialized/live
> bytes, 38 ops)
> > INFO [FlushWriter:9] 2012-07-04 19:56:05,823 Memtable.java (line 307)
> Completed flushing
> /var/lib/cassandra/data/largeobjects/content/largeobjects-content-h
> > d-1612-Data.db (1134604 bytes) for commitlog position
> ReplayPosition(segmentId=24245436475633, position=88874176)
> > ERROR [CompactionExecutor:39] 2012-07-04 19:56:06,667
> AbstractCassandraDaemon.java (line 134) Exception in thread
> Thread[CompactionExecutor:39,1,main]
> > java.io.IOError:
> org.apache.cassandra.io.compress.CorruptedBlockException:
> (/var/lib/cassandra/data/largeobjects/content/largeobjects-content-hd-1610-Data.db
> > ): corruption detected, chunk at 1573104 of length 65545.
> > at
> org.apache.cassandra.db.compaction.PrecompactedRow.merge(PrecompactedRow.java:116)
> > at
> org.apache.cassandra.db.compaction.PrecompactedRow.<init>(PrecompactedRow.java:99)
> > at
> org.apache.cassandra.db.compaction.CompactionController.getCompactedRow(CompactionController.java:164)
> > at
> org.apache.cassandra.db.compaction.CompactionIterable$Reducer.getReduced(CompactionIterable.java:83)
> > at
> org.apache.cassandra.db.compaction.CompactionIterable$Reducer.getReduced(CompactionIterable.java:68)
> > at
> org.apache.cassandra.utils.MergeIterator$ManyToOne.consume(MergeIterator.java:118)
> > at
> org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:101)
> > at
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:140)
> > at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:135)
> > at com.google.common.collect.Iterators$7.computeNext(Iterators.java:614)
> > at
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:140)
> > at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:135)
> > at
> org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:173)
> > 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:679)
> > Caused by: org.apache.cassandra.io.compress.CorruptedBlockException:
> (/var/lib/cassandra/data/largeobjects/content/largeobjects-content-hd-1610-Data.db):
> cor
> > ruption detected, chunk at 1573104 of length 65545.
> > at
> org.apache.cassandra.io.compress.CompressedRandomAccessReader.decompressChunk(CompressedRandomAccessReader.java:98)
> > at
> org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBuffer(CompressedRandomAccessReader.java:77)
> > at
> org.apache.cassandra.io.util.RandomAccessReader.read(RandomAccessReader.java:302)
> > at java.io.RandomAccessFile.readFully(RandomAccessFile.java:414)
> > at java.io.RandomAccessFile.readFully(RandomAccessFile.java:394)
> > at
> org.apache.cassandra.utils.BytesReadTracker.readFully(BytesReadTracker.java:95)
> > at
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:401)
> > at
> org.apache.cassandra.utils.ByteBufferUtil.readWithLength(ByteBufferUtil.java:363)
> > at
> org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:119)
> > at
> org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:36)
> > at
> org.apache.cassandra.db.ColumnFamilySerializer.deserializeColumns(ColumnFamilySerializer.java:144)
> > at
> org.apache.cassandra.io.sstable.SSTableIdentityIterator.getColumnFamilyWithColumns(SSTableIdentityIterator.java:234)
> > at
> org.apache.cassandra.db.compaction.PrecompactedRow.merge(PrecompactedRow.java:112)
> > ... 20 more
> > INFO [OptionalTasks:1] 2012-07-04 19:57:00,796 MeteredFlusher.java (line
> 62) flushing high-traffic column family CFS(Keyspace='largeobjects',
> ColumnFamily='
> > content') (estimated 38711275 bytes)
> > INFO [OptionalTasks:1] 2012-07-04 19:57:00,796 ColumnFamilyStore.java
> (line 633) Enqueuing flush of Memtable-content@1363920595(3096902/38711275
> serialized/
> > live bytes, 74 ops)
> > INFO [FlushWriter:9] 2012-07-04 19:57:00,796 Memtable.java (line 266)
> Writing Memtable-content@1363920595(3096902/38711275 serialized/live
> bytes, 74 ops)
> > INFO [FlushWriter:9] 2012-07-04 19:57:00,821 Memtable.java (line 307)
> Completed flushing
> /var/lib/cassandra/data/largeobjects/content/largeobjects-content-h
> > d-1613-Data.db (1553451 bytes) for commitlog position
> ReplayPosition(segmentId=24245436475633, position=91981808)
> > INFO [CompactionExecutor:40] 2012-07-04 19:57:00,822 CompactionTask.java
> (line 109) Compacting
> [SSTableReader(path='/var/lib/cassandra/data/largeobjects/con
> > tent/largeobjects-content-hd-1610-Data.db'),
> SSTableReader(path='/var/lib/cassandra/data/largeobjects/content/largeobjects-content-hd-1613-Data.db'),
> SSTable
> >
> Reader(path='/var/lib/cassandra/data/largeobjects/content/largeobjects-content-hd-1608-Data.db'),
> SSTableReader(path='/var/lib/cassandra/data/largeobjects/co
> > ntent/largeobjects-content-hd-1609-Data.db'),
> SSTableReader(path='/var/lib/cassandra/data/largeobjects/content/largeobjects-content-hd-1607-Data.db'),
> SSTabl
> >
> eReader(path='/var/lib/cassandra/data/largeobjects/content/largeobjects-content-hd-1612-Data.db')]
> > INFO [OptionalTasks:1] 2012-07-04 19:57:01,797 MeteredFlusher.java (line
> 62) flushing high-traffic column family CFS(Keyspace='largeobjects',
> ColumnFamily='
> > content') (estimated 27750950 bytes)
> > INFO [OptionalTasks:1] 2012-07-04 19:57:01,797 ColumnFamilyStore.java
> (line 633) Enqueuing flush of Memtable-content@289600485(2220076/27750950
> serialized/l
> > ive bytes, 70 ops)
> > INFO [FlushWriter:9] 2012-07-04 19:57:01,797 Memtable.java (line 266)
> Writing Memtable-content@289600485(2220076/27750950 serialized/live
> bytes, 70 ops)
> > INFO [FlushWriter:9] 2012-07-04 19:57:01,819 Memtable.java (line 307)
> Completed flushing
> /var/lib/cassandra/data/largeobjects/content/largeobjects-content-h
> > d-1615-Data.db (1114538 bytes) for commitlog position
> ReplayPosition(segmentId=24245436475633, position=94212034)
> > ERROR [ReadStage:263] 2012-07-04 19:57:02,599
> AbstractCassandraDaemon.java (line 134) Exception in thread
> Thread[ReadStage:263,5,main]
> > java.lang.RuntimeException: java.lang.RuntimeException: error reading 1
> of 1
> > at
> org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1254)
> > 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:679)
> > Caused by: java.lang.RuntimeException: error reading 1 of 1
> > at
> org.apache.cassandra.db.columniterator.SimpleSliceReader.computeNext(SimpleSliceReader.java:83)
> > at
> org.apache.cassandra.db.columniterator.SimpleSliceReader.computeNext(SimpleSliceReader.java:39)
> > at
> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:140)
> > at
> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:135)
> > at
> org.apache.cassandra.db.columniterator.SSTableSliceIterator.hasNext(SSTableSliceIterator.java:108)
> > at
> org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:147)
> > at
> org.apache.cassandra.utils.MergeIterator$ManyToOne.<init>(MergeIterator.java:90)
> > at org.apache.cassandra.utils.MergeIterator.get(MergeIterator.java:47)
> > at
> org.apache.cassandra.db.filter.QueryFilter.collateColumns(QueryFilter.java:137)
> > at
> org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:283)
> > at
> org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:63)
> > at
> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1321)
> >
> >
> >****
>
> ** **
>
> _______________________________________________
>
> This message may contain information that is confidential or privileged.
> If you are not an intended recipient of this message, please delete it and
> any attachments, and notify the sender that you have received it in error.
> Unless specifically stated in the message or otherwise indicated, you may
> not duplicate, redistribute or forward this message or any portion thereof,
> including any attachments, by any means to any other person, including any
> retail investor or customer. This message is not a recommendation, advice,
> offer or solicitation, to buy/sell any product or service, and is not an
> official confirmation of any transaction. Any opinions presented are solely
> those of the author and do not necessarily represent those of Barclays.
> This message is subject to terms available at:
> www.barclays.com/emaildisclaimer and, if received from Barclays' Sales or
> Trading desk, the terms available at:
> www.barclays.com/salesandtradingdisclaimer/. By messaging with Barclays
> you consent to the foregoing. Barclays Bank PLC is a company registered in
> England (number 1026167) with its registered office at 1 Churchill Place,
> London, E14 5HP. This email may relate to or be sent from other members of
> the Barclays group.
>
> _______________________________________________
>
>

Reply via email to