cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Jonas Borgström (Commented) (JIRA) <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-3466) Hinted handoff not working after rolling upgrade from 0.8.7 to 1.0.2
Date Wed, 09 Nov 2011 13:57:51 GMT

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

Jonas Borgström commented on CASSANDRA-3466:
--------------------------------------------

> I haven't been able to reproduce the assertion errors, but I did find what is preventing
hint delivery in some cases

Brandon, Did you verify that removing those lines of code actually fixes hint delivery? 

Instead of changing the code I just did a quick experiment with "nodetool flush" on the node
holding the hints and then restarting the other node but that was not enough to trigger hints
delivery:

{code}
Node1 notices that node2 is backup up
  INFO 14:41:50,752 Node /127.0.0.2 has restarted, now UP
  INFO 14:41:50,752 InetAddress /127.0.0.2 is now UP
  INFO 14:41:50,753 Node /127.0.0.2 state jump to normal
But no hints are delivered...

nodetool flush is used to make sure hints hit the disk on node1:

  INFO 14:42:32,675 Enqueuing flush of Memtable-Versions@1503666327(83/103 serialized/live
bytes, 3 ops)
  INFO 14:42:32,675 Writing Memtable-Versions@1503666327(83/103 serialized/live bytes, 3 ops)
  INFO 14:42:32,681 Completed flushing /tmp/node1/data/data/system/Versions-h-1-Data.db (247
bytes)
  INFO 14:42:32,682 Enqueuing flush of Memtable-HintsColumnFamily@737188401(177/221 serialized/live
bytes, 1 ops)
  INFO 14:42:32,682 Writing Memtable-HintsColumnFamily@737188401(177/221 serialized/live bytes,
1 ops)
  INFO 14:42:32,688 Completed flushing /tmp/node1/data/data/system/HintsColumnFamily-h-1-Data.db
(277 bytes)
  INFO 14:42:32,691 Enqueuing flush of Memtable-bar@1831941861(17/21 serialized/live bytes,
1 ops)
  INFO 14:42:32,691 Writing Memtable-bar@1831941861(17/21 serialized/live bytes, 1 ops)
  INFO 14:42:32,694 Completed flushing /tmp/node1/data/data/foo/bar-h-1-Data.db (68 bytes)

Node2 is restarted once more to check if this will trigger hints delivery:
  INFO 14:42:54,650 InetAddress /127.0.0.2 is now dead.
  INFO 14:43:02,628 Node /127.0.0.2 has restarted, now UP
  INFO 14:43:02,629 InetAddress /127.0.0.2 is now UP
  INFO 14:43:02,629 Node /127.0.0.2 state jump to normal

Still nothing...  Restarting node 1 will deliver the hints within a few seconds though...
{code}

Regarding reproducing the assertion error it's a bit tricky. But after letting my two node
test cluster performing hints delivery for each other a few times I was able to reproduce
it once more. Is there anything special you would like me to test?



                
> 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.0
>            Reporter: Jonas Borgström
>            Assignee: Brandon Williams
>              Labels: hintedhandoff
>             Fix For: 1.0.3
>
>
> 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

       

Mime
View raw message