[prev in list] [next in list] [prev in thread] [next in thread] 

List:       cassandra-user
Subject:    [SOLVED] Re: 0.7RC1 local_quorum -> TimedOutException
From:       Thor Carpenter <tcarpenter () mediosystems ! com>
Date:       2010-12-30 22:02:20
Message-ID: E92CFBF5-AA27-4D89-9B0E-BBC3D2715746 () medio ! com
[Download RAW message or body]

So, I found a solution, but I think there is a (trivial) bug.

The broken keyspace:

create keyspace KeyspaceDC1 with
    replication_factor = 1 and
    placement_strategy = 'org.apache.cassandra.locator.NetworkTopologyStrategy' and
    strategy_options = [{DC1:1, DC2:0}];

The fixed keyspace:

create keyspace KeyspaceDC1 with
    replication_factor = 1 and
    placement_strategy = 'org.apache.cassandra.locator.NetworkTopologyStrategy' and
    strategy_options = [{DC1:1}];

I'm not sure if the issue is that there are no nodes in DC2 or that I'm saying DC2 \
shouldn't get any replicas, or a combination of the two.  If I get time I'll try to \
poke around the code base.

-Thor

On Dec 29, 2010, at 12:38 PM, Thor Carpenter wrote:

In an attempt to replicate this error on a simpler system I have come across a \
different error but seemingly related problem.  Now I am getting the following \
exception in system.log.

DEBUG [pool-1-thread-3] 2010-12-29 12:10:38,897 CassandraServer.java (line 362) \
insert ERROR [pool-1-thread-3] 2010-12-29 12:10:38,906 Cassandra.java (line 2960) \
Internal error processing insert java.lang.AssertionError
        at org.apache.cassandra.locator.TokenMetadata.firstTokenIndex(TokenMetadata.java:392)
                
        at org.apache.cassandra.locator.TokenMetadata.ringIterator(TokenMetadata.java:417)
                
        at org.apache.cassandra.locator.NetworkTopologyStrategy.calculateNaturalEndpoints(NetworkTopologyStrategy.java:95)
                
        at org.apache.cassandra.locator.AbstractReplicationStrategy.getNaturalEndpoints(AbstractReplicationStrategy.java:99)
                
        at org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:1411)
                
        at org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:1394)
                
        at org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:109)
        at org.apache.cassandra.thrift.CassandraServer.doInsert(CassandraServer.java:442)
                
        at org.apache.cassandra.thrift.CassandraServer.insert(CassandraServer.java:379)
                
        at org.apache.cassandra.thrift.Cassandra$Processor$insert.process(Cassandra.java:2952)
                
        at org.apache.cassandra.thrift.Cassandra$Processor.process(Cassandra.java:2555)
                
        at org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:167)
                
        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
                
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
  at java.lang.Thread.run(Thread.java:619)

Looking at TokenMetadata.java:392, the assert that is triggered is:
assert ring.size() > 0;

Which led me to run nodetool ring which outputs a seemingly correct config:
-sh-3.2$ nodetool -h localhost -p 8080 ring
Address         Status State   Load            Owns    Token
10.5.64.26      Up     Normal  47.16 KB        100.00% 1


To reproduce:

* I have upgraded to 0.7RC3 so install the latest riptano 0.7rc3 rpm on a single node \
in "DC1".

* In cassandra.yaml set initial_token = 1

* cassandra-topology.properties:

10.5.64.26=DC1:R1
default=DC2:R1

* Schema loaded via cassandra-cli:

create keyspace KeyspaceDC1 with
    replication_factor = 1 and
    placement_strategy = 'org.apache.cassandra.locator.NetworkTopologyStrategy' and
    strategy_options = [{DC1:1, DC2:0}];

use KeyspaceDC1;

create column family TestCF with
    column_type = 'Standard' and
    comparator = 'BytesType' and
    keys_cached = 200000 and
    rows_cached = 2000 and
    gc_grace = 0 and
    read_repair_chance = 0.0;

