cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From aaron morton <aa...@thelastpickle.com>
Subject Re: Problems with subcolumn retrieval after upgrade from 0.6 to 0.7
Date Mon, 18 Apr 2011 23:41:32 GMT
Can you could provide an example of a get_slice request that failed and the columns that were
returned, so we can see the actual bytes for the super column and column names.

Aaron


On 19 Apr 2011, at 09:26, Abraham Sanderson wrote:

> I wish it were consistent enough that the answer were simple...  It varies between just
the requested subcolumn to all subcolumns.  It always does return the columns in order, and
the requested column is always one of the columns returned.   However, the slice start is
not consistently in the same place(like n+1 or n-1).  For example, if I have CF['key']['supercolumn'
['a','b','c','d','e']], and query for 'c', sometimes i get a slice with 'a', 'b', 'c', other
times its 'b', 'c', 'd', sometimes 'c', 'd'.  When the column name is closer to the end of
the range('d' or 'e'), sometimes it justs a slice with the column.  The sporadic behavior
makes me think that it's a race condition, but the behavior linked to the column range makes
we think I'm overrunning the buffer somewhere.  I at first suspected that I was inadvertently
making modifications to the buffers in application code during serialization/deserialization,
so I did the tests in the cli.  This limits it to just cassandra/thrift code and my custom
types.  Am I missing some other factor?  While debugging I have noticed that the byte buffers
contain more than they used to; it looks to me like tokens that contain parts of the thrift
response.  I'd see strings like "???get_slice???Foo??7c2f5d5b-b370-42e1-a6a2-77fc721440fe????"
 Is it possible that I am inadvertently using a reserved token or something on my supercolumn
name and this is screwing with the slice command?
> 
> Abe
> 
> On Mon, Apr 18, 2011 at 2:55 PM, aaron morton <aaron@thelastpickle.com> wrote:
> When you run the get_slice which columns are returned ? 
> 
> 
> Aaron
> 
> On 19 Apr 2011, at 04:12, Abraham Sanderson wrote:
> 
>> Ok, I made the changes and tried again.  Here is the before modifying my method using
a simple get, confirmed the same output in the cli:
>> 
>> DEBUG [pool-1-thread-2] 2011-04-18 09:37:23,910 CassandraServer.java (line 279) get
>> DEBUG [pool-1-thread-2] 2011-04-18 09:37:23,911 StorageProxy.java (line 322) Command/ConsistencyLevel
is SliceByNamesReadCommand(table='DocStore', key=64316337663662392d313432352d346661622d623037342d353537346335346361653038,
columnParent='QueryPath(columnFamilyName='Tran
>> slationsByTarget', superColumnName='java.nio.HeapByteBuffer[pos=95 lim=211 cap=244]',
columnName='null')', columns=[7c2f5d5b-b370-42e1-a6a2-77fc721440fe,])/ALL
>> DEBUG [pool-1-thread-2] 2011-04-18 09:37:23,911 ReadCallback.java (line 84) Blockfor/repair
is 1/true; setting up requests to localhost/127.0.0.1
>> DEBUG [pool-1-thread-2] 2011-04-18 09:37:23,911 StorageProxy.java (line 345) reading
data locally
>> DEBUG [ReadStage:4] 2011-04-18 09:37:23,911 StorageProxy.java (line 450) LocalReadRunnable
reading SliceByNamesReadCommand(table='DocStore', key=64316337663662392d313432352d346661622d623037342d353537346335346361653038,
columnParent='QueryPath(columnFamilyName='Translatio
>> nsByTarget', superColumnName='java.nio.HeapByteBuffer[pos=95 lim=211 cap=244]', columnName='null')',
columns=[7c2f5d5b-b370-42e1-a6a2-77fc721440fe,])
>> DEBUG [pool-1-thread-2] 2011-04-18 09:37:23,912 StorageProxy.java (line 395) Read:
1 ms.
>> ERROR [pool-1-thread-2] 2011-04-18 09:37:23,912 Cassandra.java (line 2665) Internal
error processing get
>> java.lang.AssertionError
>>         at org.apache.cassandra.thrift.CassandraServer.get(CassandraServer.java:300)
>>         at org.apache.cassandra.thrift.Cassandra$Processor$get.process(Cassandra.java:2655)
>>         at org.apache.cassandra.thrift.Cassandra$Processor.process(Cassandra.java:2555)
>>         at org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:206)
>>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
>>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
>>         at java.lang.Thread.run(Thread.java:636)
>> 
>> And here is the after...it succeeds here but still gives me multiple subcolumns in
the response.  Same behavior, it seems, I'm just sidestepping the original AssertionError:
>> 
>> DEBUG [pool-1-thread-6] 2011-04-18 09:50:26,617 CassandraServer.java (line 232) get_slice
>> DEBUG [pool-1-thread-6] 2011-04-18 09:50:26,617 StorageProxy.java (line 322) Command/ConsistencyLevel
is SliceByNamesReadCommand(table='DocStore', key=64316337663662392d313432352d346661622d623037342d353537346335346361653038,
columnParent='QueryPath(columnFamilyName='TranslationsByTarget', superColumnName='java.nio.HeapByteBuffer[pos=101
lim=217 cap=259]', columnName='null')', columns=[7c2f5d5b-b370-42e1-a6a2-77fc721440fe,])/ALL
>> DEBUG [pool-1-thread-6] 2011-04-18 09:50:26,617 ReadCallback.java (line 84) Blockfor/repair
is 1/true; setting up requests to localhost/127.0.0.1
>> DEBUG [pool-1-thread-6] 2011-04-18 09:50:26,617 StorageProxy.java (line 345) reading
data locally
>> DEBUG [ReadStage:3] 2011-04-18 09:50:26,618 StorageProxy.java (line 450) LocalReadRunnable
reading SliceByNamesReadCommand(table='DocStore', key=64316337663662392d313432352d346661622d623037342d353537346335346361653038,
columnParent='QueryPath(columnFamilyName='TranslationsByTarget', superColumnName='java.nio.HeapByteBuffer[pos=101
lim=217 cap=259]', columnName='null')', columns=[7c2f5d5b-b370-42e1-a6a2-77fc721440fe,])
>> DEBUG [pool-1-thread-6] 2011-04-18 09:50:26,618 StorageProxy.java (line 395) Read:
0 ms.
>> 
>> My comparators are relatively simple.  Basically I have a schema that required heterogenous
columns, but I needed to be able to deserialize them in unique ways.  So there is always a
type byte that precedes the bytes of the data.  The supercolumn in this case is a general
data type, which happens to represent a serializable object:
>> 
>>   public void validate(ByteBuffer bytes)
>>     throws MarshalException
>>   {
>>     if(bytes.remaining() == 0)
>>       return;
>> 
>>     validateDataType(bytes.get(bytes.position()));
>>     return;
>>   }
>> 
>>   public int compare(ByteBuffer bytes1, ByteBuffer bytes2)
>>   {
>>     if (bytes1.remaining() == 0)
>>       return bytes2.remaining() == 0 ? 0 : -1;
>>     else if (bytes2.remaining() == 0)
>>       return 1;
>>     else
>>     {
>>       // compare type bytes                                                     
                                                                                         
                                                                                         
          
>>       byte T1 = bytes1.get(bytes1.position());
>>       byte T2 = bytes2.get(bytes2.position());
>>       if (T1 != T2)
>>         return (T1 - T2);
>> 
>>       // compare values                                                         
                                                                                         
                                                                                         
          
>>       return ByteBufferUtil.compareUnsigned(bytes1, bytes2);
>>     }
>>   }
>> 
>> The subcolumn is similar...just a UUID with a type byte prefix:
>> 
>>   public void validate(ByteBuffer bytes)
>>     throws MarshalException
>>   {
>>     if(bytes.remaining() == 0)
>>       return;
>> 
>>     validateDataType(bytes.get(bytes.position()));
>>     if((bytes.remaining() - 1) == 0)
>>       return;
>>     else if((bytes.remaining() - 1) != 16)
>>       throw new MarshalException("UUID value must be exactly 16 bytes");
>>   }
>> 
>>   public int compare(ByteBuffer bytes1, ByteBuffer bytes2)
>>   {
>>     if (bytes1.remaining() == 0)
>>       return bytes2.remaining() == 0 ? 0 : -1;
>>     else if (bytes2.remaining() == 0)
>>       return 1;
>>     else
>>     {
>>       // compare type bytes                                                     
                                                                                         
                                                                                         
          
>>       byte T1 = bytes1.get(bytes1.position());
>>       byte T2 = bytes2.get(bytes2.position());
>>       if (T1 != T2)
>>         return (T1 - T2);
>> 
>>       // compare values                                                         
                                                                                         
                                                                                         
          
>>       UUID U1 = getUUID(bytes1, bytes1.position()+1);
>>       UUID U2 = getUUID(bytes2, bytes2.position()+1);
>>       return U1.compareTo(U2);
>>     }
>>   }
>> 
>>   static UUID getUUID(ByteBuffer bytes, int pos)
>>   {
>>     long msBits = bytes.getLong(pos);
>>     long lsBits = bytes.getLong(pos+8);
>>     return new UUID(msBits, lsBits);
>>   }
>> 
>> All of my buffer reads are done by index, the position shouldn't be changing at all.
>> 
>> Abe Sanderson
>> 
>> On Sat, Apr 16, 2011 at 5:38 PM, aaron morton <aaron@thelastpickle.com> wrote:
>> Can you run the same request as a get_slice naming the column in the SlicePredicate
and see what comes back ?
>> 
>> Can you reproduce the fault with logging set at DEBUG and send the logs ?
>> 
>> Also, whats the compare function like for your custom type ?
>> 
>> Cheers
>> Aaron
>> 
>> 
>> On 16 Apr 2011, at 07:34, Abraham Sanderson wrote:
>> 
>> > I'm having some issues with a few of my ColumnFamilies after a cassandra upgrade/import
from 0.6.1 to 0.7.4.  I followed the instructions to upgrade and everything seem to work OK...until
I got into the application and noticed some wierd behavior.  I was getting the following stacktrace
in cassandra occassionally when I did get operations for a single subcolumn for some of the
Super type CFs:
>> >
>> > ERROR 12:56:05,669 Internal error processing get
>> > java.lang.AssertionError
>> >         at org.apache.cassandra.thrift.
>> > CassandraServer.get(CassandraServer.java:300)
>> >         at org.apache.cassandra.thrift.Cassandra$Processor$get.process(Cassandra.java:2655)
>> >         at org.apache.cassandra.thrift.Cassandra$Processor.process(Cassandra.java:2555)
>> >         at org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:206)
>> >         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
>> >         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
>> >         at java.lang.Thread.run(Thread.java:636)
>> >
>> > The assertion that is failing is the check that only one column is retrieved
by the get.  I did some debugging with the cli and a remote  debugger and found a few interesting
patterns.  First, the problem does not seem consistently duplicatable.  If one supercolumn
is affected though, it will happen more frequently for subcolumns that when sorted appear
at the beginning of the range.  For columns near the end of the range, it seems to be more
intermittent, and almost never occurs when I step through the code line by line.  The only
factor I can think of that might cause issues is that I am using custom data types for all
supercolumns and columns.  I originally thought I might be reading past the end of the ByteBuffer,
but I have quadrupled checked that this is not the case.
>> >
>> > Abe Sanderson
>> 
>> 
> 
> 


Mime
View raw message