Details
-
Bug
-
Status: Resolved
-
Normal
-
Resolution: Fixed
-
None
-
Normal
Description
I don't see this happen on 1.2.6.
To reproduce (on a fresh single node cluster):
[Nicks-MacBook-Pro:11:33:06 (cassandra-1.2.7)*] cassandra$ bin/cqlsh Connected to Test Cluster at localhost:9160. [cqlsh 3.1.4 | Cassandra 1.2.7-SNAPSHOT | CQL spec 3.0.0 | Thrift protocol 19.36.0] cqlsh> CREATE KEYSPACE test_backup_restore WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}; cqlsh> use test_backup_restore; cqlsh:test_backup_restore> CREATE TABLE cf0 ( ... a text PRIMARY KEY, ... b text, ... c text ... ); cqlsh:test_backup_restore> INSERT INTO cf0 (a, b, c) VALUES ( 'a', 'b', 'c'); cqlsh:test_backup_restore> select * from cf0; a | b | c ---+---+--- a | b | c cqlsh:test_backup_restore> ^D [Nicks-MacBook-Pro:11:34:22 (cassandra-1.2.7)*] cassandra$ bin/nodetool snapshot Requested creating snapshot for: all keyspaces Snapshot directory: 1375115668449 [Nicks-MacBook-Pro:11:34:40 (cassandra-1.2.7)*] cassandra$ mkdir -p test_backup_restore/snapshots [Nicks-MacBook-Pro:11:34:48 (cassandra-1.2.7)*] cassandra$ cp /var/lib/cassandra/data/test_backup_restore/cf0/snapshots/1375115668449/* test_backup_restore/snapshots/ [Nicks-MacBook-Pro:11:35:14 (cassandra-1.2.7)*] cassandra$ bin/sstableloader --debug -v -d 127.0.0.1 test_backup_restore/snapshots Streaming revelant part of test_backup_restore/snapshots/test_backup_restore-cf0-ic-1-Data.db to [/127.0.0.1] org.apache.cassandra.io.util.CompressedSegmentedFile cannot be cast to org.apache.cassandra.io.util.CompressedPoolingSegmentedFile java.lang.ClassCastException: org.apache.cassandra.io.util.CompressedSegmentedFile cannot be cast to org.apache.cassandra.io.util.CompressedPoolingSegmentedFile at org.apache.cassandra.io.sstable.SSTableReader.getCompressionMetadata(SSTableReader.java:574) at org.apache.cassandra.streaming.StreamOut.createPendingFiles(StreamOut.java:179) at org.apache.cassandra.streaming.StreamOut.transferSSTables(StreamOut.java:154) at org.apache.cassandra.io.sstable.SSTableLoader.stream(SSTableLoader.java:145) at org.apache.cassandra.tools.BulkLoader.main(BulkLoader.java:67)