[jira] [Comment Edited] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2015-02-02 Thread Randy Fradin (JIRA)

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

Randy Fradin edited comment on CASSANDRA-6285 at 2/2/15 7:26 PM:
-

I am getting this exception using Thrift HSHA in 2.1.0:

{quote}
 INFO [CompactionExecutor:8] 2015-01-26 13:32:51,818 CompactionTask.java (line 
138) Compacting 
[SSTableReader(path='/tmp/cass_test/cassandra/TestCassandra/data/test_ks/test_cf-1c45da40a58911e4826751fbbc77b187/test_ks-test_cf-ka-2-Data.db'),
 
SSTableReader(path='/tmp/cass_test/cassandra/TestCassandra/data/test_ks/test_cf-1c45da40a58911e4826751fbbc77b187/test_ks-test_cf-ka-1-Data.db')]
 INFO [CompactionExecutor:8] 2015-01-26 13:32:51,890 ColumnFamilyStore.java 
(line 856) Enqueuing flush of compactions_in_progress: 212 (0%) on-heap, 20 
(0%) off-heap
 INFO [MemtableFlushWriter:8] 2015-01-26 13:32:51,892 Memtable.java (line 326) 
Writing Memtable-compactions_in_progress@1155018639(0 serialized bytes, 1 ops, 
0%/0% of on/off-heap limit)
 INFO [MemtableFlushWriter:8] 2015-01-26 13:32:51,896 Memtable.java (line 360) 
Completed flushing 
/tmp/cass_test/cassandra/TestCassandra/data/system/compactions_in_progress-55080ab05d9c388690a4acb25fe1f77b/system-compactions_in_progress-ka-2-Data.db
 (42 bytes) for commitlog position ReplayPosition(segmentId=1422296630707, 
position=430226)
ERROR [CompactionExecutor:8] 2015-01-26 13:32:51,906 CassandraDaemon.java (line 
166) Exception in thread Thread[CompactionExecutor:8,1,RMI Runtime]
java.lang.RuntimeException: Last written key 
DecoratedKey(131206587314004820534098544948237170809, 
80010001000c62617463685f6d757461746500) >= current key 
DecoratedKey(14775611966645399672119169777260659240, 
726f776b65793030385f31343232323937313537353835) writing into 
/tmp/cass_test/cassandra/TestCassandra/data/test_ks/test_cf-1c45da40a58911e4826751fbbc77b187/test_ks-test_cf-tmp-ka-3-Data.db
at 
org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:172)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:196) 
~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.io.sstable.SSTableRewriter.append(SSTableRewriter.java:110)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:177)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) 
~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:74)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:235)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
~[na:1.7.0_40]
at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
~[na:1.7.0_40]
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
~[na:1.7.0_40]
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
[na:1.7.0_40]
at java.lang.Thread.run(Thread.java:724) [na:1.7.0_40]
{quote}

I don't think it's caused by CASSANDRA-8211, because it happens during the 
first compaction that takes place between the first 2 SSTables to get flushed 
from an initially empty column family.

Also, I've only been able to reproduce it when using both *hsha* for the rpc 
server and *offheap_objects* for memtable allocation. If I switch either to 
sync or to offheap_buffers or heap_buffers then I cannot reproduce the problem. 
Also under the same circumstances I'm pretty sure I've seen incorrect data 
being returned to a client multiget_slice request before any SSTables had been 
flushed yet, so I presume this is corruption that happens before any 
flush/compaction takes place.

nodetool scrub yielded these errors:

{quote}
 INFO [CompactionExecutor:9] 2015-01-26 13:48:01,512 OutputHandler.java (line 
42) Scrubbing 
SSTableReader(path='/tmp/cass_test/cassandra/TestCassandra/data/test_ks/test_cf-1c45da40a58911e4826751fbbc77b187/test_ks-test_cf-ka-2-Data.db')
 (168780 bytes)
 INFO [CompactionExecutor:10] 2015-01-26 13:48:01,512 OutputHandler.java (line 
42) Scrubbing 
SSTableReader(path='/tmp/cass_test/cassandra/TestCassandra/data/test_ks/test_cf-1c45da40a58911e4826751fbbc77b187/test_ks-test_cf-ka-1-Data.db')
 (135024 bytes)
 WARN [CompactionExecutor:9] 2015-01-26 1

[jira] [Comment Edited] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-10-24 Thread Alexander Sterligov (JIRA)

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

Alexander Sterligov edited comment on CASSANDRA-6285 at 10/24/14 9:29 AM:
--

[~xedin] That NPE happend once and unfortunatelly I have not saved it. If I'll 
get it once more I'll save this sstable.
I totally removed OpsCenter keyspace (with sstables) and recreated them. I 
don't get "Last written key DecoratedKey" any more. By the way, this error 
definetely causees streams to hang on 100%.

I have several strange things happening now:
  - I've noticed that it takes about 30 minutes between "nodetool repair" and 
first pending AntiEntropySession. Is that ok?
  - Repair is already running for 24 hours (~13GB per node, 17 nodes). What's 
the number of AntiEntropySessions to finish single repair? Number of key ranges?
{quote}
Pool NameActive   Pending  Completed   Blocked  All 
time blocked
CounterMutationStage  0 0  0 0  
   0
ReadStage 0 0 392196 0  
   0
RequestResponseStage  0 05271906 0  
   0
MutationStage 0 0   19832506 0  
   0
ReadRepairStage   0 0   2280 0  
   0
GossipStage   0 0 453830 0  
   0
CacheCleanupExecutor  0 0  0 0  
   0
MigrationStage0 0  0 0  
   0
ValidationExecutor0 0  39446 0  
   0
MemtableReclaimMemory 0 0  29927 0  
   0
InternalResponseStage 0 0 588279 0  
   0
AntiEntropyStage  0 05325285 0  
   0
MiscStage 0 0  0 0  
   0
CommitLogArchiver 0 0  0 0  
   0
MemtableFlushWriter   0 0  29927 0  
   0
PendingRangeCalculator0 0 30 0  
   0
MemtablePostFlush 0 0 135734 0  
   0
CompactionExecutor   3131 502175 0  
   0
AntiEntropySessions   3 3   3446 0  
   0
HintedHandoff 0 0 44 0  
   0

Message type   Dropped
RANGE_SLICE  0
READ_REPAIR  0
PAGED_RANGE  0
BINARY   0
READ 0
MUTATION 2
_TRACE   0
REQUEST_RESPONSE 0
COUNTER_MUTATION 0
{quote}
  - Some validation compactions run for more than 100% (1923%). I thinks that 
it's CASSANDRA-7239, right?
  - the amount of sstables for some CFs is about 15 000 and continues to grow 
during repair.
  - There are several following exceptions during repair
{quote}
ERROR [RepairJobTask:80] 2014-10-24 13:27:31,717 RepairJob.java:127 - Error 
occurred during snapshot phase
java.lang.RuntimeException: Could not create snapshot at /37.140.189.163
at 
org.apache.cassandra.repair.SnapshotTask$SnapshotCallback.onFailure(SnapshotTask.java:77)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.net.MessagingService$5$1.run(MessagingService.java:347) 
~[apache-cassandra-2.1.0.jar:2.1.0]
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
~[na:1.7.0_51]
at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
~[na:1.7.0_51]
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
[na:1.7.0_51]
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
[na:1.7.0_51]
at java.lang.Thread.run(Thread.java:744) [na:1.7.0_51]
ERROR [AntiEntropySessions:141] 2014-10-24 13:27:31,724 RepairSession.java:303 
- [repair #da2cb020-5b5f-11e4-a45e-d9cec1206f33] session completed with the 
following error
java.io.IOException: Failed during snapshot creation.
at 
org.apache.cassandra.repair.RepairSession.failedSnapshot(RepairSession.java:344)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.repair.RepairJob$2.onFailure(RepairJob.java:128) 
~[apache-cassandra-2.1.0.jar:2.1.0]
at com.google.common.util.concurrent.Futures$4.run(Futures.java:1172) 
~[guava-16.0.jar:na]
at 
java.util.concurrent.ThreadPoolExecutor.runW

[jira] [Comment Edited] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-10-24 Thread Alexander Sterligov (JIRA)

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

Alexander Sterligov edited comment on CASSANDRA-6285 at 10/24/14 9:27 AM:
--

[~xedin] That NPE happend once and unfortunatelly I have not saved it. If I'll 
get it once more I'll save this sstable.
I totally removed OpsCenter keyspace (with sstables) and recreated them. I 
don't get "Last written key DecoratedKey" any more. By the way, this error 
definetely causees streams to hang on 100%.

I have several strange things happening now:
  - I've noticed that it takes about 30 minutes between "nodetool repair" and 
first pending AntiEntropySession. Is that ok?
  - Repair is already running for 24 hours (~13GB per node, 17 nodes). What's 
the number of AntiEntropySessions to finish single repair? Number of key ranges?
{quote}
Pool NameActive   Pending  Completed   Blocked  All 
time blocked
CounterMutationStage  0 0  0 0  
   0
ReadStage 0 0 392196 0  
   0
RequestResponseStage  0 05271906 0  
   0
MutationStage 0 0   19832506 0  
   0
ReadRepairStage   0 0   2280 0  
   0
GossipStage   0 0 453830 0  
   0
CacheCleanupExecutor  0 0  0 0  
   0
MigrationStage0 0  0 0  
   0
ValidationExecutor0 0  39446 0  
   0
MemtableReclaimMemory 0 0  29927 0  
   0
InternalResponseStage 0 0 588279 0  
   0
AntiEntropyStage  0 05325285 0  
   0
MiscStage 0 0  0 0  
   0
CommitLogArchiver 0 0  0 0  
   0
MemtableFlushWriter   0 0  29927 0  
   0
PendingRangeCalculator0 0 30 0  
   0
MemtablePostFlush 0 0 135734 0  
   0
CompactionExecutor   3131 502175 0  
   0
AntiEntropySessions   3 3   3446 0  
   0
HintedHandoff 0 0 44 0  
   0

Message type   Dropped
RANGE_SLICE  0
READ_REPAIR  0
PAGED_RANGE  0
BINARY   0
READ 0
MUTATION 2
_TRACE   0
REQUEST_RESPONSE 0
COUNTER_MUTATION 0
{quote}
  - Some validation compactions run for more than 100% (1923%). I thinks that 
it's CASSANDRA-7239, right?
  - the amount of sstables for some CFs is about 15 000 and continues to grow 
during repair.


was (Author: sterligovak):
[~xedin] That NPE happend once and unfortunatelly I have not saved it. If I'll 
get it once more I'll save this sstable.
I totally removed OpsCenter keyspace (with sstables) and recreated them. I 
don't get "Last written key DecoratedKey" any more. By the way, this error 
definetely causees streams to hang on 100%.

I have several strange things happening now:
  - I've noticed that it takes about 30 minutes between "nodetool repair" and 
first pending AntiEntropySession. Is that ok?
  - Repair is already running for 24 hours (~13GB per node, 17 nodes). What's 
the number of AntiEntropySessions to finish single repair? Number of key ranges?
{quote}
Pool NameActive   Pending  Completed   Blocked  All 
time blocked
CounterMutationStage  0 0  0 0  
   0
ReadStage 0 0 392196 0  
   0
RequestResponseStage  0 05271906 0  
   0
MutationStage 0 0   19832506 0  
   0
ReadRepairStage   0 0   2280 0  
   0
GossipStage   0 0 453830 0  
   0
CacheCleanupExecutor  0 0  0 0  
   0
MigrationStage0 0  0 0  
   0
ValidationExecutor0 0  39446 0  
   0
MemtableReclaimMemo

[jira] [Comment Edited] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-10-22 Thread Alexander Sterligov (JIRA)

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

Alexander Sterligov edited comment on CASSANDRA-6285 at 10/22/14 7:39 AM:
--

[~xedin] No, I've not seen such messages. sstablescrub failed with NPE. 
sstables were corrupted on all 17 nodes. I removed them manually and there was 
no errors overnight. It seems sync really impacted the problem. Maybe there are 
some another problem which hides with sync server. I still have problems - 
validation hangs on one table on all nodes :(.


was (Author: sterligovak):
[~xedin] No, I've not seen such messages. sstablescrub failed with NPE. 
sstables were corrupted on all 17 nodes. I removed them manually and there was 
no errors overnight. It seems sync really impacted on the problem. Maybe there 
are some another problem which hides with sync server. I still have problems - 
validation hangs on one table on all nodes :(.

> 2.0 HSHA server introduces corrupt data
> ---
>
> Key: CASSANDRA-6285
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6285
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
> Environment: 4 nodes, shortly updated from 1.2.11 to 2.0.2
>Reporter: David Sauer
>Assignee: Pavel Yaskevich
>Priority: Critical
> Fix For: 2.0.8
>
> Attachments: 6285_testnotes1.txt, 
> CASSANDRA-6285-disruptor-heap.patch, cassandra-attack-src.zip, 
> compaction_test.py, disruptor-high-cpu.patch, 
> disruptor-memory-corruption.patch, enable_reallocate_buffers.txt
>
>
> After altering everything to LCS the table OpsCenter.rollups60 amd one other 
> none OpsCenter-Table got stuck with everything hanging around in L0.
> The compaction started and ran until the logs showed this:
> ERROR [CompactionExecutor:111] 2013-11-01 19:14:53,865 CassandraDaemon.java 
> (line 187) Exception in thread Thread[CompactionExecutor:111,1,RMI Runtime]
> java.lang.RuntimeException: Last written key 
> DecoratedKey(1326283851463420237, 
> 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574426c6f6f6d46696c746572537061636555736564)
>  >= current key DecoratedKey(954210699457429663, 
> 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574546f74616c4469736b5370616365557365640b0f)
>  writing into 
> /var/lib/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-jb-58656-Data.db
>   at 
> org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:141)
>   at 
> org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:164)
>   at 
> org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160)
>   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$6.runMayThrow(CompactionManager.java:296)
>   at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>   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:724)
> Moving back to STC worked to keep the compactions running.
> Especialy my own Table i would like to move to LCS.
> After a major compaction with STC the move to LCS fails with the same 
> Exception.



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


[jira] [Comment Edited] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-10-21 Thread Alexander Sterligov (JIRA)

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

Alexander Sterligov edited comment on CASSANDRA-6285 at 10/21/14 11:41 PM:
---

Have you proven that it's really related to OpsCenter?

We've switched to "sync", but still get corrupted sstables. Now we get 
exception not during compaction, but at start:
{quote}
ERROR [SSTableBatchOpen:10] 2014-10-22 02:47:48,762 CassandraDaemon.java:166 - 
Exception in thread Thread[SSTableBatchOpen:10,5,main]
java.lang.IllegalStateException: SSTable first key 
DecoratedKey(4206305143314087741, 
80010001000c62617463685f6d757461746510250d00010b0d0001004e33372e3134302e3134312e3231322d6973732d736c6f745f636f6e66696775726174696f6e5f746172)
 > last key DecoratedKey(-4632241097675266745, 
80010001000c62617463685f6d757461746510260d00010b0d0001005133372e3134302e3134312e3231322d6973732d736c6f745f636f6e66696775726174696f6e5f746172676574)
at 
org.apache.cassandra.io.sstable.SSTableReader.validate(SSTableReader.java:1083) 
~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:398) 
~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:294) 
~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.io.sstable.SSTableReader$4.run(SSTableReader.java:430) 
~[apache-cassandra-2.1.0.jar:2.1.0]
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
~[na:1.7.0_51]
at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
~[na:1.7.0_51]
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
~[na:1.7.0_51]
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
[na:1.7.0_51]
at java.lang.Thread.run(Thread.java:744) [na:1.7.0_51]
{quote}

And nodetools scrub doesn't help. It finds no errors and after restart we get 
same exceptions.


was (Author: sterligovak):
Have you proven that it's really related to OpsCenter?

We've switched to "sync", but still get corrupted sstables. Now we get 
exception not during compaction, but at start:
{quote}
ERROR [SSTableBatchOpen:10] 2014-10-22 02:47:48,762 CassandraDaemon.java:166 - 
Exception in thread Thread[SSTableBatchOpen:10,5,main]
java.lang.IllegalStateException: SSTable first key 
DecoratedKey(4206305143314087741, 
80010001000c62617463685f6d757461746510250d00010b0d0001004e33372e3134302e3134312e3231322d6973732d736c6f745f636f6e66696775726174696f6e5f746172)
 > last key DecoratedKey(-4632241097675266745, 
80010001000c62617463685f6d757461746510260d00010b0d0001005133372e3134302e3134312e3231322d6973732d736c6f745f636f6e66696775726174696f6e5f746172676574)
at 
org.apache.cassandra.io.sstable.SSTableReader.validate(SSTableReader.java:1083) 
~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:398) 
~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:294) 
~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.io.sstable.SSTableReader$4.run(SSTableReader.java:430) 
~[apache-cassandra-2.1.0.jar:2.1.0]
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
~[na:1.7.0_51]
at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
~[na:1.7.0_51]
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
~[na:1.7.0_51]
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
[na:1.7.0_51]
at java.lang.Thread.run(Thread.java:744) [na:1.7.0_51]
{quote}

