[
https://issues.apache.org/jira/browse/CASSANDRA-884?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Omer van der Horst Jansen reopened CASSANDRA-884:
-------------------------------------------------
We've started using the wrapper API I've written for Cassandra for light
development here and quite unexpectedly I've started getting
NullPointerExceptions whenever a get_range_slices request is performed against
a particular CF. The CF in question has a replication factor of 2.
get_range_slices still works fine against the Standard1 CF that's exercised by
the TestApp2 file included in this ticket. In my storage-conf.xml Standard1 is
set up with a replication factor of 3.
Here is the stack trace I'm seeing over and over again, on each of my 4 servers:
ERROR [pool-1-thread-64] 2010-03-19 12:23:30,119 Cassandra.java (line 1440)
Internal error processing get_range_slices
java.lang.NullPointerException
at
org.apache.cassandra.service.RangeSliceResponseResolver$2.isEqual(RangeSliceResponseResolver.java:81)
at
org.apache.cassandra.service.RangeSliceResponseResolver$2.isEqual(RangeSliceResponseResolver.java:74)
at
org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterator.java:69)
at
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:135)
at
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:130)
at
org.apache.cassandra.service.RangeSliceResponseResolver.resolve(RangeSliceResponseResolver.java:101)
at
org.apache.cassandra.service.RangeSliceResponseResolver.resolve(RangeSliceResponseResolver.java:41)
at
org.apache.cassandra.service.QuorumResponseHandler.get(QuorumResponseHandler.java:86)
at
org.apache.cassandra.service.StorageProxy.getRangeSlice(StorageProxy.java:592)
at
org.apache.cassandra.thrift.CassandraServer.getRangeSlicesInternal(CassandraServer.java:587)
at
org.apache.cassandra.thrift.CassandraServer.get_range_slices(CassandraServer.java:559)
at
org.apache.cassandra.thrift.Cassandra$Processor$get_range_slices.process(Cassandra.java:1432)
at
org.apache.cassandra.thrift.Cassandra$Processor.process(Cassandra.java:1115)
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)
Additional notes:
I'm using revision 924837 of the Cassandra 0.6 branch which includes the patch
above. For some reason the line numbers in RangeSliceResponseResolver seem to
be off by one in the stack trace, but I'm positive that it's the patched
version being used with the isEqual() method that tests only the keys.
I guess this might be fixed by adding a few null checks to that isEqual()
method, but perhaps that would just be hiding a problem somewhere else in the
code.
> get_range_slice returns multiple copies of each row
> ---------------------------------------------------
>
> Key: CASSANDRA-884
> URL: https://issues.apache.org/jira/browse/CASSANDRA-884
> Project: Cassandra
> Issue Type: Bug
> Components: Core
> Affects Versions: 0.5
> Environment: 4-cluster Gentoo Linux 2.6.18 with a ReplicationFactor
> of 2
> Reporter: Omer van der Horst Jansen
> Assignee: Omer van der Horst Jansen
> Fix For: 0.6
>
> Attachments: 0001-RangeSliceResponseResolver.patch, 884-v2.txt,
> TestApp2.java
>
>
> I've noticed that both 0.5.1 and 0.6b2 return multiple identical copies of
> the data stored in my keyspace whenever I make a call to get_range_slice or
> get_range_slices using
> ConsistencyLevel.QUORUM and ReplicationFactor is greater than one.
> So with ReplicationFactor set to 2 for my application's KeySpace I get double
> the number of KeySlices that I expect to get. When using ConsistencyLevel.ONE
> I get only one KeySlice for each row.
> I've seen this happen with Cassandra 0.5.1 and with 0.6 beta 2. The behavior
> on 0.6 beta 2 is exhibited with both get_range_slice and get_range_slices.
> The attached Java program demonstrates the issue for 0.6 beta 2. The program
> writes a series of single-column rows into the Standard1 table, and then uses
> get_range_slice to receive a list of all row. The returned number of rows is
> consistently twice the number of rows written to the database. I wipe out the
> database completely before running the test.
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.