Return-Path: X-Original-To: apmail-cassandra-user-archive@www.apache.org Delivered-To: apmail-cassandra-user-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 5A5305B4F for ; Thu, 12 May 2011 17:06:41 +0000 (UTC) Received: (qmail 63856 invoked by uid 500); 12 May 2011 17:06:39 -0000 Delivered-To: apmail-cassandra-user-archive@cassandra.apache.org Received: (qmail 63826 invoked by uid 500); 12 May 2011 17:06:39 -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 63818 invoked by uid 99); 12 May 2011 17:06:39 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 12 May 2011 17:06:39 +0000 X-ASF-Spam-Status: No, hits=0.0 required=5.0 tests=FREEMAIL_FROM,RCVD_IN_DNSWL_NONE,SPF_PASS,T_TO_NO_BRKTS_FREEMAIL X-Spam-Check-By: apache.org Received-SPF: pass (nike.apache.org: domain of daniel.doubleday@gmx.net designates 213.165.64.23 as permitted sender) Received: from [213.165.64.23] (HELO mailout-de.gmx.net) (213.165.64.23) by apache.org (qpsmtpd/0.29) with SMTP; Thu, 12 May 2011 17:06:30 +0000 Received: (qmail invoked by alias); 12 May 2011 17:06:09 -0000 Received: from p578bde86.dip0.t-ipconnect.de (EHLO caladan.smeet.de) [87.139.222.134] by mail.gmx.net (mp031) with SMTP; 12 May 2011 19:06:09 +0200 X-Authenticated: #3445653 X-Provags-ID: V01U2FsdGVkX1+dtLdJFK6djILHI9IDNcIsXvY0y/mrMXhjRMU+Ig ZxlJ7OtTssQiNf From: Daniel Doubleday Content-Type: text/plain; charset=us-ascii Content-Transfer-Encoding: quoted-printable Subject: Monitoring bytes read per cf Date: Thu, 12 May 2011 19:06:09 +0200 Message-Id: To: user@cassandra.apache.org Mime-Version: 1.0 (Apple Message framework v1084) X-Mailer: Apple Mail (2.1084) X-Y-GMX-Trusted: 0 X-Virus-Checked: Checked by ClamAV on apache.org Hi all got a question for folks with some code insight again. To be able to better understand where our IO load is coming from we want = to monitor the number of bytes read from disc per cf. (we love stats) What I have done is wrapping the FileDataInput in SSTableReader to sum = the bytes read in CFS. This will only record data file access but that = would be good enough for us. It seems to work fine but maybe someone here knows that this is not a = good idea .... Cheers, Daniel Some code: SSTableReader: private static final boolean KEEP_IO_STATISICS =3D = Boolean.getBoolean("cassandra.keepIOStats"); public FileDataInput getFileDataInput(DecoratedKey decoratedKey, int = bufferSize) { long position =3D getPosition(decoratedKey, Operator.EQ); if (position < 0) return null; FileDataInput segment =3D dfile.getSegment(position, = bufferSize); return (KEEP_IO_STATISICS) ? new = MonitoringFileDataIInput(metadata, segment) : segment;=20 } with MonitoringFileDataIInput public class MonitoringFileDataIInput implements FileDataInput, = Closeable { private final FileDataInput fileDataInput; private final ColumnFamilyStore columnFamilyStore; public MonitoringFileDataIInput(CFMetaData cfMetaData, FileDataInput = fileDataInput) { columnFamilyStore =3D = Table.open(cfMetaData.tableName).getColumnFamilyStore(cfMetaData.cfId); this.fileDataInput =3D fileDataInput; } @Override public boolean readBoolean() throws IOException { columnFamilyStore.addBytesRead(1); return fileDataInput.readBoolean(); =20 } // ... etc and ColumnFamilyStore private final AtomicLong bytesRead =3D new AtomicLong(0L); =20 @Override // ColumnFamilyStoreMBean public long getBytesRead() { return bytesRead.get(); } =20 public void addBytesRead(int num)=20 { bytesRead.addAndGet(num); } =20=