Hi again,

After some further investigation now I'm in a situation that there are 3 nodes (of 6 nodes cluster) and all of them are falling with ValidationExecutor during compaction which is is trigger by "repair -pr PRODUCTION UserCompletions" against any node in the cluster - repair get's stuck and i have to use the forceTerminateAllRepairessions in JMX method.
Other repairs on different CFs are working fine.

So afaiu this must be just some BAD composite column name that unfortunately passed mutation and now can't be read? This is present on all replicas - which mean I can not do cassandra restart with deletion of relevant sstables an all 3 replicas - since I'll obviously loose data.
And I'm goind to hit the end of gc period really soon on this CF.

So any suggestions on what can I do?
Is there way to make cassandra throw away the offending column?
can I find which column is problematic? maybe sstable2json can somehow help?

Thanks
Alex


On 06/14/2012 04:04 PM, Sylvain Lebresne wrote:
On Thu, Jun 14, 2012 at 12:00 PM, Piavlo<lolitus...@gmail.com>  wrote:
What's the procedure to check if the compressed sstable is corrupted or not?
Since you use compression, in theory that can't be disk bitrot since
in that case
you would have got some checksum error instead. The fact that it happened
on 3 nodes during a repair also does suggest that this was not a
simple disk bitrot.

Not sure why that happened though (you don't use the StorageProxy api by any
chance?).

Is there way to delete all UserCompletions sstables on the problematic node
and then run repair on this CF only?
Like disable thrift, drain memtables so it does not read commit log on
startup and then delete the sstables and start the node again will it work?
It should work and should at least fix the compaction problem. Though to win
some time you may want to remove only those sstables that triggered the
exception in the first place. Trusting the log above that would be the ones
mentioned here:
INFO [CompactionExecutor:3335] 2012-06-14 05:14:02,088
CompactionTask.java (line 115) Compacting
[SSTableReader(path='/var/lib/cassandra/data/PRODUCTION/UserCompletions-hc-838-Data.db'),
SSTableReader(path='/var/lib/cassandra/data/PRODUCTION/UserCompletions-hc-1038-Data.db'),
SSTableReader(path='/var/lib/cassandra/data/PRODUCTION/UserCompletions-hc-846-Data.db'),
SSTableReader(path='/var/lib/cassandra/data/PRODUCTION/UserCompletions-hc-949-Data.db'),
SSTableReader(path='/var/lib/cassandra/data/PRODUCTION/UserCompletions-hc-895-Data.db'),
SSTableReader(path='/var/lib/cassandra/data/PRODUCTION/UserCompletions-hc-883-Data.db')]

can it mean that sstable is not corrupted but just some BAD column name was
inserted OK but can not be read later read by ValidationExecutor in any of
the replica nodes?
Well, the exception really only says that you have a bad column name.
What produced it is unclear.

--
Sylvain

Check the relevant cassandra logs below

dsc2b.internal/10.234.71.33
-----------------------
  INFO [AntiEntropySessions:66] 2012-06-13 18:49:24,464
