Return-Path: Delivered-To: apmail-cassandra-user-archive@www.apache.org Received: (qmail 45608 invoked from network); 17 Nov 2010 08:04:08 -0000 Received: from unknown (HELO mail.apache.org) (140.211.11.3) by 140.211.11.9 with SMTP; 17 Nov 2010 08:04:08 -0000 Received: (qmail 76898 invoked by uid 500); 17 Nov 2010 08:04:37 -0000 Delivered-To: apmail-cassandra-user-archive@cassandra.apache.org Received: (qmail 76763 invoked by uid 500); 17 Nov 2010 08:04:37 -0000 Mailing-List: contact user-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@cassandra.apache.org Delivered-To: mailing list user@cassandra.apache.org Received: (qmail 76754 invoked by uid 99); 17 Nov 2010 08:04:36 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 17 Nov 2010 08:04:36 +0000 X-ASF-Spam-Status: No, hits=0.0 required=10.0 tests=FREEMAIL_FROM,RCVD_IN_DNSWL_NONE,SPF_PASS,T_TO_NO_BRKTS_FREEMAIL X-Spam-Check-By: apache.org Received-SPF: pass (athena.apache.org: domain of hayarobipark@gmail.com designates 209.85.160.172 as permitted sender) Received: from [209.85.160.172] (HELO mail-gy0-f172.google.com) (209.85.160.172) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 17 Nov 2010 08:04:28 +0000 Received: by gyh20 with SMTP id 20so1017335gyh.31 for ; Wed, 17 Nov 2010 00:04:08 -0800 (PST) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=gamma; h=domainkey-signature:received:received:subject:from:to:content-type :organization:date:message-id:mime-version:x-mailer :content-transfer-encoding; bh=gfPqRVwAm0RnumZZfJ3/yy/se2e2Ao7xsXDFgdVKg7w=; b=AZcEUlm8DMbuVWXCIF0Fflqzwfc7A5KADWUswOmDQApsU4bDnEcLdx7FO+V1IZQI0H LUBeW7KtKw6NG+5CSFSSeGuNpi0QTFTVtAgB5tz3Nxey4bUQZVR59n/cMBIqMIL3jGs0 8c+TJLyK6dFM7AHHF6X6nO6ixu2hYOh7fNnSw= DomainKey-Signature: a=rsa-sha1; c=nofws; d=gmail.com; s=gamma; h=subject:from:to:content-type:organization:date:message-id :mime-version:x-mailer:content-transfer-encoding; b=wku0rdIQrCCTxnkelWJ8tczCixvX9NeTP4P74sZ+B8Mp30COVpS+Vxgy9mTw7iz7gU NMdVFOriJQmfSlGWeX1vA+YxmjJFuRoZiWHZD7E73nJwLAOsuAbRn72OV5QVk1ksF3aT Wv4PMiMKTWnNEN9hzLMH+9l7ylSdKmD2TvDws= Received: by 10.150.212.13 with SMTP id k13mr7039088ybg.396.1289981047837; Wed, 17 Nov 2010 00:04:07 -0800 (PST) Received: from [211.62.34.148] ([211.62.34.148]) by mx.google.com with ESMTPS id l66sm1427547yhd.20.2010.11.17.00.04.06 (version=SSLv3 cipher=RC4-MD5); Wed, 17 Nov 2010 00:04:07 -0800 (PST) Subject: Count of SliceRange in get_slice seems not to work From: Hayarobi Park To: user@cassandra.apache.org Content-Type: text/plain; charset="UTF-8" Organization: KT Hitel Date: Wed, 17 Nov 2010 17:03:59 +0900 Message-ID: <1289981039.5025.198.camel@hayarobi.paran.com> Mime-Version: 1.0 X-Mailer: Evolution 2.28.3 Content-Transfer-Encoding: 7bit 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 commands, ConsistencyLevel consistency_level) method in org.apache.cassandra.thrift.CassandraServer class. (line 224~238 in 0.7.0-beta3) private Map> getSlice(List commands, ConsistencyLevel consistency_level) throws InvalidRequestException, UnavailableException, TimedOutException { Map columnFamilies = readColumnFamily(commands, consistency_level); Map> columnFamiliesMap = new HashMap>(); for (ReadCommand command: commands) { ColumnFamily cf = columnFamilies.get(StorageService.getPartitioner().decorateKey(command.key)); boolean reverseOrder = command instanceof SliceFromReadCommand && ((SliceFromReadCommand)command).reversed; List 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.