* In cassandra-cli execute the following:

[default@unknown] use KeyspaceDC1;
Authenticated to keyspace: KeyspaceDC1
[default@KeyspaceDC1] set TestCF['some key']['some col'] = 'some value';
Internal error processing insert

* Check system.log where you should find the above assertion error.


Can anybody find a flaw in this setup?  Do others agree something looks buggy?

Thanks,
-Thor

On Dec 28, 2010, at 4:33 PM, Thor Carpenter wrote:

Third try for good luck.  Maybe this time I'll get it all in one email.  :-)

At time 15:55 I performed a write with CL = ONE, a subsequent read with CL = ONE and \
received correct results.  Similar results occur with any read CL other than \
LOCAL_QUORUM.

At time 15:58 I performed a write with CL = ONE, a subsequent read with CL = \
LOCAL_QUORUM and received a TimedOutException.

System.log and cassandra.log are from server dlc05-sea to corresponds with the hector \
error message "Closing client CassandraClient<dlc05-sea:9160-50>".  I haven't found \
anything in the logs that indicates the problem but maybe others with more discerning \
eyes can.

Please let me know if I can provide any additional information.

Thanks,
-Thor

<logs.zip><ATT00001..txt>


[Attachment #3 (text/html)]

<html><head></head><body style="word-wrap: break-word; -webkit-nbsp-mode: space; \
-webkit-line-break: after-white-space; ">So, I found a solution, but I think there is \
a (trivial) bug.<div><br></div><div>The broken \
keyspace:</div><div><br></div><div><blockquote type="cite"><div style="word-wrap: \
break-word; -webkit-nbsp-mode: space; -webkit-line-break: after-white-space; \
"><div><div><div>create keyspace KeyspaceDC1 with&nbsp;</div><div>&nbsp;&nbsp; \
&nbsp;replication_factor = 1 and&nbsp;</div><div>&nbsp;&nbsp; \
&nbsp;placement_strategy = 'org.apache.cassandra.locator.NetworkTopologyStrategy' \
and</div><div>&nbsp;&nbsp; &nbsp;strategy_options = [{DC1:1<b>, \
DC2:0</b>}];</div></div></div></div></blockquote><div><br></div>The fixed \
keyspace:</div><div><br></div><div><blockquote type="cite"><div style="word-wrap: \
break-word; -webkit-nbsp-mode: space; -webkit-line-break: after-white-space; \
"><div><div><div>create keyspace KeyspaceDC1 with&nbsp;</div><div>&nbsp;&nbsp; \
&nbsp;replication_factor = 1 and&nbsp;</div><div>&nbsp;&nbsp; \
&nbsp;placement_strategy = 'org.apache.cassandra.locator.NetworkTopologyStrategy' \
and</div><div>&nbsp;&nbsp; &nbsp;strategy_options = \
[{DC1:1}];</div></div></div></div></blockquote><div><br></div><div>I'm not sure if \
the issue is that there are no nodes in DC2 or that I'm saying DC2 shouldn't get any \
replicas, or a combination of the two. &nbsp;If I get time I'll try to poke around \
the code base.</div><div><br></div><div>-Thor</div><div><br><div><div>On Dec 29, \
2010, at 12:38 PM, Thor Carpenter wrote:</div><br \
class="Apple-interchange-newline"><blockquote type="cite"><div style="word-wrap: \
break-word; -webkit-nbsp-mode: space; -webkit-line-break: after-white-space; ">In an \
attempt to replicate this error on a simpler system I have come across a different \
error but seemingly related problem. &nbsp;Now I am getting the following exception \
in system.log.<div><br></div><div><div>DEBUG [pool-1-thread-3] 2010-12-29 \
12:10:38,897 CassandraServer.java (line 362) insert</div><div>ERROR [pool-1-thread-3] \
2010-12-29 12:10:38,906 Cassandra.java (line 2960) Internal error processing \
insert</div><div>java.lang.AssertionError</div><div>&nbsp;&nbsp; &nbsp; &nbsp; \
&nbsp;at org.apache.cassandra.locator.TokenMetadata.firstTokenIndex(TokenMetadata.java:392)</div><div>&nbsp;&nbsp; \
&nbsp; &nbsp; &nbsp;at \
org.apache.cassandra.locator.TokenMetadata.ringIterator(TokenMetadata.java:417)</div><div>&nbsp;&nbsp; \
&nbsp; &nbsp; &nbsp;at \
org.apache.cassandra.locator.NetworkTopologyStrategy.calculateNaturalEndpoints(NetworkTopologyStrategy.java:95)</div><div>&nbsp;&nbsp; \
&nbsp; &nbsp; &nbsp;at \
org.apache.cassandra.locator.AbstractReplicationStrategy.getNaturalEndpoints(AbstractReplicationStrategy.java:99)</div><div>&nbsp;&nbsp; \
&nbsp; &nbsp; &nbsp;at \
org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:1411)</div><div>&nbsp;&nbsp; \
&nbsp; &nbsp; &nbsp;at \
org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:1394)</div><div>&nbsp;&nbsp; \
&nbsp; &nbsp; &nbsp;at \
org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:109)</div><div>&nbsp;&nbsp; \
&nbsp; &nbsp; &nbsp;at \
org.apache.cassandra.thrift.CassandraServer.doInsert(CassandraServer.java:442)</div><div>&nbsp;&nbsp; \
&nbsp; &nbsp; &nbsp;at \
org.apache.cassandra.thrift.CassandraServer.insert(CassandraServer.java:379)</div><div>&nbsp;&nbsp; \
&nbsp; &nbsp; &nbsp;at \
org.apache.cassandra.thrift.Cassandra$Processor$insert.process(Cassandra.java:2952)</div><div>&nbsp;&nbsp; \
&nbsp; &nbsp; &nbsp;at \
org.apache.cassandra.thrift.Cassandra$Processor.process(Cassandra.java:2555)</div><div>&nbsp;&nbsp; \
&nbsp; &nbsp; &nbsp;at \
org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:167)</div><div>&nbsp;&nbsp; \
&nbsp; &nbsp; &nbsp;at \
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)</div><div>&nbsp;&nbsp; \
&nbsp; &nbsp; &nbsp;at \
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)</div><div>&nbsp;&nbsp; \
&nbsp; &nbsp; &nbsp;at \
java.lang.Thread.run(Thread.java:619)</div><div><br></div><div>Looking at \
TokenMetadata.java:392, the assert that is triggered is:</div><div><div \
style="margin-top: 0px; margin-right: 0px; margin-bottom: 0px; margin-left: 0px; \
font: normal normal normal 11px/normal Menlo; "><span style="color: \
#bc319c">assert</span> ring.size() &gt; <span style="color: \
#2834cf">0</span>;</div></div><div><br></div><div>Which led me to run nodetool ring \
which outputs a seemingly correct config:</div><div><div>-sh-3.2$ nodetool -h \
localhost -p 8080 ring</div><div>Address &nbsp; &nbsp; &nbsp; &nbsp; Status State \
&nbsp; Load &nbsp; &nbsp; &nbsp; &nbsp; &nbsp; &nbsp;Owns &nbsp; &nbsp;Token &nbsp; \
&nbsp; &nbsp; &nbsp; &nbsp; &nbsp; &nbsp; &nbsp; &nbsp; &nbsp; &nbsp; &nbsp; &nbsp; \
&nbsp; &nbsp; &nbsp; &nbsp; &nbsp; &nbsp;&nbsp;</div><div>10.5.64.26 &nbsp; &nbsp; \
&nbsp;Up &nbsp; &nbsp; Normal &nbsp;47.16 KB &nbsp; &nbsp; &nbsp; &nbsp;100.00% \
1&nbsp;</div></div><div><br></div><div><br></div><div>To \
reproduce:</div><div><br></div><div>* I have upgraded to 0.7RC3 so install the latest \
riptano 0.7rc3 rpm on a&nbsp;single node in "DC1".</div><div><br></div><div>* In \
cassandra.yaml set initial_token = 1</div><div><br></div><div>* \
cassandra-topology.properties:</div><div><div><br></div><div>10.5.64.26=DC1:R1</div><div>default=DC2:R1</div></div><div><br></div><div>* \
Schema loaded via cassandra-cli:</div><div><div><br></div><div>create keyspace \
KeyspaceDC1 with&nbsp;</div><div>&nbsp;&nbsp; &nbsp;replication_factor = 1 \
and&nbsp;</div><div>&nbsp;&nbsp; &nbsp;placement_strategy = \
'org.apache.cassandra.locator.NetworkTopologyStrategy' and</div><div>&nbsp;&nbsp; \
&nbsp;strategy_options = [{DC1:1, DC2:0}];</div><div><br></div><div>use \
KeyspaceDC1;</div><div><br></div><div>create column family TestCF \
with</div><div>&nbsp;&nbsp; &nbsp;column_type = 'Standard' and</div><div>&nbsp;&nbsp; \
&nbsp;comparator = 'BytesType' and</div><div>&nbsp;&nbsp; &nbsp;keys_cached = 200000 \
and</div><div>&nbsp;&nbsp; &nbsp;rows_cached = 2000 and</div><div>&nbsp;&nbsp; \
&nbsp;gc_grace = 0 and</div><div>&nbsp;&nbsp; &nbsp;read_repair_chance = \
0.0;</div></div><div><br></div><div>* In cassandra-cli execute the \
following:</div><div><br></div><div><div>[default@unknown] use KeyspaceDC1; &nbsp; \
&nbsp; &nbsp; &nbsp; &nbsp; &nbsp; &nbsp; &nbsp; &nbsp; &nbsp; &nbsp; &nbsp; &nbsp; \
&nbsp; &nbsp; &nbsp; &nbsp;</div><div>Authenticated to keyspace: \
KeyspaceDC1</div><div>[default@KeyspaceDC1] set TestCF['some key']['some col'] = \
'some value';</div><div>Internal error processing \
insert</div></div><div><br></div><div>* Check system.log where you should find the \
above assertion error.</div><div><br></div><div><br></div><div>Can anybody find a \
flaw in this setup? &nbsp;Do others agree something looks \
buggy?</div><div><br></div><div>Thanks,</div><div>-Thor</div><div><br><div><div>On \
Dec 28, 2010, at 4:33 PM, Thor Carpenter wrote:</div><br \
class="Apple-interchange-newline"><blockquote type="cite"><div>Third try for good \
luck. &nbsp;Maybe this time I'll get it all in one email. &nbsp;:-)<br><br>At time \
15:55 I performed a write with CL = ONE, a subsequent read with CL = ONE and received \
correct results. &nbsp;Similar results occur with any read CL other than \
LOCAL_QUORUM.<br><br>At time 15:58 I performed a write with CL = ONE, a subsequent \
read with CL = LOCAL_QUORUM and received a TimedOutException.<br><br>System.log and \
cassandra.log are from server dlc05-sea to corresponds with the hector error message \
"Closing client CassandraClient&lt;dlc05-sea:9160-50&gt;". &nbsp;I haven't found \
anything in the logs that indicates the problem but maybe others with more discerning \
eyes can.<br><br>Please let me know if I can provide any additional \
information.<br><br>Thanks,<br>-Thor<br><br><span>&lt;logs.zip&gt;</span><span>&lt;ATT \
00001..txt&gt;</span></div></blockquote></div><br></div></div></div></blockquote></div><br></div></div></body></html>




[prev in list] [next in list] [prev in thread] [next in thread] 

Configure | About | News | Add a list | Sponsored by KoreLogic