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

Ruoran Wang edited comment on CASSANDRA-9935 at 3/31/16 1:21 AM:
-----------------------------------------------------------------

[~pauloricardomg] I am able to download the sstables to my local machine and 
step through the code. Here are things I found interesting, 
- Whenever the row key out of order error shows up, I can find two sstables, 
say A and B, where B is the subset of A. The average cell size is 93.
- when stepping through the code, I found the unrepairedScanners in 
org.apache.cassandra.db.compaction.WrappingCompactionStrategy#getScanners are 
of type LeveledScanner. 
- I wonder why unrepaired in WrappingCompactionStrategy is set the same way as 
repaired 
(org.apache.cassandra.db.compaction.WrappingCompactionStrategy#setStrategy), 
and there are assert statements checking          assert 
repaired.getClass().equals(unrepaired.getClass()). From the documentation for 
incremental, my understanding is that unrepaired sstables should be using 
SizeTieredCompactionStrategy.

I tried the following fix locally and it worked, gonna test it on prod 
machines. I would appreciated some help here to make sure my theory is not off 
the track.

{noformat}
diff --git 
a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java 
b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
index 77ca404..498a939 100644
--- 
a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
+++ 
b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
@@ -261,13 +261,18 @@ public abstract class AbstractCompactionStrategy
         });
     }

+    public ScannerList getScanners(Collection<SSTableReader> sstables, 
Range<Token> range)
+    {
+        return getDefaultScanners(sstables, range);
+    }
+
     /**
      * Returns a list of KeyScanners given sstables and a range on which to 
scan.
      * The default implementation simply grab one SSTableScanner per-sstable, 
but overriding this method
      * allow for a more memory efficient solution if we know the sstable don't 
overlap (see
      * LeveledCompactionStrategy for instance).
      */