> 2.0 HSHA server introduces corrupt data
> ---
>
> Key: CASSANDRA-6285
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6285
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
> Environment: 4 nodes, shortly updated from 1.2.11 to 2.0.2
>Reporter: David Sauer
>Assignee: Pavel Yaskevich
>Priority: Critical
> Fix For: 2.0.8
>
> Attachments: 6285_testnotes1.txt, 
> CASSANDRA-6285-disruptor-heap.patch, cassandra-attack-src.zip, 
> compaction_test.py, disruptor-high-cpu.patch, 
> disruptor-memory-corruption.patch, enable_reallocate_buffers.txt
>
>
> After altering everything to LCS the table OpsCenter.rollups60 amd one other 
> none OpsCenter-Table got stuck with everything hanging around in L0.
> The compaction started and ran until the logs showed this:
> ERROR [CompactionExecutor:111] 2013-11-01 19:14:53,865 CassandraDaemon.java 
> (line 187) Exception in threa

[jira] [Comment Edited] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-10-21 Thread Alexander Sterligov (JIRA)

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

Alexander Sterligov edited comment on CASSANDRA-6285 at 10/21/14 6:42 PM:
--

It looks like this is not fixed in 2.1.0. We have cassandra under heavy load 
through binary interface and only OpsCenter by thrift. OpsCenter rollups are 
corrupted in about an hour after scrub.

{quote}
ERROR [CompactionExecutor:71] 2014-10-21 22:16:39,950 CassandraDaemon.java:166 
- Exception in thread Thread[CompactionExecutor:71,1,main]
java.lang.RuntimeException: Last written key DecoratedKey(-7581200918995348250, 
39352e3130382e3234322e32302d6973732d73686172645f696e666f2d676574426c6f6f6d46696c74657246616c7365506f73697469766573)
 >= current key DecoratedKey(-8301289422298317140, 
80010001000c62617463685f6d75746174650006d04a0d00010b0d00010025) 
writing into 
/ssd/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-ka-9128-Data.db
at 
org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:172)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:196) 
~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.io.sstable.SSTableRewriter.append(SSTableRewriter.java:110)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:177)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) 
~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:74)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:235)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 
~[na:1.7.0_51]
at java.util.concurrent.FutureTask.run(FutureTask.java:262) 
~[na:1.7.0_51]
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) 
~[na:1.7.0_51]
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) 
[na:1.7.0_51]
at java.lang.Thread.run(Thread.java:744) [na:1.7.0_51]
{quote}

We'll try to switch to sync and see what will happen.

Is it possible that streaming hangs because of that exception? Is it possible 
that this exception affect minor compactions of other keyspaces?


was (Author: sterligovak):
It looks like this is not fixed in 2.1.0. We have cassandra under heavy load 
through binary interface and only OpsCenter by thrift. OpsCenter rollups are 
corrupted in about an hour minutes after scrub.

{quote}
ERROR [CompactionExecutor:71] 2014-10-21 22:16:39,950 CassandraDaemon.java:166 
- Exception in thread Thread[CompactionExecutor:71,1,main]
java.lang.RuntimeException: Last written key DecoratedKey(-7581200918995348250, 
39352e3130382e3234322e32302d6973732d73686172645f696e666f2d676574426c6f6f6d46696c74657246616c7365506f73697469766573)
 >= current key DecoratedKey(-8301289422298317140, 
80010001000c62617463685f6d75746174650006d04a0d00010b0d00010025) 
writing into 
/ssd/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-ka-9128-Data.db
at 
org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:172)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:196) 
~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.io.sstable.SSTableRewriter.append(SSTableRewriter.java:110)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:177)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) 
~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:74)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
 ~[apache-cassandra-2.1.0.jar:2.1.0]
at 
org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:235)
 ~[apache-cassandra-2.1.0.jar:

[jira] [Comment Edited] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-05-19 Thread Brandon Williams (JIRA)

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

Brandon Williams edited comment on CASSANDRA-6285 at 5/19/14 7:04 PM:
--

I committed this patch to 2.0, but did not update the disruptor jar for fear of 
any further regressions, so the patch Rick tested is in there.  For 2.1, I 
committed both this patch and disruptor 0.3.5.


was (Author: brandon.williams):
I committed this patch to 2.0, but did not update the disruptor jar for fear of 
any further regressions, so the patch Rick tested in there.  For 2.1, I 
committed both this patch and disruptor 0.3.5.

> 2.0 HSHA server introduces corrupt data
> ---
>
> Key: CASSANDRA-6285
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6285
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
> Environment: 4 nodes, shortly updated from 1.2.11 to 2.0.2
>Reporter: David Sauer
>Assignee: Pavel Yaskevich
>Priority: Critical
> Fix For: 2.0.8
>
> Attachments: 6285_testnotes1.txt, 
> CASSANDRA-6285-disruptor-heap.patch, cassandra-attack-src.zip, 
> compaction_test.py, disruptor-high-cpu.patch, 
> disruptor-memory-corruption.patch, enable_reallocate_buffers.txt
>
>
> After altering everything to LCS the table OpsCenter.rollups60 amd one other 
> none OpsCenter-Table got stuck with everything hanging around in L0.
> The compaction started and ran until the logs showed this:
> ERROR [CompactionExecutor:111] 2013-11-01 19:14:53,865 CassandraDaemon.java 
> (line 187) Exception in thread Thread[CompactionExecutor:111,1,RMI Runtime]
> java.lang.RuntimeException: Last written key 
> DecoratedKey(1326283851463420237, 
> 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574426c6f6f6d46696c746572537061636555736564)
>  >= current key DecoratedKey(954210699457429663, 
> 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574546f74616c4469736b5370616365557365640b0f)
>  writing into 
> /var/lib/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-jb-58656-Data.db
>   at 
> org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:141)
>   at 
> org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:164)
>   at 
> org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160)
>   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$6.runMayThrow(CompactionManager.java:296)
>   at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>   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:724)
> Moving back to STC worked to keep the compactions running.
> Especialy my own Table i would like to move to LCS.
> After a major compaction with STC the move to LCS fails with the same 
> Exception.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Comment Edited] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-10 Thread Benedict (JIRA)

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

