You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by Alexis Lauthier <al...@yahoo.com> on 2012/01/15 19:53:17 UTC

Re : Compressed families not created on new node

I see only one line "JOINING: sleeping 30000 ms for pending range setup".

Before that, I have a lot of migration INFO messages, only for the uncompressed families.

I have currently killed the new node, so a describe cluster shows only one schema version on the first two nodes.


[default@unknown] describe cluster;
Cluster Information:
   Snitch: org.apache.cassandra.locator.SimpleSnitch
   Partitioner: org.apache.cassandra.dht.ByteOrderedPartitioner
   Schema versions: 
    b42595d0-2247-11e1-0000-a0e9ff9ab7bf: [---.36, ---.35]

    UNREACHABLE: [---.56]



Also, on one of the old nodes, I have a lot of I/O errors on the data files for some (but not all) of the compressed families. It began a few days ago. All "nodetool repair" calls have been blocking since then.

Any ideas on how I can get the data on the new node, before the old one dies?



Thanks,
Alexis Lauthier




________________________________
 De : aaron morton <aaron@>
À : user@cassandra.apache.org 
Envoyé le : Dimanche 15 Janvier 2012 19h17
Objet : Re: Compressed families not created on new node
 

Sounds like the schema has not fully migrated to the new node. It is applied to the joining node one change at a time. A quick scan of the changes file does not find anything fixed after 1.0.3

You can check schema versions in the CLI using the describe cluster command. 

Check for errors in the logs with Migration in the text.  

Are you seeing this line a lot in the log ? 
 INFO [main] 2012-01-13 14:55:00,493 StorageService.java (line 616) JOINING: sleeping 30000 ms for pending range setup
>

cheers

-----------------
Aaron Morton
Freelance Developer
@aaronmorton
http://www.thelastpickle.com 

On 14/01/2012, at 4:20 AM, Alexis Lauthier wrote:

I'm using Cassandra 1.0.3 on a 2 nodes cluster. My schema (with replication_factor=2) contains both compressed (with sstable_compression=DeflateCompressor) and uncompressed column families.
>
>
>
>When bootstrapping a third node, the uncompressed families are created on the new node as expected, but the compressed families are not. Only the uncompressed families appear in a "show schema", and the new node data size is small, which is consistent with the big compressed data not being there.
>
>
>I'm seeing frequent exceptions in the log :
>
>
> INFO [main] 2012-01-13 14:55:00,493 StorageService.java (line 616) JOINING: sleeping 30000 ms for pending range setup
>ERROR [MutationStage:1] 2012-01-13 14:55:01,511 RowMutationVerbHandler.java (line 65) Error in row
 mutation
>org.apache.cassandra.db.UnserializableColumnFamilyException: Couldn't find cfId=1008
>    at org.apache.cassandra.db.ColumnFamilySerializer.deserialize(ColumnFamilySerializer.java:129)
>    at org.apache.cassandra.db.RowMutation$RowMutationSerializer.deserialize(RowMutation.java:401)
>    at org.apache.cassandra.db.RowMutation$RowMutationSerializer.deserialize(RowMutation.java:409)
>    at org.apache.cassandra.db.RowMutation.fromBytes(RowMutation.java:357)
>    at org.apache.cassandra.db.RowMutationVerbHandler.doVerb(RowMutationVerbHandler.java:46)
>    at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:59)
>    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
>    at
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
>    at java.lang.Thread.run(Thread.java:722)
>
>
>
>
>
>After a few minutes, The column family names are shown instead of their ids ("UrlText" is one of the compressed families) :
>
>
>
>ERROR [ReadStage:46] 2012-01-13 14:59:33,924 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[ReadStage:46,5,main]
>java.lang.IllegalArgumentException: Unknown ColumnFamily UrlText in keyspace ptprod
>    at org.apache.cassandra.config.Schema.getComparator(Schema.java:226)
>    at org.apache.cassandra.db.ColumnFamily.getComparatorFor(ColumnFamily.java:300)
>    at org.apache.cassandra.db.ReadCommand.getComparator(ReadCommand.java:92)
>    at
 org.apache.cassandra.db.SliceByNamesReadCommand.<init>(SliceByNamesReadCommand.java:44)
>    at org.apache.cassandra.db.SliceByNamesReadCommandSerializer.deserialize(SliceByNamesReadCommand.java:106)
>    at org.apache.cassandra.db.SliceByNamesReadCommandSerializer.deserialize(SliceByNamesReadCommand.java:74)
>    at org.apache.cassandra.db.ReadCommandSerializer.deserialize(ReadCommand.java:132)
>    at org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:51)
>    at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:59)
>    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
>    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
>    at
 java.lang.Thread.run(Thread.java:722)
>
>
>
>
>
>How can I get the compressed families on the new node ?
>
>
>Thanks,
>Alexis Lauthier
>

Re : Compressed families not created on new node

Posted by Alexis Lauthier <al...@yahoo.com>.
That was worth a try. Unfortunately it failed in exactly the same way as before.

I had no trouble decommissioning the bad node. Only the good node remained. When I bootstrapped an empty new node, the schema migration failed again with UnserializableColumnFamilyException.

So it looks like the I/O errors are not he direct cause of the schema migration failure. I guess the cluster got corrupted at some point.

Since I can't risk remaining in the current situation for too long, I'm going to consider the current cluster is dead. I will install a completely separate cluster on the new node, create an empty schema, then stop the old node and copy its sstable data to the new node. My site will be down for less than two hours if everything goes smoothly.

Thank you for your help, Aaron. If someone is interested in investigating this "cluster corruption" problem, I'll be happy to provide more info.


Alexis Lauthier



________________________________
 De : aaron morton <aa...@thelastpickle.com>
À : user@cassandra.apache.org 
Envoyé le : Mardi 17 Janvier 2012 1h49
Objet : Re: Compressed families not created on new node
 

eeek, HW errors. 

I would guess (thats all it is) that an IO error may have stopped the schema from migrating. 

Stop cassandra on that node and copy the files off as best you can. 

I would then try a node replacement

First remove the failed new node with nodetool decomission or removetoken. 

You are now down to one server. 

Copy the yaml file from the old machine (with IO errors) to a new one. To make things potentially less complicated bump the initial token slightly (e.g. add 1 to it) so the new node will not be exactly replacing the old one. 

Now start the new node. The other node will notice the schema is out of date and send it across. 

Once all the CF's are added and the schema's match stop the new node, copy the SSTable data from the old node to the new one and restart it. There are other ways to do this, this is the simplest though. 

With the new node in place the ring should now show the IO error node as down, the new node with a token very close to the IO error node and the one other node. 

You can now remove the IO error node with decomission or removetoken. 

Now run a repair from the new node.

At any stage the rollback plan is to simply turn the IO error node back on. 

Hope that helps. 

-----------------
Aaron Morton
Freelance Developer
@aaronmorton
http://www.thelastpickle.com 

On 16/01/2012, at 10:41 PM, Alexis Lauthier wrote:

The I/O errors are caused by disk failure. Syslog contains some of those things:
>
>
>
>Jan 16 09:53:24 --- kernel: [7065781.460804] sd 4:0:0:0: [sda]  Add. Sense: Unrecovered read error
>Jan 16 09:53:24 --- kernel: [7065781.460810] sd 4:0:0:0: [sda] CDB: Read(10): 28 00 11 cf 60 70 00 00 08 00
>Jan 16 09:53:24 --- kernel: [7065781.460820] end_request: I/O error, dev sda, sector 298803312
>
>
>
>
>Scrub failed:
>
>
><a lot of lines saying it is scrubbing happily up to this point>
>
> INFO [CompactionExecutor:5818] 2012-01-16 09:45:20,650 CompactionManager.java (line 477) Scrubbing SSTableReader(path='/home/cassprod/data/ptprod/UrlInfo-hb-1326-Data.db')
>ERROR
 [CompactionExecutor:5818] 2012-01-16 09:47:51,531 PrecompactedRow.java (line 119) Skipping row DecoratedKey(Token(bytes[01f9332e566a3a8d5a1cc17e530ae46e]), 01f9332e566a3a8d5a1cc17e530ae46e) in /home/cassprod/data/ptprod/UrlInfo-hb-1326-Data.db
>java.io.IOException: (/home/cassprod/data/ptprod/UrlInfo-hb-1326-Data.db) failed to read 13705 bytes from offset 3193541.
>    at org.apache.cassandra.io.compress.CompressedRandomAccessReader.decompressChunk(CompressedRandomAccessReader.java:87)
>    at org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBuffer(CompressedRandomAccessReader.java:75)
>    at org.apache.cassandra.io.util.RandomAccessReader.read(RandomAccessReader.java:302)
>    at java.io.RandomAccessFile.readFully(RandomAccessFile.java:397)
>    at java.io.RandomAccessFile.readFully(RandomAccessFile.java:377)
>    at
 org.apache.cassandra.utils.BytesReadTracker.readFully(BytesReadTracker.java:95)
>    at org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:392)
>    at org.apache.cassandra.utils.ByteBufferUtil.readWithLength(ByteBufferUtil.java:354)
>    at org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:120)
>    at org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:37)
>    at org.apache.cassandra.db.ColumnFamilySerializer.deserializeColumns(ColumnFamilySerializer.java:147)
>    at org.apache.cassandra.io.sstable.SSTableIdentityIterator.getColumnFamilyWithColumns(SSTableIdentityIterator.java:232)
>    at org.apache.cassandra.db.compaction.PrecompactedRow.merge(PrecompactedRow.java:115)
>    at
 org.apache.cassandra.db.compaction.PrecompactedRow.<init>(PrecompactedRow.java:102)
>    at org.apache.cassandra.db.compaction.CompactionController.getCompactedRow(CompactionController.java:133)
>    at org.apache.cassandra.db.compaction.CompactionController.getCompactedRow(CompactionController.java:139)
>    at org.apache.cassandra.db.compaction.CompactionManager.scrubOne(CompactionManager.java:565)
>    at org.apache.cassandra.db.compaction.CompactionManager.doScrub(CompactionManager.java:472)
>    at org.apache.cassandra.db.compaction.CompactionManager.access$300(CompactionManager.java:63)
>    at org.apache.cassandra.db.compaction.CompactionManager$3.call(CompactionManager.java:224)
>    at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>    at
 java.util.concurrent.FutureTask.run(FutureTask.java:138)
>    at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>    at java.lang.Thread.run(Thread.java:662)
> WARN [CompactionExecutor:5818] 2012-01-16 09:47:51,531 CompactionManager.java (line 581) Non-fatal error reading row (stacktrace follows)
>java.lang.NullPointerException
> WARN [CompactionExecutor:5818] 2012-01-16 09:47:51,532 CompactionManager.java (line 623) Row at 14740167 is unreadable; skipping to next
>ERROR [CompactionExecutor:5818] 2012-01-16 09:53:24,395 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[CompactionExecutor:5818,1,RMI Runtime]
>java.io.IOException: (/home/cassprod/data/ptprod/UrlInfo-hb-1326-Data.db) failed to read 13705 bytes from offset
 3193541.
>    at org.apache.cassandra.io.compress.CompressedRandomAccessReader.decompressChunk(CompressedRandomAccessReader.java:87)
>    at org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBuffer(CompressedRandomAccessReader.java:75)
>    at org.apache.cassandra.io.util.RandomAccessReader.seek(RandomAccessReader.java:259)
>    at org.apache.cassandra.db.compaction.CompactionManager.scrubOne(CompactionManager.java:625)
>    at org.apache.cassandra.db.compaction.CompactionManager.doScrub(CompactionManager.java:472)
>    at org.apache.cassandra.db.compaction.CompactionManager.access$300(CompactionManager.java:63)
>    at org.apache.cassandra.db.compaction.CompactionManager$3.call(CompactionManager.java:224)
>    at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>    at
 java.util.concurrent.FutureTask.run(FutureTask.java:138)
>    at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>    at java.lang.Thread.run(Thread.java:662)
>
>
>
>
>The same kind of "failed to read" IOExceptions have been routinely logged for 13 days now.
>
>
>The best idea I can come up with now is: decommission the failing node, then add the new node. And hope the schema will fully replicate. This will leave me with only one node for a time, and I'm not sure it will play nice with replication_factor=2.
>This feels a lot like jumping out of a plane with an untested parachute. So any other ideas ?
>
>
>Thanks,
>Alexis Lauthier
>
>
>
>
>
>
>
>
>________________________________
> De : aaron morton <aa...@thelastpickle.com>
>À : user@cassandra.apache.org 
>Envoyé le : Lundi 16 Janvier 2012 1h05
>Objet : Re: Compressed families not created on new node
> 
>
>Without knowing what the IOErrors are I would do the following:
>
>
>
>
>nodetool scrub to fix any on disk errors, this will also take a snapshot you can use for rollback.  
>
>nodetool repair to ensure data is consistent. 
>
>
>Hope that helps. 
>
>
>-----------------
>Aaron Morton
>Freelance Developer
>@aaronmorton
>http://www.thelastpickle.com 
>
>On 16/01/2012, at 7:53 AM, Alexis Lauthier wrote:
>
>I see only one line "JOINING: sleeping 30000 ms for pending range setup".
>>
>>
>>Before that, I have a lot of migration INFO messages, only for the uncompressed families.
>>
>>
>>I have currently killed the new node, so a describe cluster shows only one schema version on the first two nodes.
>>
>>
>>
>>[default@unknown] describe cluster;
>>Cluster Information:
>>   Snitch: org.apache.cassandra.locator.SimpleSnitch
>>   Partitioner: org.apache.cassandra.dht.ByteOrderedPartitioner
>>   Schema versions: 
>>    b42595d0-2247-11e1-0000-a0e9ff9ab7bf: [---.36,
 ---.35]
>>
>>    UNREACHABLE: [---.56]
>>
>>
>>
>>
>>Also, on one of the old nodes, I have a lot of I/O errors on the data files for some (but not all) of the compressed families. It began a few days ago. All "nodetool repair" calls have been blocking since then.
>>
>>
>>Any ideas on how I can get the data on the new node, before the old one dies?
>>
>>
>>
>>
>>
>>Thanks,
>>Alexis Lauthier
>>
>>
>>
>>
>>
>>
>>________________________________
>> De : aaron morton <aaron@>
>>À : user@cassandra.apache.org 
>>Envoyé le : Dimanche 15 Janvier 2012 19h17
>>Objet : Re: Compressed families not created on new node
>> 
>>
>>Sounds like the schema has not fully migrated to the new node. It is applied to the joining node one change at a time. A quick scan of the changes file does not find anything fixed after 1.0.3
>>
>>
>>You can check schema versions in the CLI using the describe cluster command. 
>>
>>
>>Check for errors in the logs with Migration in the text.  
>>
>>
>>Are you seeing this line a lot in the log ? 
>> INFO [main] 2012-01-13 14:55:00,493 StorageService.java (line 616) JOINING: sleeping 30000 ms for pending range setup
>>>
>>
>>cheers
>>
>>
>>-----------------
>>Aaron Morton
>>Freelance Developer
>>@aaronmorton
>>http://www.thelastpickle.com 
>>
>>On 14/01/2012, at 4:20 AM, Alexis Lauthier wrote:
>>
>>I'm using Cassandra 1.0.3 on a 2 nodes cluster. My schema (with replication_factor=2) contains both compressed (with sstable_compression=DeflateCompressor) and uncompressed column families.
>>>
>>>
>>>
>>>When bootstrapping a third node, the uncompressed families are created on the new node as expected, but the compressed families are not. Only the uncompressed families appear in a "show schema", and the new node data size is small, which is consistent with the big compressed data not being there.
>>>
>>>
>>>I'm seeing frequent exceptions in the log :
>>>
>>>
>>> INFO [main] 2012-01-13 14:55:00,493 StorageService.java (line 616) JOINING: sleeping 30000 ms for pending range setup
>>>ERROR [MutationStage:1] 2012-01-13 14:55:01,511 RowMutationVerbHandler.java (line 65) Error in row
 mutation
>>>org.apache.cassandra.db.UnserializableColumnFamilyException: Couldn't find cfId=1008
>>>    at org.apache.cassandra.db.ColumnFamilySerializer.deserialize(ColumnFamilySerializer.java:129)
>>>    at org.apache.cassandra.db.RowMutation$RowMutationSerializer.deserialize(RowMutation.java:401)
>>>    at org.apache.cassandra.db.RowMutation$RowMutationSerializer.deserialize(RowMutation.java:409)
>>>    at org.apache.cassandra.db.RowMutation.fromBytes(RowMutation.java:357)
>>>    at org.apache.cassandra.db.RowMutationVerbHandler.doVerb(RowMutationVerbHandler.java:46)
>>>    at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:59)
>>>    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
>>>    at
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
>>>    at java.lang.Thread.run(Thread.java:722)
>>>
>>>
>>>
>>>
>>>
>>>After a few minutes, The column family names are shown instead of their ids ("UrlText" is one of the compressed families) :
>>>
>>>
>>>
>>>ERROR [ReadStage:46] 2012-01-13 14:59:33,924 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[ReadStage:46,5,main]
>>>java.lang.IllegalArgumentException: Unknown ColumnFamily UrlText in keyspace ptprod
>>>    at org.apache.cassandra.config.Schema.getComparator(Schema.java:226)
>>>    at org.apache.cassandra.db.ColumnFamily.getComparatorFor(ColumnFamily.java:300)
>>>    at org.apache.cassandra.db.ReadCommand.getComparator(ReadCommand.java:92)
>>>    at
 org.apache.cassandra.db.SliceByNamesReadCommand.<init>(SliceByNamesReadCommand.java:44)
