[ 
https://issues.apache.org/jira/browse/CASSANDRA-11624?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15251392#comment-15251392
 ] 

Michael Fong edited comment on CASSANDRA-11624 at 4/21/16 6:57 AM:
-------------------------------------------------------------------

Looking at the source code of Cassandra-2.0.17, it seems the cause might come 
from the following logic:

>From org.apache.cassandra.db.compaction.CompactionManager 
…
    public void performScrub(ColumnFamilyStore cfStore, final boolean 
skipCorrupted, final boolean checkData) throws InterruptedException, 
ExecutionException
    {
        performAllSSTableOperation(cfStore, new AllSSTablesOperation()
        {
…
    private void performAllSSTableOperation(final ColumnFamilyStore cfs, final 
AllSSTablesOperation operation) throws InterruptedException, ExecutionException
    {
        final Iterable<SSTableReader> sstables = cfs.markAllCompacting();
…
org.apache.cassandra.db. ColumnFamilyStore 
…
    public Iterable<SSTableReader> markAllCompacting()
    {
        Callable<Iterable<SSTableReader>> callable = new 
Callable<Iterable<SSTableReader>>()
        {
            public Iterable<SSTableReader> call() throws Exception
            {
                assert data.getCompacting().isEmpty() : data.getCompacting();
                Iterable<SSTableReader> sstables = 
Lists.newArrayList(AbstractCompactionStrategy.filterSuspectSSTables(getSSTables()));
 <-----filter out all previously marked suspected SSTables
                if (Iterables.isEmpty(sstables))
                    return null;
…
------------------
It seems scrub will not perform if SSTable was marked  as corrupted (in 
blacklist) previously; however, would this defeat the original purpose of scrub 
operation?



was (Author: michael.fong):
Looking at the source code of Cassandra-2.0.17, it seems the cause might come 
from the following logic:

>From org.apache.cassandra.db.compaction.CompactionManager 
…
    public void performScrub(ColumnFamilyStore cfStore, final boolean 
skipCorrupted, final boolean checkData) throws InterruptedException, 
ExecutionException
    {
        performAllSSTableOperation(cfStore, new AllSSTablesOperation()
        {
…
    private void performAllSSTableOperation(final ColumnFamilyStore cfs, final 
AllSSTablesOperation operation) throws InterruptedException, ExecutionException
    {
        final Iterable<SSTableReader> sstables = cfs.markAllCompacting();
…
org.apache.cassandra.db. ColumnFamilyStore 
…
    public Iterable<SSTableReader> markAllCompacting()
    {
        Callable<Iterable<SSTableReader>> callable = new 
Callable<Iterable<SSTableReader>>()
        {
            public Iterable<SSTableReader> call() throws Exception
            {
                assert data.getCompacting().isEmpty() : data.getCompacting();
                Iterable<SSTableReader> sstables = 
Lists.newArrayList(AbstractCompactionStrategy.filterSuspectSSTables(getSSTables()));
 <-----filter out all previously marked suspected SSTables
                if (Iterables.isEmpty(sstables))
                    return null;
…
------------------
It seems scrub will not perform if SSTable was marked as corrupted previously; 
however, would this defeat the original purpose of scrub operation?


> Scrub does not seem to work on previously marked corrupted SSTables
> -------------------------------------------------------------------
>
>                 Key: CASSANDRA-11624
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-11624
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Michael Fong
>
> We ran into a scenario that scrub does not seem to work on a previously 
> marked-as-corrupted SSTable. 
> Here is the log snippet related to the corrupted SSTable and scrub-attempt :
> ERROR [ReadStage:174] 2016-03-17 04:14:39,658 CassandraDaemon.java (line 258) 
> Exception in thread Thread[ReadStage:174,5,main]
> java.lang.RuntimeException: 
> org.apache.cassandra.io.sstable.CorruptSSTableException: java.io.IOException: 
> mmap segment underflow; remaining is 10197 but 30062 requested for 
> /data/ng/db/data/wsg/dpStatusRealTime/wsg-dpStatusRealTime-ic-2-Data.db
>         at 
> org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2022)
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>         at java.lang.Thread.run(Thread.java:745)
> Caused by: org.apache.cassandra.io.sstable.CorruptSSTableException: 
> java.io.IOException: mmap segment underflow; remaining is 10197 but 30062 
> requested for 
> /data/ng/db/data/wsg/dpStatusRealTime/wsg-dpStatusRealTime-ic-2-Data.db
>         at 
> org.apache.cassandra.db.columniterator.IndexedSliceReader.<init>(IndexedSliceReader.java:97)
>         at 
> org.apache.cassandra.db.columniterator.SSTableSliceIterator.createReader(SSTableSliceIterator.java:65)
>         at 
> org.apache.cassandra.db.columniterator.SSTableSliceIterator.<init>(SSTableSliceIterator.java:42)
>         at 
> org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:238)
>         at 
> org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:62)
>         at 
> org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:250)
>         at 
> org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:53)
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1642)
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1461)
>         at org.apache.cassandra.db.Keyspace.getRow(Keyspace.java:340)
>         at 
> org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:89)
>         at 
> org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1445)
>         at 
> org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2010)
>         ... 3 more
>  INFO [CompactionExecutor:98] 2016-03-17 04:14:39,693 OutputHandler.java 
> (line 42) Scrubbing 
> SSTableReader(path='/data/ng/db/data/wsg/dpStatusRealTime/wsg-dpStatusRealTime-jb-11-Data.db')
>  (2230223 bytes)
>  INFO [CompactionExecutor:98] 2016-03-17 04:14:39,751 OutputHandler.java 
> (line 42) Scrub of 
> SSTableReader(path='/data/ng/db/data/wsg/dpStatusRealTime/wsg-dpStatusRealTime-jb-11-Data.db')
>  complete: 2 rows in new sstable and 0 empty (tombstoned) rows dropped
> ------------------
> Below is the file information around that time
> ------------------
> -bash-4.1$ ls -alF /data/ng/db/data/wsg/dpStatusRealTime/
> total 2328
> drwxr-xr-x   2 root root    4096 Mar 17 04:14 ./
> drwxr-xr-x 264 root root   12288 Mar 16 06:48 ../
> -rw-r--r--   1 root root   72995 Mar 16 07:08 
> wsg-dpStatusRealTime-ic-2-Data.db
> -rw-r--r--   1 root root      75 Mar 16 07:08 
> wsg-dpStatusRealTime-ic-2-Digest.sha1
> -rw-r--r--   1 root root      16 Mar 16 07:08 
> wsg-dpStatusRealTime-ic-2-Filter.db
> -rw-r--r--   1 root root     132 Mar 16 07:08 
> wsg-dpStatusRealTime-ic-2-Index.db
> -rw-r--r--   1 root root    5956 Mar 16 07:08 
> wsg-dpStatusRealTime-ic-2-Statistics.db
> -rw-r--r--   1 root root     244 Mar 16 07:20 
> wsg-dpStatusRealTime-ic-2-Summary.db
> -rw-r--r--   1 root root      72 Mar 16 07:08 
> wsg-dpStatusRealTime-ic-2-TOC.txt
> -rw-r--r--   1 root root     144 Mar 17 04:14 
> wsg-dpStatusRealTime-jb-12-CRC.db
> -rw-r--r--   1 root root 2230223 Mar 17 04:14 
> wsg-dpStatusRealTime-jb-12-Data.db
> -rw-r--r--   1 root root      76 Mar 17 04:14 
> wsg-dpStatusRealTime-jb-12-Digest.sha1
> -rw-r--r--   1 root root     336 Mar 17 04:14 
> wsg-dpStatusRealTime-jb-12-Filter.db
> -rw-r--r--   1 root root    1424 Mar 17 04:14 
> wsg-dpStatusRealTime-jb-12-Index.db
> -rw-r--r--   1 root root    6004 Mar 17 04:14 
> wsg-dpStatusRealTime-jb-12-Statistics.db
> -rw-r--r--   1 root root     244 Mar 17 04:14 
> wsg-dpStatusRealTime-jb-12-Summary.db
> -rw-r--r--   1 root root      79 Mar 17 04:14 
> wsg-dpStatusRealTime-jb-12-TOC.txt
> --------------------------
> 1. Please note that the corrupted file is in (ic) version, which is 1.2.19. 
> This test bed was upgraded and attempted to upgradesstable a day ago. There 
> has been some read/write to that column family before upgradesstable was 
> performed - but I do not have log for that :(
> 2. I manually performed the scrub/compact operation with nodetool and the 
> result is the same: 
> INFO [CompactionExecutor:103] 2016-03-17 07:09:53,444 CompactionTask.java 
> (line 123) Compacting 
> [SSTableReader(path='/data/ng/db/data/wsg/dpStatusRealTime/wsg-dpStatusRealTime-jb-14-Data.db'),
>  
> SSTableReader(path='/data/ng/db/data/wsg/dpStatusRealTime/wsg-dpStatusRealTime-jb-15-Data.db')]
>  
> Our test environment was Cassandra 2.0.17 with max heap set with 8 gigabytes. 



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

Reply via email to