Benedict edited comment on CASSANDRA-6285 at 3/10/14 9:52 AM:
--

Hmm. Just taking a look at Viktor's patch, I realised that my initial 
conclusions were actually quite plausible and probably (one of) the causes of 
the problem. When I dismissed them, I didn't realise we were using a custom 
TBinaryProtocol implementation. In particular (1) is definitely possible, and 
probably the cause of the issue, although the attack jar source would be 
helpful to figure out of there are any other potential causes. We should be 
able to force the problem to occur by artificially delaying the commit log 
write to prove this.

Either way, I don't think Viktor's patch is the best way to deal with this 
problem, as it leaves cleaning up the direct buffers to GC. Since we could be 
creating a lot of these, we could create an awful lot of artificial memory 
pressure. Honestly, I think the best solution is to simply avoid using direct 
buffers with thrift, at least until 2.1, which should fix this problem by 
ensuring the CL _write_ (if not commit) has happened before performing the 
memtable insertion.


was (Author: benedict):
Hmm. Just taking a look at Viktor's patch, I realised that my initial 
conclusions were actually quite plausible and probably the causes of the 
problem. When I dismissed them, I didn't realise we were using a custom 
TBinaryProtocol implementation. In particular (1) is definitely possible, and 
probably the cause of the issue, although the attack jar source would be 
helpful to figure thatout of there are any other potential causes. We should be 
able to force the problem to occur by artificially delaying the commit log 
write to prove this.

Either way, I don't think Viktor's patch is the best way to deal with this 
problem, as it leaves cleaning up the direct buffers to GC. Since we could be 
creating a lot of these, we could create an awful lot of artificial memory 
pressure. Honestly, I think the best solution is to simply avoid using direct 
buffers with thrift, at least until 2.1, which should fix this problem by 
ensuring the CL _write_ (if not commit) has happened before performing the 
memtable insertion.

> 2.0 HSHA server introduces corrupt data
> ---
>
> Key: CASSANDRA-6285
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6285
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
> Environment: 4 nodes, shortly updated from 1.2.11 to 2.0.2
>Reporter: David Sauer
>Assignee: Pavel Yaskevich
>Priority: Critical
> Fix For: 2.0.6
>
> Attachments: 6285_testnotes1.txt, 
> CASSANDRA-6285-disruptor-heap.patch, cassandra-attack-src.zip, 
> compaction_test.py, disruptor-high-cpu.patch, 
> disruptor-memory-corruption.patch
>
>
> After altering everything to LCS the table OpsCenter.rollups60 amd one other 
> none OpsCenter-Table got stuck with everything hanging around in L0.
> The compaction started and ran until the logs showed this:
> ERROR [CompactionExecutor:111] 2013-11-01 19:14:53,865 CassandraDaemon.java 
> (line 187) Exception in thread Thread[CompactionExecutor:111,1,RMI Runtime]
> java.lang.RuntimeException: Last written key 
> DecoratedKey(1326283851463420237, 
> 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574426c6f6f6d46696c746572537061636555736564)
>  >= current key DecoratedKey(954210699457429663, 
> 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574546f74616c4469736b5370616365557365640b0f)
>  writing into 
> /var/lib/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-jb-58656-Data.db
>   at 
> org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:141)
>   at 
> org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:164)
>   at 
> org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160)
>   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$6.runMayThrow(CompactionManager.java:296)
>   at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>   at 
> java.util.concurrent.Executors$Runna

[jira] [Comment Edited] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-10 Thread Viktor Kuzmin (JIRA)

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

Viktor Kuzmin edited comment on CASSANDRA-6285 at 3/10/14 9:40 AM:
---

Attached patches for on-heap disruptor.

First pacth (disruptor-high-cpu.patch) turns off any key interests in case 
we're waiting for message to be processed. We need that cause processing may be 
delayed in case of high load and there may be something available to read from 
stream. In that case we'll have 100% cpu core usage.

Second patch (disruptor-memory-corruption.patch) makes copy from off-heap 
ByteBuffer when reading binary data. This binary data may be stored inside 
cassandra as is even after message processing. And binary data can be corrupted 
- cause it's memory may be already deallocated.


was (Author: kvaster):
Attached patches for on-heap disruptor.

First pacth turns off any key interests in case we're waiting for message to be 
processed. We need that cause processing may be delayed in case of high load 
and there may be something available to read from stream. In that case we'll 
have 100% cpu core usage.

Second patch makes copy from off-heap ByteBuffer when reading binary data. This 
binary data may be stored inside cassandra as is even after message processing. 
And binary data can be corrupted - cause it's memory may be already deallocated.

