cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "bjc (JIRA)" <j...@apache.org>
Subject [jira] Commented: (CASSANDRA-781) in a cluster, get_range_slice() does not return all the keys it should
Date Tue, 09 Feb 2010 23:25:28 GMT

    [ https://issues.apache.org/jira/browse/CASSANDRA-781?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12831752#action_12831752
] 

bjc commented on CASSANDRA-781:
-------------------------------

Ok, got the patch applied properly and things look better! The simple test passes. Awesome!!
:) However, the more complicated test uses a "start" offset after the first get_range_slice(),
and that still causes an exception. From the log:

 INFO 23:15:52,425 scanning node range (20123910036548544936247138992367052936,67283373037552029587203789575295250400]
ERROR 23:15:52,425 Internal error processing get_range_slice
java.lang.AssertionError: [124451343962032323897724984972289130546,67283373037552029587203789575295250400]
        at org.apache.cassandra.dht.Bounds.<init>(Bounds.java:26)
        at org.apache.cassandra.dht.Bounds.getRangeOrBounds(Bounds.java:74)
        at org.apache.cassandra.dht.Bounds.restrictTo(Bounds.java:59)
        at org.apache.cassandra.service.StorageProxy.getRangeSlice(StorageProxy.java:559)
        at org.apache.cassandra.thrift.CassandraServer.get_range_slice(CassandraServer.java:560)
        at org.apache.cassandra.thrift.Cassandra$Processor$get_range_slice.process(Cassandra.java:1189)
        at org.apache.cassandra.thrift.Cassandra$Processor.process(Cassandra.java:984)
        at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:253)
        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)

If you have trouble duplicating this one I can give you more information, but I did not have
difficulty getting it to happen. It happened every time I tried. For example, you can get
it to happen just by modifying one line of the simple test:

result = client.get_range_slice(ks, parent, predicate, "b37e14bb37304e0096e2e77a8fc88a5b",
"", 1000, ConsistencyLevel.ONE)

Of course, if you don't scan starting from "" then the simple test doesn't make sense, because
you might specifically exclude keys you are looking for by starting from the string I put
in.

However, the more complicated test I posted earlier makes sense and exercises the start and
end ranges of get_range_slice(). So..can we go back to the complicated test and specifically
make that one work? It's also nice because you should be able to run it over and over, since
it removes the keys at the end. With the simple test you have to manually flush the data and
restart the servers each time.

Thanks so much for fixing this! I am getting more familiar with the java so soon I might be
able to fix some bugs like this.

> in a cluster, get_range_slice() does not return all the keys it should
> ----------------------------------------------------------------------
>
>                 Key: CASSANDRA-781
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-781
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 0.5
>         Environment: Debian 5 lenny on EC2, Gentoo linux, Windows XP
>            Reporter: bjc
>            Assignee: Jonathan Ellis
>             Fix For: 0.5, 0.6
>
>         Attachments: 0001-fix-timeout-bug.txt, 0002-fix-slices-over-non-trivial-wrapped-ranges.txt,
AbstractBounds.java, Bounds.java
>
>
> get_range_slice() does not return the same set of keys as get_key_range() in 0.5.0 final.
> I posted a program to reproduce the behavior:
> http://www.mail-archive.com/cassandra-dev@incubator.apache.org/msg01474.html
> Apparently, you must have more than one node to get the behavior. Also, it may depend
on the locations of the nodes on the ring.. I.e., if you don't generate enough keys randomly,
then by chance they could all fall on the same host and you might not see the behavior, although
I was able to get it to happen using only 2 nodes and 10 keys.
> Here are the other emails describing the issue:
> http://www.mail-archive.com/cassandra-user@incubator.apache.org/msg02423.html

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


Mime
View raw message