[ 
https://issues.apache.org/jira/browse/CASSANDRA-2076?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12990491#comment-12990491
 ] 

Thibaut commented on CASSANDRA-2076:
------------------------------------

I was running yesterday's version with the Consitency fix and getting a similar 
error messages while reading the commitlog at start. I had to delete the 
commitlog (data loss) to restart cassandra.

Is this also related to CASSANDRA-2102 or shall I open a new bug report?


 INFO 23:50:00,922 Replaying 
/hd1/cassandra_md5/commitlog/CommitLog-1296731219671.log, 
/hd1/cassandra_md5/commitlog/CommitLog-1296759540149.log, 
/hd1/cassandra_md5/commitlog/CommitLog-1296760444366.log, 
/hd1/cassandra_md5/commitlog/CommitLog-1296761120546.log, 
/hd1/cassandra_md5/commitlog/CommitLog-1296762054192.log, 
/hd1/cassandra_md5/commitlog/CommitLog-1296773137101.log, 
/hd1/cassandra_md5/commitlog/CommitLog-1296773242671.log
 INFO 23:50:02,192 Finished reading 
/hd1/cassandra_md5/commitlog/CommitLog-1296731219671.log
ERROR 23:50:02,235 Fatal exception in thread Thread[MutationStage:7,5,main]
java.lang.RuntimeException: java.lang.RuntimeException: The provided key was 
not UTF8 encoded.
        at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
        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)
Caused by: java.lang.RuntimeException: The provided key was not UTF8 encoded.
        at 
org.apache.cassandra.dht.OrderPreservingPartitioner.getToken(OrderPreservingPartitioner.java:159)
        at 
org.apache.cassandra.dht.OrderPreservingPartitioner.decorateKey(OrderPreservingPartitioner.java:44)
        at org.apache.cassandra.db.Table.apply(Table.java:406)
        at 
org.apache.cassandra.db.commitlog.CommitLog$2.runMayThrow(CommitLog.java:294)
        at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
        ... 6 more
Caused by: java.nio.charset.MalformedInputException: Input length = 1
        at java.nio.charset.CoderResult.throwException(CoderResult.java:260)
        at java.nio.charset.CharsetDecoder.decode(CharsetDecoder.java:781)
        at 
org.apache.cassandra.utils.FBUtilities.decodeToUTF8(FBUtilities.java:403)
        at 
org.apache.cassandra.dht.OrderPreservingPartitioner.getToken(OrderPreservingPartitioner.java:155)
        ... 10 more
 INFO 23:50:02,245 Finished reading 
/hd1/cassandra_md5/commitlog/CommitLog-1296759540149.log
ERROR 23:50:02,245 Exception encountered during startup.
java.lang.RuntimeException: java.util.concurrent.ExecutionException: 
java.lang.RuntimeException: java.lang.RuntimeException: The provided key was 
not UTF8 encoded.
        at 
org.apache.cassandra.utils.FBUtilities.waitOnFutures(FBUtilities.java:455)
        at 
org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:301)
        at 
org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:159)
        at 
org.apache.cassandra.service.AbstractCassandraDaemon.setup(AbstractCassandraDaemon.java:166)
        at 
org.apache.cassandra.service.AbstractCassandraDaemon.activate(AbstractCassandraDaemon.java:307)
        at 
org.apache.cassandra.thrift.CassandraDaemon.main(CassandraDaemon.java:81)
Caused by: java.util.concurrent.ExecutionException: java.lang.RuntimeException: 
java.lang.RuntimeException: The provided key was not UTF8 encoded.
        at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222)
        at java.util.concurrent.FutureTask.get(FutureTask.java:83)
        at 
org.apache.cassandra.utils.FBUtilities.waitOnFutures(FBUtilities.java:451)
        ... 5 more
Caused by: java.lang.RuntimeException: java.lang.RuntimeException: The provided 
key was not UTF8 encoded.
        at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
        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)
Caused by: java.lang.RuntimeException: The provided key was not UTF8 encoded.
        at 
org.apache.cassandra.dht.OrderPreservingPartitioner.getToken(OrderPreservingPartitioner.java:159)
        at 
org.apache.cassandra.dht.OrderPreservingPartitioner.decorateKey(OrderPreservingPartitioner.java:44)
        at org.apache.cassandra.db.Table.apply(Table.java:406)
        at 
org.apache.cassandra.db.commitlog.CommitLog$2.runMayThrow(CommitLog.java:294)
        at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
        ... 6 more
Caused by: java.nio.charset.MalformedInputException: Input length = 1
        at java.nio.charset.CoderResult.throwException(CoderResult.java:260)
        at java.nio.charset.CharsetDecoder.decode(CharsetDecoder.java:781)
        at 
org.apache.cassandra.utils.FBUtilities.decodeToUTF8(FBUtilities.java:403)
        at 
org.apache.cassandra.dht.OrderPreservingPartitioner.getToken(OrderPreservingPartitioner.java:155)
        ... 10 more
Exception encountered during startup.
java.lang.RuntimeException: java.util.concurrent.ExecutionException: 
java.lang.RuntimeException: java.lang.RuntimeException: The provided key was 
not UTF8 encoded.
        at 
org.apache.cassandra.utils.FBUtilities.waitOnFutures(FBUtilities.java:455)
        at 
org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:301)
        at 
org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:159)
        at 
org.apache.cassandra.service.AbstractCassandraDaemon.setup(AbstractCassandraDaemon.java:166)
        at 
org.apache.cassandra.service.AbstractCassandraDaemon.activate(AbstractCassandraDaemon.java:307)
        at 
org.apache.cassandra.thrift.CassandraDaemon.main(CassandraDaemon.java:81)
Caused by: java.util.concurrent.ExecutionException: java.lang.RuntimeException: 
java.lang.RuntimeException: The provided key was not UTF8 encoded.
        at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222)
        at java.util.concurrent.FutureTask.get(FutureTask.java:83)
        at 
org.apache.cassandra.utils.FBUtilities.waitOnFutures(FBUtilities.java:451)
        ... 5 more
Caused by: java.lang.RuntimeException: java.lang.RuntimeException: The provided 
key was not UTF8 encoded.
        at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
        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)
Caused by: java.lang.RuntimeException: The provided key was not UTF8 encoded.
        at 
org.apache.cassandra.dht.OrderPreservingPartitioner.getToken(OrderPreservingPartitioner.java:159)
        at 
org.apache.cassandra.dht.OrderPreservingPartitioner.decorateKey(OrderPreservingPartitioner.java:44)
        at org.apache.cassandra.db.Table.apply(Table.java:406)
        at 
org.apache.cassandra.db.commitlog.CommitLog$2.runMayThrow(CommitLog.java:294)
        at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
        ... 6 more
Caused by: java.nio.charset.MalformedInputException: Input length = 1
        at java.nio.charset.CoderResult.throwException(CoderResult.java:260)
        at java.nio.charset.CharsetDecoder.decode(CharsetDecoder.java:781)
        at 
org.apache.cassandra.utils.FBUtilities.decodeToUTF8(FBUtilities.java:403)
        at 
org.apache.cassandra.dht.OrderPreservingPartitioner.getToken(OrderPreservingPartitioner.java:155)
        ... 10 more