> 2.0 HSHA server introduces corrupt data
> ---
>
> Key: CASSANDRA-6285
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6285
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
> Environment: 4 nodes, shortly updated from 1.2.11 to 2.0.2
>Reporter: David Sauer
>Assignee: Pavel Yaskevich
>Priority: Critical
> Fix For: 2.0.6
>
> Attachments: 6285_testnotes1.txt, 
> CASSANDRA-6285-disruptor-heap.patch, compaction_test.py, 
> disruptor-high-cpu.patch, disruptor-memory-corruption.patch
>
>
> After altering everything to LCS the table OpsCenter.rollups60 amd one other 
> none OpsCenter-Table got stuck with everything hanging around in L0.
> The compaction started and ran until the logs showed this:
> ERROR [CompactionExecutor:111] 2013-11-01 19:14:53,865 CassandraDaemon.java 
> (line 187) Exception in thread Thread[CompactionExecutor:111,1,RMI Runtime]
> java.lang.RuntimeException: Last written key 
> DecoratedKey(1326283851463420237, 
> 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574426c6f6f6d46696c746572537061636555736564)
>  >= current key DecoratedKey(954210699457429663, 
> 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574546f74616c4469736b5370616365557365640b0f)
>  writing into 
> /var/lib/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-jb-58656-Data.db
>   at 
> org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:141)
>   at 
> org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:164)
>   at 
> org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160)
>   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$6.runMayThrow(CompactionManager.java:296)
>   at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>   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:724)
> Moving back to STC worked to keep the compactions running.
> Especialy my own Table i would like to move to LCS.
> After a major compaction with STC the move to LCS fails with the same 
> Exception.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Comment Edited] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-07 Thread Miles Shang (JIRA)

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

Miles Shang edited comment on CASSANDRA-6285 at 3/7/14 9:29 PM:


To add to [~rbranson]'s input, we're also seeing the same stacktrace as 
[~mshuler] (TimeUUID MarshalException). I inspected the row mutations that 
caused it. Three ranges were nonsensical: the key, the column name, and the 
value. By nonsensical, I mean that they don't match my expectation of what we 
are inserting in production data. All other ranges seemed fine (timestamps, 
masks, sizes, cfid). The key, column name, and value were read successfully, so 
their length metadata was good. For our data, the column comparator is 
TimeUUID. Our client library is pycassa. Whereas pycassa generates tuuids like 
this: 913d7fea-a631-11e3-8080-808080808080, the nonsensical column names look 
like this: 22050aa4-de11-e380-8080-80808080800b and this: 
10c326eb-86a4-e211-e380-808080808080. Most are of the first form. By shifting 
these nonsensical tuuids to the left or right by an octet, you get a reasonable 
tuuid. I don't have a similar insight into the nonsensical keys and values, but 
they could also be left or right shifted.


was (Author: mshang):
To add to [~rbranson]'s input, we're also seeing the same stacktrace as 
[~mshuler] (TimeUUID MarshalException). I inspected the row mutations that 
caused it. Three ranges were nonsensical: the key, the column name, and the 
value. By nonsensical, I mean that they don't match my expectation of what we 
are inserting in production data. All other ranges seemed fine (timestamps, 
masks, sizes, cfid). The key, column name, and value were read successfully, so 
their length metadata was good. For our data, the column comparator is 
TimeUUID. Our client library is pycassa. Whereas pycassa generates tuuids like 
this: 913d7fea-a631-11e3-8080-808080808080, the nonsensical column names look 
like this: 22050aa4-de11-e380-8080-80808080800b and this: 
10c326eb-86a4-e211-e380-808080808080. Most are of the first form. By shifting 
these nonsensical tuuids to the left or right by 2 octets, you get a reasonable 
tuuid. I don't have a similar insight into the nonsensical keys and values, but 
they could also be left or right shifted.

> 2.0 HSHA server introduces corrupt data
> ---
>
> Key: CASSANDRA-6285
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6285
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
> Environment: 4 nodes, shortly updated from 1.2.11 to 2.0.2
>Reporter: David Sauer
>Assignee: Pavel Yaskevich
>Priority: Critical
> Fix For: 2.0.6
>
> Attachments: 6285_testnotes1.txt, 
> CASSANDRA-6285-disruptor-heap.patch, compaction_test.py
>
>
> After altering everything to LCS the table OpsCenter.rollups60 amd one other 
> none OpsCenter-Table got stuck with everything hanging around in L0.
> The compaction started and ran until the logs showed this:
> ERROR [CompactionExecutor:111] 2013-11-01 19:14:53,865 CassandraDaemon.java 
> (line 187) Exception in thread Thread[CompactionExecutor:111,1,RMI Runtime]
> java.lang.RuntimeException: Last written key 
> DecoratedKey(1326283851463420237, 
> 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574426c6f6f6d46696c746572537061636555736564)
>  >= current key DecoratedKey(954210699457429663, 
> 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574546f74616c4469736b5370616365557365640b0f)
>  writing into 
> /var/lib/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-jb-58656-Data.db
>   at 
> org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:141)
>   at 
> org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:164)
>   at 
> org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160)
>   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$6.runMayThrow(CompactionManager.java:296)
>   at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>   at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
>   at java.util.concurrent.FutureTask.run(Fut

[jira] [Comment Edited] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-06 Thread Benedict (JIRA)

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

Benedict edited comment on CASSANDRA-6285 at 3/6/14 9:57 PM:
-

So, I think there may potentially be at least two races in the off heap 
deallocation. I suspect that may not be everything, though, as these two races 
probably won't cause the problem often. These are predicated on the assumption 
that thrift doesn't copy data from the DirectByteBuffer that the hsha server 
provides to it, so could be wrong, but anyway:

1) CL appends can be lagged behind the memtable update and, as a result, the 
acknowledgment to the client of success writing. If the CL record contains the 
ByteBuffer when it is freed, and that address is then reused in another 
allocation, it will write incorrect data to the commit log.
2) I believe thrift calls are two stage. If this is the case, and the client 
disconnects in between sending the first stage and receiving the result in the 
second stage, the buffer could be freed whilst still in flight to the 
memtable/CL

