[
https://issues.apache.org/jira/browse/CASSANDRA-5133?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13547590#comment-13547590
]
Terrance Shepherd commented on CASSANDRA-5133:
----------------------------------------------
Will trying to help Matt I can verify, that GossipPropertySnitch does not seem
to work as expected.
Below is a detailed explanation of what happened to me while working with Matt,
3 nodes A, B, C.
setup:
Node A -> seeds="node A ip address"
Node B -> seeds="node A ip address"
Node C -> seeds="node A ip address"
1) start node A and let it fully start up. ( first node needs to be a seed of
itself to start )
2) start node B and let it fully join the ring.
3) start node C and let it fully join the ring.
4) nodetool ring shows all nodes up
5) stop node C and update the cassandra-rackdc.protperites file to rack=2
6) start node C ( fails to start error about cant find DC for node A, maybe
GossipPropertySnitch wont let a seed node talk about itself? )
7) stop node A and update seeds="node B ip address", to try and solve question
above
8) start node A ( fails to start cant find DC of node C )
9) stuck not being able to start node A and C
> Nodes can't rejoin after stopping, when using GossipingPropertyFileSnitch
> -------------------------------------------------------------------------
>
> Key: CASSANDRA-5133
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5133
> Project: Cassandra
> Issue Type: Bug
> Affects Versions: 1.2.0
> Environment: 3 ec2 instances (CentOS 6.3; java 1.7.0_05; Cassandra
> 1.2)
> Reporter: Matt Jurik
> Assignee: Brandon Williams
>
> I can establish a 1.2 ring with GossipingPropertyFileSnitch, but after
> killing a node and restarting it, the node cannot rejoin.
> [Node 1] ./bin/cassandra -f
> [Node 2] ./bin/cassandra -f
> [Node 3] ./bin/cassandra -f
> [Node 1] ./bin/nodetool ring
> ... ok ...
> [Node 1] ^C
> ... node shutdown ...
> [Node 1] ./bin/cassandra -f
> ... Exception! ...
> ERROR 05:45:39,305 Exception encountered during startup
> java.lang.RuntimeException: Could not retrieve DC for /10.114.18.51 from
> gossip and PFS compatibility is disabled
> at
> org.apache.cassandra.locator.GossipingPropertyFileSnitch.getDatacenter(GossipingPropertyFileSnitch.java:109)
> at
> org.apache.cassandra.locator.DynamicEndpointSnitch.getDatacenter(DynamicEndpointSnitch.java:127)
> at
> org.apache.cassandra.locator.TokenMetadata$Topology.addEndpoint(TokenMetadata.java:1040)
> at
> org.apache.cassandra.locator.TokenMetadata.updateNormalTokens(TokenMetadata.java:185)
> at
> org.apache.cassandra.locator.TokenMetadata.updateNormalTokens(TokenMetadata.java:157)
> at
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:441)
> at
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:397)
> at
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:309)
> at
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:397)
> at
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:440)
> java.lang.RuntimeException: Could not retrieve DC for /10.114.18.51 from
> gossip and PFS compatibility is disabled
> at
> org.apache.cassandra.locator.GossipingPropertyFileSnitch.getDatacenter(GossipingPropertyFileSnitch.java:109)
> at
> org.apache.cassandra.locator.DynamicEndpointSnitch.getDatacenter(DynamicEndpointSnitch.java:127)
> at
> org.apache.cassandra.locator.TokenMetadata$Topology.addEndpoint(TokenMetadata.java:1040)
> at
> org.apache.cassandra.locator.TokenMetadata.updateNormalTokens(TokenMetadata.java:185)
> at
> org.apache.cassandra.locator.TokenMetadata.updateNormalTokens(TokenMetadata.java:157)
> at
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:441)
> at
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:397)
> at
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:309)
> at
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:397)
> at
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:440)
> Full environment + exceptions + stacktraces:
> https://gist.github.com/1e74ff02c2d4f622ce8f
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira