cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Alexander Sterligov (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-6285) 2.0 HSHA server introduces corrupt data
Date Fri, 24 Oct 2014 09:26:36 GMT

    [ https://issues.apache.org/jira/browse/CASSANDRA-6285?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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 Name                    Active   Pending      Completed   Blocked  All time blocked
CounterMutationStage              0         0              0         0                 0
ReadStage                         0         0         392196         0                 0
RequestResponseStage              0         0        5271906         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
MigrationStage                    0         0              0         0                 0
ValidationExecutor                0         0          39446         0                 0
MemtableReclaimMemory             0         0          29927         0                 0
InternalResponseStage             0         0         588279         0                 0
AntiEntropyStage                  0         0        5325285         0                 0
MiscStage                         0         0              0         0                 0
CommitLogArchiver                 0         0              0         0                 0
MemtableFlushWriter               0         0          29927         0                 0
PendingRangeCalculator            0         0             30         0                 0
MemtablePostFlush                 0         0         135734         0                 0
CompactionExecutor               31        31         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 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)

Mime
View raw message