Well, I finally resolved this issue by modifying cassandra to ignore
sstables that had size bigger than a threshold.

The leveled compaction will fall back to sized tiered compaction in some
situation and that's why I always got some old huge sstables compacted.
More details can be found in 'LeveledManifest.java' , the
'getCompactionCandidates' function. I modified the 'mostInterestingBucket'
method of 'SizeTieredCompactionStrategy.java' and added a filter before
function return:

        Iterator<SSTableReader> iter = hottest.left.iterator();
>         while (iter.hasNext()) {
>             SSTableReader mysstable = iter.next();
>             if (mysstable.onDiskLength() > 1099511627776L) {
>                 logger.info("Removed candidate {} ",
> mysstable.toString());
>                 iter.remove();
>             }
>         }
>

 I don't have much time to do some more research to figure out if this has
side effect or not, but this is a solution for me. I hope this would be
useful to those who had similar issues.


On Sun, May 4, 2014 at 5:10 PM, Yatong Zhang <bluefl...@gmail.com> wrote:

> I am using the latest 2.0.7. The 'nodetool tpstats' shows as:
>
> [root@storage5 bin]# ./nodetool tpstats
>> Pool Name                    Active   Pending      Completed   Blocked
>> All time blocked
>> ReadStage                         0         0         628220
>> 0                 0
>> RequestResponseStage              0         0        3342234
>> 0                 0
>> MutationStage                     0         0        3172116
>> 0                 0
>> ReadRepairStage                   0         0          47666
>> 0                 0
>> ReplicateOnWriteStage             0         0              0
>> 0                 0
>> GossipStage                       0         0         756024
>> 0                 0
>> AntiEntropyStage                  0         0              0
>> 0                 0
>> MigrationStage                    0         0              0
>> 0                 0
>> MemoryMeter                       0         0           6652
>> 0                 0
>> MemtablePostFlusher               0         0           7042
>> 0                 0
>> FlushWriter                       0         0           4023
>> 0                 0
>> MiscStage                         0         0              0
>> 0                 0
>> PendingRangeCalculator            0         0             27
>> 0                 0
>> commitlog_archiver                0         0              0
>> 0                 0
>> InternalResponseStage             0         0              0
>> 0                 0
>> HintedHandoff                     0         0             28
>> 0                 0
>>
>> Message type           Dropped
>> RANGE_SLICE                  0
>> READ_REPAIR                  0
>> PAGED_RANGE                  0
>> BINARY                       0
>> READ                         0
>> MUTATION                     0
>> _TRACE                       0
>> REQUEST_RESPONSE             0
>> COUNTER_MUTATION             0
>>
>
>  And here is another type of error, and these errors seem to occur after
> 'disk is full'
>
> ERROR [SSTableBatchOpen:2] 2014-04-30 13:47:48,348 CassandraDaemon.java
>> (line 198) Exception in thread Thread[SSTableBatchOpen:2,5,main]
>> org.apache.cassandra.io.sstable.CorruptSSTableException:
>> java.io.EOFException
>>         at
>> org.apache.cassandra.io.compress.CompressionMetadata.<init>(CompressionMetadata.java:110)
>>         at
>> org.apache.cassandra.io.compress.CompressionMetadata.create(CompressionMetadata.java:64)
>>         at
>> org.apache.cassandra.io.util.CompressedPoolingSegmentedFile$Builder.complete(CompressedPoolingSegmentedFile.java:42)
>>         at
>> org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:458)
>>         at
>> org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:422)
>>         at
>> org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:203)
>>         at
>> org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:184)
>>         at
>> org.apache.cassandra.io.sstable.SSTableReader$1.run(SSTableReader.java:264)
>>
>>         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:85)
>>         ... 12 more
>>
>
>
>
> On Sun, May 4, 2014 at 4:59 PM, DuyHai Doan <doanduy...@gmail.com> wrote:
>
>> The symptoms looks like there are pending compactions stacking up or
>> failed compactions so temporary files (-tmp-Data.db) are not properly
>> cleaned up.
>>
>>  What is your Cassandra version ? Can you do a "nodetool tpstats" and
>> look into Cassandra logs to see whether there are issues with compactions ?
>>
>> I've found one discussion thread that have the same symptoms:
>> http://comments.gmane.org/gmane.comp.db.cassandra.user/22089
>>
>>
>>
>>
>> On Sun, May 4, 2014 at 10:39 AM, Yatong Zhang <bluefl...@gmail.com>wrote:
>>
>>> Yes after a while the disk fills up again. So I changed the compaction
>>> strategy from 'sized tiered' to 'leveled' to reduce the disk usage when
>>> compacting, but the problem still occurs.
>>>
>>> This table has lots of write and a relative very small read, and no
>>> update. here is the schema of the table:
>>>
>>> CREATE TABLE mydb.images (
>>>   image_id uuid PRIMARY KEY,
>>>   available boolean,
>>>   message text,
>>>   raw_data blob,
>>>   time_created timestamp,
>>>   url text
>>> ) WITH
>>>   bloom_filter_fp_chance=0.010000 AND
>>>   caching='KEYS_ONLY' AND
>>>   comment='' AND
>>>   dclocal_read_repair_chance=0.000000 AND
>>>   gc_grace_seconds=864000 AND
>>>   read_repair_chance=0.100000 AND
>>>   replicate_on_write='true' AND
>>>   populate_io_cache_on_flush='false' AND
>>>   compaction={'sstable_size_in_mb': '192', 'class':
>>> 'LeveledCompactionStrategy'} AND
>>>   compression={'sstable_compression': 'LZ4Compressor'};
>>>
>>>
>>> On Sun, May 4, 2014 at 4:31 PM, DuyHai Doan <doanduy...@gmail.com>wrote:
>>>
>>>> And after a while the /data6 drive fills up again right ?
>>>>
>>>>  One question, can you please give the CQL3 definition of your 
>>>> "mydb-images-tmp"
>>>> table ?
>>>>
>>>> What is the access pattern for this table ? Lots of write ? Lots of
>>>> update ?
>>>>
>>>>
>>>>
>>>>
>>>> On Sun, May 4, 2014 at 10:00 AM, Yatong Zhang <bluefl...@gmail.com>wrote:
>>>>
>>>>> after restarting or 'cleanup' the big tmp file has gone and all looks
>>>>> like fine:
>>>>>
>>>>> -rw-r--r-- 1 root root  19K Apr 30 13:58
>>>>>> mydb_oe-images-tmp-jb-96242-CompressionInfo.db
>>>>>> -rw-r--r-- 1 root root 145M Apr 30 13:58
>>>>>> mydb_oe-images-tmp-jb-96242-Data.db
>>>>>> -rw-r--r-- 1 root root  64K Apr 30 13:58
>>>>>> mydb_oe-images-tmp-jb-96242-Index.db
>>>>>>
>>>>>
>>>>> [root@node5 images]# df -hl
>>>>> Filesystem      Size  Used Avail Use% Mounted on
>>>>> /dev/sda1        49G  7.5G   39G  17% /
>>>>> tmpfs           7.8G     0  7.8G   0% /dev/shm
>>>>> /dev/sda3       3.6T  1.3T  2.1T  38% /data1
>>>>> /dev/sdb1       3.6T  1.4T  2.1T  39% /data2
>>>>> /dev/sdc1       3.6T  466G  3.0T  14% /data3
>>>>> /dev/sdd1       3.6T  1.3T  2.2T  38% /data4
>>>>> /dev/sde1       3.6T  1.3T  2.2T  38% /data5
>>>>> /dev/sdf1       3.6T  662M  3.4T   1% /data6
>>>>>
>>>>> I didn't perform repair, not even for one time
>>>>>
>>>>>
>>>>> On Sun, May 4, 2014 at 2:37 PM, DuyHai Doan <doanduy...@gmail.com>wrote:
>>>>>
>>>>>> Hello Yatong
>>>>>>
>>>>>> "If I restart the node or using 'cleanup', it will resume to normal."
>>>>>> --> what does df -hl shows for /data6 when you restart or cleanup the 
>>>>>> node ?
>>>>>>
>>>>>> By the way, a single SSTable of 3.6Tb is kind of huge. Do you perform
>>>>>> manual repair frequently ?
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Sun, May 4, 2014 at 1:51 AM, Yatong Zhang <bluefl...@gmail.com>wrote:
>>>>>>
>>>>>>> My Cassandra cluster has plenty of free space, for now only about
>>>>>>> 30% of space are used
>>>>>>>
>>>>>>>
>>>>>>> On Sun, May 4, 2014 at 6:36 AM, Yatong Zhang <bluefl...@gmail.com>wrote:
>>>>>>>
>>>>>>>> Hi there,
>>>>>>>>
>>>>>>>> It was strange that the 'xxx-tmp-xxx.db' file kept increasing until
>>>>>>>> Cassandra throw exceptions with 'No space left on device'. I am using 
>>>>>>>> CQL 3
>>>>>>>> to create a table to store data about 200K ~ 500K per record. I have 6
>>>>>>>> harddisks per node and cassandra was configured with 6 data
>>>>>>>> directories(ext4 file systems, Centos 6.5):
>>>>>>>>
>>>>>>>> data_file_directories:
>>>>>>>>>     - /data1/cass
>>>>>>>>>     - /data2/cass
>>>>>>>>>     - /data3/cass
>>>>>>>>>     - /data4/cass
>>>>>>>>>     - /data5/cass
>>>>>>>>>     - /data6/cass
>>>>>>>>>
>>>>>>>>
>>>>>>>> And every directory is on a standalone disk. But I just found when
>>>>>>>> the error occurred:
>>>>>>>>
>>>>>>>> [root@node5 images]# ll -hl
>>>>>>>>> total 3.6T
>>>>>>>>> drwxr-xr-x 4 root root 4.0K Jan 20 09:44 snapshots
>>>>>>>>> -rw-r--r-- 1 root root 456M Apr 30 13:42
>>>>>>>>> mydb-images-tmp-jb-91068-CompressionInfo.db
>>>>>>>>> -rw-r--r-- 1 root root 3.5T Apr 30 13:42
>>>>>>>>> mydb-images-tmp-jb-91068-Data.db
>>>>>>>>> -rw-r--r-- 1 root root    0 Apr 30 13:42
>>>>>>>>> mydb-images-tmp-jb-91068-Filter.db
>>>>>>>>> -rw-r--r-- 1 root root 2.0G Apr 30 13:42
>>>>>>>>> mydb-images-tmp-jb-91068-Index.db
>>>>>>>>>
>>>>>>>>
>>>>>>>> [root@node5 images]# df -hl
>>>>>>>> Filesystem      Size  Used Avail Use% Mounted on
>>>>>>>> /dev/sda1        49G  7.5G   39G  17% /
>>>>>>>> tmpfs           7.8G     0  7.8G   0% /dev/shm
>>>>>>>> /dev/sda3       3.6T  1.3T  2.1T  38% /data1
>>>>>>>> /dev/sdb1       3.6T  1.4T  2.1T  39% /data2
>>>>>>>> /dev/sdc1       3.6T  466G  3.0T  14% /data3
>>>>>>>> /dev/sdd1       3.6T  1.3T  2.2T  38% /data4
>>>>>>>> /dev/sde1       3.6T  1.3T  2.2T  38% /data5
>>>>>>>> /dev/sdf1       3.6T  3.6T     0 100% /data6
>>>>>>>>
>>>>>>>> *mydb-images-tmp-jb-91068-Data.db *almost occupied all the disk
>>>>>>>> space (4T harddisk with 3.6T actual usable size) and the error looks 
>>>>>>>> like:
>>>>>>>>
>>>>>>>> INFO [FlushWriter:4174] 2014-05-04 05:15:15,744 Memtable.java (line
>>>>>>>>> 403) Completed flushing
>>>>>>>>> /data3/cass/system/compactions_in_progress/system-compactions_in_progress-jb-16942-Data.db
>>>>>>>>> (42 bytes) for commitlog position 
>>>>>>>>> ReplayPosition(segmentId=1398900356204,
>>>>>>>>> position=25024609)
>>>>>>>>>  INFO [CompactionExecutor:3689] 2014-05-04 05:15:15,745
>>>>>>>>> CompactionTask.java (line 115) Compacting
>>>>>>>>> [SSTableReader(path='/data3/cass/system/compactions_in_progress/system-compactions_in_progress-jb-16940-Data.db'),
>>>>>>>>> SSTableReader(path='/data3/cass/system/compactions_in_progress/system-compactions_in_progress-jb-16942-Data.db'),
>>>>>>>>> SSTableReader(path='/data3/cass/system/compactions_in_progress/system-compactions_in_progress-jb-16941-Data.db'),
>>>>>>>>> SSTableReader(path='/data3/cass/system/compactions_in_progress/system-compactions_in_progress-jb-16939-Data.db')]
>>>>>>>>> ERROR [CompactionExecutor:1245] 2014-05-04 05:15:15,745
>>>>>>>>> CassandraDaemon.java (line 198) Exception in thread
>>>>>>>>> Thread[CompactionExecutor:1245,1,main]
>>>>>>>>> FSWriteError in
>>>>>>>>> /data2/cass/mydb/images/mydb-images-tmp-jb-92181-Filter.db
>>>>>>>>>         at
>>>>>>>>> org.apache.cassandra.io.sstable.SSTableWriter$IndexWriter.close(SSTableWriter.java:475)
>>>>>>>>>         at
>>>>>>>>> org.apache.cassandra.io.util.FileUtils.closeQuietly(FileUtils.java:212)
>>>>>>>>>         at
>>>>>>>>> org.apache.cassandra.io.sstable.SSTableWriter.abort(SSTableWriter.java:301)
>>>>>>>>>         at
>>>>>>>>> org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:209)
>>>>>>>>>         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)
>>>>>>>>> Caused by: java.io.IOException: No space left on device
>>>>>>>>>         at java.io.FileOutputStream.write(Native Method)
>>>>>>>>>         at
>>>>>>>>> java.io.FileOutputStream.write(FileOutputStream.java:295)
>>>>>>>>>         at
>>>>>>>>> java.io.DataOutputStream.writeInt(DataOutputStream.java:197)
>>>>>>>>>         at
>>>>>>>>> org.apache.cassandra.utils.BloomFilterSerializer.serialize(BloomFilterSerializer.java:34)
>>>>>>>>>         at
>>>>>>>>> org.apache.cassandra.utils.Murmur3BloomFilter$Murmur3BloomFilterSerializer.serialize(Murmur3BloomFilter.java:44)
>>>>>>>>>         at
>>>>>>>>> org.apache.cassandra.utils.FilterFactory.serialize(FilterFactory.java:41)
>>>>>>>>>         at
>>>>>>>>> org.apache.cassandra.io.sstable.SSTableWriter$IndexWriter.close(SSTableWriter.java:468)
>>>>>>>>>         ... 13 more
>>>>>>>>> ERROR [CompactionExecutor:1245] 2014-05-04 05:15:15,800
>>>>>>>>> StorageService.java (line 367) Stopping gossiper
>>>>>>>>>  WARN [CompactionExecutor:1245] 2014-05-04 05:15:15,800
>>>>>>>>> StorageService.java (line 281) Stopping gossip by operator request
>>>>>>>>>  INFO [CompactionExecutor:1245] 2014-05-04 05:15:15,800
>>>>>>>>> Gossiper.java (line 1271) Announcing shutdown
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> I have changed my table to "LeveledCompactionStrategy" to reduce
>>>>>>>> the disk size needed when compaction, with:
>>>>>>>>
>>>>>>>> ALTER TABLE images WITH compaction = { 'class' :
>>>>>>>>> 'LeveledCompactionStrategy', 'sstable_size_in_mb' : '192' };
>>>>>>>>>
>>>>>>>>
>>>>>>>> But the problem still exists: the file keep increasing, and after
>>>>>>>> about 2 or 3 days cassandra will fail due to 'No space left on device'
>>>>>>>> error.  If I restart the node or using 'cleanup', it will resume to 
>>>>>>>> normal.
>>>>>>>>
>>>>>>>> I don't know is it because my configuration or it's just a bug, so
>>>>>>>> would any one please help to solve this issue?
>>>>>>>>
>>>>>>>> Thanks
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>
>>>>>
>>>>
>>>
>>
>

Reply via email to