-    public ScannerList getScanners(Collection<SSTableReader> sstables, 
Range<Token> range)
+    public ScannerList getDefaultScanners(Collection<SSTableReader> sstables, 
Range<Token> range)
     {
         RateLimiter limiter = CompactionManager.instance.getRateLimiter();
         ArrayList<ISSTableScanner> scanners = new ArrayList<ISSTableScanner>();
diff --git 
a/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java 
b/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java
index 71a6bc1..f398067 100644
--- 
a/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java
+++ 
b/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java
@@ -404,7 +404,7 @@ public final class WrappingCompactionStrategy extends 
AbstractCompactionStrategy
             else
                 unrepairedSSTables.add(sstable);
         ScannerList repairedScanners = repaired.getScanners(repairedSSTables, 
range);
-        ScannerList unrepairedScanners = 
unrepaired.getScanners(unrepairedSSTables, range);
+        ScannerList unrepairedScanners = 
unrepaired.getDefaultScanners(unrepairedSSTables, range);
         List<ISSTableScanner> scanners = new 
ArrayList<>(repairedScanners.scanners.size() + 
unrepairedScanners.scanners.size());
         scanners.addAll(repairedScanners.scanners);
         scanners.addAll(unrepairedScanners.scanners);
{noformat}


was (Author: ruoranwang):
[~pauloricardomg] I am able to download the sstables to my local machine and 
step through the code. Here are things I found interesting, 
- Whenever the row key out of order error shows up, I can find two sstables, 
say A and B, where B is the subset of A. The average cell size is 93.
- when stepping through the code, I found the unrepairedScanners in 
org.apache.cassandra.db.compaction.WrappingCompactionStrategy#getScanners are 
of type LeveledScanner. 
- I wonder why unrepaired in WrappingCompactionStrategy is set the same way as 
repaired 
(org.apache.cassandra.db.compaction.WrappingCompactionStrategy#setStrategy), 
and there are assert statements checking          assert 
repaired.getClass().equals(unrepaired.getClass()). From the documentation for 
incremental, my understanding is that unrepaired sstables should be using 
SizeTieredCompactionStrategy.

I tried the following fix locally and it worked, gonna test it on prod 
machines. I would appreciated some help here to make sure my theory is not off 
the track.

```
diff --git 
a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java 
b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
index 77ca404..498a939 100644
--- 
a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
+++ 
b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
@@ -261,13 +261,18 @@ public abstract class AbstractCompactionStrategy
         });
     }

+    public ScannerList getScanners(Collection<SSTableReader> sstables, 
Range<Token> range)
+    {
+        return getDefaultScanners(sstables, range);
+    }
+
     /**
      * Returns a list of KeyScanners given sstables and a range on which to 
scan.
      * The default implementation simply grab one SSTableScanner per-sstable, 
but overriding this method
      * allow for a more memory efficient solution if we know the sstable don't 
overlap (see
      * LeveledCompactionStrategy for instance).
      */
-    public ScannerList getScanners(Collection<SSTableReader> sstables, 
Range<Token> range)
+    public ScannerList getDefaultScanners(Collection<SSTableReader> sstables, 
Range<Token> range)
     {
         RateLimiter limiter = CompactionManager.instance.getRateLimiter();
         ArrayList<ISSTableScanner> scanners = new ArrayList<ISSTableScanner>();
diff --git 
a/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java 
b/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java
index 71a6bc1..f398067 100644
--- 
a/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java
+++ 
b/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java
@@ -404,7 +404,7 @@ public final class WrappingCompactionStrategy extends 
AbstractCompactionStrategy
             else
                 unrepairedSSTables.add(sstable);
         ScannerList repairedScanners = repaired.getScanners(repairedSSTables, 
range);
-        ScannerList unrepairedScanners = 
unrepaired.getScanners(unrepairedSSTables, range);
+        ScannerList unrepairedScanners = 
unrepaired.getDefaultScanners(unrepairedSSTables, range);
         List<ISSTableScanner> scanners = new 
ArrayList<>(repairedScanners.scanners.size() + 
unrepairedScanners.scanners.size());
         scanners.addAll(repairedScanners.scanners);
         scanners.addAll(unrepairedScanners.scanners);
```

> Repair fails with RuntimeException
> ----------------------------------
>
>                 Key: CASSANDRA-9935
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-9935
>             Project: Cassandra
>          Issue Type: Bug
>         Environment: C* 2.1.8, Debian Wheezy
>            Reporter: mlowicki
>            Assignee: Yuki Morishita
>             Fix For: 2.1.x
>
>         Attachments: db1.sync.lati.osa.cassandra.log, 
> db5.sync.lati.osa.cassandra.log, system.log.10.210.3.117, 
> system.log.10.210.3.221, system.log.10.210.3.230
>
>
> We had problems with slow repair in 2.1.7 (CASSANDRA-9702) but after upgrade 
> to 2.1.8 it started to work faster but now it fails with:
> {code}
> ...
> [2015-07-29 20:44:03,956] Repair session 23a811b0-3632-11e5-a93e-4963524a8bde 
> for range (-5474076923322749342,-5468600594078911162] finished
> [2015-07-29 20:44:03,957] Repair session 336f8740-3632-11e5-a93e-4963524a8bde 
> for range (-8631877858109464676,-8624040066373718932] finished
> [2015-07-29 20:44:03,957] Repair session 4ccd8430-3632-11e5-a93e-4963524a8bde 
> for range (-5372806541854279315,-5369354119480076785] finished
> [2015-07-29 20:44:03,957] Repair session 59f129f0-3632-11e5-a93e-4963524a8bde 
> for range (8166489034383821955,8168408930184216281] finished
> [2015-07-29 20:44:03,957] Repair session 6ae7a9a0-3632-11e5-a93e-4963524a8bde 
> for range (6084602890817326921,6088328703025510057] finished
> [2015-07-29 20:44:03,957] Repair session 8938e4a0-3632-11e5-a93e-4963524a8bde 
> for range (-781874602493000830,-781745173070807746] finished
> [2015-07-29 20:44:03,957] Repair command #4 finished
> error: nodetool failed, check server logs
> -- StackTrace --
> java.lang.RuntimeException: nodetool failed, check server logs
>         at 
> org.apache.cassandra.tools.NodeTool$NodeToolCmd.run(NodeTool.java:290)
>         at org.apache.cassandra.tools.NodeTool.main(NodeTool.java:202)
> {code}
> After running:
> {code}
> nodetool repair --partitioner-range --parallel --in-local-dc sync
> {code}
> Last records in logs regarding repair are:
> {code}
> INFO  [Thread-173887] 2015-07-29 20:44:03,956 StorageService.java:2952 - 
> Repair session 09ff9e40-3632-11e5-a93e-4963524a8bde for range 
> (-7695808664784761779,-7693529816291585568] finished
> INFO  [Thread-173887] 2015-07-29 20:44:03,956 StorageService.java:2952 - 
> Repair session 17d8d860-3632-11e5-a93e-4963524a8bde for range 
> (8063716953988492222,8065203836608925992] finished
> INFO  [Thread-173887] 2015-07-29 20:44:03,956 StorageService.java:2952 - 
> Repair session 23a811b0-3632-11e5-a93e-4963524a8bde for range 
> (-5474076923322749342,-5468600594078911162] finished
> INFO  [Thread-173887] 2015-07-29 20:44:03,956 StorageService.java:2952 - 
> Repair session 336f8740-3632-11e5-a93e-4963524a8bde for range 
> (-8631877858109464676,-8624040066373718932] finished
> INFO  [Thread-173887] 2015-07-29 20:44:03,957 StorageService.java:2952 - 
> Repair session 4ccd8430-3632-11e5-a93e-4963524a8bde for range 
> (-5372806541854279315,-5369354119480076785] finished
> INFO  [Thread-173887] 2015-07-29 20:44:03,957 StorageService.java:2952 - 
> Repair session 59f129f0-3632-11e5-a93e-4963524a8bde for range 
> (8166489034383821955,8168408930184216281] finished
> INFO  [Thread-173887] 2015-07-29 20:44:03,957 StorageService.java:2952 - 
> Repair session 6ae7a9a0-3632-11e5-a93e-4963524a8bde for range 
> (6084602890817326921,6088328703025510057] finished
> INFO  [Thread-173887] 2015-07-29 20:44:03,957 StorageService.java:2952 - 
> Repair session 8938e4a0-3632-11e5-a93e-4963524a8bde for range 
> (-781874602493000830,-781745173070807746] finished
> {code}
> but a bit above I see (at least two times in attached log):
> {code}
> ERROR [Thread-173887] 2015-07-29 20:44:03,853 StorageService.java:2959 - 
> Repair session 1b07ea50-3608-11e5-a93e-4963524a8bde for range 
> (5765414319217852786,5781018794516851576] failed with error 
> org.apache.cassandra.exceptions.RepairException: [repair 
> #1b07ea50-3608-11e5-a93e-4963524a8bde on sync/entity_by_id2, 
> (5765414319217852786,5781018794516851576]] Validation failed in /10.195.15.162
> java.util.concurrent.ExecutionException: java.lang.RuntimeException: 
> org.apache.cassandra.exceptions.RepairException: [repair 
> #1b07ea50-3608-11e5-a93e-4963524a8bde on sync/entity_by_id2, 
> (5765414319217852786,5781018794516851576]] Validation failed in /10.195.15.162
>         at java.util.concurrent.FutureTask.report(FutureTask.java:122) 
> [na:1.7.0_80]
>         at java.util.concurrent.FutureTask.get(FutureTask.java:188) 
> [na:1.7.0_80]
>         at 
> org.apache.cassandra.service.StorageService$4.runMayThrow(StorageService.java:2950)
>  ~[apache-cassandra-2.1.8.jar:2.1.8]
>         at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) 
> [apache-cassandra-2.1.8.jar:2.1.8]
>         at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
> [na:1.7.0_80]
>         at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
> [na:1.7.0_80]
>         at java.lang.Thread.run(Thread.java:745) [na:1.7.0_80]
> Caused by: java.lang.RuntimeException: 
> org.apache.cassandra.exceptions.RepairException: [repair 
> #1b07ea50-3608-11e5-a93e-4963524a8bde on sync/entity_by_id2, 
> (5765414319217852786,5781018794516851576]] Validation failed in /10.195.15.162
>         at com.google.common.base.Throwables.propagate(Throwables.java:160) 
> ~[guava-16.0.jar:na]
>         at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32) 
> [apache-cassandra-2.1.8.jar:2.1.8]
>         at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
> [na:1.7.0_80]
>         at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
> [na:1.7.0_80]
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>  ~[na:1.7.0_80]
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>  ~[na:1.7.0_80]        ... 1 common frames omitted
> Caused by: org.apache.cassandra.exceptions.RepairException: [repair 
> #1b07ea50-3608-11e5-a93e-4963524a8bde on sync/entity_by_id2, 
> (5765414319217852786,5781018794516851576]] Validation failed in /10.195.15.162
>         at 
> org.apache.cassandra.repair.RepairSession.validationComplete(RepairSession.java:166)
>  ~[apache-cassandra-2.1.8.jar:2.1.8]        at 
> org.apache.cassandra.service.ActiveRepairService.handleMessage(ActiveRepairService.java:406)
>  ~[apache-cassandra-2.1.8.jar:2.1.8]
>         at 
> org.apache.cassandra.repair.RepairMessageVerbHandler.doVerb(RepairMessageVerbHandler.java:134)
>  ~[apache-cassandra-2.1.8.jar:2.1.8]        at 
> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:62) 
> ~[apache-cassandra-2.1.8.jar:2.1.8]
>         ... 3 common frames omittedINFO  [Thread-173887] 2015-07-29 
> 20:44:03,854 StorageService.java:2952 - Repair session 
> 846d9300-3608-11e5-a93e-4963524a8bde for range (-6705935
> 742755245856,-6704072966568763453] finished
> {code}



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

Reply via email to