[ https://issues.apache.org/jira/browse/CASSANDRA-3466?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13146165#comment-13146165 ]
Jonas Borgström commented on CASSANDRA-3466: -------------------------------------------- > Do you observe the same behavior if both nodes start life as 1.x? I spoke too soon. I did some more testing with fresh 1.0.2 installs (and empty data directories). {code} 1. Start node1 and node2 and create keyspace and cf. 2. Stop node2, wait until "is now dead". 3. Trigger hint using "set bar[x][x]='x';" 4. Start node2, wait for hint delivery... Nothing happens 5. Restart node1, hints are finally delivered. {code} Now it gets really weird... {code} 2. Stop node2, wait until "is now dead". 3. Trigger hint using "set bar[x][x]='x';" 4. Start node2. Node1 attempts to deliver hints but fails with: INFO 09:29:35,153 Started hinted handoff for token: 23495828435496583962471242736585511198 with IP: /127.0.0.2 ERROR 09:29:35,212 Fatal exception in thread Thread[HintedHandoff:1,1,main] java.lang.AssertionError at org.apache.cassandra.db.HintedHandOffManager.deliverHintsToEndpoint(HintedHandOffManager.java:301) at org.apache.cassandra.db.HintedHandOffManager.access$100(HintedHandOffManager.java:81) at org.apache.cassandra.db.HintedHandOffManager$2.runMayThrow(HintedHandOffManager.java:353) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30) 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:680) On node1: [default@system] list HintsColumnFamily; Using default limit of 100 ------------------- RowKey: 11ad21c7d112e903d71645d1d951911e => (super_column=b2cd983009e311e10000fe8ebeead9cf, (column=6b6579, value=78, timestamp=1320740935987, ttl=864000) (column=6d75746174696f6e, value=0003666f6f00017800000001000003e801000003e880000000800000000000000000000001000178000004b134f65401d80000000139, timestamp=1320740935987, ttl=864000) (column=7461626c65, value=666f6f, timestamp=1320740935987, ttl=864000) (column=76657273696f6e, value=00000003, timestamp=1320740935987, ttl=864000)) => (super_column=f929a36009e211e10000fe8ebeead9df, (column=6b6579, value=78, timestamp=1320740624538, ttl=864000) (column=7461626c65, value=666f6f, timestamp=1320740624538, ttl=864000)) 1 Row Returned. {code} Is that a 0.8 hint showing up out of the blue on a pure 1.0.2 cluster? Or did I mess up my test somehow? > Hinted handoff not working after rolling upgrade from 0.8.7 to 1.0.2 > -------------------------------------------------------------------- > > Key: CASSANDRA-3466 > URL: https://issues.apache.org/jira/browse/CASSANDRA-3466 > Project: Cassandra > Issue Type: Bug > Affects Versions: 1.0.2 > Reporter: Jonas Borgström > Labels: hintedhandoff > > While testing rolling upgrades from 0.8.7 to 1.0.2 on a test cluster I've > noticed that hinted hand-off didn't always work properly. Hints generated on > an upgraded node does not seem to be delivered to other newly upgraded nodes > once they rejoin the ring. They only way I've found to get a node to deliver > its hints is to restart it. > Here's some steps to reproduce this issue: > 1. Install cassandra 0.8.7 on node1 and node2 using default settings. > 2. Create keyspace foo with {replication_factor: 2}. Create column family bar > 3. Shutdown node2 > 4. Insert data into bar and verify that HintsColumnFamily on node2 contains > hints > 5. Start node2 and verify that hinted handoff is performed and > HintsColumnFamily becomes empty again. > 6. Upgrade and restart node1 > 7. Shutdown node2 > 8. Insert data into bar and verify that HintsColumnFamily on node2 contains > hints > 9. Upgrade and start node2 > 10. Notice that hinted handoff is *not* performed when "node2" comes back. > (Only if node1 is restarted) -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira