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

2015-01-26 Thread Randy Fradin (JIRA)

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

Randy Fradin commented on CASSANDRA-6285:
-

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 13:48:01,531 OutputHandler.java (line 
52) Out of 

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

2014-10-29 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-6285:


I think the cause of the latest exceptions in this ticket is CASSANDRA-8211

 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] [Commented] (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-tabpanelfocusedCommentId=14182626#comment-14182626
 ] 

Alexander Sterligov commented on CASSANDRA-6285:


[~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?

 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 
 

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

2014-10-23 Thread Nikolai Grigoriev (JIRA)

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

Nikolai Grigoriev commented on CASSANDRA-6285:
--

By the way, I am getting 

{code}
ERROR [CompactionExecutor:2333] 2014-10-23 18:29:53,590 CassandraDaemon.java 
(line 199) Exception in thread Thread[Compactio
nExecutor:2333,1,main]
java.lang.RuntimeException: Last written key DecoratedKey(1156541975678546868, 
001003bc510f1
003bc510f10400100) = 
current key DecoratedKey(36735936098318717, 001000
00015feb8a10015feb8a10400100)
 writing into /
cassandra-data/disk2/myks/mytable/myks-mytable-tmp-jb-94445-Data.db
at 
org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:142)
at 
org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:165)
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$BackgroundCompactionTask.run(CompactionManager.java:198)
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:745)
{code}

with 2.0.10 release. I am using native protocol. I believe native protocol 
handler is based on HSHA, am I right? Anyway, I am getting those too.

 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 
 

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

2014-10-23 Thread Pavel Yaskevich (JIRA)

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

Pavel Yaskevich commented on CASSANDRA-6285:


[~ngrigoriev] No, native protocol is not using Thrift, which further confirms 
that it's cross transport problem, I think we should create a separate ticket 
to handle it.
[~sterligovak] Do you still have stacktrace for the NPE you've got while 
scrubbing?

 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-10-22 Thread Pavel Yaskevich (JIRA)

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

Pavel Yaskevich commented on CASSANDRA-6285:


[~sterligovak] Did you get any WARN messages like this N out of order rows 
found while scrubbing file; Those have been written (in order) to a new 
sstable new-file while running scrub? Anyhow, you will have to delete 
affected files and repair from the neighbors, I'm also not sure how much of an 
involvement Thrift has in this because the only thing that could go wrong 
(shared buffers) was already fixed to be copied for every request and 
everything is allocated on-heap

[~rbranson] Are you running HsHa with 2.1 or still on 2.0 ? 

 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] [Commented] (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-tabpanelfocusedCommentId=14179687#comment-14179687
 ] 

Alexander Sterligov commented on CASSANDRA-6285:


[~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] [Commented] (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-tabpanelfocusedCommentId=14178814#comment-14178814
 ] 

Alexander Sterligov commented on CASSANDRA-6285:


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: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?

 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 
 

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

2014-10-21 Thread Nikolai Grigoriev (JIRA)

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

Nikolai Grigoriev commented on CASSANDRA-6285:
--

[~sterligovak] I was always wondering why did I always see these problems 
appearing for OpsCenter keyspace. My keyspace had much more traffic but when I 
had this problem - it always manifested itself with OpsCenter keyspace. Even 
when I was also using Thrift (we use native protocol now).

I even remember disabling OpsCenter to prove the point :) 



 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] [Commented] (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-tabpanelfocusedCommentId=14179321#comment-14179321
 ] 

Alexander Sterligov commented on CASSANDRA-6285:


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 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.
 

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

2014-10-21 Thread Nikolai Grigoriev (JIRA)

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

Nikolai Grigoriev commented on CASSANDRA-6285:
--

I think this is the error that you cannot fix by scrubbing. Corrupted sstable. 
I was fixing those by deleting the sstables and doing repairs. Unfortunately, 
if that happens on many nodes there is a risk of data loss.