AntiEntropyService.java (line 658) [repair
#7ec142c0-b588-11e1-0000-f423231d3fff] new session: will sync
dsc2b.internal/10.234.71.33, /10.49.127.4, /10.58.249.118 on range
(85070591730234615865843651857942052864,113427455640312821154458202477256070485]
for PRODUCTION.[UserCompletions]
  INFO [AntiEntropySessions:66] 2012-06-13 18:49:24,465
AntiEntropyService.java (line 837) [repair
#7ec142c0-b588-11e1-0000-f423231d3fff] requests for merkle tree sent for
UserCompletions (to [/10.49.127.4, /10.58.249.118,
dsc2b.internal/10.234.71.33])
  INFO [ValidationExecutor:129] 2012-06-13 18:49:24,466
ColumnFamilyStore.java (line 705) Enqueuing flush of
Memtable-UserCompletions@843906517(9952311/21343163 serialized/live bytes,
41801 ops)
  INFO [FlushWriter:2563] 2012-06-13 18:49:24,467 Memtable.java (line 246)
Writing Memtable-UserCompletions@843906517(9952311/21343163 serialized/live
bytes, 41801 ops)
  INFO [FlushWriter:2563] 2012-06-13 18:49:24,828 Memtable.java (line 283)
Completed flushing
/var/lib/cassandra/data/PRODUCTION/UserCompletions-hc-515-Data.db (1671566
bytes)
ERROR [ValidationExecutor:129] 2012-06-13 18:55:32,236
AbstractCassandraDaemon.java (line 139) Fatal exception in thread
Thread[ValidationExecutor:129,1,main]
java.lang.IllegalArgumentException
        at java.nio.Buffer.limit(Buffer.java:249)
....
-----------------------

dsc1a.internal/10.49.127.4
-----------------------
  INFO [ValidationExecutor:125] 2012-06-13 18:49:24,457
ColumnFamilyStore.java (line 705) Enqueuing flush of
Memtable-UserCompletions@266077104(9047552/76151840 serialized/live bytes,
38000 ops)
  INFO [FlushWriter:2670] 2012-06-13 18:49:24,466 Memtable.java (line 246)
Writing Memtable-UserCompletions@266077104(9047552/76151840 serialized/live
bytes, 38000 ops)
  INFO [FlushWriter:2670] 2012-06-13 18:49:24,969 Memtable.java (line 283)
Completed flushing
/var/lib/cassandra/data/PRODUCTION/UserCompletions-hc-1030-Data.db (1508368
bytes)
  INFO [CompactionExecutor:3299] 2012-06-13 18:49:24,971 CompactionTask.java
(line 115) Compacting
[SSTableReader(path='/var/lib/cassandra/data/PRODUCTION/UserCompletions-hc-1027-Data.db'),
SSTableReader(path='/var/lib/cassandra/data/PRODUCTION/UserCompletions-hc-1030-Data.db'),
SSTableReader(path='/var/lib/cassandra/data/PRODUCTION/UserCompletions-hc-1028-Data.db'),
SSTableReader(path='/var/lib/cassandra/data/PRODUCTION/UserCompletions-hc-1029-Data.db')]
  INFO [CompactionExecutor:3299] 2012-06-13 18:50:03,554 CompactionTask.java
(line 223) Compacted to
[/var/lib/cassandra/data/PRODUCTION/UserCompletions-hc-1031-Data.db,].
  23,417,251 to 23,832,802 (~101% of original) bytes for 116,956 keys at
0.589102MB/s.  Time: 38,582ms.
ERROR [ValidationExecutor:125] 2012-06-13 18:56:58,961
AbstractCassandraDaemon.java (line 139) Fatal exception in thread
Thread[ValidationExecutor:125,1,main]
java.lang.IllegalArgumentException
        at java.nio.Buffer.limit(Buffer.java:249)
...
-------------------------

dsc2c.internal/10.58.249.118
-------------------------

  INFO [ValidationExecutor:119] 2012-06-13 18:49:24,305
ColumnFamilyStore.java (line 705) Enqueuing flush of
Memtable-UserCompletions@1279460811(19014066/66201229 serialized/live bytes,
79838 ops)
  INFO [FlushWriter:2001] 2012-06-13 18:49:24,326 Memtable.java (line 246)
Writing Memtable-UserCompletions@1279460811(19014066/66201229
serialized/live bytes, 79838 ops)
  INFO [FlushWriter:2001] 2012-06-13 18:49:24,848 Memtable.java (line 283)
Completed flushing
/var/lib/cassandra/data/PRODUCTION/UserCompletions-hc-548-Data.db (3177074
bytes)
ERROR [ValidationExecutor:119] 2012-06-13 18:55:50,387
AbstractCassandraDaemon.java (line 139) Fatal exception in thread
Thread[ValidationExecutor:119,1,main]
java.lang.IllegalArgumentException
        at java.nio.Buffer.limit(Buffer.java:249)
...
-------------------------

Thanks for your help.



On 06/14/2012 11:09 AM, Sylvain Lebresne wrote:
On Thu, Jun 14, 2012 at 8:26 AM, Piavlo<lolitus...@gmail.com>    wrote:
I started looking for similar messages on other nodes saw a SINGLE
IllegalArgumentException on
ValidationExecutor on the same node and 2 other nodes (this is a 6 node
cluster) which happened
at almost the same time , in all nodes while flushing same
UserCompletions CF memtable. This
happened about 12hours before the IllegalArgumentException in
  CompactionExecutor.
This actually does not happen during a flush but during a validation
compaction, which happens during a repair.
The exception is basically saying there is invalid composite column
name (you do use a composite comparator right?).
I guess that could result from some on-disk corruption. Are you using
sstable compression on UserCompletions? (I am asking because
compressed sstables have checksums)

And even bigger problem now is that running repairs on other CFs against
different nodes does not have any effect, for example running
/usr/bin/nodetool -h dsc2b.internal -pr repair PRODUCTION
UserDirectVendors
does not trigger any repair activity and nothing in the logs to indicate
a
start of repair. And I have ~24hours left to repair some CFs before the
gc
period ends :(
Does that happen on every node?
What can happen is that some failed repair may block other from
starting. One thing you can try is to run the method called
forceTerminateAllRepairessions in JMX under
org.apache.cassandra.db->StorageService->Operations (I'm afraid there
is no nodetool hook so you will have to use jconsole). After that, try
starting a repair again. If that doesn't work, it's worth trying to
restart the node.

--
Sylvain


Reply via email to