New node unable to stream (0.8.5)
Hi. We've been running a 7-node cluster with RF 3, QUORUM reads/writes in our production environment for a few months. It's been consistently stable during this period, particularly once we got out maintenance strategy fully worked out (per node, one repair a week, one major compaction a week, the latter due to the nature of our data model and usage). While this cluster started, back in June or so, on the 0.7 series, it's been running 0.8.3 for a while now with no issues. We upgraded to 0.8.5 two days ago, having tested the upgrade in our staging cluster (with an otherwise identical configuration) previously and verified that our application's various use cases appeared successful. One of our nodes suffered a disk failure yesterday. We attempted to replace the dead node by placing a new node at OldNode.initial_token - 1 with auto_bootstrap on. A few things went awry from there: 1. We never saw the new node in bootstrap mode; it became available pretty much immediately upon joining the ring, and never reported a joining state. I did verify that auto_bootstrap was on. 2. I mistakenly ran repair on the new node rather than removetoken on the old node, due to a delightful mental error. The repair got nowhere fast, as it attempts to repair against the down node which throws an exception. So I interrupted the repair, restarted the node to clear any pending validation compactions, and... 3. Ran removetoken for the old node. 4. We let this run for some time and saw eventually that all the nodes appeared to be done various compactions and were stuck at streaming. Many streams listed as open, none making any progress. 5. I observed an Rpc-related exception on the new node (where the removetoken was launched) and concluded that the streams were broken so the process wouldn't ever finish. 6. Ran a removetoken force to get the dead node out of the mix. No problems. 7. Ran a repair on the new node. 8. Validations ran, streams opened up, and again things got stuck in streaming, hanging for over an hour with no progress. 9. Musing that lingering tasks from the removetoken could be a factor, I performed a rolling restart and attempted a repair again. 10. Same problem. Did another rolling restart and attempted a fresh repair on the most important column family alone. 11. Same problem. Streams included CFs not specified, so I guess they must be for hinted handoff. In concluding that streaming is stuck, I've observed: - streams will be open to the new node from other nodes, but the new node doesn't list them - streams will be open to the other nodes from the new node, but the other nodes don't list them - the streams reported may make some initial progress, but then they hang at a particular point and do not move on for an hour or more. - The logs report repair-related activity, until NPEs on incoming TCP connections show up, which appear likely to be the culprit. I can provide more exact details when I'm done commuting. With streaming broken on this node, I'm unable to run repairs, which is obviously problematic. The application didn't suffer any operational issues as a consequence of this, but I need to review the overnight results to verify we're not suffering data loss (I doubt we are). At this point, I'm considering a couple options: 1. Remove the new node and let the adjacent node take over its range 2. Bring the new node down, add a new one in front of it, and properly removetoken the problematic one. 3. Bring the new node down, remove all its data except for the system keyspace, then bring it back up and repair it. 4. Revert to 0.8.3 and see if that helps. Recommendations? Thanks. - Ethan
Re: New node unable to stream (0.8.5)
On Thu, Sep 15, 2011 at 1:16 PM, Ethan Rowe et...@the-rowes.com wrote: Hi. We've been running a 7-node cluster with RF 3, QUORUM reads/writes in our production environment for a few months. It's been consistently stable during this period, particularly once we got out maintenance strategy fully worked out (per node, one repair a week, one major compaction a week, the latter due to the nature of our data model and usage). While this cluster started, back in June or so, on the 0.7 series, it's been running 0.8.3 for a while now with no issues. We upgraded to 0.8.5 two days ago, having tested the upgrade in our staging cluster (with an otherwise identical configuration) previously and verified that our application's various use cases appeared successful. One of our nodes suffered a disk failure yesterday. We attempted to replace the dead node by placing a new node at OldNode.initial_token - 1 with auto_bootstrap on. A few things went awry from there: 1. We never saw the new node in bootstrap mode; it became available pretty much immediately upon joining the ring, and never reported a joining state. I did verify that auto_bootstrap was on. 2. I mistakenly ran repair on the new node rather than removetoken on the old node, due to a delightful mental error. The repair got nowhere fast, as it attempts to repair against the down node which throws an exception. So I interrupted the repair, restarted the node to clear any pending validation compactions, and... 3. Ran removetoken for the old node. 4. We let this run for some time and saw eventually that all the nodes appeared to be done various compactions and were stuck at streaming. Many streams listed as open, none making any progress. 5. I observed an Rpc-related exception on the new node (where the removetoken was launched) and concluded that the streams were broken so the process wouldn't ever finish. 6. Ran a removetoken force to get the dead node out of the mix. No problems. 7. Ran a repair on the new node. 8. Validations ran, streams opened up, and again things got stuck in streaming, hanging for over an hour with no progress. 9. Musing that lingering tasks from the removetoken could be a factor, I performed a rolling restart and attempted a repair again. 10. Same problem. Did another rolling restart and attempted a fresh repair on the most important column family alone. 11. Same problem. Streams included CFs not specified, so I guess they must be for hinted handoff. In concluding that streaming is stuck, I've observed: - streams will be open to the new node from other nodes, but the new node doesn't list them - streams will be open to the other nodes from the new node, but the other nodes don't list them - the streams reported may make some initial progress, but then they hang at a particular point and do not move on for an hour or more. - The logs report repair-related activity, until NPEs on incoming TCP connections show up, which appear likely to be the culprit. Can you send the stack trace from those NPE. I can provide more exact details when I'm done commuting. With streaming broken on this node, I'm unable to run repairs, which is obviously problematic. The application didn't suffer any operational issues as a consequence of this, but I need to review the overnight results to verify we're not suffering data loss (I doubt we are). At this point, I'm considering a couple options: 1. Remove the new node and let the adjacent node take over its range 2. Bring the new node down, add a new one in front of it, and properly removetoken the problematic one. 3. Bring the new node down, remove all its data except for the system keyspace, then bring it back up and repair it. 4. Revert to 0.8.3 and see if that helps. Recommendations? Thanks. - Ethan
Re: New node unable to stream (0.8.5)
Here's a typical log slice (not terribly informative, I fear): INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,106 AntiEntropyService.java (l ine 884) Performing streaming repair of 1003 ranges with /10.34.90.8 for (299 90798416657667504332586989223299634,54296681768153272037430773234349600451] INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,427 StreamOut.java (line 181) Stream context metadata [/mnt/cassandra/data/events_production/FitsByShip-g-1 0-Data.db sections=88 progress=0/11707163 - 0%, /mnt/cassandra/data/events_pr oduction/FitsByShip-g-11-Data.db sections=169 progress=0/6133240 - 0%, /mnt/c assandra/data/events_production/FitsByShip-g-6-Data.db sections=1 progress=0/ 6918814 - 0%, /mnt/cassandra/data/events_production/FitsByShip-g-12-Data.db s ections=260 progress=0/9091780 - 0%], 4 sstables. INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,428 StreamOutSession.java (lin e 174) Streaming to /10.34.90.8 ERROR [Thread-56] 2011-09-15 05:41:38,515 AbstractCassandraDaemon.java (line 139) Fatal exception in thread Thread[Thread-56,5,main] java.lang.NullPointerException at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpC onnection.java:174) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConn ection.java:114) Not sure if the exception is related to the outbound streaming above; other nodes are actively trying to stream to this node, so perhaps it comes from those and temporal adjacency to the outbound stream is just coincidental. I have other snippets that look basically identical to the above, except if I look at the logs to which this node is trying to stream, I see that it has concurrently opened a stream in the other direction, which could be the one that the exception pertains to. On Thu, Sep 15, 2011 at 7:41 AM, Sylvain Lebresne sylv...@datastax.comwrote: On Thu, Sep 15, 2011 at 1:16 PM, Ethan Rowe et...@the-rowes.com wrote: Hi. We've been running a 7-node cluster with RF 3, QUORUM reads/writes in our production environment for a few months. It's been consistently stable during this period, particularly once we got out maintenance strategy fully worked out (per node, one repair a week, one major compaction a week, the latter due to the nature of our data model and usage). While this cluster started, back in June or so, on the 0.7 series, it's been running 0.8.3 for a while now with no issues. We upgraded to 0.8.5 two days ago, having tested the upgrade in our staging cluster (with an otherwise identical configuration) previously and verified that our application's various use cases appeared successful. One of our nodes suffered a disk failure yesterday. We attempted to replace the dead node by placing a new node at OldNode.initial_token - 1 with auto_bootstrap on. A few things went awry from there: 1. We never saw the new node in bootstrap mode; it became available pretty much immediately upon joining the ring, and never reported a joining state. I did verify that auto_bootstrap was on. 2. I mistakenly ran repair on the new node rather than removetoken on the old node, due to a delightful mental error. The repair got nowhere fast, as it attempts to repair against the down node which throws an exception. So I interrupted the repair, restarted the node to clear any pending validation compactions, and... 3. Ran removetoken for the old node. 4. We let this run for some time and saw eventually that all the nodes appeared to be done various compactions and were stuck at streaming. Many streams listed as open, none making any progress. 5. I observed an Rpc-related exception on the new node (where the removetoken was launched) and concluded that the streams were broken so the process wouldn't ever finish. 6. Ran a removetoken force to get the dead node out of the mix. No problems. 7. Ran a repair on the new node. 8. Validations ran, streams opened up, and again things got stuck in streaming, hanging for over an hour with no progress. 9. Musing that lingering tasks from the removetoken could be a factor, I performed a rolling restart and attempted a repair again. 10. Same problem. Did another rolling restart and attempted a fresh repair on the most important column family alone. 11. Same problem. Streams included CFs not specified, so I guess they must be for hinted handoff. In concluding that streaming is stuck, I've observed: - streams will be open to the new node from other nodes, but the new node doesn't list them - streams will be open to the other nodes from the new node, but the other nodes don't list them - the streams reported may make some initial progress, but then they hang at a particular point and do not move on for an hour or more. - The logs report repair-related activity, until NPEs on incoming TCP connections show up, which appear likely to be the
Re: New node unable to stream (0.8.5)
I just noticed the following from one of Jonathan Ellis' messages yesterday: Added to NEWS: - After upgrading, run nodetool scrub against each node before running repair, moving nodes, or adding new ones. We did not do this, as it was not indicated as necessary in the news when we were dealing with the upgrade. So perhaps I need to scrub everything before going any further, though the question is what to do with the problematic node. Additionally, it would be helpful to know if scrub will affect the hinted handoffs that have accumulated, as these seem likely to be part of the set of failing streams. On Thu, Sep 15, 2011 at 8:13 AM, Ethan Rowe et...@the-rowes.com wrote: Here's a typical log slice (not terribly informative, I fear): INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,106 AntiEntropyService.java (l ine 884) Performing streaming repair of 1003 ranges with /10.34.90.8 for (299 90798416657667504332586989223299634,54296681768153272037430773234349600451] INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,427 StreamOut.java (line 181) Stream context metadata [/mnt/cassandra/data/events_production/FitsByShip-g-1 0-Data.db sections=88 progress=0/11707163 - 0%, /mnt/cassandra/data/events_pr oduction/FitsByShip-g-11-Data.db sections=169 progress=0/6133240 - 0%, /mnt/c assandra/data/events_production/FitsByShip-g-6-Data.db sections=1 progress=0/ 6918814 - 0%, /mnt/cassandra/data/events_production/FitsByShip-g-12-Data.db s ections=260 progress=0/9091780 - 0%], 4 sstables. INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,428 StreamOutSession.java (lin e 174) Streaming to /10.34.90.8 ERROR [Thread-56] 2011-09-15 05:41:38,515 AbstractCassandraDaemon.java (line 139) Fatal exception in thread Thread[Thread-56,5,main] java.lang.NullPointerException at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpC onnection.java:174) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConn ection.java:114) Not sure if the exception is related to the outbound streaming above; other nodes are actively trying to stream to this node, so perhaps it comes from those and temporal adjacency to the outbound stream is just coincidental. I have other snippets that look basically identical to the above, except if I look at the logs to which this node is trying to stream, I see that it has concurrently opened a stream in the other direction, which could be the one that the exception pertains to. On Thu, Sep 15, 2011 at 7:41 AM, Sylvain Lebresne sylv...@datastax.comwrote: On Thu, Sep 15, 2011 at 1:16 PM, Ethan Rowe et...@the-rowes.com wrote: Hi. We've been running a 7-node cluster with RF 3, QUORUM reads/writes in our production environment for a few months. It's been consistently stable during this period, particularly once we got out maintenance strategy fully worked out (per node, one repair a week, one major compaction a week, the latter due to the nature of our data model and usage). While this cluster started, back in June or so, on the 0.7 series, it's been running 0.8.3 for a while now with no issues. We upgraded to 0.8.5 two days ago, having tested the upgrade in our staging cluster (with an otherwise identical configuration) previously and verified that our application's various use cases appeared successful. One of our nodes suffered a disk failure yesterday. We attempted to replace the dead node by placing a new node at OldNode.initial_token - 1 with auto_bootstrap on. A few things went awry from there: 1. We never saw the new node in bootstrap mode; it became available pretty much immediately upon joining the ring, and never reported a joining state. I did verify that auto_bootstrap was on. 2. I mistakenly ran repair on the new node rather than removetoken on the old node, due to a delightful mental error. The repair got nowhere fast, as it attempts to repair against the down node which throws an exception. So I interrupted the repair, restarted the node to clear any pending validation compactions, and... 3. Ran removetoken for the old node. 4. We let this run for some time and saw eventually that all the nodes appeared to be done various compactions and were stuck at streaming. Many streams listed as open, none making any progress. 5. I observed an Rpc-related exception on the new node (where the removetoken was launched) and concluded that the streams were broken so the process wouldn't ever finish. 6. Ran a removetoken force to get the dead node out of the mix. No problems. 7. Ran a repair on the new node. 8. Validations ran, streams opened up, and again things got stuck in streaming, hanging for over an hour with no progress. 9. Musing that lingering tasks from the removetoken could be a factor, I performed a rolling restart and attempted a repair again. 10. Same problem. Did another rolling
Re: New node unable to stream (0.8.5)
After further review, I'm definitely going to scrub all the original nodes in the cluster. We've lost some data as a result of this situation. It can be restored, but the question is what to do with the problematic new node first. I don't particularly care about the data that's on it, since I'm going to re-import the critical data from files anyway, and then I can recreate derivative data afterwards. So it's purely a matter of getting the cluster healthy again as quickly as possible so I can begin that import process. Any issue with running scrubs on multiple nodes at a time, provided they aren't replication neighbors? On Thu, Sep 15, 2011 at 8:18 AM, Ethan Rowe et...@the-rowes.com wrote: I just noticed the following from one of Jonathan Ellis' messages yesterday: Added to NEWS: - After upgrading, run nodetool scrub against each node before running repair, moving nodes, or adding new ones. We did not do this, as it was not indicated as necessary in the news when we were dealing with the upgrade. So perhaps I need to scrub everything before going any further, though the question is what to do with the problematic node. Additionally, it would be helpful to know if scrub will affect the hinted handoffs that have accumulated, as these seem likely to be part of the set of failing streams. On Thu, Sep 15, 2011 at 8:13 AM, Ethan Rowe et...@the-rowes.com wrote: Here's a typical log slice (not terribly informative, I fear): INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,106 AntiEntropyService.java (l ine 884) Performing streaming repair of 1003 ranges with /10.34.90.8 for (299 90798416657667504332586989223299634,54296681768153272037430773234349600451] INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,427 StreamOut.java (line 181) Stream context metadata [/mnt/cassandra/data/events_production/FitsByShip-g-1 0-Data.db sections=88 progress=0/11707163 - 0%, /mnt/cassandra/data/events_pr oduction/FitsByShip-g-11-Data.db sections=169 progress=0/6133240 - 0%, /mnt/c assandra/data/events_production/FitsByShip-g-6-Data.db sections=1 progress=0/ 6918814 - 0%, /mnt/cassandra/data/events_production/FitsByShip-g-12-Data.db s ections=260 progress=0/9091780 - 0%], 4 sstables. INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,428 StreamOutSession.java (lin e 174) Streaming to /10.34.90.8 ERROR [Thread-56] 2011-09-15 05:41:38,515 AbstractCassandraDaemon.java (line 139) Fatal exception in thread Thread[Thread-56,5,main] java.lang.NullPointerException at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpC onnection.java:174) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConn ection.java:114) Not sure if the exception is related to the outbound streaming above; other nodes are actively trying to stream to this node, so perhaps it comes from those and temporal adjacency to the outbound stream is just coincidental. I have other snippets that look basically identical to the above, except if I look at the logs to which this node is trying to stream, I see that it has concurrently opened a stream in the other direction, which could be the one that the exception pertains to. On Thu, Sep 15, 2011 at 7:41 AM, Sylvain Lebresne sylv...@datastax.comwrote: On Thu, Sep 15, 2011 at 1:16 PM, Ethan Rowe et...@the-rowes.com wrote: Hi. We've been running a 7-node cluster with RF 3, QUORUM reads/writes in our production environment for a few months. It's been consistently stable during this period, particularly once we got out maintenance strategy fully worked out (per node, one repair a week, one major compaction a week, the latter due to the nature of our data model and usage). While this cluster started, back in June or so, on the 0.7 series, it's been running 0.8.3 for a while now with no issues. We upgraded to 0.8.5 two days ago, having tested the upgrade in our staging cluster (with an otherwise identical configuration) previously and verified that our application's various use cases appeared successful. One of our nodes suffered a disk failure yesterday. We attempted to replace the dead node by placing a new node at OldNode.initial_token - 1 with auto_bootstrap on. A few things went awry from there: 1. We never saw the new node in bootstrap mode; it became available pretty much immediately upon joining the ring, and never reported a joining state. I did verify that auto_bootstrap was on. 2. I mistakenly ran repair on the new node rather than removetoken on the old node, due to a delightful mental error. The repair got nowhere fast, as it attempts to repair against the down node which throws an exception. So I interrupted the repair, restarted the node to clear any pending validation compactions, and... 3. Ran removetoken for the old node. 4. We let this run for some time and saw eventually that all the nodes appeared to be done various
Re: New node unable to stream (0.8.5)
That means we missed a place we needed to special-case for backwards compatibility -- the workaround is, add an empty encryption_options section to cassandra.yaml: encryption_options: internode_encryption: none keystore: conf/.keystore keystore_password: cassandra truststore: conf/.truststore truststore_password: cassandra Created https://issues.apache.org/jira/browse/CASSANDRA-3212 to fix this. On Thu, Sep 15, 2011 at 7:13 AM, Ethan Rowe et...@the-rowes.com wrote: Here's a typical log slice (not terribly informative, I fear): INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,106 AntiEntropyService.java (l ine 884) Performing streaming repair of 1003 ranges with /10.34.90.8 for (299 90798416657667504332586989223299634,54296681768153272037430773234349600451] INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,427 StreamOut.java (line 181) Stream context metadata [/mnt/cassandra/data/events_production/FitsByShip-g-1 0-Data.db sections=88 progress=0/11707163 - 0%, /mnt/cassandra/data/events_pr oduction/FitsByShip-g-11-Data.db sections=169 progress=0/6133240 - 0%, /mnt/c assandra/data/events_production/FitsByShip-g-6-Data.db sections=1 progress=0/ 6918814 - 0%, /mnt/cassandra/data/events_production/FitsByShip-g-12-Data.db s ections=260 progress=0/9091780 - 0%], 4 sstables. INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,428 StreamOutSession.java (lin e 174) Streaming to /10.34.90.8 ERROR [Thread-56] 2011-09-15 05:41:38,515 AbstractCassandraDaemon.java (line 139) Fatal exception in thread Thread[Thread-56,5,main] java.lang.NullPointerException at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpC onnection.java:174) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConn ection.java:114) Not sure if the exception is related to the outbound streaming above; other nodes are actively trying to stream to this node, so perhaps it comes from those and temporal adjacency to the outbound stream is just coincidental. I have other snippets that look basically identical to the above, except if I look at the logs to which this node is trying to stream, I see that it has concurrently opened a stream in the other direction, which could be the one that the exception pertains to. On Thu, Sep 15, 2011 at 7:41 AM, Sylvain Lebresne sylv...@datastax.com wrote: On Thu, Sep 15, 2011 at 1:16 PM, Ethan Rowe et...@the-rowes.com wrote: Hi. We've been running a 7-node cluster with RF 3, QUORUM reads/writes in our production environment for a few months. It's been consistently stable during this period, particularly once we got out maintenance strategy fully worked out (per node, one repair a week, one major compaction a week, the latter due to the nature of our data model and usage). While this cluster started, back in June or so, on the 0.7 series, it's been running 0.8.3 for a while now with no issues. We upgraded to 0.8.5 two days ago, having tested the upgrade in our staging cluster (with an otherwise identical configuration) previously and verified that our application's various use cases appeared successful. One of our nodes suffered a disk failure yesterday. We attempted to replace the dead node by placing a new node at OldNode.initial_token - 1 with auto_bootstrap on. A few things went awry from there: 1. We never saw the new node in bootstrap mode; it became available pretty much immediately upon joining the ring, and never reported a joining state. I did verify that auto_bootstrap was on. 2. I mistakenly ran repair on the new node rather than removetoken on the old node, due to a delightful mental error. The repair got nowhere fast, as it attempts to repair against the down node which throws an exception. So I interrupted the repair, restarted the node to clear any pending validation compactions, and... 3. Ran removetoken for the old node. 4. We let this run for some time and saw eventually that all the nodes appeared to be done various compactions and were stuck at streaming. Many streams listed as open, none making any progress. 5. I observed an Rpc-related exception on the new node (where the removetoken was launched) and concluded that the streams were broken so the process wouldn't ever finish. 6. Ran a removetoken force to get the dead node out of the mix. No problems. 7. Ran a repair on the new node. 8. Validations ran, streams opened up, and again things got stuck in streaming, hanging for over an hour with no progress. 9. Musing that lingering tasks from the removetoken could be a factor, I performed a rolling restart and attempted a repair again. 10. Same problem. Did another rolling restart and attempted a fresh repair on the most important column family alone. 11. Same problem. Streams included CFs not specified, so I guess they must be for hinted handoff.
Re: New node unable to stream (0.8.5)
Where did the data loss come in? Scrub is safe to run in parallel. On Thu, Sep 15, 2011 at 8:08 AM, Ethan Rowe et...@the-rowes.com wrote: After further review, I'm definitely going to scrub all the original nodes in the cluster. We've lost some data as a result of this situation. It can be restored, but the question is what to do with the problematic new node first. I don't particularly care about the data that's on it, since I'm going to re-import the critical data from files anyway, and then I can recreate derivative data afterwards. So it's purely a matter of getting the cluster healthy again as quickly as possible so I can begin that import process. Any issue with running scrubs on multiple nodes at a time, provided they aren't replication neighbors? On Thu, Sep 15, 2011 at 8:18 AM, Ethan Rowe et...@the-rowes.com wrote: I just noticed the following from one of Jonathan Ellis' messages yesterday: Added to NEWS: - After upgrading, run nodetool scrub against each node before running repair, moving nodes, or adding new ones. We did not do this, as it was not indicated as necessary in the news when we were dealing with the upgrade. So perhaps I need to scrub everything before going any further, though the question is what to do with the problematic node. Additionally, it would be helpful to know if scrub will affect the hinted handoffs that have accumulated, as these seem likely to be part of the set of failing streams. On Thu, Sep 15, 2011 at 8:13 AM, Ethan Rowe et...@the-rowes.com wrote: Here's a typical log slice (not terribly informative, I fear): INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,106 AntiEntropyService.java (l ine 884) Performing streaming repair of 1003 ranges with /10.34.90.8 for (299 90798416657667504332586989223299634,54296681768153272037430773234349600451] INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,427 StreamOut.java (line 181) Stream context metadata [/mnt/cassandra/data/events_production/FitsByShip-g-1 0-Data.db sections=88 progress=0/11707163 - 0%, /mnt/cassandra/data/events_pr oduction/FitsByShip-g-11-Data.db sections=169 progress=0/6133240 - 0%, /mnt/c assandra/data/events_production/FitsByShip-g-6-Data.db sections=1 progress=0/ 6918814 - 0%, /mnt/cassandra/data/events_production/FitsByShip-g-12-Data.db s ections=260 progress=0/9091780 - 0%], 4 sstables. INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,428 StreamOutSession.java (lin e 174) Streaming to /10.34.90.8 ERROR [Thread-56] 2011-09-15 05:41:38,515 AbstractCassandraDaemon.java (line 139) Fatal exception in thread Thread[Thread-56,5,main] java.lang.NullPointerException at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpC onnection.java:174) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConn ection.java:114) Not sure if the exception is related to the outbound streaming above; other nodes are actively trying to stream to this node, so perhaps it comes from those and temporal adjacency to the outbound stream is just coincidental. I have other snippets that look basically identical to the above, except if I look at the logs to which this node is trying to stream, I see that it has concurrently opened a stream in the other direction, which could be the one that the exception pertains to. On Thu, Sep 15, 2011 at 7:41 AM, Sylvain Lebresne sylv...@datastax.com wrote: On Thu, Sep 15, 2011 at 1:16 PM, Ethan Rowe et...@the-rowes.com wrote: Hi. We've been running a 7-node cluster with RF 3, QUORUM reads/writes in our production environment for a few months. It's been consistently stable during this period, particularly once we got out maintenance strategy fully worked out (per node, one repair a week, one major compaction a week, the latter due to the nature of our data model and usage). While this cluster started, back in June or so, on the 0.7 series, it's been running 0.8.3 for a while now with no issues. We upgraded to 0.8.5 two days ago, having tested the upgrade in our staging cluster (with an otherwise identical configuration) previously and verified that our application's various use cases appeared successful. One of our nodes suffered a disk failure yesterday. We attempted to replace the dead node by placing a new node at OldNode.initial_token - 1 with auto_bootstrap on. A few things went awry from there: 1. We never saw the new node in bootstrap mode; it became available pretty much immediately upon joining the ring, and never reported a joining state. I did verify that auto_bootstrap was on. 2. I mistakenly ran repair on the new node rather than removetoken on the old node, due to a delightful mental error. The repair got nowhere fast, as it attempts to repair against the down node which throws an exception. So I interrupted the repair, restarted the node to clear any pending validation
Re: New node unable to stream (0.8.5)
Thanks, Jonathan. I'll try the workaround and see if that gets the streams flowing properly. As I mentioned before, we did not run scrub yet. What is the consequence of letting the streams from the hinted handoffs complete if scrub hasn't been run on these nodes? I'm currently running scrub on one node to get a sense of the time frame. Thanks again. - Ethan On Thu, Sep 15, 2011 at 9:09 AM, Jonathan Ellis jbel...@gmail.com wrote: That means we missed a place we needed to special-case for backwards compatibility -- the workaround is, add an empty encryption_options section to cassandra.yaml: encryption_options: internode_encryption: none keystore: conf/.keystore keystore_password: cassandra truststore: conf/.truststore truststore_password: cassandra Created https://issues.apache.org/jira/browse/CASSANDRA-3212 to fix this. On Thu, Sep 15, 2011 at 7:13 AM, Ethan Rowe et...@the-rowes.com wrote: Here's a typical log slice (not terribly informative, I fear): INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,106 AntiEntropyService.java (l ine 884) Performing streaming repair of 1003 ranges with /10.34.90.8for (299 90798416657667504332586989223299634,54296681768153272037430773234349600451] INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,427 StreamOut.java (line 181) Stream context metadata [/mnt/cassandra/data/events_production/FitsByShip-g-1 0-Data.db sections=88 progress=0/11707163 - 0%, /mnt/cassandra/data/events_pr oduction/FitsByShip-g-11-Data.db sections=169 progress=0/6133240 - 0%, /mnt/c assandra/data/events_production/FitsByShip-g-6-Data.db sections=1 progress=0/ 6918814 - 0%, /mnt/cassandra/data/events_production/FitsByShip-g-12-Data.db s ections=260 progress=0/9091780 - 0%], 4 sstables. INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,428 StreamOutSession.java (lin e 174) Streaming to /10.34.90.8 ERROR [Thread-56] 2011-09-15 05:41:38,515 AbstractCassandraDaemon.java (line 139) Fatal exception in thread Thread[Thread-56,5,main] java.lang.NullPointerException at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpC onnection.java:174) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConn ection.java:114) Not sure if the exception is related to the outbound streaming above; other nodes are actively trying to stream to this node, so perhaps it comes from those and temporal adjacency to the outbound stream is just coincidental. I have other snippets that look basically identical to the above, except if I look at the logs to which this node is trying to stream, I see that it has concurrently opened a stream in the other direction, which could be the one that the exception pertains to. On Thu, Sep 15, 2011 at 7:41 AM, Sylvain Lebresne sylv...@datastax.com wrote: On Thu, Sep 15, 2011 at 1:16 PM, Ethan Rowe et...@the-rowes.com wrote: Hi. We've been running a 7-node cluster with RF 3, QUORUM reads/writes in our production environment for a few months. It's been consistently stable during this period, particularly once we got out maintenance strategy fully worked out (per node, one repair a week, one major compaction a week, the latter due to the nature of our data model and usage). While this cluster started, back in June or so, on the 0.7 series, it's been running 0.8.3 for a while now with no issues. We upgraded to 0.8.5 two days ago, having tested the upgrade in our staging cluster (with an otherwise identical configuration) previously and verified that our application's various use cases appeared successful. One of our nodes suffered a disk failure yesterday. We attempted to replace the dead node by placing a new node at OldNode.initial_token - 1 with auto_bootstrap on. A few things went awry from there: 1. We never saw the new node in bootstrap mode; it became available pretty much immediately upon joining the ring, and never reported a joining state. I did verify that auto_bootstrap was on. 2. I mistakenly ran repair on the new node rather than removetoken on the old node, due to a delightful mental error. The repair got nowhere fast, as it attempts to repair against the down node which throws an exception. So I interrupted the repair, restarted the node to clear any pending validation compactions, and... 3. Ran removetoken for the old node. 4. We let this run for some time and saw eventually that all the nodes appeared to be done various compactions and were stuck at streaming. Many streams listed as open, none making any progress. 5. I observed an Rpc-related exception on the new node (where the removetoken was launched) and concluded that the streams were broken so the process wouldn't ever finish. 6. Ran a removetoken force to get the dead node out of the mix.
Re: New node unable to stream (0.8.5)
On Thu, Sep 15, 2011 at 9:21 AM, Jonathan Ellis jbel...@gmail.com wrote: Where did the data loss come in? The outcome of the analytical jobs run overnight while some of these repairs were (not) running is consistent with what I would expect if perhaps 20-30% of the source data was missing. Given the strong consistency model we're using, this is surprising to me, since the jobs did not report any read or write failures. I wonder if this is a consequence of the dead node missing and the new node being operational but having received basically none of its hinted handoff streams. Perhaps with streaming fixed the data will reappear, which would be a happy outcome, but if not, I can reimport the critical stuff from files. Scrub is safe to run in parallel. Is it somewhat analogous to a major compaction in terms of I/O impact, with perhaps less greedy use of disk space? On Thu, Sep 15, 2011 at 8:08 AM, Ethan Rowe et...@the-rowes.com wrote: After further review, I'm definitely going to scrub all the original nodes in the cluster. We've lost some data as a result of this situation. It can be restored, but the question is what to do with the problematic new node first. I don't particularly care about the data that's on it, since I'm going to re-import the critical data from files anyway, and then I can recreate derivative data afterwards. So it's purely a matter of getting the cluster healthy again as quickly as possible so I can begin that import process. Any issue with running scrubs on multiple nodes at a time, provided they aren't replication neighbors? On Thu, Sep 15, 2011 at 8:18 AM, Ethan Rowe et...@the-rowes.com wrote: I just noticed the following from one of Jonathan Ellis' messages yesterday: Added to NEWS: - After upgrading, run nodetool scrub against each node before running repair, moving nodes, or adding new ones. We did not do this, as it was not indicated as necessary in the news when we were dealing with the upgrade. So perhaps I need to scrub everything before going any further, though the question is what to do with the problematic node. Additionally, it would be helpful to know if scrub will affect the hinted handoffs that have accumulated, as these seem likely to be part of the set of failing streams. On Thu, Sep 15, 2011 at 8:13 AM, Ethan Rowe et...@the-rowes.com wrote: Here's a typical log slice (not terribly informative, I fear): INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,106 AntiEntropyService.java (l ine 884) Performing streaming repair of 1003 ranges with /10.34.90.8for (299 90798416657667504332586989223299634,54296681768153272037430773234349600451] INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,427 StreamOut.java (line 181) Stream context metadata [/mnt/cassandra/data/events_production/FitsByShip-g-1 0-Data.db sections=88 progress=0/11707163 - 0%, /mnt/cassandra/data/events_pr oduction/FitsByShip-g-11-Data.db sections=169 progress=0/6133240 - 0%, /mnt/c assandra/data/events_production/FitsByShip-g-6-Data.db sections=1 progress=0/ 6918814 - 0%, /mnt/cassandra/data/events_production/FitsByShip-g-12-Data.db s ections=260 progress=0/9091780 - 0%], 4 sstables. INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,428 StreamOutSession.java (lin e 174) Streaming to /10.34.90.8 ERROR [Thread-56] 2011-09-15 05:41:38,515 AbstractCassandraDaemon.java (line 139) Fatal exception in thread Thread[Thread-56,5,main] java.lang.NullPointerException at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpC onnection.java:174) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConn ection.java:114) Not sure if the exception is related to the outbound streaming above; other nodes are actively trying to stream to this node, so perhaps it comes from those and temporal adjacency to the outbound stream is just coincidental. I have other snippets that look basically identical to the above, except if I look at the logs to which this node is trying to stream, I see that it has concurrently opened a stream in the other direction, which could be the one that the exception pertains to. On Thu, Sep 15, 2011 at 7:41 AM, Sylvain Lebresne sylv...@datastax.com wrote: On Thu, Sep 15, 2011 at 1:16 PM, Ethan Rowe et...@the-rowes.com wrote: Hi. We've been running a 7-node cluster with RF 3, QUORUM reads/writes in our production environment for a few months. It's been consistently stable during this period, particularly once we got out maintenance strategy fully worked out (per node, one repair a week, one major compaction a week, the latter due to the nature of our data model and usage). While this cluster started, back in June or so, on the 0.7 series, it's been running 0.8.3 for a while now with no issues. We upgraded to 0.8.5 two days ago,
Re: New node unable to stream (0.8.5)
Hinted handoff doesn't use streaming mode, so it doesn't care. (Streaming to Cassandra means sending raw sstable file ranges to another node. HH just uses the normal column-based write path.) On Thu, Sep 15, 2011 at 8:24 AM, Ethan Rowe et...@the-rowes.com wrote: Thanks, Jonathan. I'll try the workaround and see if that gets the streams flowing properly. As I mentioned before, we did not run scrub yet. What is the consequence of letting the streams from the hinted handoffs complete if scrub hasn't been run on these nodes? I'm currently running scrub on one node to get a sense of the time frame. Thanks again. - Ethan On Thu, Sep 15, 2011 at 9:09 AM, Jonathan Ellis jbel...@gmail.com wrote: That means we missed a place we needed to special-case for backwards compatibility -- the workaround is, add an empty encryption_options section to cassandra.yaml: encryption_options: internode_encryption: none keystore: conf/.keystore keystore_password: cassandra truststore: conf/.truststore truststore_password: cassandra Created https://issues.apache.org/jira/browse/CASSANDRA-3212 to fix this. On Thu, Sep 15, 2011 at 7:13 AM, Ethan Rowe et...@the-rowes.com wrote: Here's a typical log slice (not terribly informative, I fear): INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,106 AntiEntropyService.java (l ine 884) Performing streaming repair of 1003 ranges with /10.34.90.8 for (299 90798416657667504332586989223299634,54296681768153272037430773234349600451] INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,427 StreamOut.java (line 181) Stream context metadata [/mnt/cassandra/data/events_production/FitsByShip-g-1 0-Data.db sections=88 progress=0/11707163 - 0%, /mnt/cassandra/data/events_pr oduction/FitsByShip-g-11-Data.db sections=169 progress=0/6133240 - 0%, /mnt/c assandra/data/events_production/FitsByShip-g-6-Data.db sections=1 progress=0/ 6918814 - 0%, /mnt/cassandra/data/events_production/FitsByShip-g-12-Data.db s ections=260 progress=0/9091780 - 0%], 4 sstables. INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,428 StreamOutSession.java (lin e 174) Streaming to /10.34.90.8 ERROR [Thread-56] 2011-09-15 05:41:38,515 AbstractCassandraDaemon.java (line 139) Fatal exception in thread Thread[Thread-56,5,main] java.lang.NullPointerException at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpC onnection.java:174) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConn ection.java:114) Not sure if the exception is related to the outbound streaming above; other nodes are actively trying to stream to this node, so perhaps it comes from those and temporal adjacency to the outbound stream is just coincidental. I have other snippets that look basically identical to the above, except if I look at the logs to which this node is trying to stream, I see that it has concurrently opened a stream in the other direction, which could be the one that the exception pertains to. On Thu, Sep 15, 2011 at 7:41 AM, Sylvain Lebresne sylv...@datastax.com wrote: On Thu, Sep 15, 2011 at 1:16 PM, Ethan Rowe et...@the-rowes.com wrote: Hi. We've been running a 7-node cluster with RF 3, QUORUM reads/writes in our production environment for a few months. It's been consistently stable during this period, particularly once we got out maintenance strategy fully worked out (per node, one repair a week, one major compaction a week, the latter due to the nature of our data model and usage). While this cluster started, back in June or so, on the 0.7 series, it's been running 0.8.3 for a while now with no issues. We upgraded to 0.8.5 two days ago, having tested the upgrade in our staging cluster (with an otherwise identical configuration) previously and verified that our application's various use cases appeared successful. One of our nodes suffered a disk failure yesterday. We attempted to replace the dead node by placing a new node at OldNode.initial_token - 1 with auto_bootstrap on. A few things went awry from there: 1. We never saw the new node in bootstrap mode; it became available pretty much immediately upon joining the ring, and never reported a joining state. I did verify that auto_bootstrap was on. 2. I mistakenly ran repair on the new node rather than removetoken on the old node, due to a delightful mental error. The repair got nowhere fast, as it attempts to repair against the down node which throws an exception. So I interrupted the repair, restarted the node to clear any pending validation compactions, and... 3. Ran removetoken for the old node. 4. We let this run for some time and saw eventually that all the nodes appeared to be done various compactions and were stuck at streaming. Many
Re: New node unable to stream (0.8.5)
If you added the new node as a seed, it would ignore bootstrap mode. And bootstrap / repair *do* use streaming so you'll want to re-run repair post-scrub. (No need to re-bootstrap since you're repairing.) Scrub is a little less heavyweight than major compaction but same ballpark. It runs sstable-at-a-time so (as long as you haven't been in the habit of forcing majors) space should not be a concern. On Thu, Sep 15, 2011 at 8:40 AM, Ethan Rowe et...@the-rowes.com wrote: On Thu, Sep 15, 2011 at 9:21 AM, Jonathan Ellis jbel...@gmail.com wrote: Where did the data loss come in? The outcome of the analytical jobs run overnight while some of these repairs were (not) running is consistent with what I would expect if perhaps 20-30% of the source data was missing. Given the strong consistency model we're using, this is surprising to me, since the jobs did not report any read or write failures. I wonder if this is a consequence of the dead node missing and the new node being operational but having received basically none of its hinted handoff streams. Perhaps with streaming fixed the data will reappear, which would be a happy outcome, but if not, I can reimport the critical stuff from files. Scrub is safe to run in parallel. Is it somewhat analogous to a major compaction in terms of I/O impact, with perhaps less greedy use of disk space? On Thu, Sep 15, 2011 at 8:08 AM, Ethan Rowe et...@the-rowes.com wrote: After further review, I'm definitely going to scrub all the original nodes in the cluster. We've lost some data as a result of this situation. It can be restored, but the question is what to do with the problematic new node first. I don't particularly care about the data that's on it, since I'm going to re-import the critical data from files anyway, and then I can recreate derivative data afterwards. So it's purely a matter of getting the cluster healthy again as quickly as possible so I can begin that import process. Any issue with running scrubs on multiple nodes at a time, provided they aren't replication neighbors? On Thu, Sep 15, 2011 at 8:18 AM, Ethan Rowe et...@the-rowes.com wrote: I just noticed the following from one of Jonathan Ellis' messages yesterday: Added to NEWS: - After upgrading, run nodetool scrub against each node before running repair, moving nodes, or adding new ones. We did not do this, as it was not indicated as necessary in the news when we were dealing with the upgrade. So perhaps I need to scrub everything before going any further, though the question is what to do with the problematic node. Additionally, it would be helpful to know if scrub will affect the hinted handoffs that have accumulated, as these seem likely to be part of the set of failing streams. On Thu, Sep 15, 2011 at 8:13 AM, Ethan Rowe et...@the-rowes.com wrote: Here's a typical log slice (not terribly informative, I fear): INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,106 AntiEntropyService.java (l ine 884) Performing streaming repair of 1003 ranges with /10.34.90.8 for (299 90798416657667504332586989223299634,54296681768153272037430773234349600451] INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,427 StreamOut.java (line 181) Stream context metadata [/mnt/cassandra/data/events_production/FitsByShip-g-1 0-Data.db sections=88 progress=0/11707163 - 0%, /mnt/cassandra/data/events_pr oduction/FitsByShip-g-11-Data.db sections=169 progress=0/6133240 - 0%, /mnt/c assandra/data/events_production/FitsByShip-g-6-Data.db sections=1 progress=0/ 6918814 - 0%, /mnt/cassandra/data/events_production/FitsByShip-g-12-Data.db s ections=260 progress=0/9091780 - 0%], 4 sstables. INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,428 StreamOutSession.java (lin e 174) Streaming to /10.34.90.8 ERROR [Thread-56] 2011-09-15 05:41:38,515 AbstractCassandraDaemon.java (line 139) Fatal exception in thread Thread[Thread-56,5,main] java.lang.NullPointerException at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpC onnection.java:174) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConn ection.java:114) Not sure if the exception is related to the outbound streaming above; other nodes are actively trying to stream to this node, so perhaps it comes from those and temporal adjacency to the outbound stream is just coincidental. I have other snippets that look basically identical to the above, except if I look at the logs to which this node is trying to stream, I see that it has concurrently opened a stream in the other direction, which could be the one that the exception pertains to. On Thu, Sep 15, 2011 at 7:41 AM, Sylvain Lebresne sylv...@datastax.com wrote: On Thu, Sep 15, 2011 at 1:16 PM, Ethan Rowe et...@the-rowes.com wrote: Hi. We've been running a 7-node cluster
Re: New node unable to stream (0.8.5)
On Thu, Sep 15, 2011 at 10:03 AM, Jonathan Ellis jbel...@gmail.com wrote: If you added the new node as a seed, it would ignore bootstrap mode. And bootstrap / repair *do* use streaming so you'll want to re-run repair post-scrub. (No need to re-bootstrap since you're repairing.) Ah, of course. That's what happened; the chef recipe added the node to its own seed list, which is a problem I thought we'd fixed but apparently not. That definitely explains the bootstrap issue. But no matter, so long as the repairs can eventually run. Scrub is a little less heavyweight than major compaction but same ballpark. It runs sstable-at-a-time so (as long as you haven't been in the habit of forcing majors) space should not be a concern. Cool. We've deactivated all tasks against these nodes and will scrub them all in parallel, apply the encryption options you specified, and see where that gets us. Thanks for the assistance. - Ethan On Thu, Sep 15, 2011 at 8:40 AM, Ethan Rowe et...@the-rowes.com wrote: On Thu, Sep 15, 2011 at 9:21 AM, Jonathan Ellis jbel...@gmail.com wrote: Where did the data loss come in? The outcome of the analytical jobs run overnight while some of these repairs were (not) running is consistent with what I would expect if perhaps 20-30% of the source data was missing. Given the strong consistency model we're using, this is surprising to me, since the jobs did not report any read or write failures. I wonder if this is a consequence of the dead node missing and the new node being operational but having received basically none of its hinted handoff streams. Perhaps with streaming fixed the data will reappear, which would be a happy outcome, but if not, I can reimport the critical stuff from files. Scrub is safe to run in parallel. Is it somewhat analogous to a major compaction in terms of I/O impact, with perhaps less greedy use of disk space? On Thu, Sep 15, 2011 at 8:08 AM, Ethan Rowe et...@the-rowes.com wrote: After further review, I'm definitely going to scrub all the original nodes in the cluster. We've lost some data as a result of this situation. It can be restored, but the question is what to do with the problematic new node first. I don't particularly care about the data that's on it, since I'm going to re-import the critical data from files anyway, and then I can recreate derivative data afterwards. So it's purely a matter of getting the cluster healthy again as quickly as possible so I can begin that import process. Any issue with running scrubs on multiple nodes at a time, provided they aren't replication neighbors? On Thu, Sep 15, 2011 at 8:18 AM, Ethan Rowe et...@the-rowes.com wrote: I just noticed the following from one of Jonathan Ellis' messages yesterday: Added to NEWS: - After upgrading, run nodetool scrub against each node before running repair, moving nodes, or adding new ones. We did not do this, as it was not indicated as necessary in the news when we were dealing with the upgrade. So perhaps I need to scrub everything before going any further, though the question is what to do with the problematic node. Additionally, it would be helpful to know if scrub will affect the hinted handoffs that have accumulated, as these seem likely to be part of the set of failing streams. On Thu, Sep 15, 2011 at 8:13 AM, Ethan Rowe et...@the-rowes.com wrote: Here's a typical log slice (not terribly informative, I fear): INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,106 AntiEntropyService.java (l ine 884) Performing streaming repair of 1003 ranges with / 10.34.90.8 for (299 90798416657667504332586989223299634,54296681768153272037430773234349600451] INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,427 StreamOut.java (line 181) Stream context metadata [/mnt/cassandra/data/events_production/FitsByShip-g-1 0-Data.db sections=88 progress=0/11707163 - 0%, /mnt/cassandra/data/events_pr oduction/FitsByShip-g-11-Data.db sections=169 progress=0/6133240 - 0%, /mnt/c assandra/data/events_production/FitsByShip-g-6-Data.db sections=1 progress=0/ 6918814 - 0%, /mnt/cassandra/data/events_production/FitsByShip-g-12-Data.db s ections=260 progress=0/9091780 - 0%], 4 sstables. INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,428 StreamOutSession.java (lin e 174) Streaming to /10.34.90.8 ERROR [Thread-56] 2011-09-15 05:41:38,515 AbstractCassandraDaemon.java (line 139) Fatal exception in thread Thread[Thread-56,5,main] java.lang.NullPointerException at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpC onnection.java:174) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConn ection.java:114) Not sure if the exception is related to the outbound
Re: New node unable to stream (0.8.5)
Cool. We've deactivated all tasks against these nodes and will scrub them all in parallel, apply the encryption options you specified, and see where that gets us. Thanks for the assistance. To follow up: * We scrubbed all the nodes * We applied the encryption options specified * A repair is continuing (for about an hour so far, perhaps more) on the new, problematic node; it's successfully streaming data from its neighbors and has built up a roughly equivalent data volume on disk We'll see if the data is fully restored once this process completes. Even if it isn't, it seems likely that the cluster will be in a healthy state soon, so we can reimport as necessary and we'll be out of the woods. Now that I've said all that, something will inevitably go wrong, but until that happens, thanks again for the feedback. - Ethan On Thu, Sep 15, 2011 at 8:40 AM, Ethan Rowe et...@the-rowes.com wrote: On Thu, Sep 15, 2011 at 9:21 AM, Jonathan Ellis jbel...@gmail.com wrote: Where did the data loss come in? The outcome of the analytical jobs run overnight while some of these repairs were (not) running is consistent with what I would expect if perhaps 20-30% of the source data was missing. Given the strong consistency model we're using, this is surprising to me, since the jobs did not report any read or write failures. I wonder if this is a consequence of the dead node missing and the new node being operational but having received basically none of its hinted handoff streams. Perhaps with streaming fixed the data will reappear, which would be a happy outcome, but if not, I can reimport the critical stuff from files. Scrub is safe to run in parallel. Is it somewhat analogous to a major compaction in terms of I/O impact, with perhaps less greedy use of disk space? On Thu, Sep 15, 2011 at 8:08 AM, Ethan Rowe et...@the-rowes.com wrote: After further review, I'm definitely going to scrub all the original nodes in the cluster. We've lost some data as a result of this situation. It can be restored, but the question is what to do with the problematic new node first. I don't particularly care about the data that's on it, since I'm going to re-import the critical data from files anyway, and then I can recreate derivative data afterwards. So it's purely a matter of getting the cluster healthy again as quickly as possible so I can begin that import process. Any issue with running scrubs on multiple nodes at a time, provided they aren't replication neighbors? On Thu, Sep 15, 2011 at 8:18 AM, Ethan Rowe et...@the-rowes.com wrote: I just noticed the following from one of Jonathan Ellis' messages yesterday: Added to NEWS: - After upgrading, run nodetool scrub against each node before running repair, moving nodes, or adding new ones. We did not do this, as it was not indicated as necessary in the news when we were dealing with the upgrade. So perhaps I need to scrub everything before going any further, though the question is what to do with the problematic node. Additionally, it would be helpful to know if scrub will affect the hinted handoffs that have accumulated, as these seem likely to be part of the set of failing streams. On Thu, Sep 15, 2011 at 8:13 AM, Ethan Rowe et...@the-rowes.com wrote: Here's a typical log slice (not terribly informative, I fear): INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,106 AntiEntropyService.java (l ine 884) Performing streaming repair of 1003 ranges with / 10.34.90.8 for (299 90798416657667504332586989223299634,54296681768153272037430773234349600451] INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,427 StreamOut.java (line 181) Stream context metadata [/mnt/cassandra/data/events_production/FitsByShip-g-1 0-Data.db sections=88 progress=0/11707163 - 0%, /mnt/cassandra/data/events_pr oduction/FitsByShip-g-11-Data.db sections=169 progress=0/6133240 - 0%, /mnt/c assandra/data/events_production/FitsByShip-g-6-Data.db sections=1 progress=0/ 6918814 - 0%, /mnt/cassandra/data/events_production/FitsByShip-g-12-Data.db s ections=260 progress=0/9091780 - 0%], 4 sstables. INFO [AntiEntropyStage:2] 2011-09-15 05:41:36,428 StreamOutSession.java (lin e 174) Streaming to /10.34.90.8 ERROR [Thread-56] 2011-09-15 05:41:38,515 AbstractCassandraDaemon.java (line 139) Fatal exception in thread Thread[Thread-56,5,main] java.lang.NullPointerException at org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpC onnection.java:174) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConn ection.java:114) Not sure if the exception is related to the outbound streaming above; other nodes are actively trying to stream to this node, so perhaps it comes from