> Not restarting due to Invalid saved cache
> -----------------------------------------
>
>                 Key: CASSANDRA-2076
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2076
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>    Affects Versions: 0.7.0
>         Environment: linux
>            Reporter: Thibaut
>            Assignee: Matthew F. Dennis
>            Priority: Critical
>             Fix For: 0.7.1, 0.7.2
>
>         Attachments: 2076-cassandra-0.7.txt
>
>
> This occured on two nodes on me (running 0.7.1 from svn)
> One node was killed by the kernel due to a OOM and the other node was haning 
> and I had to kill it manually with kill -9 (kill didn't work). (maybe these 
> were faulty hardware nodes, I don't know)
> The saved_cache was corrupt afterwards and I couldn't start the nodes. 
> After deleting the saved_caches directory I could start the nodes again. 
> Instead of not starting when an error occurs, cassandra could simply delete 
> the errornous file and continue to start?
>  INFO 22:31:11,570 reading saved cache
> /hd1/cassandra_md5/saved_caches/table_attributes-table_attributes-KeyCache
> ERROR 22:31:11,595 Exception encountered during startup.
> java.lang.RuntimeException: The provided key was not UTF8 encoded.
>        at 
> org.apache.cassandra.dht.OrderPreservingPartitioner.getToken(OrderPreservingPartitioner.java:159)
>        at 
> org.apache.cassandra.dht.OrderPreservingPartitioner.decorateKey(OrderPreservingPartitioner.java:44)
>        at 
> org.apache.cassandra.db.ColumnFamilyStore.readSavedCache(ColumnFamilyStore.java:281)
>        at 
> org.apache.cassandra.db.ColumnFamilyStore.<init>(ColumnFamilyStore.java:218)
>        at 
> org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:458)
>        at 
> org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:440)
>        at org.apache.cassandra.db.Table.initCf(Table.java:360)
>        at org.apache.cassandra.db.Table.<init>(Table.java:290)
>        at org.apache.cassandra.db.Table.open(Table.java:107)
>        at 
> org.apache.cassandra.service.AbstractCassandraDaemon.setup(AbstractCassandraDaemon.java:167)
>        at 
> org.apache.cassandra.service.AbstractCassandraDaemon.activate(AbstractCassandraDaemon.java:312)
>        at 
> org.apache.cassandra.thrift.CassandraDaemon.main(CassandraDaemon.java:81)
> Caused by: java.nio.charset.MalformedInputException: Input length = 1
>        at java.nio.charset.CoderResult.throwException(CoderResult.java:260)
>        at java.nio.charset.CharsetDecoder.decode(CharsetDecoder.java:781)
>        at 
> org.apache.cassandra.utils.FBUtilities.decodeToUTF8(FBUtilities.java:403)
>        at 
> org.apache.cassandra.dht.OrderPreservingPartitioner.getToken(OrderPreservingPartitioner.java:155)
>        ... 11 more
> Exception encountered during startup.
> java.lang.RuntimeException: The provided key was not UTF8 encoded.
>        at 
> org.apache.cassandra.dht.OrderPreservingPartitioner.getToken(OrderPreservingPartitioner.java:159)
>        at 
> org.apache.cassandra.dht.OrderPreservingPartitioner.decorateKey(OrderPreservingPartitioner.java:44)
>        at 
> org.apache.cassandra.db.ColumnFamilyStore.readSavedCache(ColumnFamilyStore.java:281)
>        at 
> org.apache.cassandra.db.ColumnFamilyStore.<init>(ColumnFamilyStore.java:218)
>        at 
> org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:458)
>        at 
> org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:440)
>        at org.apache.cassandra.db.Table.initCf(Table.java:360)
>        at org.apache.cassandra.db.Table.<init>(Table.java:290)
>        at org.apache.cassandra.db.Table.open(Table.java:107)
>        at 
> org.apache.cassandra.service.AbstractCassandraDaemon.setup(AbstractCassandraDaemon.java:167)
>        at 
> org.apache.cassandra.service.AbstractCassandraDaemon.activate(AbstractCassandraDaemon.java:312)
>        at 
> org.apache.cassandra.thrift.CassandraDaemon.main(CassandraDaemon.java:81)
> Caused by: java.nio.charset.MalformedInputException: Input length = 1
>        at java.nio.charset.CoderResult.throwException(CoderResult.java:260)
>        at java.nio.charset.CharsetDecoder.decode(CharsetDecoder.java:781)
>        at 
> org.apache.cassandra.utils.FBUtilities.decodeToUTF8(FBUtilities.java:403)
>        at 
> org.apache.cassandra.dht.OrderPreservingPartitioner.getToken(OrderPreservingPartitioner.java:155)
>        ... 11 more

-- 
This message is automatically generated by JIRA.
-
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to