>>>    at org.apache.cassandra.db.SliceByNamesReadCommandSerializer.deserialize(SliceByNamesReadCommand.java:106)
>>>    at org.apache.cassandra.db.SliceByNamesReadCommandSerializer.deserialize(SliceByNamesReadCommand.java:74)
>>>    at org.apache.cassandra.db.ReadCommandSerializer.deserialize(ReadCommand.java:132)
>>>    at org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:51)
>>>    at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:59)
>>>    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
>>>    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
>>>    at
 java.lang.Thread.run(Thread.java:722)
>>>
>>>
>>>
>>>
>>>
>>>How can I get the compressed families on the new node ?
>>>
>>>
>>>Thanks,
>>>Alexis Lauthier
>>>
>>
>>
>>
>
>
>

Re: Compressed families not created on new node

Posted by aaron morton <aa...@thelastpickle.com>.
eeek, HW errors. 

I would guess (thats all it is) that an IO error may have stopped the schema from migrating. 

Stop cassandra on that node and copy the files off as best you can. 

I would then try a node replacement

First remove the failed new node with nodetool decomission or removetoken. 

You are now down to one server. 

Copy the yaml file from the old machine (with IO errors) to a new one. To make things potentially less complicated bump the initial token slightly (e.g. add 1 to it) so the new node will not be exactly replacing the old one. 

Now start the new node. The other node will notice the schema is out of date and send it across. 

Once all the CF's are added and the schema's match stop the new node, copy the SSTable data from the old node to the new one and restart it. There are other ways to do this, this is the simplest though. 

With the new node in place the ring should now show the IO error node as down, the new node with a token very close to the IO error node and the one other node. 

You can now remove the IO error node with decomission or removetoken. 

Now run a repair from the new node.

At any stage the rollback plan is to simply turn the IO error node back on. 

Hope that helps. 

-----------------
Aaron Morton
Freelance Developer
@aaronmorton
http://www.thelastpickle.com

On 16/01/2012, at 10:41 PM, Alexis Lauthier wrote:

> The I/O errors are caused by disk failure. Syslog contains some of those things:
> 
> Jan 16 09:53:24 --- kernel: [7065781.460804] sd 4:0:0:0: [sda]  Add. Sense: Unrecovered read error
> Jan 16 09:53:24 --- kernel: [7065781.460810] sd 4:0:0:0: [sda] CDB: Read(10): 28 00 11 cf 60 70 00 00 08 00
> Jan 16 09:53:24 --- kernel: [7065781.460820] end_request: I/O error, dev sda, sector 298803312
> 
> 
> Scrub failed:
> 
> <a lot of lines saying it is scrubbing happily up to this point>
>  INFO [CompactionExecutor:5818] 2012-01-16 09:45:20,650 CompactionManager.java (line 477) Scrubbing SSTableReader(path='/home/cassprod/data/ptprod/UrlInfo-hb-1326-Data.db')
> ERROR [CompactionExecutor:5818] 2012-01-16 09:47:51,531 PrecompactedRow.java (line 119) Skipping row DecoratedKey(Token(bytes[01f9332e566a3a8d5a1cc17e530ae46e]), 01f9332e566a3a8d5a1cc17e530ae46e) in /home/cassprod/data/ptprod/UrlInfo-hb-1326-Data.db
> java.io.IOException: (/home/cassprod/data/ptprod/UrlInfo-hb-1326-Data.db) failed to read 13705 bytes from offset 3193541.
>     at org.apache.cassandra.io.compress.CompressedRandomAccessReader.decompressChunk(CompressedRandomAccessReader.java:87)
>     at org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBuffer(CompressedRandomAccessReader.java:75)
>     at org.apache.cassandra.io.util.RandomAccessReader.read(RandomAccessReader.java:302)
>     at java.io.RandomAccessFile.readFully(RandomAccessFile.java:397)
>     at java.io.RandomAccessFile.readFully(RandomAccessFile.java:377)
>     at org.apache.cassandra.utils.BytesReadTracker.readFully(BytesReadTracker.java:95)
>     at org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:392)
>     at org.apache.cassandra.utils.ByteBufferUtil.readWithLength(ByteBufferUtil.java:354)
>     at org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:120)
>     at org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:37)
>     at org.apache.cassandra.db.ColumnFamilySerializer.deserializeColumns(ColumnFamilySerializer.java:147)
>     at org.apache.cassandra.io.sstable.SSTableIdentityIterator.getColumnFamilyWithColumns(SSTableIdentityIterator.java:232)
>     at org.apache.cassandra.db.compaction.PrecompactedRow.merge(PrecompactedRow.java:115)
>     at org.apache.cassandra.db.compaction.PrecompactedRow.<init>(PrecompactedRow.java:102)
>     at org.apache.cassandra.db.compaction.CompactionController.getCompactedRow(CompactionController.java:133)
>     at org.apache.cassandra.db.compaction.CompactionController.getCompactedRow(CompactionController.java:139)
>     at org.apache.cassandra.db.compaction.CompactionManager.scrubOne(CompactionManager.java:565)
>     at org.apache.cassandra.db.compaction.CompactionManager.doScrub(CompactionManager.java:472)
>     at org.apache.cassandra.db.compaction.CompactionManager.access$300(CompactionManager.java:63)
>     at org.apache.cassandra.db.compaction.CompactionManager$3.call(CompactionManager.java:224)
>     at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>     at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>     at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>     at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>     at java.lang.Thread.run(Thread.java:662)
>  WARN [CompactionExecutor:5818] 2012-01-16 09:47:51,531 CompactionManager.java (line 581) Non-fatal error reading row (stacktrace follows)
> java.lang.NullPointerException
>  WARN [CompactionExecutor:5818] 2012-01-16 09:47:51,532 CompactionManager.java (line 623) Row at 14740167 is unreadable; skipping to next
> ERROR [CompactionExecutor:5818] 2012-01-16 09:53:24,395 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[CompactionExecutor:5818,1,RMI Runtime]
> java.io.IOException: (/home/cassprod/data/ptprod/UrlInfo-hb-1326-Data.db) failed to read 13705 bytes from offset 3193541.
>     at org.apache.cassandra.io.compress.CompressedRandomAccessReader.decompressChunk(CompressedRandomAccessReader.java:87)
>     at org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBuffer(CompressedRandomAccessReader.java:75)
>     at org.apache.cassandra.io.util.RandomAccessReader.seek(RandomAccessReader.java:259)
>     at org.apache.cassandra.db.compaction.CompactionManager.scrubOne(CompactionManager.java:625)
>     at org.apache.cassandra.db.compaction.CompactionManager.doScrub(CompactionManager.java:472)
>     at org.apache.cassandra.db.compaction.CompactionManager.access$300(CompactionManager.java:63)
>     at org.apache.cassandra.db.compaction.CompactionManager$3.call(CompactionManager.java:224)
>     at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
>     at java.util.concurrent.FutureTask.run(FutureTask.java:138)
>     at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>     at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>     at java.lang.Thread.run(Thread.java:662)
> 
> 
> The same kind of "failed to read" IOExceptions have been routinely logged for 13 days now.
> 
> The best idea I can come up with now is: decommission the failing node, then add the new node. And hope the schema will fully replicate. This will leave me with only one node for a time, and I'm not sure it will play nice with replication_factor=2.
> This feels a lot like jumping out of a plane with an untested parachute. So any other ideas ?
> 
> Thanks,
> Alexis Lauthier
> 
> 
> 
> De : aaron morton <aa...@thelastpickle.com>
> À : user@cassandra.apache.org 
> Envoyé le : Lundi 16 Janvier 2012 1h05
> Objet : Re: Compressed families not created on new node
> 
> Without knowing what the IOErrors are I would do the following:
> 
> 
> nodetool scrub to fix any on disk errors, this will also take a snapshot you can use for rollback.  
> nodetool repair to ensure data is consistent. 
> 
> Hope that helps. 
> 
> -----------------
> Aaron Morton
> Freelance Developer
> @aaronmorton
> http://www.thelastpickle.com
> 
> On 16/01/2012, at 7:53 AM, Alexis Lauthier wrote:
> 
>> I see only one line "JOINING: sleeping 30000 ms for pending range setup".
>> 
>> Before that, I have a lot of migration INFO messages, only for the uncompressed families.
>> 
>> I have currently killed the new node, so a describe cluster shows only one schema version on the first two nodes.
>> 
>> [default@unknown] describe cluster;
>> Cluster Information:
>>    Snitch: org.apache.cassandra.locator.SimpleSnitch
>>    Partitioner: org.apache.cassandra.dht.ByteOrderedPartitioner
>>    Schema versions: 
>>     b42595d0-2247-11e1-0000-a0e9ff9ab7bf: [---.36, ---.35]
>> 
>>     UNREACHABLE: [---.56]
>> 
>> 
>> Also, on one of the old nodes, I have a lot of I/O errors on the data files for some (but not all) of the compressed families. It began a few days ago. All "nodetool repair" calls have been blocking since then.
>> 
>> Any ideas on how I can get the data on the new node, before the old one dies?
>> 
>> 
>> Thanks,
>> Alexis Lauthier
>> 
>> 
>> De : aaron morton <aaron@>
>> À : user@cassandra.apache.org 
>> Envoyé le : Dimanche 15 Janvier 2012 19h17
>> Objet : Re: Compressed families not created on new node
>> 
>> Sounds like the schema has not fully migrated to the new node. It is applied to the joining node one change at a time. A quick scan of the changes file does not find anything fixed after 1.0.3
>> 
>> You can check schema versions in the CLI using the describe cluster command. 
>> 
>> Check for errors in the logs with Migration in the text.  
>> 
>> Are you seeing this line a lot in the log ? 
>>>  INFO [main] 2012-01-13 14:55:00,493 StorageService.java (line 616) JOINING: sleeping 30000 ms for pending range setup
>> 
>> cheers
>> 
>> -----------------
>> Aaron Morton
>> Freelance Developer
>> @aaronmorton
>> http://www.thelastpickle.com
>> 
>> On 14/01/2012, at 4:20 AM, Alexis Lauthier wrote:
>> 
>>> I'm using Cassandra 1.0.3 on a 2 nodes cluster. My schema (with replication_factor=2) contains both compressed (with sstable_compression=DeflateCompressor) and uncompressed column families.
>>> 
>>> When bootstrapping a third node, the uncompressed families are created on the new node as expected, but the compressed families are not. Only the uncompressed families appear in a "show schema", and the new node data size is small, which is consistent with the big compressed data not being there.
>>> 
>>> I'm seeing frequent exceptions in the log :
>>> 
>>>  INFO [main] 2012-01-13 14:55:00,493 StorageService.java (line 616) JOINING: sleeping 30000 ms for pending range setup
>>> ERROR [MutationStage:1] 2012-01-13 14:55:01,511 RowMutationVerbHandler.java (line 65) Error in row mutation
>>> org.apache.cassandra.db.UnserializableColumnFamilyException: Couldn't find cfId=1008
>>>     at org.apache.cassandra.db.ColumnFamilySerializer.deserialize(ColumnFamilySerializer.java:129)
>>>     at org.apache.cassandra.db.RowMutation$RowMutationSerializer.deserialize(RowMutation.java:401)
>>>     at org.apache.cassandra.db.RowMutation$RowMutationSerializer.deserialize(RowMutation.java:409)
>>>     at org.apache.cassandra.db.RowMutation.fromBytes(RowMutation.java:357)
>>>     at org.apache.cassandra.db.RowMutationVerbHandler.doVerb(RowMutationVerbHandler.java:46)
>>>     at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:59)
>>>     at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
>>>     at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
>>>     at java.lang.Thread.run(Thread.java:722)
>>> 
>>> 
>>> After a few minutes, The column family names are shown instead of their ids ("UrlText" is one of the compressed families) :
>>> 
>>> ERROR [ReadStage:46] 2012-01-13 14:59:33,924 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[ReadStage:46,5,main]
>>> java.lang.IllegalArgumentException: Unknown ColumnFamily UrlText in keyspace ptprod
>>>     at org.apache.cassandra.config.Schema.getComparator(Schema.java:226)
>>>     at org.apache.cassandra.db.ColumnFamily.getComparatorFor(ColumnFamily.java:300)
>>>     at org.apache.cassandra.db.ReadCommand.getComparator(ReadCommand.java:92)
>>>     at org.apache.cassandra.db.SliceByNamesReadCommand.<init>(SliceByNamesReadCommand.java:44)
>>>     at org.apache.cassandra.db.SliceByNamesReadCommandSerializer.deserialize(SliceByNamesReadCommand.java:106)
>>>     at org.apache.cassandra.db.SliceByNamesReadCommandSerializer.deserialize(SliceByNamesReadCommand.java:74)
>>>     at org.apache.cassandra.db.ReadCommandSerializer.deserialize(ReadCommand.java:132)
>>>     at org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:51)
>>>     at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:59)
>>>     at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
>>>     at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
>>>     at java.lang.Thread.run(Thread.java:722)
>>> 
>>> 
>>> How can I get the compressed families on the new node ?
>>> 
>>> Thanks,
>>> Alexis Lauthier
>> 
>> 
>> 
> 
> 
> 


