You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Gary Dusbabek (JIRA)" <ji...@apache.org> on 2010/06/23 14:40:51 UTC

[jira] Created: (CASSANDRA-1221) loadbalance operation never completes on a 3 node cluster

loadbalance operation never completes on a 3 node cluster
---------------------------------------------------------

                 Key: CASSANDRA-1221
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1221
             Project: Cassandra
          Issue Type: Bug
    Affects Versions: 0.7
            Reporter: Gary Dusbabek
            Assignee: Gary Dusbabek
             Fix For: 0.7


Arya Goudarzi reports:

Please confirm if this is an issue and should be reported or I am doing something wrong. I could not find anything relevant on JIRA:

Playing with 0.7 nightly (today's build), I setup a 3 node cluster this way:

 - Added one node;
 - Loaded default schema with RF 1 from YAML using JMX;
 - Loaded 2M keys using py_stress;
 - Bootstrapped a second node;
 - Cleaned up the first node;
 - Bootstrapped a third node;
 - Cleaned up the second node;

I got the following ring:

Address       Status     Load          Range                                      Ring
                                      154293670372423273273390365393543806425
10.50.26.132  Up         518.63 MB     69164917636305877859094619660693892452     |<--|
10.50.26.134  Up         234.8 MB      111685517405103688771527967027648896391    |   |
10.50.26.133  Up         235.26 MB     154293670372423273273390365393543806425    |-->|

Now I ran:

nodetool --host 10.50.26.132 loadbalance

It's been going for a while. I checked the streams

nodetool --host 10.50.26.134 streams
Mode: Normal
Not sending any streams.
Streaming from: /10.50.26.132
  Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-3-Data.db/[(0,22206096), (22206096,27271682)]
  Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-4-Data.db/[(0,15180462), (15180462,18656982)]
  Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-5-Data.db/[(0,353139829), (353139829,433883659)]
  Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-6-Data.db/[(0,366336059), (366336059,450095320)]

nodetool --host 10.50.26.132 streams
Mode: Leaving: streaming data to other nodes
Streaming to: /10.50.26.134
  /var/lib/cassandra/data/Keyspace1/Standard1-d-48-Data.db/[(0,366336059), (366336059,450095320)]
Not receiving any streams.

These have been going for the past 2 hours.

I see in the logs of the node with 134 IP address and I saw this:

INFO [GOSSIP_STAGE:1] 2010-06-22 16:30:54,679 StorageService.java (line 603) Will not change my token ownership to /10.50.26.132

So, to my understanding from wikis loadbalance supposed to decommission and re-bootstrap again by sending its tokens to other nodes and then bootstrap again. It's been stuck in streaming for the past 2 hours and the size of ring has not changed. The log in the first node says it has started streaming for the past hours:

INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,255 StreamOut.java (line 72) Beginning transfer process to /10.50.26.134 for ranges (154293670372423273273390365393543806425,69164917636305877859094619660693892452]
 INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,255 StreamOut.java (line 82) Flushing memtables for Keyspace1...
 INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,266 StreamOut.java (line 128) Stream context metadata [/var/lib/cassandra/data/Keyspace1/Standard1-d-48-Data.db/[(0,366336059), (366336059,450095320)]] 1 sstables.
 INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,267 StreamOut.java (line 135) Sending a stream initiate message to /10.50.26.134 ...
 INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,267 StreamOut.java (line 140) Waiting for transfer to /10.50.26.134 to complete
 INFO [FLUSH-TIMER] 2010-06-22 17:36:53,370 ColumnFamilyStore.java (line 359) LocationInfo has reached its threshold; switching in a fresh Memtable at CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1277249454413.log', position=720)
 INFO [FLUSH-TIMER] 2010-06-22 17:36:53,370 ColumnFamilyStore.java (line 622) Enqueuing flush of Memtable(LocationInfo)@1637794189
 INFO [FLUSH-WRITER-POOL:1] 2010-06-22 17:36:53,370 Memtable.java (line 149) Writing Memtable(LocationInfo)@1637794189
 INFO [FLUSH-WRITER-POOL:1] 2010-06-22 17:36:53,528 Memtable.java (line 163) Completed flushing /var/lib/cassandra/data/system/LocationInfo-d-9-Data.db
 INFO [MEMTABLE-POST-FLUSHER:1] 2010-06-22 17:36:53,529 ColumnFamilyStore.java (line 374) Discarding 1000


Nothing more after this line.

Am I doing something wrong?

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (CASSANDRA-1221) loadbalance operation never completes on a 3 node cluster

Posted by "Gary Dusbabek (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-1221?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12888913#action_12888913 ] 

Gary Dusbabek commented on CASSANDRA-1221:
------------------------------------------

Thanks Arya. I can reproduce this.  Now just to fix it.

> loadbalance operation never completes on a 3 node cluster
> ---------------------------------------------------------
>
>                 Key: CASSANDRA-1221
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1221
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 0.7
>            Reporter: Gary Dusbabek
>            Assignee: Gary Dusbabek
>             Fix For: 0.7
>
>         Attachments: system1.log, system2.log, system3.log
>
>
> Arya Goudarzi reports:
> Please confirm if this is an issue and should be reported or I am doing something wrong. I could not find anything relevant on JIRA:
> Playing with 0.7 nightly (today's build), I setup a 3 node cluster this way:
>  - Added one node;
>  - Loaded default schema with RF 1 from YAML using JMX;
>  - Loaded 2M keys using py_stress;
>  - Bootstrapped a second node;
>  - Cleaned up the first node;
>  - Bootstrapped a third node;
>  - Cleaned up the second node;
> I got the following ring:
> Address       Status     Load          Range                                      Ring
>                                       154293670372423273273390365393543806425
> 10.50.26.132  Up         518.63 MB     69164917636305877859094619660693892452     |<--|
> 10.50.26.134  Up         234.8 MB      111685517405103688771527967027648896391    |   |
> 10.50.26.133  Up         235.26 MB     154293670372423273273390365393543806425    |-->|
> Now I ran:
> nodetool --host 10.50.26.132 loadbalance
> It's been going for a while. I checked the streams
> nodetool --host 10.50.26.134 streams
> Mode: Normal
> Not sending any streams.
> Streaming from: /10.50.26.132
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-3-Data.db/[(0,22206096), (22206096,27271682)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-4-Data.db/[(0,15180462), (15180462,18656982)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-5-Data.db/[(0,353139829), (353139829,433883659)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-6-Data.db/[(0,366336059), (366336059,450095320)]
> nodetool --host 10.50.26.132 streams
> Mode: Leaving: streaming data to other nodes
> Streaming to: /10.50.26.134
>   /var/lib/cassandra/data/Keyspace1/Standard1-d-48-Data.db/[(0,366336059), (366336059,450095320)]
> Not receiving any streams.
> These have been going for the past 2 hours.
> I see in the logs of the node with 134 IP address and I saw this:
> INFO [GOSSIP_STAGE:1] 2010-06-22 16:30:54,679 StorageService.java (line 603) Will not change my token ownership to /10.50.26.132
> So, to my understanding from wikis loadbalance supposed to decommission and re-bootstrap again by sending its tokens to other nodes and then bootstrap again. It's been stuck in streaming for the past 2 hours and the size of ring has not changed. The log in the first node says it has started streaming for the past hours:
> INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,255 StreamOut.java (line 72) Beginning transfer process to /10.50.26.134 for ranges (154293670372423273273390365393543806425,69164917636305877859094619660693892452]
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,255 StreamOut.java (line 82) Flushing memtables for Keyspace1...
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,266 StreamOut.java (line 128) Stream context metadata [/var/lib/cassandra/data/Keyspace1/Standard1-d-48-Data.db/[(0,366336059), (366336059,450095320)]] 1 sstables.
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,267 StreamOut.java (line 135) Sending a stream initiate message to /10.50.26.134 ...
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,267 StreamOut.java (line 140) Waiting for transfer to /10.50.26.134 to complete
>  INFO [FLUSH-TIMER] 2010-06-22 17:36:53,370 ColumnFamilyStore.java (line 359) LocationInfo has reached its threshold; switching in a fresh Memtable at CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1277249454413.log', position=720)
>  INFO [FLUSH-TIMER] 2010-06-22 17:36:53,370 ColumnFamilyStore.java (line 622) Enqueuing flush of Memtable(LocationInfo)@1637794189
>  INFO [FLUSH-WRITER-POOL:1] 2010-06-22 17:36:53,370 Memtable.java (line 149) Writing Memtable(LocationInfo)@1637794189
>  INFO [FLUSH-WRITER-POOL:1] 2010-06-22 17:36:53,528 Memtable.java (line 163) Completed flushing /var/lib/cassandra/data/system/LocationInfo-d-9-Data.db
>  INFO [MEMTABLE-POST-FLUSHER:1] 2010-06-22 17:36:53,529 ColumnFamilyStore.java (line 374) Discarding 1000
> Nothing more after this line.
> Am I doing something wrong?

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (CASSANDRA-1221) loadbalance operation never completes on a 3 node cluster

Posted by "Gary Dusbabek (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-1221?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Gary Dusbabek updated CASSANDRA-1221:
-------------------------------------

    Attachment: 0001-Gossiper-and-FD-never-called-MS.convict-to-shut-down.patch

> loadbalance operation never completes on a 3 node cluster
> ---------------------------------------------------------
>
>                 Key: CASSANDRA-1221
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1221
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 0.7
>            Reporter: Gary Dusbabek
>            Assignee: Gary Dusbabek
>             Fix For: 0.7
>
>         Attachments: 0001-Gossiper-and-FD-never-called-MS.convict-to-shut-down.patch, system1.log, system2.log, system3.log
>
>
> Arya Goudarzi reports:
> Please confirm if this is an issue and should be reported or I am doing something wrong. I could not find anything relevant on JIRA:
> Playing with 0.7 nightly (today's build), I setup a 3 node cluster this way:
>  - Added one node;
>  - Loaded default schema with RF 1 from YAML using JMX;
>  - Loaded 2M keys using py_stress;
>  - Bootstrapped a second node;
>  - Cleaned up the first node;
>  - Bootstrapped a third node;
>  - Cleaned up the second node;
> I got the following ring:
> Address       Status     Load          Range                                      Ring
>                                       154293670372423273273390365393543806425
> 10.50.26.132  Up         518.63 MB     69164917636305877859094619660693892452     |<--|
> 10.50.26.134  Up         234.8 MB      111685517405103688771527967027648896391    |   |
> 10.50.26.133  Up         235.26 MB     154293670372423273273390365393543806425    |-->|
> Now I ran:
> nodetool --host 10.50.26.132 loadbalance
> It's been going for a while. I checked the streams
> nodetool --host 10.50.26.134 streams
> Mode: Normal
> Not sending any streams.
> Streaming from: /10.50.26.132
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-3-Data.db/[(0,22206096), (22206096,27271682)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-4-Data.db/[(0,15180462), (15180462,18656982)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-5-Data.db/[(0,353139829), (353139829,433883659)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-6-Data.db/[(0,366336059), (366336059,450095320)]
> nodetool --host 10.50.26.132 streams
> Mode: Leaving: streaming data to other nodes
> Streaming to: /10.50.26.134
>   /var/lib/cassandra/data/Keyspace1/Standard1-d-48-Data.db/[(0,366336059), (366336059,450095320)]
> Not receiving any streams.
> These have been going for the past 2 hours.
> I see in the logs of the node with 134 IP address and I saw this:
> INFO [GOSSIP_STAGE:1] 2010-06-22 16:30:54,679 StorageService.java (line 603) Will not change my token ownership to /10.50.26.132
> So, to my understanding from wikis loadbalance supposed to decommission and re-bootstrap again by sending its tokens to other nodes and then bootstrap again. It's been stuck in streaming for the past 2 hours and the size of ring has not changed. The log in the first node says it has started streaming for the past hours:
> INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,255 StreamOut.java (line 72) Beginning transfer process to /10.50.26.134 for ranges (154293670372423273273390365393543806425,69164917636305877859094619660693892452]
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,255 StreamOut.java (line 82) Flushing memtables for Keyspace1...
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,266 StreamOut.java (line 128) Stream context metadata [/var/lib/cassandra/data/Keyspace1/Standard1-d-48-Data.db/[(0,366336059), (366336059,450095320)]] 1 sstables.
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,267 StreamOut.java (line 135) Sending a stream initiate message to /10.50.26.134 ...
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,267 StreamOut.java (line 140) Waiting for transfer to /10.50.26.134 to complete
>  INFO [FLUSH-TIMER] 2010-06-22 17:36:53,370 ColumnFamilyStore.java (line 359) LocationInfo has reached its threshold; switching in a fresh Memtable at CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1277249454413.log', position=720)
>  INFO [FLUSH-TIMER] 2010-06-22 17:36:53,370 ColumnFamilyStore.java (line 622) Enqueuing flush of Memtable(LocationInfo)@1637794189
>  INFO [FLUSH-WRITER-POOL:1] 2010-06-22 17:36:53,370 Memtable.java (line 149) Writing Memtable(LocationInfo)@1637794189
>  INFO [FLUSH-WRITER-POOL:1] 2010-06-22 17:36:53,528 Memtable.java (line 163) Completed flushing /var/lib/cassandra/data/system/LocationInfo-d-9-Data.db
>  INFO [MEMTABLE-POST-FLUSHER:1] 2010-06-22 17:36:53,529 ColumnFamilyStore.java (line 374) Discarding 1000
> Nothing more after this line.
> Am I doing something wrong?

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (CASSANDRA-1221) loadbalance operation never completes on a 3 node cluster

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-1221?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12890295#action_12890295 ] 

Jonathan Ellis commented on CASSANDRA-1221:
-------------------------------------------

+1

(but fix brace placement in onDead please)

> loadbalance operation never completes on a 3 node cluster
> ---------------------------------------------------------
>
>                 Key: CASSANDRA-1221
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1221
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 0.7
>            Reporter: Gary Dusbabek
>            Assignee: Gary Dusbabek
>             Fix For: 0.7
>
>         Attachments: 0.6-conviction-fix.diff, 0001-Gossiper-and-FD-never-called-MS.convict-to-shut-down.patch, system1.log, system2.log, system3.log
>
>
> Arya Goudarzi reports:
> Please confirm if this is an issue and should be reported or I am doing something wrong. I could not find anything relevant on JIRA:
> Playing with 0.7 nightly (today's build), I setup a 3 node cluster this way:
>  - Added one node;
>  - Loaded default schema with RF 1 from YAML using JMX;
>  - Loaded 2M keys using py_stress;
>  - Bootstrapped a second node;
>  - Cleaned up the first node;
>  - Bootstrapped a third node;
>  - Cleaned up the second node;
> I got the following ring:
> Address       Status     Load          Range                                      Ring
>                                       154293670372423273273390365393543806425
> 10.50.26.132  Up         518.63 MB     69164917636305877859094619660693892452     |<--|
> 10.50.26.134  Up         234.8 MB      111685517405103688771527967027648896391    |   |
> 10.50.26.133  Up         235.26 MB     154293670372423273273390365393543806425    |-->|
> Now I ran:
> nodetool --host 10.50.26.132 loadbalance
> It's been going for a while. I checked the streams
> nodetool --host 10.50.26.134 streams
> Mode: Normal
> Not sending any streams.
> Streaming from: /10.50.26.132
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-3-Data.db/[(0,22206096), (22206096,27271682)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-4-Data.db/[(0,15180462), (15180462,18656982)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-5-Data.db/[(0,353139829), (353139829,433883659)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-6-Data.db/[(0,366336059), (366336059,450095320)]
> nodetool --host 10.50.26.132 streams
> Mode: Leaving: streaming data to other nodes
> Streaming to: /10.50.26.134
>   /var/lib/cassandra/data/Keyspace1/Standard1-d-48-Data.db/[(0,366336059), (366336059,450095320)]
> Not receiving any streams.
> These have been going for the past 2 hours.
> I see in the logs of the node with 134 IP address and I saw this:
> INFO [GOSSIP_STAGE:1] 2010-06-22 16:30:54,679 StorageService.java (line 603) Will not change my token ownership to /10.50.26.132
> So, to my understanding from wikis loadbalance supposed to decommission and re-bootstrap again by sending its tokens to other nodes and then bootstrap again. It's been stuck in streaming for the past 2 hours and the size of ring has not changed. The log in the first node says it has started streaming for the past hours:
> INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,255 StreamOut.java (line 72) Beginning transfer process to /10.50.26.134 for ranges (154293670372423273273390365393543806425,69164917636305877859094619660693892452]
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,255 StreamOut.java (line 82) Flushing memtables for Keyspace1...
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,266 StreamOut.java (line 128) Stream context metadata [/var/lib/cassandra/data/Keyspace1/Standard1-d-48-Data.db/[(0,366336059), (366336059,450095320)]] 1 sstables.
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,267 StreamOut.java (line 135) Sending a stream initiate message to /10.50.26.134 ...
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,267 StreamOut.java (line 140) Waiting for transfer to /10.50.26.134 to complete
>  INFO [FLUSH-TIMER] 2010-06-22 17:36:53,370 ColumnFamilyStore.java (line 359) LocationInfo has reached its threshold; switching in a fresh Memtable at CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1277249454413.log', position=720)
>  INFO [FLUSH-TIMER] 2010-06-22 17:36:53,370 ColumnFamilyStore.java (line 622) Enqueuing flush of Memtable(LocationInfo)@1637794189
>  INFO [FLUSH-WRITER-POOL:1] 2010-06-22 17:36:53,370 Memtable.java (line 149) Writing Memtable(LocationInfo)@1637794189
>  INFO [FLUSH-WRITER-POOL:1] 2010-06-22 17:36:53,528 Memtable.java (line 163) Completed flushing /var/lib/cassandra/data/system/LocationInfo-d-9-Data.db
>  INFO [MEMTABLE-POST-FLUSHER:1] 2010-06-22 17:36:53,529 ColumnFamilyStore.java (line 374) Discarding 1000
> Nothing more after this line.
> Am I doing something wrong?

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (CASSANDRA-1221) loadbalance operation never completes on a 3 node cluster

Posted by "Gary Dusbabek (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-1221?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12888560#action_12888560 ] 

Gary Dusbabek commented on CASSANDRA-1221:
------------------------------------------

Arya,  can you supply the nodetool commands you are using that constitute "cleanup"?  I've tried a few times now and can't get the failure you describe.  In your latest test was .132 the second or third node booted?

> loadbalance operation never completes on a 3 node cluster
> ---------------------------------------------------------
>
>                 Key: CASSANDRA-1221
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1221
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 0.7
>            Reporter: Gary Dusbabek
>            Assignee: Gary Dusbabek
>             Fix For: 0.7
>
>
> Arya Goudarzi reports:
> Please confirm if this is an issue and should be reported or I am doing something wrong. I could not find anything relevant on JIRA:
> Playing with 0.7 nightly (today's build), I setup a 3 node cluster this way:
>  - Added one node;
>  - Loaded default schema with RF 1 from YAML using JMX;
>  - Loaded 2M keys using py_stress;
>  - Bootstrapped a second node;
>  - Cleaned up the first node;
>  - Bootstrapped a third node;
>  - Cleaned up the second node;
> I got the following ring:
> Address       Status     Load          Range                                      Ring
>                                       154293670372423273273390365393543806425
> 10.50.26.132  Up         518.63 MB     69164917636305877859094619660693892452     |<--|
> 10.50.26.134  Up         234.8 MB      111685517405103688771527967027648896391    |   |
> 10.50.26.133  Up         235.26 MB     154293670372423273273390365393543806425    |-->|
> Now I ran:
> nodetool --host 10.50.26.132 loadbalance
> It's been going for a while. I checked the streams
> nodetool --host 10.50.26.134 streams
> Mode: Normal
> Not sending any streams.
> Streaming from: /10.50.26.132
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-3-Data.db/[(0,22206096), (22206096,27271682)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-4-Data.db/[(0,15180462), (15180462,18656982)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-5-Data.db/[(0,353139829), (353139829,433883659)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-6-Data.db/[(0,366336059), (366336059,450095320)]
> nodetool --host 10.50.26.132 streams
> Mode: Leaving: streaming data to other nodes
> Streaming to: /10.50.26.134
>   /var/lib/cassandra/data/Keyspace1/Standard1-d-48-Data.db/[(0,366336059), (366336059,450095320)]
> Not receiving any streams.
> These have been going for the past 2 hours.
> I see in the logs of the node with 134 IP address and I saw this:
> INFO [GOSSIP_STAGE:1] 2010-06-22 16:30:54,679 StorageService.java (line 603) Will not change my token ownership to /10.50.26.132
> So, to my understanding from wikis loadbalance supposed to decommission and re-bootstrap again by sending its tokens to other nodes and then bootstrap again. It's been stuck in streaming for the past 2 hours and the size of ring has not changed. The log in the first node says it has started streaming for the past hours:
> INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,255 StreamOut.java (line 72) Beginning transfer process to /10.50.26.134 for ranges (154293670372423273273390365393543806425,69164917636305877859094619660693892452]
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,255 StreamOut.java (line 82) Flushing memtables for Keyspace1...
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,266 StreamOut.java (line 128) Stream context metadata [/var/lib/cassandra/data/Keyspace1/Standard1-d-48-Data.db/[(0,366336059), (366336059,450095320)]] 1 sstables.
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,267 StreamOut.java (line 135) Sending a stream initiate message to /10.50.26.134 ...
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,267 StreamOut.java (line 140) Waiting for transfer to /10.50.26.134 to complete
>  INFO [FLUSH-TIMER] 2010-06-22 17:36:53,370 ColumnFamilyStore.java (line 359) LocationInfo has reached its threshold; switching in a fresh Memtable at CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1277249454413.log', position=720)
>  INFO [FLUSH-TIMER] 2010-06-22 17:36:53,370 ColumnFamilyStore.java (line 622) Enqueuing flush of Memtable(LocationInfo)@1637794189
>  INFO [FLUSH-WRITER-POOL:1] 2010-06-22 17:36:53,370 Memtable.java (line 149) Writing Memtable(LocationInfo)@1637794189
>  INFO [FLUSH-WRITER-POOL:1] 2010-06-22 17:36:53,528 Memtable.java (line 163) Completed flushing /var/lib/cassandra/data/system/LocationInfo-d-9-Data.db
>  INFO [MEMTABLE-POST-FLUSHER:1] 2010-06-22 17:36:53,529 ColumnFamilyStore.java (line 374) Discarding 1000
> Nothing more after this line.
> Am I doing something wrong?

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (CASSANDRA-1221) loadbalance operation never completes on a 3 node cluster

Posted by "Arya Goudarzi (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-1221?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Arya Goudarzi updated CASSANDRA-1221:
-------------------------------------

    Attachment: system1.log
                system2.log
                system3.log

Cassandra System Logs for node 1-3

> loadbalance operation never completes on a 3 node cluster
> ---------------------------------------------------------
>
>                 Key: CASSANDRA-1221
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1221
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 0.7
>            Reporter: Gary Dusbabek
>            Assignee: Gary Dusbabek
>             Fix For: 0.7
>
>         Attachments: system1.log, system2.log, system3.log
>
>
> Arya Goudarzi reports:
> Please confirm if this is an issue and should be reported or I am doing something wrong. I could not find anything relevant on JIRA:
> Playing with 0.7 nightly (today's build), I setup a 3 node cluster this way:
>  - Added one node;
>  - Loaded default schema with RF 1 from YAML using JMX;
>  - Loaded 2M keys using py_stress;
>  - Bootstrapped a second node;
>  - Cleaned up the first node;
>  - Bootstrapped a third node;
>  - Cleaned up the second node;
> I got the following ring:
> Address       Status     Load          Range                                      Ring
>                                       154293670372423273273390365393543806425
> 10.50.26.132  Up         518.63 MB     69164917636305877859094619660693892452     |<--|
> 10.50.26.134  Up         234.8 MB      111685517405103688771527967027648896391    |   |
> 10.50.26.133  Up         235.26 MB     154293670372423273273390365393543806425    |-->|
> Now I ran:
> nodetool --host 10.50.26.132 loadbalance
> It's been going for a while. I checked the streams
> nodetool --host 10.50.26.134 streams
> Mode: Normal
> Not sending any streams.
> Streaming from: /10.50.26.132
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-3-Data.db/[(0,22206096), (22206096,27271682)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-4-Data.db/[(0,15180462), (15180462,18656982)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-5-Data.db/[(0,353139829), (353139829,433883659)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-6-Data.db/[(0,366336059), (366336059,450095320)]
> nodetool --host 10.50.26.132 streams
> Mode: Leaving: streaming data to other nodes
> Streaming to: /10.50.26.134
>   /var/lib/cassandra/data/Keyspace1/Standard1-d-48-Data.db/[(0,366336059), (366336059,450095320)]
> Not receiving any streams.
> These have been going for the past 2 hours.
> I see in the logs of the node with 134 IP address and I saw this:
> INFO [GOSSIP_STAGE:1] 2010-06-22 16:30:54,679 StorageService.java (line 603) Will not change my token ownership to /10.50.26.132
> So, to my understanding from wikis loadbalance supposed to decommission and re-bootstrap again by sending its tokens to other nodes and then bootstrap again. It's been stuck in streaming for the past 2 hours and the size of ring has not changed. The log in the first node says it has started streaming for the past hours:
> INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,255 StreamOut.java (line 72) Beginning transfer process to /10.50.26.134 for ranges (154293670372423273273390365393543806425,69164917636305877859094619660693892452]
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,255 StreamOut.java (line 82) Flushing memtables for Keyspace1...
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,266 StreamOut.java (line 128) Stream context metadata [/var/lib/cassandra/data/Keyspace1/Standard1-d-48-Data.db/[(0,366336059), (366336059,450095320)]] 1 sstables.
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,267 StreamOut.java (line 135) Sending a stream initiate message to /10.50.26.134 ...
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,267 StreamOut.java (line 140) Waiting for transfer to /10.50.26.134 to complete
>  INFO [FLUSH-TIMER] 2010-06-22 17:36:53,370 ColumnFamilyStore.java (line 359) LocationInfo has reached its threshold; switching in a fresh Memtable at CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1277249454413.log', position=720)
>  INFO [FLUSH-TIMER] 2010-06-22 17:36:53,370 ColumnFamilyStore.java (line 622) Enqueuing flush of Memtable(LocationInfo)@1637794189
>  INFO [FLUSH-WRITER-POOL:1] 2010-06-22 17:36:53,370 Memtable.java (line 149) Writing Memtable(LocationInfo)@1637794189
>  INFO [FLUSH-WRITER-POOL:1] 2010-06-22 17:36:53,528 Memtable.java (line 163) Completed flushing /var/lib/cassandra/data/system/LocationInfo-d-9-Data.db
>  INFO [MEMTABLE-POST-FLUSHER:1] 2010-06-22 17:36:53,529 ColumnFamilyStore.java (line 374) Discarding 1000
> Nothing more after this line.
> Am I doing something wrong?

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (CASSANDRA-1221) loadbalance operation never completes on a 3 node cluster

Posted by "Arya Goudarzi (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-1221?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12888116#action_12888116 ] 

Arya Goudarzi commented on CASSANDRA-1221:
------------------------------------------

Hi Gary,

I was able to reproduce this using today's nightly build. This time i used a smaller data set (500000 keys) and I got the following:

[agoudarzi@cas-test3 scripts]$ nodetool --host 10.50.26.132 ring   
Address         Status State   Load            Token                                       
                                       160348796167900510561059505917619274541    
10.50.26.134    Up     Normal  116.98 MB       32717880524093094169411234083126184860      
10.50.26.132    Up     Leaving 58.58 MB        75101027859180840627831025901565139619      
10.50.26.133    Up     Normal  117.09 MB       160348796167900510561059505917619274541    

[agoudarzi@cas-test3 scripts]$ nodetool --host 10.50.26.132 streams
Mode: Leaving: streaming data to other nodes
Streaming to: /10.50.26.133
   /var/lib/cassandra/data/Keyspace1/Standard1-d-17-Data.db/[(0,54080834)]
Not receiving any streams.
[agoudarzi@cas-test3 scripts]$ nodetool --host 10.50.26.133 streams
Mode: Normal
Not sending any streams.
Not receiving any streams.

>From the logs of 10.50.26.132 it seams that it tried to tell 10.50.26.133 to claim its stream:

INFO [STREAM-STAGE:1] 2010-07-13 16:50:35,994 StreamOut.java (line 135) Sending a stream initiate message to /10.50.26.133 ...
INFO [STREAM-STAGE:1] 2010-07-13 16:50:35,994 StreamOut.java (line 140) Waiting for transfer to /10.50.26.133 to complete

But nothing in 133's log acknowledges the receipt of the request from 132 and as you see above it shows that it is getting no streams and this has been going for the past hour or so.

-Arya



> loadbalance operation never completes on a 3 node cluster
> ---------------------------------------------------------
>
>                 Key: CASSANDRA-1221
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1221
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 0.7
>            Reporter: Gary Dusbabek
>            Assignee: Gary Dusbabek
>             Fix For: 0.7
>
>
> Arya Goudarzi reports:
> Please confirm if this is an issue and should be reported or I am doing something wrong. I could not find anything relevant on JIRA:
> Playing with 0.7 nightly (today's build), I setup a 3 node cluster this way:
>  - Added one node;
>  - Loaded default schema with RF 1 from YAML using JMX;
>  - Loaded 2M keys using py_stress;
>  - Bootstrapped a second node;
>  - Cleaned up the first node;
>  - Bootstrapped a third node;
>  - Cleaned up the second node;
> I got the following ring:
> Address       Status     Load          Range                                      Ring
>                                       154293670372423273273390365393543806425
> 10.50.26.132  Up         518.63 MB     69164917636305877859094619660693892452     |<--|
> 10.50.26.134  Up         234.8 MB      111685517405103688771527967027648896391    |   |
> 10.50.26.133  Up         235.26 MB     154293670372423273273390365393543806425    |-->|
> Now I ran:
> nodetool --host 10.50.26.132 loadbalance
> It's been going for a while. I checked the streams
> nodetool --host 10.50.26.134 streams
> Mode: Normal
> Not sending any streams.
> Streaming from: /10.50.26.132
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-3-Data.db/[(0,22206096), (22206096,27271682)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-4-Data.db/[(0,15180462), (15180462,18656982)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-5-Data.db/[(0,353139829), (353139829,433883659)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-6-Data.db/[(0,366336059), (366336059,450095320)]
> nodetool --host 10.50.26.132 streams
> Mode: Leaving: streaming data to other nodes
> Streaming to: /10.50.26.134
>   /var/lib/cassandra/data/Keyspace1/Standard1-d-48-Data.db/[(0,366336059), (366336059,450095320)]
> Not receiving any streams.
> These have been going for the past 2 hours.
> I see in the logs of the node with 134 IP address and I saw this:
> INFO [GOSSIP_STAGE:1] 2010-06-22 16:30:54,679 StorageService.java (line 603) Will not change my token ownership to /10.50.26.132
> So, to my understanding from wikis loadbalance supposed to decommission and re-bootstrap again by sending its tokens to other nodes and then bootstrap again. It's been stuck in streaming for the past 2 hours and the size of ring has not changed. The log in the first node says it has started streaming for the past hours:
> INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,255 StreamOut.java (line 72) Beginning transfer process to /10.50.26.134 for ranges (154293670372423273273390365393543806425,69164917636305877859094619660693892452]
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,255 StreamOut.java (line 82) Flushing memtables for Keyspace1...
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,266 StreamOut.java (line 128) Stream context metadata [/var/lib/cassandra/data/Keyspace1/Standard1-d-48-Data.db/[(0,366336059), (366336059,450095320)]] 1 sstables.
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,267 StreamOut.java (line 135) Sending a stream initiate message to /10.50.26.134 ...
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,267 StreamOut.java (line 140) Waiting for transfer to /10.50.26.134 to complete
>  INFO [FLUSH-TIMER] 2010-06-22 17:36:53,370 ColumnFamilyStore.java (line 359) LocationInfo has reached its threshold; switching in a fresh Memtable at CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1277249454413.log', position=720)
>  INFO [FLUSH-TIMER] 2010-06-22 17:36:53,370 ColumnFamilyStore.java (line 622) Enqueuing flush of Memtable(LocationInfo)@1637794189
>  INFO [FLUSH-WRITER-POOL:1] 2010-06-22 17:36:53,370 Memtable.java (line 149) Writing Memtable(LocationInfo)@1637794189
>  INFO [FLUSH-WRITER-POOL:1] 2010-06-22 17:36:53,528 Memtable.java (line 163) Completed flushing /var/lib/cassandra/data/system/LocationInfo-d-9-Data.db
>  INFO [MEMTABLE-POST-FLUSHER:1] 2010-06-22 17:36:53,529 ColumnFamilyStore.java (line 374) Discarding 1000
> Nothing more after this line.
> Am I doing something wrong?

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (CASSANDRA-1221) loadbalance operation never completes on a 3 node cluster

Posted by "Arya Goudarzi (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-1221?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12888600#action_12888600 ] 

Arya Goudarzi commented on CASSANDRA-1221:
------------------------------------------

132 is node1
133 is node2
134 is node3

Give me some time and I'll regenerate all the commands for you in details. 

> loadbalance operation never completes on a 3 node cluster
> ---------------------------------------------------------
>
>                 Key: CASSANDRA-1221
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1221
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 0.7
>            Reporter: Gary Dusbabek
>            Assignee: Gary Dusbabek
>             Fix For: 0.7
>
>
> Arya Goudarzi reports:
> Please confirm if this is an issue and should be reported or I am doing something wrong. I could not find anything relevant on JIRA:
> Playing with 0.7 nightly (today's build), I setup a 3 node cluster this way:
>  - Added one node;
>  - Loaded default schema with RF 1 from YAML using JMX;
>  - Loaded 2M keys using py_stress;
>  - Bootstrapped a second node;
>  - Cleaned up the first node;
>  - Bootstrapped a third node;
>  - Cleaned up the second node;
> I got the following ring:
> Address       Status     Load          Range                                      Ring
>                                       154293670372423273273390365393543806425
> 10.50.26.132  Up         518.63 MB     69164917636305877859094619660693892452     |<--|
> 10.50.26.134  Up         234.8 MB      111685517405103688771527967027648896391    |   |
> 10.50.26.133  Up         235.26 MB     154293670372423273273390365393543806425    |-->|
> Now I ran:
> nodetool --host 10.50.26.132 loadbalance
> It's been going for a while. I checked the streams
> nodetool --host 10.50.26.134 streams
> Mode: Normal
> Not sending any streams.
> Streaming from: /10.50.26.132
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-3-Data.db/[(0,22206096), (22206096,27271682)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-4-Data.db/[(0,15180462), (15180462,18656982)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-5-Data.db/[(0,353139829), (353139829,433883659)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-6-Data.db/[(0,366336059), (366336059,450095320)]
> nodetool --host 10.50.26.132 streams
> Mode: Leaving: streaming data to other nodes
> Streaming to: /10.50.26.134
>   /var/lib/cassandra/data/Keyspace1/Standard1-d-48-Data.db/[(0,366336059), (366336059,450095320)]
> Not receiving any streams.
> These have been going for the past 2 hours.
> I see in the logs of the node with 134 IP address and I saw this:
> INFO [GOSSIP_STAGE:1] 2010-06-22 16:30:54,679 StorageService.java (line 603) Will not change my token ownership to /10.50.26.132
> So, to my understanding from wikis loadbalance supposed to decommission and re-bootstrap again by sending its tokens to other nodes and then bootstrap again. It's been stuck in streaming for the past 2 hours and the size of ring has not changed. The log in the first node says it has started streaming for the past hours:
> INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,255 StreamOut.java (line 72) Beginning transfer process to /10.50.26.134 for ranges (154293670372423273273390365393543806425,69164917636305877859094619660693892452]
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,255 StreamOut.java (line 82) Flushing memtables for Keyspace1...
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,266 StreamOut.java (line 128) Stream context metadata [/var/lib/cassandra/data/Keyspace1/Standard1-d-48-Data.db/[(0,366336059), (366336059,450095320)]] 1 sstables.
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,267 StreamOut.java (line 135) Sending a stream initiate message to /10.50.26.134 ...
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,267 StreamOut.java (line 140) Waiting for transfer to /10.50.26.134 to complete
>  INFO [FLUSH-TIMER] 2010-06-22 17:36:53,370 ColumnFamilyStore.java (line 359) LocationInfo has reached its threshold; switching in a fresh Memtable at CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1277249454413.log', position=720)
>  INFO [FLUSH-TIMER] 2010-06-22 17:36:53,370 ColumnFamilyStore.java (line 622) Enqueuing flush of Memtable(LocationInfo)@1637794189
>  INFO [FLUSH-WRITER-POOL:1] 2010-06-22 17:36:53,370 Memtable.java (line 149) Writing Memtable(LocationInfo)@1637794189
>  INFO [FLUSH-WRITER-POOL:1] 2010-06-22 17:36:53,528 Memtable.java (line 163) Completed flushing /var/lib/cassandra/data/system/LocationInfo-d-9-Data.db
>  INFO [MEMTABLE-POST-FLUSHER:1] 2010-06-22 17:36:53,529 ColumnFamilyStore.java (line 374) Discarding 1000
> Nothing more after this line.
> Am I doing something wrong?

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (CASSANDRA-1221) loadbalance operation never completes on a 3 node cluster

Posted by "Arya Goudarzi (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-1221?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12888635#action_12888635 ] 

Arya Goudarzi commented on CASSANDRA-1221:
------------------------------------------

Gary, I missed one thing (Step 9-13) where I took a node down and insert. I tested this without step 9-13 and loadbalance worked. Not sure why step 9-13 changes everything. Here are the full production steps (I am also attaching the logs from all 3 nodes if it helps):

This is the ring topology I discuss here:

Node1 10.50.26.132 (Hostname: cas-test1)
Node2 10.50.26.133 (Hostname: cas-test2)
Node3 10.50.26.134 (Hostname: cas-test3)

This run is using today's nightly built from a clean setup.

Step 1: Startup Node1

[agoudarzi@cas-test1 ~]$ sudo /etc/init.d/cassandra start

Step 2: LoadSchemadFromYAML

I go to JConsole and call the function from o.a.c.service StorageService MBeans

Step 3: I insert 500000 keys into Standard1 CF using py_stress

$ python stress.py --num-keys 500000 --threads 8 --nodes 10.50.26.132 --keep-going --operation insert
Keyspace already exists.
total,interval_op_rate,interval_key_rate,avg_latency,elapsed_time
62455,6245,6245,0.00128478354559,10
121893,5943,5943,0.00134767375398,21
184298,6240,6240,0.00128336335573,31
248124,6382,6382,0.00124898537112,42
297205,4908,4908,0.00163303957852,52
340338,4313,4313,0.00189026848124,63
380233,3989,3989,0.00203818801591,73
444452,6421,6421,0.00124198496903,84
500000,5554,5554,0.00114441244599,93

Step 3: Let's Take a Look at Ring

[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.132 ring
Address         Status State   Load            Token                                       
10.50.26.132    Up     Normal  206.05 MB       139380634429053457983268837561452509806     

Step 4: Bootstrap Node 2 into cluster

[agoudarzi@cas-test2 ~]$ sudo /etc/init.d/cassandra start

Step 5: Check the ring
[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.132 ring
Address         Status State   Load            Token                                       
                                       139380634429053457983268837561452509806    
10.50.26.133    Up     Joining 5.84 KB         54081521187303805945240848606999860232      
10.50.26.132    Up     Normal  206.05 MB       139380634429053457983268837561452509806     

Step 6: Check the streams on Node 1

[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.132 streams
Mode: Normal
Streaming to: /10.50.26.133
   /var/lib/cassandra/data/Keyspace1/Standard1-d-5-Data.db/[(0,34658183), (89260032,109057810)]
   /var/lib/cassandra/data/Keyspace1/Standard1-d-6-Data.db/[(0,8746823), (22272929,27264363)]
   /var/lib/cassandra/data/Keyspace1/Standard1-d-8-Data.db/[(0,8749389), (22336617,27264253)]
   /var/lib/cassandra/data/Keyspace1/Standard1-d-9-Data.db/[(0,8235190), (21174782,25822054)]
   /var/lib/cassandra/data/Keyspace1/Standard1-d-7-Data.db/[(0,8642472), (22333239,27264347)]
Not receiving any streams.

Step 7: Check the ring from Node1 and Node2 and make sure they agree

[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.132 ring
Address         Status State   Load            Token                                       
                                       139380634429053457983268837561452509806    
10.50.26.133    Up     Normal  116.94 MB       54081521187303805945240848606999860232      
10.50.26.132    Up     Normal  233.93 MB       139380634429053457983268837561452509806     
[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.133 ring
Address         Status State   Load            Token                                       
                                       139380634429053457983268837561452509806    
10.50.26.133    Up     Normal  116.94 MB       54081521187303805945240848606999860232      
10.50.26.132    Up     Normal  233.93 MB       139380634429053457983268837561452509806

Step 8: Cleanup Node1

[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.132 cleanup

Step 9: Check the ring agreement again

[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.132 ring
Address         Status State   Load            Token                                       
                                       139380634429053457983268837561452509806    
10.50.26.133    Up     Normal  116.94 MB       54081521187303805945240848606999860232      
10.50.26.132    Up     Normal  117 MB          139380634429053457983268837561452509806     
[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.133 ring
Address         Status State   Load            Token                                       
                                       139380634429053457983268837561452509806    
10.50.26.133    Up     Normal  116.94 MB       54081521187303805945240848606999860232      
10.50.26.132    Up     Normal  117 MB          139380634429053457983268837561452509806     

Step 9: Let's kill Node 2

[agoudarzi@cas-test2 ~]$ sudo /etc/init.d/cassandra stop 

Step 10: Check the ring on Node 1

[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.132 ring
Address         Status State   Load            Token                                       
                                       139380634429053457983268837561452509806    
10.50.26.133    Down   Normal  116.94 MB       54081521187303805945240848606999860232      
10.50.26.132    Up     Normal  117 MB          139380634429053457983268837561452509806

Step 11: Let's try to insert 500000 more keys expecting lots of unavailable exceptions ad the only replica for some keys is dead and py_stress does not use CLevel.ANY or ZERO

$ python stress.py --num-keys 500000 --threads 8 --nodes 10.50.26.132 --keep-going --operation insert

Keyspace already exists.
UnavailableException()
UnavailableException()
UnavailableException()
....
...
..
.
total,interval_op_rate,interval_key_rate,avg_latency,elapsed_time
500000,2922,2922,0.000816067281446,67

Step 12: Check the ring

[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.132 ring
Address         Status State   Load            Token                                       
                                       139380634429053457983268837561452509806    
10.50.26.133    Down   Normal  116.94 MB       54081521187303805945240848606999860232      
10.50.26.132    Up     Normal  205.35 MB       139380634429053457983268837561452509806     

Node 1 got more data as expected

Step 13: Bring up Node 2 again

[agoudarzi@cas-test2 ~]$ sudo /etc/init.d/cassandra start

Step 14: Check the Ring

[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.132 ring
Address         Status State   Load            Token                                       
                                       139380634429053457983268837561452509806    
10.50.26.133    Up     Normal  116.94 MB       54081521187303805945240848606999860232      
10.50.26.132    Up     Normal  205.35 MB       139380634429053457983268837561452509806     
[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.133 ring
Address         Status State   Load            Token                                       
                                       139380634429053457983268837561452509806    
10.50.26.133    Up     Normal  116.94 MB       54081521187303805945240848606999860232      
10.50.26.132    Up     Normal  205.35 MB       139380634429053457983268837561452509806

Step 15: Bootstrap Node 3

[agoudarzi@cas-test3 ~]$ sudo /etc/init.d/cassandra start

Step 11: Check Ring 

[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.132 ring
Address         Status State   Load            Token                                       
                                       139380634429053457983268837561452509806    
10.50.26.133    Up     Normal  116.94 MB       54081521187303805945240848606999860232      
10.50.26.134    Up     Normal  116.68 MB       96565427321648203609592911083606603165      
10.50.26.132    Up     Normal  234 MB          139380634429053457983268837561452509806     
[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.133 ring
Address         Status State   Load            Token                                       
                                       139380634429053457983268837561452509806    
10.50.26.133    Up     Normal  116.94 MB       54081521187303805945240848606999860232      
10.50.26.134    Up     Normal  116.68 MB       96565427321648203609592911083606603165      
10.50.26.132    Up     Normal  234 MB          139380634429053457983268837561452509806     
[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.134 ring
Address         Status State   Load            Token                                       
                                       139380634429053457983268837561452509806    
10.50.26.133    Up     Normal  116.94 MB       54081521187303805945240848606999860232      
10.50.26.134    Up     Normal  116.68 MB       96565427321648203609592911083606603165      
10.50.26.132    Up     Normal  234 MB          139380634429053457983268837561452509806     

Step 12: Cleanup Node 1 (132)

[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.132 cleanup

Step 13: Check Ring

[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.132 ring
Address         Status State   Load            Token                                       
                                       139380634429053457983268837561452509806    
10.50.26.133    Up     Normal  116.94 MB       54081521187303805945240848606999860232      
10.50.26.134    Up     Normal  116.68 MB       96565427321648203609592911083606603165      
10.50.26.132    Up     Normal  58.89 MB        139380634429053457983268837561452509806     
[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.133 ring
Address         Status State   Load            Token                                       
                                       139380634429053457983268837561452509806    
10.50.26.133    Up     Normal  116.94 MB       54081521187303805945240848606999860232      
10.50.26.134    Up     Normal  116.68 MB       96565427321648203609592911083606603165      
10.50.26.132    Up     Normal  58.89 MB        139380634429053457983268837561452509806     
[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.134 ring
Address         Status State   Load            Token                                       
                                       139380634429053457983268837561452509806    
10.50.26.133    Up     Normal  116.94 MB       54081521187303805945240848606999860232      
10.50.26.134    Up     Normal  116.68 MB       96565427321648203609592911083606603165      
10.50.26.132    Up     Normal  58.89 MB        139380634429053457983268837561452509806     

Looks as expected. Node 1 (132) has the least load. Let's loadbalance it.

Step 14: Loadbalance Node 1

[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.132 loadbalance &
[1] 27457

Step 15: Check Ring

[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.132 ring
Address         Status State   Load            Token                                       
                                       139380634429053457983268837561452509806    
10.50.26.133    Up     Normal  116.94 MB       54081521187303805945240848606999860232      
10.50.26.134    Up     Normal  116.68 MB       96565427321648203609592911083606603165      
10.50.26.132    Up     Leaving 58.89 MB        139380634429053457983268837561452509806     
[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.133 ring
Address         Status State   Load            Token                                       
                                       139380634429053457983268837561452509806    
10.50.26.133    Up     Normal  116.94 MB       54081521187303805945240848606999860232      
10.50.26.134    Up     Normal  116.68 MB       96565427321648203609592911083606603165      
10.50.26.132    Up     Leaving 58.89 MB        139380634429053457983268837561452509806     
[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.134 ring
Address         Status State   Load            Token                                       
                                       139380634429053457983268837561452509806    
10.50.26.133    Up     Normal  116.94 MB       54081521187303805945240848606999860232      
10.50.26.134    Up     Normal  116.68 MB       96565427321648203609592911083606603165      
10.50.26.132    Up     Leaving 58.89 MB        139380634429053457983268837561452509806     

Step 16: Check the Streams

[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.132 streams
Mode: Leaving: streaming data to other nodes
Streaming to: /10.50.26.133
   /var/lib/cassandra/data/Keyspace1/Standard1-d-17-Data.db/[(0,54278564)]
Not receiving any streams.
[agoudarzi@cas-test1 ~]$ nodetool --host=10.50.26.133 streams
Mode: Normal
Not sending any streams.
Not receiving any streams.

PROBLEM:
Notice 132 says I am streaming to 133 but 133 says "Not receiving any streams!"  




> loadbalance operation never completes on a 3 node cluster
> ---------------------------------------------------------
>
>                 Key: CASSANDRA-1221
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1221
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 0.7
>            Reporter: Gary Dusbabek
>            Assignee: Gary Dusbabek
>             Fix For: 0.7
>
>
> Arya Goudarzi reports:
> Please confirm if this is an issue and should be reported or I am doing something wrong. I could not find anything relevant on JIRA:
> Playing with 0.7 nightly (today's build), I setup a 3 node cluster this way:
>  - Added one node;
>  - Loaded default schema with RF 1 from YAML using JMX;
>  - Loaded 2M keys using py_stress;
>  - Bootstrapped a second node;
>  - Cleaned up the first node;
>  - Bootstrapped a third node;
>  - Cleaned up the second node;
> I got the following ring:
> Address       Status     Load          Range                                      Ring
>                                       154293670372423273273390365393543806425
> 10.50.26.132  Up         518.63 MB     69164917636305877859094619660693892452     |<--|
> 10.50.26.134  Up         234.8 MB      111685517405103688771527967027648896391    |   |
> 10.50.26.133  Up         235.26 MB     154293670372423273273390365393543806425    |-->|
> Now I ran:
> nodetool --host 10.50.26.132 loadbalance
> It's been going for a while. I checked the streams
> nodetool --host 10.50.26.134 streams
> Mode: Normal
> Not sending any streams.
> Streaming from: /10.50.26.132
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-3-Data.db/[(0,22206096), (22206096,27271682)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-4-Data.db/[(0,15180462), (15180462,18656982)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-5-Data.db/[(0,353139829), (353139829,433883659)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-6-Data.db/[(0,366336059), (366336059,450095320)]
> nodetool --host 10.50.26.132 streams
> Mode: Leaving: streaming data to other nodes
> Streaming to: /10.50.26.134
>   /var/lib/cassandra/data/Keyspace1/Standard1-d-48-Data.db/[(0,366336059), (366336059,450095320)]
> Not receiving any streams.
> These have been going for the past 2 hours.
> I see in the logs of the node with 134 IP address and I saw this:
> INFO [GOSSIP_STAGE:1] 2010-06-22 16:30:54,679 StorageService.java (line 603) Will not change my token ownership to /10.50.26.132
> So, to my understanding from wikis loadbalance supposed to decommission and re-bootstrap again by sending its tokens to other nodes and then bootstrap again. It's been stuck in streaming for the past 2 hours and the size of ring has not changed. The log in the first node says it has started streaming for the past hours:
> INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,255 StreamOut.java (line 72) Beginning transfer process to /10.50.26.134 for ranges (154293670372423273273390365393543806425,69164917636305877859094619660693892452]
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,255 StreamOut.java (line 82) Flushing memtables for Keyspace1...
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,266 StreamOut.java (line 128) Stream context metadata [/var/lib/cassandra/data/Keyspace1/Standard1-d-48-Data.db/[(0,366336059), (366336059,450095320)]] 1 sstables.
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,267 StreamOut.java (line 135) Sending a stream initiate message to /10.50.26.134 ...
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,267 StreamOut.java (line 140) Waiting for transfer to /10.50.26.134 to complete
>  INFO [FLUSH-TIMER] 2010-06-22 17:36:53,370 ColumnFamilyStore.java (line 359) LocationInfo has reached its threshold; switching in a fresh Memtable at CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1277249454413.log', position=720)
>  INFO [FLUSH-TIMER] 2010-06-22 17:36:53,370 ColumnFamilyStore.java (line 622) Enqueuing flush of Memtable(LocationInfo)@1637794189
>  INFO [FLUSH-WRITER-POOL:1] 2010-06-22 17:36:53,370 Memtable.java (line 149) Writing Memtable(LocationInfo)@1637794189
>  INFO [FLUSH-WRITER-POOL:1] 2010-06-22 17:36:53,528 Memtable.java (line 163) Completed flushing /var/lib/cassandra/data/system/LocationInfo-d-9-Data.db
>  INFO [MEMTABLE-POST-FLUSHER:1] 2010-06-22 17:36:53,529 ColumnFamilyStore.java (line 374) Discarding 1000
> Nothing more after this line.
> Am I doing something wrong?

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (CASSANDRA-1221) loadbalance operation never completes on a 3 node cluster

Posted by "Hudson (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-1221?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12890713#action_12890713 ] 

Hudson commented on CASSANDRA-1221:
-----------------------------------

Integrated in Cassandra #496 (See [http://hudson.zones.apache.org/hudson/job/Cassandra/496/])
    failure detection wasn't closing sockets. patch by gdusbabek, reviewed by jbellis. CASSANDRA-1221


> loadbalance operation never completes on a 3 node cluster
> ---------------------------------------------------------
>
>                 Key: CASSANDRA-1221
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1221
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 0.7
>            Reporter: Gary Dusbabek
>            Assignee: Gary Dusbabek
>             Fix For: 0.6.4
>
>         Attachments: 0.6-conviction-fix.diff, 0001-Gossiper-and-FD-never-called-MS.convict-to-shut-down.patch, system1.log, system2.log, system3.log
>
>
> Arya Goudarzi reports:
> Please confirm if this is an issue and should be reported or I am doing something wrong. I could not find anything relevant on JIRA:
> Playing with 0.7 nightly (today's build), I setup a 3 node cluster this way:
>  - Added one node;
>  - Loaded default schema with RF 1 from YAML using JMX;
>  - Loaded 2M keys using py_stress;
>  - Bootstrapped a second node;
>  - Cleaned up the first node;
>  - Bootstrapped a third node;
>  - Cleaned up the second node;
> I got the following ring:
> Address       Status     Load          Range                                      Ring
>                                       154293670372423273273390365393543806425
> 10.50.26.132  Up         518.63 MB     69164917636305877859094619660693892452     |<--|
> 10.50.26.134  Up         234.8 MB      111685517405103688771527967027648896391    |   |
> 10.50.26.133  Up         235.26 MB     154293670372423273273390365393543806425    |-->|
> Now I ran:
> nodetool --host 10.50.26.132 loadbalance
> It's been going for a while. I checked the streams
> nodetool --host 10.50.26.134 streams
> Mode: Normal
> Not sending any streams.
> Streaming from: /10.50.26.132
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-3-Data.db/[(0,22206096), (22206096,27271682)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-4-Data.db/[(0,15180462), (15180462,18656982)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-5-Data.db/[(0,353139829), (353139829,433883659)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-6-Data.db/[(0,366336059), (366336059,450095320)]
> nodetool --host 10.50.26.132 streams
> Mode: Leaving: streaming data to other nodes
> Streaming to: /10.50.26.134
>   /var/lib/cassandra/data/Keyspace1/Standard1-d-48-Data.db/[(0,366336059), (366336059,450095320)]
> Not receiving any streams.
> These have been going for the past 2 hours.
> I see in the logs of the node with 134 IP address and I saw this:
> INFO [GOSSIP_STAGE:1] 2010-06-22 16:30:54,679 StorageService.java (line 603) Will not change my token ownership to /10.50.26.132
> So, to my understanding from wikis loadbalance supposed to decommission and re-bootstrap again by sending its tokens to other nodes and then bootstrap again. It's been stuck in streaming for the past 2 hours and the size of ring has not changed. The log in the first node says it has started streaming for the past hours:
> INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,255 StreamOut.java (line 72) Beginning transfer process to /10.50.26.134 for ranges (154293670372423273273390365393543806425,69164917636305877859094619660693892452]
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,255 StreamOut.java (line 82) Flushing memtables for Keyspace1...
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,266 StreamOut.java (line 128) Stream context metadata [/var/lib/cassandra/data/Keyspace1/Standard1-d-48-Data.db/[(0,366336059), (366336059,450095320)]] 1 sstables.
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,267 StreamOut.java (line 135) Sending a stream initiate message to /10.50.26.134 ...
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,267 StreamOut.java (line 140) Waiting for transfer to /10.50.26.134 to complete
>  INFO [FLUSH-TIMER] 2010-06-22 17:36:53,370 ColumnFamilyStore.java (line 359) LocationInfo has reached its threshold; switching in a fresh Memtable at CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1277249454413.log', position=720)
>  INFO [FLUSH-TIMER] 2010-06-22 17:36:53,370 ColumnFamilyStore.java (line 622) Enqueuing flush of Memtable(LocationInfo)@1637794189
>  INFO [FLUSH-WRITER-POOL:1] 2010-06-22 17:36:53,370 Memtable.java (line 149) Writing Memtable(LocationInfo)@1637794189
>  INFO [FLUSH-WRITER-POOL:1] 2010-06-22 17:36:53,528 Memtable.java (line 163) Completed flushing /var/lib/cassandra/data/system/LocationInfo-d-9-Data.db
>  INFO [MEMTABLE-POST-FLUSHER:1] 2010-06-22 17:36:53,529 ColumnFamilyStore.java (line 374) Discarding 1000
> Nothing more after this line.
> Am I doing something wrong?

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Updated: (CASSANDRA-1221) loadbalance operation never completes on a 3 node cluster

Posted by "Gary Dusbabek (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-1221?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Gary Dusbabek updated CASSANDRA-1221:
-------------------------------------

    Attachment: 0.6-conviction-fix.diff

patch for 0.6.  I couldn't get stress.py to work in my branch, but the same problem should be present.  All tests pass with this patch.

> loadbalance operation never completes on a 3 node cluster
> ---------------------------------------------------------
>
>                 Key: CASSANDRA-1221
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1221
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 0.7
>            Reporter: Gary Dusbabek
>            Assignee: Gary Dusbabek
>             Fix For: 0.7
>
>         Attachments: 0.6-conviction-fix.diff, 0001-Gossiper-and-FD-never-called-MS.convict-to-shut-down.patch, system1.log, system2.log, system3.log
>
>
> Arya Goudarzi reports:
> Please confirm if this is an issue and should be reported or I am doing something wrong. I could not find anything relevant on JIRA:
> Playing with 0.7 nightly (today's build), I setup a 3 node cluster this way:
>  - Added one node;
>  - Loaded default schema with RF 1 from YAML using JMX;
>  - Loaded 2M keys using py_stress;
>  - Bootstrapped a second node;
>  - Cleaned up the first node;
>  - Bootstrapped a third node;
>  - Cleaned up the second node;
> I got the following ring:
> Address       Status     Load          Range                                      Ring
>                                       154293670372423273273390365393543806425
> 10.50.26.132  Up         518.63 MB     69164917636305877859094619660693892452     |<--|
> 10.50.26.134  Up         234.8 MB      111685517405103688771527967027648896391    |   |
> 10.50.26.133  Up         235.26 MB     154293670372423273273390365393543806425    |-->|
> Now I ran:
> nodetool --host 10.50.26.132 loadbalance
> It's been going for a while. I checked the streams
> nodetool --host 10.50.26.134 streams
> Mode: Normal
> Not sending any streams.
> Streaming from: /10.50.26.132
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-3-Data.db/[(0,22206096), (22206096,27271682)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-4-Data.db/[(0,15180462), (15180462,18656982)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-5-Data.db/[(0,353139829), (353139829,433883659)]
>   Keyspace1: /var/lib/cassandra/data/Keyspace1/Standard1-tmp-d-6-Data.db/[(0,366336059), (366336059,450095320)]
> nodetool --host 10.50.26.132 streams
> Mode: Leaving: streaming data to other nodes
> Streaming to: /10.50.26.134
>   /var/lib/cassandra/data/Keyspace1/Standard1-d-48-Data.db/[(0,366336059), (366336059,450095320)]
> Not receiving any streams.
> These have been going for the past 2 hours.
> I see in the logs of the node with 134 IP address and I saw this:
> INFO [GOSSIP_STAGE:1] 2010-06-22 16:30:54,679 StorageService.java (line 603) Will not change my token ownership to /10.50.26.132
> So, to my understanding from wikis loadbalance supposed to decommission and re-bootstrap again by sending its tokens to other nodes and then bootstrap again. It's been stuck in streaming for the past 2 hours and the size of ring has not changed. The log in the first node says it has started streaming for the past hours:
> INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,255 StreamOut.java (line 72) Beginning transfer process to /10.50.26.134 for ranges (154293670372423273273390365393543806425,69164917636305877859094619660693892452]
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,255 StreamOut.java (line 82) Flushing memtables for Keyspace1...
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,266 StreamOut.java (line 128) Stream context metadata [/var/lib/cassandra/data/Keyspace1/Standard1-d-48-Data.db/[(0,366336059), (366336059,450095320)]] 1 sstables.
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,267 StreamOut.java (line 135) Sending a stream initiate message to /10.50.26.134 ...
>  INFO [STREAM-STAGE:1] 2010-06-22 16:35:56,267 StreamOut.java (line 140) Waiting for transfer to /10.50.26.134 to complete
>  INFO [FLUSH-TIMER] 2010-06-22 17:36:53,370 ColumnFamilyStore.java (line 359) LocationInfo has reached its threshold; switching in a fresh Memtable at CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1277249454413.log', position=720)
>  INFO [FLUSH-TIMER] 2010-06-22 17:36:53,370 ColumnFamilyStore.java (line 622) Enqueuing flush of Memtable(LocationInfo)@1637794189
>  INFO [FLUSH-WRITER-POOL:1] 2010-06-22 17:36:53,370 Memtable.java (line 149) Writing Memtable(LocationInfo)@1637794189
>  INFO [FLUSH-WRITER-POOL:1] 2010-06-22 17:36:53,528 Memtable.java (line 163) Completed flushing /var/lib/cassandra/data/system/LocationInfo-d-9-Data.db
>  INFO [MEMTABLE-POST-FLUSHER:1] 2010-06-22 17:36:53,529 ColumnFamilyStore.java (line 374) Discarding 1000
> Nothing more after this line.
> Am I doing something wrong?

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.