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 CF2D9FA34 for ; Wed, 17 Apr 2013 04:30:46 +0000 (UTC) Received: (qmail 3484 invoked by uid 500); 17 Apr 2013 04:30:44 -0000 Delivered-To: apmail-cassandra-user-archive@cassandra.apache.org Received: (qmail 3459 invoked by uid 500); 17 Apr 2013 04:30:44 -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 3404 invoked by uid 99); 17 Apr 2013 04:30:41 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 17 Apr 2013 04:30:41 +0000 X-ASF-Spam-Status: No, hits=2.2 required=5.0 tests=HTML_MESSAGE,RCVD_IN_DNSWL_NONE,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (athena.apache.org: local policy) Received: from [208.113.200.5] (HELO homiemail-a46.g.dreamhost.com) (208.113.200.5) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 17 Apr 2013 04:30:36 +0000 Received: from homiemail-a46.g.dreamhost.com (localhost [127.0.0.1]) by homiemail-a46.g.dreamhost.com (Postfix) with ESMTP id 9ECA23E405B for ; Tue, 16 Apr 2013 21:30:15 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha1; c=relaxed; d=thelastpickle.com; h=from :content-type:subject:message-id:date:to:mime-version; s= thelastpickle.com; bh=Vwuxlu+t1wlwCqZUW6agVafaCtg=; b=bhH0wYeSEG iktaVn3zg4qrvwOWRt4GySD+T21EcY+ytDdjq7FxuKpRb4BOD+4YdyN9iKTUjwNM e/+zK3Hjb0gNgT00v5zf9ziSYa7AzNaHqo80WyavgpOkT+iwwiZUNPLjbRq6kyGM 9Er9VMk+w4ZL6aEBs2yynjTOlLAxcK9bs= Received: from [172.16.1.8] (unknown [203.86.207.101]) (using TLSv1 with cipher AES128-SHA (128/128 bits)) (No client certificate requested) (Authenticated sender: aaron@thelastpickle.com) by homiemail-a46.g.dreamhost.com (Postfix) with ESMTPSA id 1A14B3E4057 for ; Tue, 16 Apr 2013 21:30:14 -0700 (PDT) From: aaron morton Content-Type: multipart/alternative; boundary="Apple-Mail=_5172EEA8-1F6B-48FD-B472-3D4913421F5A" Subject: Commit Log question Message-Id: <9B0C0513-718A-48E9-AC89-B3D495ACCB0B@thelastpickle.com> Date: Wed, 17 Apr 2013 16:30:13 +1200 To: Cassandra User Mime-Version: 1.0 (Mac OS X Mail 6.2 \(1499\)) X-Mailer: Apple Mail (2.1499) X-Virus-Checked: Checked by ClamAV on apache.org --Apple-Mail=_5172EEA8-1F6B-48FD-B472-3D4913421F5A Content-Transfer-Encoding: quoted-printable Content-Type: text/plain; charset=windows-1252 I'm looking into a case where it appears that recycling a commit log = segment and flushing the dirty CF's results in 46 CF's being flushed. = Out of 47 in the keyspace. All this flush activity blocks writes.=20 Before I dig further I wanted to confirm my understanding. At 10:46 the MeteredFlusher kicks and flushed the CF=85.=20 2013-03-18T10:46:59-07:00 INFO [OptionalTasks:1] 2013-03-18 10:46:59,009 = MeteredFlusher.java (line 62) flushing high-traffic column family = CFS(Keyspace=3D'KSNAME', ColumnFamily=3D'CFNAME') (estimated 198959043 = bytes) 2013-03-18T10:46:59-07:00 INFO [OptionalTasks:1] 2013-03-18 10:46:59,009 = ColumnFamilyStore.java (line 659) Enqueuing flush of = Memtable-CFNAME@172178999(158553073/198959043 serialized/live bytes, = 470065 ops) 2013-03-18T10:46:59-07:00 INFO [FlushWriter:482] 2013-03-18 10:46:59,010 = Memtable.java (line 264) Writing = Memtable-CFNAME@172178999(158553073/198959043 serialized/live bytes, = 470065 ops) 2013-03-18T10:47:01-07:00 INFO [FlushWriter:482] 2013-03-18 10:47:01,503 = Memtable.java (line 305) Completed flushing = /mnt/cassandra/data/KSNAME/CFNAME/KSNAME-CFNAME-hf-590-Data.db (29212692 = bytes) for commitlog position ReplayPosition(segmentId=3D1363374084494, = position=3D982002808) I expect that the CF is marked as clean in all CL segments other than = the active one (1363374084494), and in the active one if the last write = location for the CF is before the current context position (982002808). There is nothing else in the log (other than liveRatio measuring) then a = storm of flushing without the MeteredFlusher being involved. I attribute = this to the CL growing over capacity and the CF's marked as dirty in the = oldest segment being flushed. As part of that the CF from above is flushed.=20 2013-03-18T10:50:05-07:00 INFO [OptionalTasks:1] 2013-03-18 10:50:05,777 = ColumnFamilyStore.java (line 659) Enqueuing flush of = Memtable-CFNAME@1841064520(93240219/117001736 serialized/live bytes, = 71513 ops) 2013-03-18T10:50:05-07:00 INFO [FlushWriter:483] 2013-03-18 10:50:05,942 = Memtable.java (line 264) Writing = Memtable-CFNAME@1841064520(93240219/117001736 serialized/live bytes, = 71513 ops) 2013-03-18T10:50:07-07:00 INFO [FlushWriter:483] 2013-03-18 10:50:07,277 = Memtable.java (line 305) Completed flushing = /mnt/cassandra/data/KSNAME/CFNAME/KSNAME-CFNAME-hf-591-Data.db (12050522 = bytes) for commitlog position ReplayPosition(segmentId=3D1363374084495, = position=3D8542666) Not that the segmentId has only increased by 1, so we have not put a = huge amount of traffic through the commit log.=20 My understanding is that after the first flush the CF would be clean in = the oldest CL segment and I should not see the CF being flushed a second = time. Is that correct?=20 running on 1.1.6 =20 Cheers ----------------- Aaron Morton Freelance Cassandra Consultant New Zealand @aaronmorton http://www.thelastpickle.com --Apple-Mail=_5172EEA8-1F6B-48FD-B472-3D4913421F5A Content-Transfer-Encoding: quoted-printable Content-Type: text/html; charset=windows-1252 I'm = looking into a case where it appears that recycling a commit log segment = and flushing the dirty CF's results in 46 CF's being flushed. Out of 47 = in the keyspace. All this flush activity blocks = writes. 