As for the OpsCenter - do not get me wrong ;) I did not want to say that 
OpsCenter was directly responsible for these troubles. But I do believe that 
OpsCenter does something particular that reveals the bug in hsha server. At 
least this was my impression. After disabling OpsCenter and fixing the 
outstanding problems I do not recall seeing those errors anymore. And I was 
also using Thrift and I was writing and reading 100x more data than OpsCenter.



 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-05-19 Thread Rick Branson (JIRA)

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

Rick Branson commented on CASSANDRA-6285:
-

Haven't been able to repro in over 5 days. We're considering the 
enable_reallocate_buffers.txt patch fixed and production-ready.

 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-05-19 Thread Mikhail Stepura (JIRA)

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

Mikhail Stepura commented on CASSANDRA-6285:


[~brandon.williams] done.

 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-05-16 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-6285:
---

I thought we were reallocating by default but I must have gotten that confused 
with on-heap buffers above.  If Viktor is right, reusing buffers is always 
potentially dangerous and should just be removed.  Can you comment, [~xedin]?

 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-05-15 Thread Brandon Williams (JIRA)

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

Brandon Williams commented on CASSANDRA-6285:
-

I have no issue with doing a) _AND_ b), just to be extra safe, if we know this 
puts the nail in this ticket's coffin.

 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-05-15 Thread Brandon Williams (JIRA)

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

Brandon Williams commented on CASSANDRA-6285:
-

The line in question: 
https://github.com/xedin/disruptor_thrift_server/commit/77d6715af0eeba4c52f42fa6ba6549c8ae52ffa7#diff-18c889f19dc9fbeb73af99dcff152b6eR421

 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-05-15 Thread Pavel Yaskevich (JIRA)

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

Pavel Yaskevich commented on CASSANDRA-6285:


No, by default it's turned off, because Thrift side expectation is that once 
the invocation is complete nobody else holds the buffers, but it seems like the 
problem is that on Cassandra side we actually never copy the buffer for the 
commit log (or was it something else?). So we need to set thrift server to 
alwayReallocate explicitly.

[~rbranson] I can give you updated jar so you don't have to wait for the 
release of Cassandra which would have alwaysReallocate set to true by default.

 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-05-15 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-6285:
---

Yeah, we do treat BB as immutable so CL would understandably not expect Thrift 
to pull the rug out from under it.

I'm fine with calling alwaysReallocate on the Cassandra side in the interest of 
not changing things out from under any other users.

 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-05-14 Thread Edward Capriolo (JIRA)

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

Edward Capriolo commented on CASSANDRA-6285:


@Pavel I understand what you are saying. I understand what unit test is. 
Whenever I submit patches to the ASF they come with tests. ::cough:: ::cough::. 
In any case, what I was saying is the external dependency does not do load 
testing. Cassandra does not default to hsha. I DO NOT see any reference in this 
conversation into how exactly the HSHA server is now load/correctness tested. 
If such a test exists great, if not potentially should be added.

 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-05-14 Thread Edward Capriolo (JIRA)

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

Edward Capriolo commented on CASSANDRA-6285:


Also while we are on the topic.

{quote}
# The default is sync because on Windows hsha is about 30% slower.  On Linux,
# sync/hsha performance is about the same, with hsha of course using less 
memory.
#
# Alternatively,  can provide your own RPC server by providing the 
fully-qualified class name
# of an o.a.c.t.TServerFactory that can create an instance of it.
rpc_server_type: sync
{quote}

The logic behind this default confuses me. The the vast majority of the 
cassandra user base is linux. We chose 'sync' so the uncommon case is not 
slowed down. Clearly anyone using linux should switch to hsha because it uses 
less memory and is wiched fast according to github tests. But not being the 
default does it really get performance/correctness evaluated in any meaningful 
way?



 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-05-13 Thread Rick Branson (JIRA)

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

Rick Branson commented on CASSANDRA-6285:
-

I think what might help this specific quality issue out is just moving to the 
new HSHA implementation entirely in a later version and removing the choice. 
The new HSHA supposedly eliminates the performance issues that made it not a 
good default choice, so it appears as if there's no advantage to having the 
other choices.

 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-05-13 Thread Rick Branson (JIRA)

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

Rick Branson commented on CASSANDRA-6285:
-

I did some more digging around on our cluster that was running 2.0.6 when it 
saw the corruption: it took anywhere from a few hours to 48 hours for the first 
compaction with the out of order key exception to throw. These nodes are 
receiving thousands of writes per second, so it's not going to be trivially 
reproducible. We've been running one of the nodes with 2.0.8-tenative + 
enable_reallocate_buffers.txt and will report back once we've reached 72 hours 
and are comfortable rolling this out wide to our own clusters.

 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-05-13 Thread Brandon Williams (JIRA)

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

Brandon Williams commented on CASSANDRA-6285:
-

For 2.1, I can get behind that I think, especially calling it 'disruptor' or 
pretty much anything besides 'HSHA.'  For 2.0 though it's hard to swallow in a 
minor.

 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-05-12 Thread Pavel Yaskevich (JIRA)

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

Pavel Yaskevich commented on CASSANDRA-6285:


[~brandon.williams] 0.3.5 is already available on [maven 
central|http://search.maven.org/remotecontent?filepath=com/thinkaurelius/thrift/thrift-server/0.3.5/thrift-server-0.3.5.jar]
 so act as you think appropriate.

 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-05-12 Thread Benedict (JIRA)

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

Benedict commented on CASSANDRA-6285:
-

For many workloads sync is faster than async on linux also (by a significant 
margin), so perhaps the docs should be updated.

 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-05-12 Thread Pavel Yaskevich (JIRA)

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

Pavel Yaskevich commented on CASSANDRA-6285:


So I can do two things, a). set alwaysReuse to true by default and release 
0.3.5 today b). you can just switch to alwaysReallocate(true) in the 
configuration for 2.0.8, either works for me.

 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-05-11 Thread Edward Capriolo (JIRA)

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

Edward Capriolo commented on CASSANDRA-6285:


I was poking around the dependency a bit

{quote}
Running com.thinkaurelius.thrift.OffHeapMultiRequestTest
Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 2.515 sec
Running com.thinkaurelius.thrift.OnHeapMultiConnectionWithReallocateTest
Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.481 sec
Running com.thinkaurelius.thrift.OffheapMultiConnectionWithRellocateTest
Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.478 sec
Running com.thinkaurelius.thrift.OnHeapMultiConnectionTest
Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.417 sec
Running com.thinkaurelius.thrift.OnHeapMultiRequestWithReallocateTest
Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.735 sec
Running com.thinkaurelius.thrift.OffHeapMultiRequestWithReallocateTest
Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.437 sec
Running com.thinkaurelius.thrift.OffHeapMultiConnectionTest
Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.282 sec
Running com.thinkaurelius.thrift.OnHeapMultiRequestTest
Tests run: 2, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 0.491 sec
{quote}

Q. Are a few tests that run in roughly 20 seconds enough to prove that this 
component fit for production? These highly concurrent off heap systems can have 
very subtle bugs as this ticket shows.

If I understand correct HSHA is not the default, the artifact has good test 
coverage, and only a handful of findbugs issues. Is there any piece that is 
going to run end-to-end or attempt to load/concurrently test these classes and 
be more rigorous then the previous system? Can that be made?

 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 

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

2014-05-11 Thread Pavel Yaskevich (JIRA)

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

Pavel Yaskevich commented on CASSANDRA-6285:


[~brandon.williams] I have released 0.3.5 just now, with reallocation and 
on-heap buffers turned on by default. +1 on the change so it's either we commit 
0.3.5 or your patch.

[~appodictic] Here is a definition of [unit 
testing|http://en.wikipedia.org/wiki/Unit_testing], in my tests I cover 
single/multi connection on-heap/off-heap +/- reallocation scenarios, basically 
everything to prove that server functions properly in all of the modes and 
returns correct results based on the operation being used. end-to-end tests are 
what systems which integrate project are supposed to do and that is done by 
stress and bpdlab testing. If you have been following discussion in this ticket 
you must have already realized that the problem is not caused by HsHa server 
directly but rather by the fact that Cassandra holds Thrift buffers even after 
blocking evaluation is finished.

 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-20 Thread Edward Capriolo (JIRA)

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

Edward Capriolo commented on CASSANDRA-6285:


I read thought this. Does it make sense to call this HSHA2 and restore the old 
code and call it HSHA? I

 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$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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-18 Thread Robert Coli (JIRA)

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

Robert Coli commented on CASSANDRA-6285:


{quote}
... Telling the truth, I don't think that this is really fixed in 2.0.6.
{quote}
If hsha is irrevocably broken with data corruption risk in 2.0 line, could we 
either get it wired off in the next point release, or some messaging in 
NEWS.txt that instructs people not to use it? My preference is the former to 
cover upgraders who are foolish enough to not read NEWS.txt; I am unable to see 
the benefit of leaving it usable if it is known broken.


 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$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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-14 Thread Viktor Kuzmin (JIRA)

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

Viktor Kuzmin commented on CASSANDRA-6285:
--

Telling the truth, I don't think that this is really fixed in 2.0.6.
It's not easy to reproduce bug right now, but I think it can be. 
thrift-disruptor server does not allocate new Buffer for new message in case 
new message is of equal size with previous. In that case bug can be reproduced 
even with on-heap allocation.

 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$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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-14 Thread Benedict (JIRA)

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

Benedict commented on CASSANDRA-6285:
-

+1. That needs to be fixed as well.

 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$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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-12 Thread Viktor Kuzmin (JIRA)

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

Viktor Kuzmin commented on CASSANDRA-6285:
--

I've tried my test with beta1 and I can confirm that I was not able to 
reproduce bug.
I think that it will be better to not use disruptor on 2.0.x even with on-heap 
allocation (we can still reuse buffer in case message will be of equal size 
when previous).
And it should be safe to use disruptor on 2.1 branch.

We'll be waiting for 2.1 release, cause it really impressed me over 2.0

 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$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] [Commented] (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-tabpanelfocusedCommentId=13925605#comment-13925605
 ] 

Benedict commented on CASSANDRA-6285:
-

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$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] [Commented] (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-tabpanelfocusedCommentId=13925609#comment-13925609
 ] 

Viktor Kuzmin commented on CASSANDRA-6285:
--

My patch is defenetly NOT GOOD. Also for me that patch means anothe thing: it 
seems that we have 'success' answer  before data is passed to commitlog... I 
don't think that this is good.

 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$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] [Commented] (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-tabpanelfocusedCommentId=13925614#comment-13925614
 ] 

Benedict commented on CASSANDRA-6285:
-

bq. we have 'success' answer before data is passed to commitlog

Yes, see my comment from a few days ago:

bq. 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.

This is an absolutely plausible scenario since we do actually slice directly 
from the DirectByteBuffer, which I previously thought we did not.

 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$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] [Commented] (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-tabpanelfocusedCommentId=13925617#comment-13925617
 ] 

Benedict commented on CASSANDRA-6285:
-

Has anybody tested this problem against 2.1? As if this is the only issue, it 
should be fixed there.

 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$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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-10 Thread Ryan McGuire (JIRA)

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

Ryan McGuire commented on CASSANDRA-6285:
-

[~benedict] I haven't yet been able to reproduce this with anything other than 
Viktor's attack jar. I'm thinking Java's threading is beating Python's 
threading here, so I [created a 
dtest|https://github.com/riptano/cassandra-dtest/blob/master/thrift_hsha_test.py]
 that just run's his jar directly. This test is currently passing on 
cassandra-2.0 and cassandra-2.1 HEAD. 

 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$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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-10 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-6285:
---

You'd want to revert Pavel's patch from 2.1 to test Benedict's theory.

 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$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] [Commented] (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-tabpanelfocusedCommentId=13926233#comment-13926233
 ] 

Viktor Kuzmin commented on CASSANDRA-6285:
--

You may set threads count to only one in cass-atack jar and you will be still 
able to reproduce error.

 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$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] [Commented] (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-tabpanelfocusedCommentId=13929762#comment-13929762
 ] 

Benedict commented on CASSANDRA-6285:
-

bq. You'd want to revert Pavel's patch from 2.1

beta1 should be fine to test against for this

 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$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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-09 Thread Ryan McGuire (JIRA)

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

Ryan McGuire commented on CASSANDRA-6285:
-

I'd like to be able to reproduce this in dtests to track this bug. Seeing as 
[~rhatch]'s python test wasn't able to repro this issue, and a [quick test I 
wrote|https://github.com/riptano/cassandra-dtest/blob/cassandra-6285/test_6285.py]
 doesn't either, does anyone have a simple way to reproduce this issue? 

[~kvaster] would you mind sharing the source code for your attack jar?

 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-07 Thread Pavel Yaskevich (JIRA)

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

Pavel Yaskevich commented on CASSANDRA-6285:


Ah, so they have finally made transport a protected field in FrameBuffer... 
Well, that considerably complicates things with switching back 
TThreadedSelectorServer.

 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] [Commented] (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-tabpanelfocusedCommentId=13924330#comment-13924330
 ] 

Miles Shang commented on CASSANDRA-6285:


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(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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-06 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-6285:
---

[~mshuler] Can you test hsha with Viktor's jar above?  
(https://issues.apache.org/jira/browse/CASSANDRA-6285?focusedCommentId=13917950page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13917950)

I want to know if
# you can reproduce with a single node
# if not, if you can reproduce with multiple nodes
# assuming either 1 or 2, if you can still reproduce after applying Pavel's 
heap allocation path

 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: 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] [Commented] (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-tabpanelfocusedCommentId=13922903#comment-13922903
 ] 

Michael Shuler commented on CASSANDRA-6285:
---

Sure - let me see what I can find out.

 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: 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] [Commented] (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-tabpanelfocusedCommentId=13923022#comment-13923022
 ] 

Pavel Yaskevich commented on CASSANDRA-6285:


[~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)


[jira] [Commented] (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-tabpanelfocusedCommentId=13923026#comment-13923026
 ] 

Michael Shuler commented on CASSANDRA-6285:
---

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] [Commented] (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-tabpanelfocusedCommentId=13923104#comment-13923104
 ] 

Michael Shuler commented on CASSANDRA-6285:
---

Both a single local hsha node and 3x node ccm cluster with hsha (jna on both) 
throw the above errors after attack.jar run and restart.  The patch does appear 
to fix both single and ccm cluster.  My pre-patch ccm cluster never fully 
restarted, but do we need logs or anything from before/after?

 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] [Commented] (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-tabpanelfocusedCommentId=13923112#comment-13923112
 ] 

Pavel Yaskevich commented on CASSANDRA-6285:


I don't think we need logs, [~jbellis] I'm thinking of commiting attached patch 
which should help meanwhile I working on resolving off-heap problem, WDYT?

 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] [Commented] (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-tabpanelfocusedCommentId=13923125#comment-13923125
 ] 

Benedict commented on CASSANDRA-6285:
-

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)
   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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-06 Thread Rick Branson (JIRA)

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

Rick Branson commented on CASSANDRA-6285:
-

Unfortunately this hit us during our 2.0.5 upgrade and 'sync' is not an option 
for the # of connections we have per node (tried this). We've been running 
Marcus' patch in prod and limping along on it, but it looks like the 
requestInvoke() override is causing the requests to get executed on the 
selector pool (which is limited by CPU #) instead of the executor service so 
our response times are pretty bad. The lack of anything showing up in the JMX 
for the executor service definitely points towards this. 

 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] [Commented] (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-tabpanelfocusedCommentId=13923275#comment-13923275
 ] 

Pavel Yaskevich commented on CASSANDRA-6285:


[~rbranson] Which Marcus' patch are you talking about? Also I want to clarify 
one thing - disruptor server doesn't use requestInvoke(FrameBuffer) but 
dispatchInvoke(Message) which schedules message to executor pool based on ring 
buffer (WorkerPool) so actual execution is done in the separate thread. I can 
attach a patch which would switch back to TThreadedSelectorServer which is 
packed with Thrift (the only different between it and disruptor is that it 
schedules to classic thread pool), maybe disruptor server wasn't as good an 
idea for all of the real world use cases...

 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] [Commented] (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-tabpanelfocusedCommentId=13923318#comment-13923318
 ] 

Benedict commented on CASSANDRA-6285:
-

It looks like thrift doesn't retain the DirectByteBuffer, just reads straight 
from them. The only possible window for corruption is during the construction 
of the thrift args object, which is a fairly narrow window.

 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-06 Thread Rick Branson (JIRA)

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

Rick Branson commented on CASSANDRA-6285:
-

We put in the TThreadedSelectorServer patch from Marcus. On top of that, to get 
our response times down from the 10x what they should be, I rolled out a larger 
hard-coded selector thread pool size of 256 (instead of the # of processors -- 
a measly 16). This is shaping up nicely.

 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] [Commented] (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-tabpanelfocusedCommentId=13923453#comment-13923453
 ] 

Pavel Yaskevich commented on CASSANDRA-6285:


[~rbranson] So what you are saying is that after problem with disruptor you 
never tried it again with on-heap buffers but switched to 
TThreadedSelectorServer and increased selector pool size and the 
requestInvoke() is the problem with TThreadedSelectorServer?

 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-06 Thread Rick Branson (JIRA)

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

Rick Branson commented on CASSANDRA-6285:
-

[~xedin]: We had perf issues with the disruptor as well (sudden spikes of CPU 
to 100%) + this so I just wanted to get production away from it ASAP.

 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] [Commented] (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-tabpanelfocusedCommentId=13923484#comment-13923484
 ] 

Pavel Yaskevich commented on CASSANDRA-6285:


[~rbranson] But the most important question for this ticket at least is - did 
you run with on or off heap buffers? I can bring TThreadedSelectorServer back 
in this ticket or just go with on-heap buffers and disruptor. [~mshuler] do you 
have any performance tests related to Thrift server? Maybe there is a low 
hanging fruit in there to fix up the spikes that Rick mentioned if we can 
reproduce.

 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-06 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-6285:
---

bq. I'm thinking of commiting attached patch which should help meanwhile I 
working on resolving off-heap problem

Yeah, let's do this for now and roll 2.0.6 so we can stop the bleeding, then 
figure out whether doing more work on disruptor or TTSS is better for 2.0.7.

 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-06 Thread Rick Branson (JIRA)

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

Rick Branson commented on CASSANDRA-6285:
-

[~xedin]: off-heap for disruptor.

I think that we should really consider bringing back the old HSHA 
implementation from 1.2 as the hsha and allow switching to the disruptor 
implementation as another rpc_server_type for those that want to try it out.

 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-06 Thread Brandon Williams (JIRA)

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

Brandon Williams commented on CASSANDRA-6285:
-

bq. I think that we should really consider bringing back the old HSHA 
implementation from 1.2 as the hsha and allow switching to the disruptor 
implementation as another rpc_server_type for those that want to try it out.

I think I'm inclined to agree with this, aside from creating yaml creation 
problems in a minor.  If we just give up and effectively revert, what have we 
lost?  We need 2.0 stabilization sooner rather than later now, in all aspects.  
If we can't trust disruptor except with a small change, let's just not trust it 
yet and worry about that in a future release.  With 2.1 beta already out, we 
can't tolerate much instability in the 2.0 branch.

 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] [Commented] (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-tabpanelfocusedCommentId=13923541#comment-13923541
 ] 

Pavel Yaskevich commented on CASSANDRA-6285:


I'm not sure if there is a point of going all the way back to original HsHa 
when there is TThrededSelectorServer, but I'm fine with going with disruptor as 
a separate option, something like disruptor_hsha and making hsha - 
TThrededSelectorServer from Thrift, that's how I wanted it originally. Also I 
just want to mention that people have reported that disruptor works for them 
with on-heap buffers, so I am not sure if we need to go all paranoid about 
this...

 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-06 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-6285:


My branch from above needed a tiny hack to thrift 
(https://github.com/krummas/thrift/commit/01ba2a3f3d386d0981371aab2494470e2a78e596),
 so if we want to roll with TTSS we should refactor our thrift usage a bit to 
avoid that hack

 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-04 Thread Christopher Wirt (JIRA)

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

Christopher Wirt commented on CASSANDRA-6285:
-

[~Jonathan Ellis] [~Marcus Eriksson] That was my post from the user mail list.
After our 1.2.14 - 2.0.5 upgrade and failure to get the new HsHa stable in our 
system, we moved to using the thrift 0.9.1 TTSS with reasonable success. We've 
now been running for two weeks under a relatively high read load.
We haven't seen any DecoratedKey != ... errors.
We have seen some warnings on start up about SSTable rows being out of order.
We have seen commit logs starting to build up with All time blocked on the 
FlushWriter incrementing.

Performance comparisons might be a little unfair, but certainly our p95, p99 
have overall improved.

Obviously very keen to not be running a custom build of C*.

 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: 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-04 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-6285:


[~chris.wirt] could you paste those startup log lines?

 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: 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-04 Thread Christopher Wirt (JIRA)

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

Christopher Wirt commented on CASSANDRA-6285:
-

These have since disappeared. I just restarted this node just now to check. We 
haven't run a scrub.

 WARN [main] 2014-02-16 23:23:02,032 LeveledManifest.java (line 171) At level 
1, 
SSTableReader(path='/disk2/cassandra/data/struqrealtime/impressionstorev2/struqrealtime-impressionstorev2-jb-118905-Data.db')
 [DecoratedKey(-1513272878957942943, c41c955b40274acfa466ccb6079a21e5), 
DecoratedKey(6301362410765453237, 43ae61aacbc446be92c8bdea1d43e342)] overlaps 
SSTableReader(path='/disk2/cassandra/data/struqrealtime/impressionstorev2/struqrealtime-impressionstorev2-jb-116400-Data.db')
 [DecoratedKey(3953001739649874864, 5811ce41b7014917ab82eb32e8861ca5), 
DecoratedKey(9190609424240623933, 4e5b00a5a7594289924674974f44a995)].  This 
could be caused by a bug in Cassandra 1.1.0 .. 1.1.3 or due to the fact that 
you have dropped sstables from another node into the data directory. Sending 
back to L0.  If you didn't drop in sstables, and have not yet run scrub, you 
should do so since you may also have rows out-of-order within an sstable


 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: 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-04 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-6285:


[~chris.wirt] those entries are unrelated (and fixed in CASSANDRA-6688)

 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: 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-03 Thread Viktor Kuzmin (JIRA)

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

Viktor Kuzmin commented on CASSANDRA-6285:
--

I've tried to investigate problem with HsHaDistruptorServer, but with no luck. 
Telling the truth, I see no reason for that server to corrupt data.
Also HsHaDistruptor do not corrupt data in case useHeapBasedAllocation is 
turned on.
More over if you look at disruptor-thrift-server code - 
Message.reallocateDataBuffer and turn on heap based allocation only for 
dataBuffer then you will not see corruption.

 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: 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-03 Thread Pavel Yaskevich (JIRA)

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

Pavel Yaskevich commented on CASSANDRA-6285:


[~ngrigoriev] and [~brandon.kearby] can you try setting useHeapBasedAllocation 
to true ? I'm fine with switch back to TThreadedSelectorServer if that helps.

 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: 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-03 Thread Nikolai Grigoriev (JIRA)

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

Nikolai Grigoriev commented on CASSANDRA-6285:
--

[~xedin]

That seems to be a parameter of the Thrift server...How do I control this 
parameter? Or I should just disable JNA?

 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: 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] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data

2014-03-03 Thread Pavel Yaskevich (JIRA)

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

Pavel Yaskevich commented on CASSANDRA-6285:


You can do it via JMX or disable JNA, I can also make a patch with would set it 
explicitly in Cassandra code.

 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: 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)