These are just quick ideas for where it might be, I haven't familiarised myself 
fully with thrift, the disruptor etc. to be certain if these are plausible, but 
it may turn out to be useful so thought I'd share.


was (Author: benedict):
So, I think there may potentially be at least two races in the off heap 
deallocation. I suspect that may not be everything, though, as these two races 
probably won't cause the problem often. These are predicated on the assumption 
that thrift doesn't copy data from the DirectByteBuffer that the hsha server 
provides to it, so could be wrong, but anyway:

1) CL appends can be lagged behind the memtable update and, as a result, the 
acknowledgment to the client of success writing. If the CL record contains the 
ByteBuffer when it is freed, and that data is then reused in another 
allocation, it will write incorrect data to the commit log.
2) I believe thrift calls are two stage. If this is the case, and the client 
disconnects in between sending the first stage and receiving the result in the 
second stage, the buffer could be freed whilst still in flight to the 
memtable/CL

These are just quick ideas for where it might be, I haven't familiarised myself 
fully with thrift, the disruptor etc. to be certain if these are plausible, but 
it may turn out to be useful so thought I'd share.

> 2.0 HSHA server introduces corrupt data
> ---
>
> Key: CASSANDRA-6285
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6285
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
> Environment: 4 nodes, shortly updated from 1.2.11 to 2.0.2
>Reporter: David Sauer
>Assignee: Pavel Yaskevich
>Priority: Critical
> Fix For: 2.0.6
>
> Attachments: 6285_testnotes1.txt, 
> CASSANDRA-6285-disruptor-heap.patch, compaction_test.py
>
>
> After altering everything to LCS the table OpsCenter.rollups60 amd one other 
> none OpsCenter-Table got stuck with everything hanging around in L0.
> The compaction started and ran until the logs showed this:
> ERROR [CompactionExecutor:111] 2013-11-01 19:14:53,865 CassandraDaemon.java 
> (line 187) Exception in thread Thread[CompactionExecutor:111,1,RMI Runtime]
> java.lang.RuntimeException: Last written key 
> DecoratedKey(1326283851463420237, 
> 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574426c6f6f6d46696c746572537061636555736564)
>  >= current key DecoratedKey(954210699457429663, 
> 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574546f74616c4469736b5370616365557365640b0f)
>  writing into 
> /var/lib/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-jb-58656-Data.db
>   at 
> org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:141)
>   at 
> org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:164)
>   at 
> org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160)
>   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$6.runMayThrow(CompactionManager.java:296)
>   at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>

[jira] [Comment Edited] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-06 Thread Michael Shuler (JIRA)

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

Michael Shuler edited comment on CASSANDRA-6285 at 3/6/14 8:54 PM:
---

With jna enabled, yes, on a single node, after running the attack jar and 
restarting c*, I get:
{noformat}
 INFO [main] 2014-03-06 14:46:51,272 ColumnFamilyStore.java (line 254) 
Initializing tmp.CF
 INFO [main] 2014-03-06 14:46:51,277 ColumnFamilyStore.java (line 254) 
Initializing system_traces.sessions
 INFO [main] 2014-03-06 14:46:51,280 ColumnFamilyStore.java (line 254) 
Initializing system_traces.events
 INFO [main] 2014-03-06 14:46:51,281 CassandraDaemon.java (line 291) completed 
pre-loading (5 keys) key cache.
 INFO [main] 2014-03-06 14:46:51,288 CommitLog.java (line 130) Replaying 
/var/lib/cassandra/commitlog/CommitLog-3-1394138577628.log, /var/lib/
cassandra/commitlog/CommitLog-3-1394138577629.log
 INFO [main] 2014-03-06 14:46:51,311 CommitLogReplayer.java (line 184) 
Replaying /var/lib/cassandra/commitlog/CommitLog-3-1394138577628.log (C
L version 3, messaging version 7)
ERROR [main] 2014-03-06 14:46:51,432 CommitLogReplayer.java (line 306) 
Unexpected error deserializing mutation; saved to /tmp/mutation77387084
28696995512dat and ignored.  This may be caused by replaying a mutation against 
a table with the same name but incompatible schema.  Exception
 follows: 
org.apache.cassandra.serializers.MarshalException: Invalid version for TimeUUID 
type.
at 
org.apache.cassandra.serializers.TimeUUIDSerializer.validate(TimeUUIDSerializer.java:39)
at 
org.apache.cassandra.db.marshal.AbstractType.validate(AbstractType.java:172)
at 
org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:276)
at 
org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:97)
at 
org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:151)
at 
org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:131)
at 
org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:312)
at 
org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:471)
at 
org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:560)
{noformat}

I'll double-check a 3 node cluster, then patch and see where I get.

(edit) this looks quite different than the previously posted errors - not sure 
if I'm on the right track, here..


was (Author: mshuler):
With jna enabled, yes, on a single node, after running the attack jar and 
restarting c*, I get:
{noformat}
 INFO [main] 2014-03-06 14:46:51,272 ColumnFamilyStore.java (line 254) 
Initializing tmp.CF
 INFO [main] 2014-03-06 14:46:51,277 ColumnFamilyStore.java (line 254) 
Initializing system_traces.sessions
 INFO [main] 2014-03-06 14:46:51,280 ColumnFamilyStore.java (line 254) 
Initializing system_traces.events
 INFO [main] 2014-03-06 14:46:51,281 CassandraDaemon.java (line 291) completed 
pre-loading (5 keys) key cache.
 INFO [main] 2014-03-06 14:46:51,288 CommitLog.java (line 130) Replaying 
