[ https://issues.apache.org/jira/browse/CASSANDRA-1840?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12970115#action_12970115 ]
Peter Schuller commented on CASSANDRA-1840: ------------------------------------------- I could reproduce it consistently. I tried once per node; in each case a restart was required. Still same after stopping all nodes and starting them from scratch. I will update to today's 0.7 branch and re-try. > nodetool move caused the moved node to drop itself from 'nodetool ring' > output; others think it's 'joining' > ----------------------------------------------------------------------------------------------------------- > > Key: CASSANDRA-1840 > URL: https://issues.apache.org/jira/browse/CASSANDRA-1840 > Project: Cassandra > Issue Type: Bug > Reporter: Peter Schuller > > I have a test cluster with three nodes on a very recent 0.7 (last few days > branch). It has very little data in it (so maybe timing can be an issue given > how fast operations complete). It was otherwise healthy; nodetool ring was > consistent on all nodes and I had just run some compactions and 'repair' > commands on all nodes repeatedly. > I had a single client doing some reads/writes of single columns; nothing > extreme (low load). > When I did a 'nodetool move' the node exited the ring, stopped responding to > thrift RPC, entered the ring again, and started accepting RPC requests via > thrift. It reports in the log that it is joined. > However, at this point 'nodetool ring' on the node I moved does *not* show > its own location in the ring, and other nodes show it as 'joining' (with the > new token, not the old token). I will include nodetool ring output and log > output below. > The situation was un-wedged by restarting the node that I had moved. After it > started and a few seconds passed, nodetool ring looked correct on the node in > question and other nodes now reported it as 'up' rather than 'joining'. > Moved node said post-move (.61 in the below pastes is the node that I moved): > Address Status State Load Owns Token > > > 110288156320304836825416347816186393502 > 78.31.15.204 Up Normal 224.34 KB 61.44% > 44678687293344048155696022135861768368 > 193.182.3.229 Up Normal 251.84 KB 38.56% > 110288156320304836825416347816186393502 > And the other two: > Address Status State Load Owns Token > > > 164957594472845753490452447750528540018 > 78.31.15.204 Up Normal 224.34 KB 29.31% > 44678687293344048155696022135861768368 > 193.182.3.229 Up Normal 251.84 KB 38.56% > 110288156320304836825416347816186393502 > 193.182.3.61 Up Joining 194.76 KB 32.13% > 164957594472845753490452447750528540018 > Address Status State Load Owns Token > > > 164957594472845753490452447750528540018 > 78.31.15.204 Up Normal > 224.34 KB 29.31% 44678687293344048155696022135861768368 > 193.182.3.229 Up Normal 251.84 KB 38.56% > 110288156320304836825416347816186393502 > 193.182.3.61 Up Joining 194.76 KB 32.13% > 164957594472845753490452447750528540018 > I'll try reproducing a few times, and also merge latest 0.7. > Here is some system log output from the node that got moved; it looks good to > me: > INFO [RMI TCP Connection(32)-193.182.3.61] 2010-12-10 09:38:17,560 > StorageService.java (line 455) Leaving: sleeping 30000 ms for pending range > setup > INFO [RMI TCP Connection(32)-193.182.3.61] 2010-12-10 09:38:47,564 > StorageService.java (line 455) Leaving: streaming data to other nodes > INFO [StreamStage:1] 2010-12-10 09:38:47,566 StreamOut.java (line 75) > Beginning transfer to /78.31.15.204 > INFO [StreamStage:1] 2010-12-10 09:38:47,566 StreamOut.java (line 98) > Flushing memtables for KeyspaceSlask... > INFO [StreamStage:1] 2010-12-10 09:38:47,567 ColumnFamilyStore.java (line > 639) switching in a fresh Memtable for KeyValue at > CommitLogContext(file='/var/lib/spotify-cassandra/slask/commitlog/CommitLog-1291973418600.log', > position=2573610) > INFO [StreamStage:1] 2010-12-10 09:38:47,567 ColumnFamilyStore.java (line > 943) Enqueuing flush of memtable-keyva...@1131602880(370711 bytes, 5533 > operations) > INFO [FlushWriter:1] 2010-12-10 09:38:47,567 Memtable.java (line 155) > Writing memtable-keyva...@1131602880(370711 bytes, 5533 operations) > INFO [FlushWriter:1] 2010-12-10 09:38:47,599 Memtable.java (line 162) > Completed flushing > /var/lib/spotify-cassandra/slask/data/KeyspaceSlask/KeyValue-e-68-Data.db > (15042 bytes) > INFO [StreamStage:1] 2010-12-10 09:38:47,601 StreamOut.java (line 171) > Stream context metadata > [/var/lib/spotify-cassandra/slask/data/KeyspaceSlask/KeyValue-e-67-Data.db/(0,10094) > progress=0/10094 - 0%, > /var/lib/spotify-cassandra/slask/data/KeyspaceSlask/KeyValue-e-68-Data.db/(0,10094) > progress=0/10094 - 0%], 2 sstables. > INFO [StreamStage:1] 2010-12-10 09:38:47,601 StreamOutSession.java (line > 175) Streaming to /78.31.15.204 > INFO [StreamStage:1] 2010-12-10 09:38:47,601 StreamOut.java (line 75) > Beginning transfer to /193.182.3.229 > INFO [StreamStage:1] 2010-12-10 09:38:47,602 StreamOut.java (line 98) > Flushing memtables for KeyspaceSlask... > INFO [StreamStage:1] 2010-12-10 09:38:47,602 ColumnFamilyStore.java (line > 639) switching in a fresh Memtable for KeyValue at > CommitLogContext(file='/var/lib/spotify-cassandra/slask/commitlog/CommitLog-1291973418600.log', > position=2573755) > INFO [StreamStage:1] 2010-12-10 09:38:47,602 ColumnFamilyStore.java (line > 943) Enqueuing flush of memtable-keyva...@1894688899(67 bytes, 1 operations) > INFO [FlushWriter:1] 2010-12-10 09:38:47,604 Memtable.java (line 155) > Writing memtable-keyva...@1894688899(67 bytes, 1 operations) > INFO [FlushWriter:1] 2010-12-10 09:38:47,635 Memtable.java (line 162) > Completed flushing > /var/lib/spotify-cassandra/slask/data/KeyspaceSlask/KeyValue-e-69-Data.db > (198 bytes) > INFO [StreamStage:1] 2010-12-10 09:38:47,637 StreamOut.java (line 171) > Stream context metadata > [/var/lib/spotify-cassandra/slask/data/KeyspaceSlask/KeyValue-e-67-Data.db/(10094,15042) > progress=0/4948 - 0%, > /var/lib/spotify-cassandra/slask/data/KeyspaceSlask/KeyValue-e-68-Data.db/(10094,15042) > progress=0/4948 - 0%], 3 sstables. > INFO [StreamStage:1] 2010-12-10 09:38:47,637 StreamOutSession.java (line > 175) Streaming to /193.182.3.229 > INFO [RMI TCP Connection(32)-193.182.3.61] 2010-12-10 09:38:49,734 > StorageService.java (line 1682) re-bootstrapping to new token > 164957594472845753490452447750528540018 > INFO [RMI TCP Connection(32)-193.182.3.61] 2010-12-10 09:38:49,735 > ColumnFamilyStore.java (line 639) switching in a fresh Memtable for > LocationInfo at > CommitLogContext(file='/var/lib/spotify-cassandra/slask/commitlog/CommitLog-1291973418600.log', > position=2576080) > INFO [RMI TCP Connection(32)-193.182.3.61] 2010-12-10 09:38:49,736 > ColumnFamilyStore.java (line 943) Enqueuing flush of > memtable-locationi...@578162504(53 bytes, 2 operations) > INFO [FlushWriter:1] 2010-12-10 09:38:49,737 Memtable.java (line 155) > Writing memtable-locationi...@578162504(53 bytes, 2 operations) > INFO [FlushWriter:1] 2010-12-10 09:38:49,865 Memtable.java (line 162) > Completed flushing > /var/lib/spotify-cassandra/slask/data/system/LocationInfo-e-17-Data.db (301 > bytes) > INFO [RMI TCP Connection(32)-193.182.3.61] 2010-12-10 09:38:49,866 > StorageService.java (line 455) Joining: sleeping 30000 ms for pending range > setup > INFO [RMI TCP Connection(32)-193.182.3.61] 2010-12-10 09:39:19,866 > StorageService.java (line 455) Bootstrapping > INFO [CompactionExecutor:1] 2010-12-10 09:39:19,960 SSTableReader.java (line > 170) Opening /var/lib/spotify-cassandra/slask/data/KeyspaceSlask/KeyValue-e-70 > INFO [CompactionExecutor:1] 2010-12-10 09:39:19,981 SSTableReader.java (line > 170) Opening /var/lib/spotify-cassandra/slask/data/KeyspaceSlask/KeyValue-e-71 > INFO [Thread-71] 2010-12-10 09:39:19,983 StreamInSession.java (line 160) > Finished streaming session 258244662990017 from /193.182.3.229 > INFO [RMI TCP Connection(32)-193.182.3.61] 2010-12-10 09:39:19,983 > StorageService.java (line 249) Bootstrap/move completed! Now serving reads. > So based on the output the node certainly claims to have finished the entire > bootstrapping procedure and started serving reads (which it is; reads are > working against it). -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.