Before I dig further I wanted to = confirm my understanding.

At 10:46 the = MeteredFlusher kicks and flushed the = CF=85. 

2013-03-18T10:46:59-07:00 = INFO [OptionalTasks:1] 2013-03-18 10:46:59,009 MeteredFlusher.java (line = 62) flushing high-traffic column family CFS(Keyspace=3D'KSNAME', = ColumnFamily=3D'CFNAME') (estimated 198959043 = bytes)
2013-03-18T10:46:59-07:00 INFO [OptionalTasks:1] = 2013-03-18 10:46:59,009 ColumnFamilyStore.java (line 659) Enqueuing = flush of Memtable-CFNAME@172178999(158553073/198959043 serialized/live = bytes, 470065 ops)
2013-03-18T10:46:59-07:00 INFO = [FlushWriter:482] 2013-03-18 10:46:59,010 Memtable.java (line 264) = Writing Memtable-CFNAME@172178999(158553073/198959043 serialized/live = bytes, 470065 ops)
2013-03-18T10:47:01-07:00 INFO = [FlushWriter:482] 2013-03-18 10:47:01,503 Memtable.java (line 305) = Completed flushing = /mnt/cassandra/data/KSNAME/CFNAME/KSNAME-CFNAME-hf-590-Data.db (29212692 = bytes) for commitlog position ReplayPosition(segmentId=3D1363374084494, = position=3D982002808)

I expect that the CF is = marked as clean in all CL segments other than the active one = (1363374084494), and in the active one if the last write location for = the CF is before the current context position = (982002808).

There is nothing else in the log = (other than liveRatio measuring) then a storm of flushing without the = MeteredFlusher being involved. I attribute this to the CL growing over = capacity and the CF's marked as dirty in the oldest segment being = flushed.

As part of that the CF from above is = flushed. 

2013-03-18T10:50:05-07:00 INFO = [OptionalTasks:1] 2013-03-18 10:50:05,777 ColumnFamilyStore.java (line = 659) Enqueuing flush of Memtable-CFNAME@1841064520(93240219/117001736 = serialized/live bytes, 71513 ops)
2013-03-18T10:50:05-07:00 = INFO [FlushWriter:483] 2013-03-18 10:50:05,942 Memtable.java (line 264) = Writing Memtable-CFNAME@1841064520(93240219/117001736 serialized/live = bytes, 71513 ops)
2013-03-18T10:50:07-07:00 INFO = [FlushWriter:483] 2013-03-18 10:50:07,277 Memtable.java (line 305) = Completed flushing = /mnt/cassandra/data/KSNAME/CFNAME/KSNAME-CFNAME-hf-591-Data.db (12050522 = bytes) for commitlog position ReplayPosition(segmentId=3D1363374084495, = position=3D8542666)

Not that the = segmentId has only increased by 1, so we have not put a huge amount of = traffic through the commit = log. 

My understanding is that = after the first flush the CF would be clean in the oldest CL segment and = I should not see the CF being flushed a second time. Is that = correct? 

running on = 1.1.6
  
Cheers

http://www.thelastpickle.com

= --Apple-Mail=_5172EEA8-1F6B-48FD-B472-3D4913421F5A--