Re : Compressed families not created on new node

Posted by Alexis Lauthier <al...@yahoo.com>.
The I/O errors are caused by disk failure. Syslog contains some of those things:


Jan 16 09:53:24 --- kernel: [7065781.460804] sd 4:0:0:0: [sda]  Add. Sense: Unrecovered read error
Jan 16 09:53:24 --- kernel: [7065781.460810] sd 4:0:0:0: [sda] CDB: Read(10): 28 00 11 cf 60 70 00 00 08 00
Jan 16 09:53:24 --- kernel: [7065781.460820] end_request: I/O error, dev sda, sector 298803312



Scrub failed:

<a lot of lines saying it is scrubbing happily up to this point>

 INFO [CompactionExecutor:5818] 2012-01-16 09:45:20,650 CompactionManager.java (line 477) Scrubbing SSTableReader(path='/home/cassprod/data/ptprod/UrlInfo-hb-1326-Data.db')
ERROR [CompactionExecutor:5818] 2012-01-16 09:47:51,531 PrecompactedRow.java (line 119) Skipping row DecoratedKey(Token(bytes[01f9332e566a3a8d5a1cc17e530ae46e]), 01f9332e566a3a8d5a1cc17e530ae46e) in /home/cassprod/data/ptprod/UrlInfo-hb-1326-Data.db
java.io.IOException: (/home/cassprod/data/ptprod/UrlInfo-hb-1326-Data.db) failed to read 13705 bytes from offset 3193541.
    at org.apache.cassandra.io.compress.CompressedRandomAccessReader.decompressChunk(CompressedRandomAccessReader.java:87)
    at org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBuffer(CompressedRandomAccessReader.java:75)
    at org.apache.cassandra.io.util.RandomAccessReader.read(RandomAccessReader.java:302)
    at java.io.RandomAccessFile.readFully(RandomAccessFile.java:397)
    at java.io.RandomAccessFile.readFully(RandomAccessFile.java:377)
    at org.apache.cassandra.utils.BytesReadTracker.readFully(BytesReadTracker.java:95)
    at org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:392)
    at org.apache.cassandra.utils.ByteBufferUtil.readWithLength(ByteBufferUtil.java:354)
    at org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:120)
    at org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:37)
    at org.apache.cassandra.db.ColumnFamilySerializer.deserializeColumns(ColumnFamilySerializer.java:147)
    at org.apache.cassandra.io.sstable.SSTableIdentityIterator.getColumnFamilyWithColumns(SSTableIdentityIterator.java:232)
    at org.apache.cassandra.db.compaction.PrecompactedRow.merge(PrecompactedRow.java:115)
    at org.apache.cassandra.db.compaction.PrecompactedRow.<init>(PrecompactedRow.java:102)
    at org.apache.cassandra.db.compaction.CompactionController.getCompactedRow(CompactionController.java:133)
    at org.apache.cassandra.db.compaction.CompactionController.getCompactedRow(CompactionController.java:139)
    at org.apache.cassandra.db.compaction.CompactionManager.scrubOne(CompactionManager.java:565)
    at org.apache.cassandra.db.compaction.CompactionManager.doScrub(CompactionManager.java:472)
    at org.apache.cassandra.db.compaction.CompactionManager.access$300(CompactionManager.java:63)
    at org.apache.cassandra.db.compaction.CompactionManager$3.call(CompactionManager.java:224)
    at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
    at java.util.concurrent.FutureTask.run(FutureTask.java:138)
    at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
    at java.lang.Thread.run(Thread.java:662)
 WARN [CompactionExecutor:5818] 2012-01-16 09:47:51,531 CompactionManager.java (line 581) Non-fatal error reading row (stacktrace follows)
java.lang.NullPointerException
 WARN [CompactionExecutor:5818] 2012-01-16 09:47:51,532 CompactionManager.java (line 623) Row at 14740167 is unreadable; skipping to next
ERROR [CompactionExecutor:5818] 2012-01-16 09:53:24,395 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[CompactionExecutor:5818,1,RMI Runtime]
java.io.IOException: (/home/cassprod/data/ptprod/UrlInfo-hb-1326-Data.db) failed to read 13705 bytes from offset 3193541.
    at org.apache.cassandra.io.compress.CompressedRandomAccessReader.decompressChunk(CompressedRandomAccessReader.java:87)
    at org.apache.cassandra.io.compress.CompressedRandomAccessReader.reBuffer(CompressedRandomAccessReader.java:75)
    at org.apache.cassandra.io.util.RandomAccessReader.seek(RandomAccessReader.java:259)
    at org.apache.cassandra.db.compaction.CompactionManager.scrubOne(CompactionManager.java:625)
    at org.apache.cassandra.db.compaction.CompactionManager.doScrub(CompactionManager.java:472)
    at org.apache.cassandra.db.compaction.CompactionManager.access$300(CompactionManager.java:63)
    at org.apache.cassandra.db.compaction.CompactionManager$3.call(CompactionManager.java:224)
    at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
    at java.util.concurrent.FutureTask.run(FutureTask.java:138)
    at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
    at java.lang.Thread.run(Thread.java:662)



