I inspected some code of the cluster ring. Thread [ReadStage:6] (Suspended) SuperColumn.isMarkedForDelete() line: 87 SliceQueryFilter.collectReducedColumns(IColumnContainer, Iterator<IColumn>, int) line: 138 QueryFilter.collectCollatedColumns(ColumnFamily, Iterator<IColumn>, int) line: 146 ColumnFamilyStore.getTopLevelColumns(QueryFilter, int) line: 1157 ColumnFamilyStore.getColumnFamily(QueryFilter, int) line: 1034 ColumnFamilyStore.getColumnFamily(QueryFilter) line: 1004 Table.getRow(QueryFilter) line: 359 SliceFromReadCommand.getRow(Table) line: 63 ReadVerbHandler.doVerb(Message) line: 73 MessageDeliveryTask.run() line: 62 ThreadPoolExecutor$Worker.runTask(Runnable) line: 886 ThreadPoolExecutor$Worker.run() line: 908 Thread.run() line: 619
In method of SliceQueryFilter.collectReducedColumns(), all columns has same values of SuperColumn.localDeletionTime and SuperColumn.markedForDeleteAt, which seemed to be current time. That caused SuperColumn.isMarkedForDelete() to return true, and then no liveColumns but all non-gc-able columns to be returned. This happened in super column family with LongType super column. (see description below.) SCF with UTF8Type super column worked normally. I'm not sure if the column was inserted with bad deletion time ,by bug of client API or etc, or there was a problem to read super columns with Long Type comparator, or else. ColumnFamily: LongtypeSCF (Super) Columns sorted by: org.apache.cassandra.db.marshal.LongType/org.apache.cassandra.db.marshal.UTF8Type Row cache size / save period: 0.0/0 Key cache size / save period: 200000.0/3600 Memtable thresholds: 0.19218749999999998/41/60 GC grace seconds: 864000 Compaction min/max thresholds: 4/32 Read repair chance: 1.0 ColumnFamily: StringTypeSCF (Super) Columns sorted by: org.apache.cassandra.db.marshal.UTF8Type/org.apache.cassandra.db.marshal.UTF8Type Row cache size / save period: 0.0/0 Key cache size / save period: 200000.0/3600 Memtable thresholds: 0.19218749999999998/41/60 GC grace seconds: 864000 Compaction min/max thresholds: 4/32 Read repair chance: 1.0 2010-11-18 (목), 04:28 +0000, Aaron Morton: > Just had a quick look at an 0.7b2 install and it appeared to be > working as expected. > > > Here's what I got for a row with 50 super columns, that each have 50 > columns. I ran the following get_slice calls . > > > get_slice with no super column specified, count=100 > returned 50 super columns, each with 50 columns > > > get_slice with no super column specified, count = 5 > returned 5 super column, each with 50 columns > > > If your get_slice does not specify a super column (on the ColumParent > arg) the count applies to the number of SuperColumn objects to return. > Each of those will have all of it's columns. If a super column is > specified on ColumnParent then the count refers to the number of > Columns to return. > > > If you're seeing something else can you send an example. > > > Thanks. > Aaron > > > On 18 Nov, 2010,at 03:04 PM, Hayarobi Park <hayarobip...@gmail.com> > wrote: > > > > It returned all columns within the range of start and end without > > regard > > to the count. the CF is super column family and I send the range of > > super column names of type Long. (and sub column name was UTF8) > > > > I put 2000 super columns in a row, and tried to read the first 50 > > columns in some range of columns. I inspected > > StorageProxy.readProtocol() after read your reply, and got the > > command > > object, of class SliceFromReadCommand, has the 'count' member > > variable > > having int value 50. > > > > I test get_slice request to super column family of UTF8Type/UTF8Type > > for > > super column name/column name, and this test was successfully return > > the > > columns with requested count. > > > > > > 2010-11-18 (목), 00:35 +1300, aaron morton: > > > The CassandraServer is not doing the read, step through the code > > from the call to readColumnFamily() in getSlice(). > > > > > > The read is passed to the StorageProxy.readProtocol() which looks > > at the CL and determines if its a weak or strong read, sends it out > > to all the replicas and manages everything. Eventually the request > > ends up as the ReadVerbHandler() where it will deserialise an > > instance of the SliceFromReadCommand and call it's getRow(). From > > there you can trace through how the count is used. > > > > > > Do you have a case where a call to the API returned more or less > > data than expected? > > > > > > Hope that helps. > > > Aaron > > > > > > On 17 Nov 2010, at 21:03, Hayarobi Park wrote: > > > > > > > Hello. > > > > > > > > I'm using cassandra (currently 0.7.0-beta3) in JAVA; with > > library > > > > hector. > > > > > > > > It seems that cassandra ignore the count of SliceRange when > > received > > > > get_slice request. > > > > > > > > > > > > I traced cassandra source code, and the part of code that > > retrieving > > > > columns does not get count as parameter. See, > > > > getSlice(List<ReadCommand> commands, ConsistencyLevel > > consistency_level) > > > > method in org.apachecassandra.thrift.CassandraServer class. > > (line > > > > 224~238 in 0.7.0-beta3) > > > > > > > > > > > > private Map<ByteBuffer, List<ColumnOrSuperColumn>> > > > > getSlice(List<ReadCommand> commands, ConsistencyLevel > > consistency_level) > > > > throws InvalidRequestException, UnavailableException, > > > > TimedOutException > > > > { > > > > Map<DecoratedKey, ColumnFamily> columnFamilies = > > > > readColumnFamily(commands, consistency_level); > > > > Map<ByteBuffer, List<ColumnOrSuperColumn>> columnFamiliesMap = > > > > new HashMap<ByteBuffer, List<ColumnOrSuperColumn>>(); > > > > for (ReadCommand command: commands) > > > > { > > > > ColumnFamily cf = > > > > > > columnFamilies.get(StorageService.getPartitioner().decorateKey(command.key)); > > > > boolean reverseOrder = command instanceof > > > > SliceFromReadCommand && > > ((SliceFromReadCommand)command).reversed; > > > > List<ColumnOrSuperColumn> thriftifiedColumns = > > > > thriftifyColumnFamily(cf, command.queryPath.superColumnName != > > null, > > > > reverseOrder); > > > > columnFamiliesMap.put(command.key, thriftifiedColumns); > > > > } > > > > > > > > return columnFamiliesMap; > > > > } > > > > > > > > When I inspected in debug mode, the command variable in for loop > > has the > > > > valid count value. The thriftifyColumnFamily(cf, > > > > command.queryPath.superColumnName != null, reverseOrder) method > > actually > > > > get columns but it has no way to get count value, and return all > > value > > > > that were not limit by the count. > > > > > > > > > > > > > > > > >