Return-Path: Delivered-To: apmail-cassandra-user-archive@www.apache.org Received: (qmail 62907 invoked from network); 11 Apr 2011 00:14:42 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) by minotaur.apache.org with SMTP; 11 Apr 2011 00:14:42 -0000 Received: (qmail 24861 invoked by uid 500); 11 Apr 2011 00:14:39 -0000 Delivered-To: apmail-cassandra-user-archive@cassandra.apache.org Received: (qmail 24833 invoked by uid 500); 11 Apr 2011 00:14: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 24825 invoked by uid 99); 11 Apr 2011 00:14:39 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 11 Apr 2011 00:14:39 +0000 X-ASF-Spam-Status: No, hits=0.7 required=5.0 tests=RCVD_IN_DNSWL_NONE,SPF_NEUTRAL X-Spam-Check-By: apache.org Received-SPF: neutral (athena.apache.org: local policy) Received: from [66.33.216.122] (HELO hapkido.dreamhost.com) (66.33.216.122) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 11 Apr 2011 00:14:32 +0000 Received: from homiemail-a45.g.dreamhost.com (caibbdcaaaaf.dreamhost.com [208.113.200.5]) by hapkido.dreamhost.com (Postfix) with ESMTP id 7D557179281 for ; Sun, 10 Apr 2011 17:14:11 -0700 (PDT) Received: from homiemail-a45.g.dreamhost.com (localhost [127.0.0.1]) by homiemail-a45.g.dreamhost.com (Postfix) with ESMTP id 9127448056 for ; Sun, 10 Apr 2011 17:14:05 -0700 (PDT) DomainKey-Signature: a=rsa-sha1; c=nofws; d=thelastpickle.com; h=content-type :mime-version:subject:from:in-reply-to:date :content-transfer-encoding:message-id:references:to; q=dns; s= thelastpickle.com; b=YwNI8xpjUnhhr4dIQF+yftYwr0mQmzsywlx00qpg+z/ mv2PT6eUZfW9bpGgKUaRLGZHy/P93Z8DW29t2h8GwP6Nwtb8WoZ+Y7o/Et/fX/dr ew29KdYmTcBkIkgWId+Cw7aMaTlQ+gEzo+Z1TEIKK6IY0vykGg73lwQI9NM07iXc = DKIM-Signature: v=1; a=rsa-sha1; c=relaxed; d=thelastpickle.com; h= content-type:mime-version:subject:from:in-reply-to:date :content-transfer-encoding:message-id:references:to; s= thelastpickle.com; bh=N7ptZEsqd1lyLLiraYlNxDykNU4=; b=Yp8xfYmb3o XsZ6hLbXVDAA5wiLkN9NLKtg/lOZG565D7BApU2GmhJ3BGAzCz8P9DAUrAAY0U+x ze88bxIlA3Fn8cHOeVQWngM/gZJdCSqy8Aox98bsA4yKzSDXAsiYyLB0DjNLaTvj OttGdrjHMW48kMTe/2qIOMjWk7FmbcqhA= Received: from [10.0.1.155] (121-73-157-230.cable.telstraclear.net [121.73.157.230]) (using TLSv1 with cipher AES128-SHA (128/128 bits)) (No client certificate requested) (Authenticated sender: aaron@thelastpickle.com) by homiemail-a45.g.dreamhost.com (Postfix) with ESMTPSA id C9B8748051 for ; Sun, 10 Apr 2011 17:14:04 -0700 (PDT) Content-Type: text/plain; charset=us-ascii Mime-Version: 1.0 (Apple Message framework v1084) Subject: Re: unrepairable sstable data rows From: aaron morton In-Reply-To: <0D174140-3132-4DCA-9BB3-A6BEB2FB9BCE@gmail.com> Date: Mon, 11 Apr 2011 12:14:02 +1200 Content-Transfer-Encoding: quoted-printable Message-Id: References: <0D174140-3132-4DCA-9BB3-A6BEB2FB9BCE@gmail.com> To: user@cassandra.apache.org X-Mailer: Apple Mail (2.1084) The WARN messages are the "emergency pressure valve" kicking in, search = for that test in conf/cassandra.yaml. These are settings to reduce the = chance of going OOM. It means you should take a look at your memtable = and cache settings as you are getting close to running out of memory.=20 Jonathan or Sylvain will have a better idea of why/how the error is = occurring.=20 But if you wanted to get fresh data on the node, a simple approach is to = delete/move just the SSTable that is causing problems then run a repair. = That should reduce the amount of data that needs to be moved.=20 Hope that helps. Aaron On 11 Apr 2011, at 00:15, Jonathan Colby wrote: > It appears we have several unserializable or unreadable rows. These = were not fixed even after doing a "scrub" on all nodes - even though = the scrub seemed to have completed successfully. >=20 > I trying to fix these by doing a "repair", but these exceptions are = thrown exactly when doing a repair. Anyone run into this issue? = What's the best way to fix this? =20 >=20 > I was thinking that flushing and reloading the data with a move = (reusing the same token) might be a way to get out of this. >=20 >=20 > Exception seem multiple times for different keys during a repair: >=20 > ERROR [CompactionExecutor:1] 2011-04-10 14:05:55,528 = PrecompactedRow.java (line 82) Skipping row = DecoratedKey(58054163627659284217684165071269705317, = 64396663313763662d383432622d343439652d623761312d643164663936333738306565) = in /var/lib/cassandra/data/DFS/main-f-232-Data.db > java.io.EOFException > at java.io.RandomAccessFile.readFully(RandomAccessFile.java:383) > at java.io.RandomAccessFile.readFully(RandomAccessFile.java:361) > at = org.apache.cassandra.io.util.BufferedRandomAccessFile.readBytes(BufferedRa= ndomAccessFile.java:268) > at = org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:310) > at = org.apache.cassandra.utils.ByteBufferUtil.readWithLength(ByteBufferUtil.ja= va:267) > at = org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java= :94) > at = org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java= :35) > at = org.apache.cassandra.db.ColumnFamilySerializer.deserializeColumns(ColumnFa= milySerializer.java:129) > at = org.apache.cassandra.io.sstable.SSTableIdentityIterator.getColumnFamilyWit= hColumns(SSTableIdentityIterator.java:176) > at = org.apache.cassandra.io.PrecompactedRow.(PrecompactedRow.java:78) > at = org.apache.cassandra.io.CompactionIterator.getCompactedRow(CompactionItera= tor.java:139) > at = org.apache.cassandra.io.CompactionIterator.getReduced(CompactionIterator.j= ava:108) > at = org.apache.cassandra.io.CompactionIterator.getReduced(CompactionIterator.j= ava:43) > at = org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterator.j= ava:73) > at = com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterat= or.java:136) > at = com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:1= 31) > at = org.apache.commons.collections.iterators.FilterIterator.setNextObject(Filt= erIterator.java:183) > at = org.apache.commons.collections.iterators.FilterIterator.hasNext(FilterIter= ator.java:94) > at = org.apache.cassandra.db.CompactionManager.doValidationCompaction(Compactio= nManager.java:803) > at = org.apache.cassandra.db.CompactionManager.access$800(CompactionManager.jav= a:56) > at = org.apache.cassandra.db.CompactionManager$6.call(CompactionManager.java:35= 8) > at = java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303) > at java.util.concurrent.FutureTask.run(FutureTask.java:138) > at = java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.= java:886) > at = java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java= :908) > at java.lang.Thread.run(Thread.java:662) >=20 >=20 > This WARN also seems to come up often during a repair. Not sure if it = related to this problem: >=20 > WARN [ScheduledTasks:1] 2011-04-10 14:10:24,991 GCInspector.java (line = 149) Heap is 0.8675910480028087 full. You may need to reduce memtable = and/or cache sizes. Cassandra will now flush up to the two largest = memtables to free up memory. Adjust flush_largest_memtables_at = threshold in cassandra.yaml if you don't want Cassandra to do this = automatically > WARN [ScheduledTasks:1] 2011-04-10 14:10:24,992 StorageService.java = (line 2206) Flushing ColumnFamilyStore(table=3D'DFS', = columnFamily=3D'main') to relieve memory pressure > INFO [ScheduledTasks:1] 2011-04-10 14:10:24,992 ColumnFamilyStore.java = (line 695) switching in a fresh Memtable for main at = CommitLogContext(file=3D'/var/lib/cassandra/commitlog/CommitLog-1302435708= 131.log', position=3D28257053) >=20