The same kind of "failed to read" IOExceptions have been routinely logged for 13 days now.

The best idea I can come up with now is: decommission the failing node, then add the new node. And hope the schema will fully replicate. This will leave me with only one node for a time, and I'm not sure it will play nice with replication_factor=2.
This feels a lot like jumping out of a plane with an untested parachute. So any other ideas ?

Thanks,
Alexis Lauthier





________________________________
 De : aaron morton <aa...@thelastpickle.com>
À : user@cassandra.apache.org 
Envoyé le : Lundi 16 Janvier 2012 1h05
Objet : Re: Compressed families not created on new node
 

Without knowing what the IOErrors are I would do the following:


nodetool scrub to fix any on disk errors, this will also take a snapshot you can use for rollback.  

nodetool repair to ensure data is consistent. 

Hope that helps. 

-----------------
Aaron Morton
Freelance Developer
@aaronmorton
http://www.thelastpickle.com 

On 16/01/2012, at 7:53 AM, Alexis Lauthier wrote:

I see only one line "JOINING: sleeping 30000 ms for pending range setup".
>
>
>Before that, I have a lot of migration INFO messages, only for the uncompressed families.
>
>
>I have currently killed the new node, so a describe cluster shows only one schema version on the first two nodes.
>
>
>
>[default@unknown] describe cluster;
>Cluster Information:
>   Snitch: org.apache.cassandra.locator.SimpleSnitch
>   Partitioner: org.apache.cassandra.dht.ByteOrderedPartitioner
>   Schema versions: 
>    b42595d0-2247-11e1-0000-a0e9ff9ab7bf: [---.36, ---.35]
>
>    UNREACHABLE: [---.56]
>
>
>
>
>Also, on one of the old nodes, I have a lot of I/O errors on the data files for some (but not all) of the compressed families. It began a few days ago. All "nodetool repair" calls have been blocking since then.
>
>
>Any ideas on how I can get the data on the new node, before the old one dies?
>
>
>
>
>
>Thanks,
>Alexis Lauthier
>
>
>
>
>
>
>________________________________
> De : aaron morton <aaron@>
>À : user@cassandra.apache.org 
>Envoyé le : Dimanche 15 Janvier 2012 19h17
>Objet : Re: Compressed families not created on new node
> 
>
>Sounds like the schema has not fully migrated to the new node. It is applied to the joining node one change at a time. A quick scan of the changes file does not find anything fixed after 1.0.3
>
>
>You can check schema versions in the CLI using the describe cluster command. 
>
>
>Check for errors in the logs with Migration in the text.  
>
>
>Are you seeing this line a lot in the log ? 
> INFO [main] 2012-01-13 14:55:00,493 StorageService.java (line 616) JOINING: sleeping 30000 ms for pending range setup
>>
>
>cheers
>
>
>-----------------
>Aaron Morton
>Freelance Developer
>@aaronmorton
>http://www.thelastpickle.com 
>
>On 14/01/2012, at 4:20 AM, Alexis Lauthier wrote:
>
>I'm using Cassandra 1.0.3 on a 2 nodes cluster. My schema (with replication_factor=2) contains both compressed (with sstable_compression=DeflateCompressor) and uncompressed column families.
>>
>>
>>
>>When bootstrapping a third node, the uncompressed families are created on the new node as expected, but the compressed families are not. Only the uncompressed families appear in a "show schema", and the new node data size is small, which is consistent with the big compressed data not being there.
>>
>>
>>I'm seeing frequent exceptions in the log :
>>
>>
>> INFO [main] 2012-01-13 14:55:00,493 StorageService.java (line 616) JOINING: sleeping 30000 ms for pending range setup
>>ERROR [MutationStage:1] 2012-01-13 14:55:01,511 RowMutationVerbHandler.java (line 65) Error in row
 mutation
>>org.apache.cassandra.db.UnserializableColumnFamilyException: Couldn't find cfId=1008
>>    at org.apache.cassandra.db.ColumnFamilySerializer.deserialize(ColumnFamilySerializer.java:129)
>>    at org.apache.cassandra.db.RowMutation$RowMutationSerializer.deserialize(RowMutation.java:401)
>>    at org.apache.cassandra.db.RowMutation$RowMutationSerializer.deserialize(RowMutation.java:409)
>>    at org.apache.cassandra.db.RowMutation.fromBytes(RowMutation.java:357)
>>    at org.apache.cassandra.db.RowMutationVerbHandler.doVerb(RowMutationVerbHandler.java:46)
>>    at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:59)
>>    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
>>    at
 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
>>    at java.lang.Thread.run(Thread.java:722)
>>
>>
>>
>>
>>
>>After a few minutes, The column family names are shown instead of their ids ("UrlText" is one of the compressed families) :
>>
>>
>>
>>ERROR [ReadStage:46] 2012-01-13 14:59:33,924 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[ReadStage:46,5,main]
>>java.lang.IllegalArgumentException: Unknown ColumnFamily UrlText in keyspace ptprod
>>    at org.apache.cassandra.config.Schema.getComparator(Schema.java:226)
>>    at org.apache.cassandra.db.ColumnFamily.getComparatorFor(ColumnFamily.java:300)
>>    at org.apache.cassandra.db.ReadCommand.getComparator(ReadCommand.java:92)
>>    at
 org.apache.cassandra.db.SliceByNamesReadCommand.<init>(SliceByNamesReadCommand.java:44)