/var/lib/cassandra/commitlog/CommitLog-3-1394138577628.log, /var/lib/
cassandra/commitlog/CommitLog-3-1394138577629.log
 INFO [main] 2014-03-06 14:46:51,311 CommitLogReplayer.java (line 184) 
Replaying /var/lib/cassandra/commitlog/CommitLog-3-1394138577628.log (C
L version 3, messaging version 7)
ERROR [main] 2014-03-06 14:46:51,432 CommitLogReplayer.java (line 306) 
Unexpected error deserializing mutation; saved to /tmp/mutation77387084
28696995512dat and ignored.  This may be caused by replaying a mutation against 
a table with the same name but incompatible schema.  Exception
 follows: 
org.apache.cassandra.serializers.MarshalException: Invalid version for TimeUUID 
type.
at 
org.apache.cassandra.serializers.TimeUUIDSerializer.validate(TimeUUIDSerializer.java:39)
at 
org.apache.cassandra.db.marshal.AbstractType.validate(AbstractType.java:172)
at 
org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:276)
at 
org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:97)
at 
org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:151)
at 
org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:131)
at 
org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:312)
at 
org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:471)
at 
org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:560)
{noformat}

I'll double-check a 3 node cluster, then patch and see where I get.

> 2.0 HSHA server introduces corrupt data
> ---
>
> Key: CASSANDRA-6285
> URL: https://issues.a

[jira] [Comment Edited] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-06 Thread Michael Shuler (JIRA)

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

Michael Shuler edited comment on CASSANDRA-6285 at 3/6/14 8:41 PM:
---

I'm using a linux machine  :)  - and will link in JNA - good suggestion.


was (Author: mshuler):
I'm using a linux machine  :)

> 2.0 HSHA server introduces corrupt data
> ---
>
> Key: CASSANDRA-6285
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6285
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
> Environment: 4 nodes, shortly updated from 1.2.11 to 2.0.2
>Reporter: David Sauer
>Assignee: Pavel Yaskevich
>Priority: Critical
> Fix For: 2.0.6
>
> Attachments: 6285_testnotes1.txt, 
> CASSANDRA-6285-disruptor-heap.patch, compaction_test.py
>
>
> After altering everything to LCS the table OpsCenter.rollups60 amd one other 
> none OpsCenter-Table got stuck with everything hanging around in L0.
> The compaction started and ran until the logs showed this:
> ERROR [CompactionExecutor:111] 2013-11-01 19:14:53,865 CassandraDaemon.java 
> (line 187) Exception in thread Thread[CompactionExecutor:111,1,RMI Runtime]
> java.lang.RuntimeException: Last written key 
> DecoratedKey(1326283851463420237, 
> 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574426c6f6f6d46696c746572537061636555736564)
>  >= current key DecoratedKey(954210699457429663, 
> 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574546f74616c4469736b5370616365557365640b0f)
>  writing into 
> /var/lib/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-jb-58656-Data.db
>   at 
> org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:141)
>   at 
> org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:164)
>   at 
> org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160)
>   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$6.runMayThrow(CompactionManager.java:296)
>   at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>   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:724)
> Moving back to STC worked to keep the compactions running.
> Especialy my own Table i would like to move to LCS.
> After a major compaction with STC the move to LCS fails with the same 
> Exception.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Comment Edited] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-06 Thread Pavel Yaskevich (JIRA)

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

Pavel Yaskevich edited comment on CASSANDRA-6285 at 3/6/14 8:39 PM:


[~mshuler] Can you try the same on the machine running Linux (if you haven't 
done that yet)? 

Edit: from the log it looks like Disruptor wasn't using the off-heap memory 
because JNA is disabled, "Off-heap allocation couldn't be used as JNA is not 
present in classpath or broken, using on-heap instead." So it would be great if 
you could test this on Linux with jna enabled.

Thanks!


was (Author: xedin):
[~mshuler] Can you try the same on the machine running Linux (if you haven't 
done that yet)?

> 2.0 HSHA server introduces corrupt data
> ---
>
> Key: CASSANDRA-6285
> URL: https://issues.apache.org/jira/browse/CASSANDRA-6285
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
> Environment: 4 nodes, shortly updated from 1.2.11 to 2.0.2
>Reporter: David Sauer
>Assignee: Pavel Yaskevich
>Priority: Critical
> Fix For: 2.0.6
>
> Attachments: 6285_testnotes1.txt, 
> CASSANDRA-6285-disruptor-heap.patch, compaction_test.py
>
>
> After altering everything to LCS the table OpsCenter.rollups60 amd one other 
> none OpsCenter-Table got stuck with everything hanging around in L0.
> The compaction started and ran until the logs showed this:
> ERROR [CompactionExecutor:111] 2013-11-01 19:14:53,865 CassandraDaemon.java 
> (line 187) Exception in thread Thread[CompactionExecutor:111,1,RMI Runtime]
> java.lang.RuntimeException: Last written key 
> DecoratedKey(1326283851463420237, 
> 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574426c6f6f6d46696c746572537061636555736564)
>  >= current key DecoratedKey(954210699457429663, 
> 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574546f74616c4469736b5370616365557365640b0f)
>  writing into 
> /var/lib/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-jb-58656-Data.db
>   at 
> org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:141)
>   at 
> org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:164)
>   at 
> org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160)
>   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$6.runMayThrow(CompactionManager.java:296)
>   at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>   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:724)
> Moving back to STC worked to keep the compactions running.
> Especialy my own Table i would like to move to LCS.
> After a major compaction with STC the move to LCS fails with the same 
> Exception.



--
This message was sent by Atlassian JIRA
(v6.2#6252)