>>    at org.apache.cassandra.db.SliceByNamesReadCommandSerializer.deserialize(SliceByNamesReadCommand.java:106)
>>    at org.apache.cassandra.db.SliceByNamesReadCommandSerializer.deserialize(SliceByNamesReadCommand.java:74)
>>    at org.apache.cassandra.db.ReadCommandSerializer.deserialize(ReadCommand.java:132)
>>    at org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:51)
>>    at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:59)
>>    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
>>    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
>>    at
 java.lang.Thread.run(Thread.java:722)
>>
>>
>>
>>
>>
>>How can I get the compressed families on the new node ?
>>
>>
>>Thanks,
>>Alexis Lauthier
>>
>
>
>

Re: Compressed families not created on new node

Posted by aaron morton <aa...@thelastpickle.com>.
Without knowing what the IOErrors are I would do the following:


nodetool scrub to fix any on disk errors, this will also take a snapshot you can use for rollback.  
nodetool repair to ensure data is consistent. 

Hope that helps. 

-----------------
Aaron Morton
Freelance Developer
@aaronmorton
http://www.thelastpickle.com

On 16/01/2012, at 7:53 AM, Alexis Lauthier wrote:

> I see only one line "JOINING: sleeping 30000 ms for pending range setup".
> 
> Before that, I have a lot of migration INFO messages, only for the uncompressed families.
> 
> I have currently killed the new node, so a describe cluster shows only one schema version on the first two nodes.
> 
> [default@unknown] describe cluster;
> Cluster Information:
>    Snitch: org.apache.cassandra.locator.SimpleSnitch
>    Partitioner: org.apache.cassandra.dht.ByteOrderedPartitioner
>    Schema versions: 
>     b42595d0-2247-11e1-0000-a0e9ff9ab7bf: [---.36, ---.35]
> 
>     UNREACHABLE: [---.56]
> 
> 
> Also, on one of the old nodes, I have a lot of I/O errors on the data files for some (but not all) of the compressed families. It began a few days ago. All "nodetool repair" calls have been blocking since then.
> 
> Any ideas on how I can get the data on the new node, before the old one dies?
> 
> 
> Thanks,
> Alexis Lauthier
> 
> 
> De : aaron morton <aaron@>
> À : user@cassandra.apache.org 
> Envoyé le : Dimanche 15 Janvier 2012 19h17
> Objet : Re: Compressed families not created on new node
> 
> Sounds like the schema has not fully migrated to the new node. It is applied to the joining node one change at a time. A quick scan of the changes file does not find anything fixed after 1.0.3
> 
> You can check schema versions in the CLI using the describe cluster command. 
> 
> Check for errors in the logs with Migration in the text.  
> 
> Are you seeing this line a lot in the log ? 
>>  INFO [main] 2012-01-13 14:55:00,493 StorageService.java (line 616) JOINING: sleeping 30000 ms for pending range setup
> 
> cheers
> 
> -----------------
> Aaron Morton
> Freelance Developer
> @aaronmorton
> http://www.thelastpickle.com
> 
> On 14/01/2012, at 4:20 AM, Alexis Lauthier wrote:
> 
>> I'm using Cassandra 1.0.3 on a 2 nodes cluster. My schema (with replication_factor=2) contains both compressed (with sstable_compression=DeflateCompressor) and uncompressed column families.
>> 
>> When bootstrapping a third node, the uncompressed families are created on the new node as expected, but the compressed families are not. Only the uncompressed families appear in a "show schema", and the new node data size is small, which is consistent with the big compressed data not being there.
>> 
>> I'm seeing frequent exceptions in the log :
>> 
>>  INFO [main] 2012-01-13 14:55:00,493 StorageService.java (line 616) JOINING: sleeping 30000 ms for pending range setup
>> ERROR [MutationStage:1] 2012-01-13 14:55:01,511 RowMutationVerbHandler.java (line 65) Error in row mutation
>> org.apache.cassandra.db.UnserializableColumnFamilyException: Couldn't find cfId=1008
>>     at org.apache.cassandra.db.ColumnFamilySerializer.deserialize(ColumnFamilySerializer.java:129)
>>     at org.apache.cassandra.db.RowMutation$RowMutationSerializer.deserialize(RowMutation.java:401)
>>     at org.apache.cassandra.db.RowMutation$RowMutationSerializer.deserialize(RowMutation.java:409)
>>     at org.apache.cassandra.db.RowMutation.fromBytes(RowMutation.java:357)
>>     at org.apache.cassandra.db.RowMutationVerbHandler.doVerb(RowMutationVerbHandler.java:46)
>>     at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:59)
>>     at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
>>     at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
>>     at java.lang.Thread.run(Thread.java:722)
>> 
>> 
>> After a few minutes, The column family names are shown instead of their ids ("UrlText" is one of the compressed families) :
>> 
>> ERROR [ReadStage:46] 2012-01-13 14:59:33,924 AbstractCassandraDaemon.java (line 133) Fatal exception in thread Thread[ReadStage:46,5,main]
>> java.lang.IllegalArgumentException: Unknown ColumnFamily UrlText in keyspace ptprod
>>     at org.apache.cassandra.config.Schema.getComparator(Schema.java:226)
>>     at org.apache.cassandra.db.ColumnFamily.getComparatorFor(ColumnFamily.java:300)
>>     at org.apache.cassandra.db.ReadCommand.getComparator(ReadCommand.java:92)
>>     at org.apache.cassandra.db.SliceByNamesReadCommand.<init>(SliceByNamesReadCommand.java:44)
>>     at org.apache.cassandra.db.SliceByNamesReadCommandSerializer.deserialize(SliceByNamesReadCommand.java:106)
>>     at org.apache.cassandra.db.SliceByNamesReadCommandSerializer.deserialize(SliceByNamesReadCommand.java:74)
>>     at org.apache.cassandra.db.ReadCommandSerializer.deserialize(ReadCommand.java:132)
>>     at org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:51)
>>     at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:59)
>>     at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
>>     at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
>>     at java.lang.Thread.run(Thread.java:722)
>> 
>> 
>> How can I get the compressed families on the new node ?
>> 
>> Thanks,
>> Alexis Lauthier
> 
> 
>