Return-Path: Delivered-To: apmail-cassandra-user-archive@www.apache.org Received: (qmail 33285 invoked from network); 8 Apr 2011 15:56:10 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) by minotaur.apache.org with SMTP; 8 Apr 2011 15:56:10 -0000 Received: (qmail 27880 invoked by uid 500); 8 Apr 2011 15:56:08 -0000 Delivered-To: apmail-cassandra-user-archive@cassandra.apache.org Received: (qmail 27860 invoked by uid 500); 8 Apr 2011 15:56:08 -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 27852 invoked by uid 99); 8 Apr 2011 15:56:08 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 08 Apr 2011 15:56:08 +0000 X-ASF-Spam-Status: No, hits=-0.7 required=5.0 tests=RCVD_IN_DNSWL_LOW,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (athena.apache.org: domain of sylvain@datastax.com designates 209.85.213.172 as permitted sender) Received: from [209.85.213.172] (HELO mail-yx0-f172.google.com) (209.85.213.172) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 08 Apr 2011 15:56:00 +0000 Received: by yxk30 with SMTP id 30so1753718yxk.31 for ; Fri, 08 Apr 2011 08:55:39 -0700 (PDT) MIME-Version: 1.0 Received: by 10.236.123.227 with SMTP id v63mr2832693yhh.455.1302278139025; Fri, 08 Apr 2011 08:55:39 -0700 (PDT) Received: by 10.147.167.9 with HTTP; Fri, 8 Apr 2011 08:55:38 -0700 (PDT) X-Originating-IP: [64.132.24.248] In-Reply-To: <0B5765C5-D5C5-44CD-8607-00B1493C1F82@gmail.com> References: <0B5765C5-D5C5-44CD-8607-00B1493C1F82@gmail.com> Date: Fri, 8 Apr 2011 10:55:38 -0500 Message-ID: Subject: Re: Is the repair still going on or did it fail because of exceptions? From: Sylvain Lebresne To: user@cassandra.apache.org Cc: Jonathan Colby Content-Type: text/plain; charset=ISO-8859-1 Content-Transfer-Encoding: quoted-printable Sadly repair isn't very resilient to errors and has failed. There is a few ticket open to improve this and repair in general but right now, if any problems occurs during repairs, it will fail (and nodetool repair won't return, so you could just ctrl-c). Provided you're on a recent enough cassandra, I suggest you run scrub on the node giving the error. -- Sylvain On Fri, Apr 8, 2011 at 9:19 AM, Jonathan Colby w= rote: > It seems on my cluster there are a few unserializable Rows. =A0I'm trying= to run a repair on the nodes, but it also seems that the replica nodes hav= e unreadable or unserializable rows. =A0 =A0The problem is, I cannot determ= ine if the repair is still going on, or if was interrupted because of these= errors. =A0 It is unclear because nothing else related to the repair show = up in the logs. =A0It's been about 5 hours and I also don't see anything ha= ppening when I perform a "nodetool netstats" on the nodes. =A0The nodetool = repair command is still blocking from the console. > > On the node I'm trying to repair, I see this after launching a "repair": > > ... > =A0INFO [manual-repair-6160b400-2c82-4ccb-9451-79caafd7d3cc] 2011-04-08 1= 1:41:55,520 AntiEntropyService.java (line 770) Waiting for repair requests:= [# 9caafd7d3cc, /10.46.108.102, (DFS,main)>, #, # , /10.46.108.100, (DFS,main)>, #] > ... > > In the log of the node "10.46.108.102" where the repair tries to compare = the replica data, =A0 I see a couple of the below exceptions a few minutes = later. =A0 =A0Are the exceptions bad enough to cause the repair to fail? > > > ERROR [CompactionExecutor:1] 2011-04-08 11:43:01,177 PrecompactedRow.java= (line 82) Skipping row DecoratedKey(1782314446006375058060694305099335169,= 4d657373616765456e726963686d656e743a31343236) in /va > r/lib/cassandra/data/DFS/main-f-177-Data.db > java.io.EOFException > =A0 =A0 =A0 =A0at java.io.RandomAccessFile.readFully(RandomAccessFile.jav= a:383) > =A0 =A0 =A0 =A0at java.io.RandomAccessFile.readFully(RandomAccessFile.jav= a:361) > =A0 =A0 =A0 =A0at org.apache.cassandra.io.util.BufferedRandomAccessFile.r= eadBytes(BufferedRandomAccessFile.java:268) > =A0 =A0 =A0 =A0at org.apache.cassandra.utils.ByteBufferUtil.read(ByteBuff= erUtil.java:310) > =A0 =A0 =A0 =A0at org.apache.cassandra.utils.ByteBufferUtil.readWithLengt= h(ByteBufferUtil.java:267) > =A0 =A0 =A0 =A0at org.apache.cassandra.db.ColumnSerializer.deserialize(Co= lumnSerializer.java:94) > =A0 =A0 =A0 =A0at org.apache.cassandra.db.ColumnSerializer.deserialize(Co= lumnSerializer.java:35) > =A0 =A0 =A0 =A0at org.apache.cassandra.db.ColumnFamilySerializer.deserial= izeColumns(ColumnFamilySerializer.java:129) =A0 =A0 =A0 =A0at org.apache.ca= ssandra.io.sstable.SSTableIdentityIterator.getColumnFamilyWithColumns(SSTab= leIdentityIterator.java:176) > =A0 =A0 =A0 =A0at org.apache.cassandra.io.PrecompactedRow.(Precompa= ctedRow.java:78) > =A0 =A0 =A0 =A0at org.apache.cassandra.io.CompactionIterator.getCompacted= Row(CompactionIterator.java:139) > =A0 =A0 =A0 =A0at org.apache.cassandra.io.CompactionIterator.getReduced(C= ompactionIterator.java:108) > =A0 =A0 =A0 =A0at org.apache.cassandra.io.CompactionIterator.getReduced(C= ompactionIterator.java:43) > =A0 =A0 =A0 =A0at org.apache.cassandra.utils.ReducingIterator.computeNext= (ReducingIterator.java:73) > =A0 =A0 =A0 =A0at com.google.common.collect.AbstractIterator.tryToCompute= Next(AbstractIterator.java:136) > =A0 =A0 =A0 =A0at com.google.common.collect.AbstractIterator.hasNext(Abst= ractIterator.java:131) > =A0 =A0 =A0 =A0at org.apache.commons.collections.iterators.FilterIterator= .setNextObject(FilterIterator.java:183) > =A0 =A0 =A0 =A0at org.apache.commons.collections.iterators.FilterIterator= .hasNext(FilterIterator.java:94) > =A0 =A0 =A0 =A0at org.apache.cassandra.db.CompactionManager.doValidationC= ompaction(CompactionManager.java:803) > =A0 =A0 =A0 =A0at org.apache.cassandra.db.CompactionManager.access$800(Co= mpactionManager.java:56) > =A0 =A0 =A0 =A0at org.apache.cassandra.db.CompactionManager$6.call(Compac= tionManager.java:358) > =A0 =A0 =A0 =A0at java.util.concurrent.FutureTask$Sync.innerRun(FutureTas= k.java:303) > =A0 =A0 =A0 =A0at java.util.concurrent.FutureTask.run(FutureTask.java:138= ) > =A0 =A0 =A0 =A0at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(= ThreadPoolExecutor.java:886) > =A0 =A0 =A0 =A0at java.util.concurrent.ThreadPoolExecutor$Worker.run(Thre= adPoolExecutor.java:908) > =A0 =A0 =A0 =A0at java.lang.Thread.run(Thread.java:662) > ERROR [CompactionExecutor:1] 2011-04-08 11:43:53,762 PrecompactedRow.java= (line 82) Skipping row DecoratedKey(8073554114801607394928746621229606383,= 34393734663734382d316330302d346164372d613333372d3162 > 34303866613933333832) in /var/lib/cassandra/data/DFS/main-f-177-Data.db > java.io.EOFException > =A0 =A0 =A0 =A0at java.io.RandomAccessFile.readFully(RandomAccessFile.jav= a:383) > =A0 =A0 =A0 =A0at java.io.RandomAccessFile.readFully(RandomAccessFile.jav= a:361) > =A0 =A0 =A0 =A0at org.apache.cassandra.io.util.BufferedRandomAccessFile.r= eadBytes(BufferedRandomAccessFile.java:268) > =A0 =A0 =A0 =A0at org.apache.cassandra.utils.ByteBufferUtil.read(ByteBuff= erUtil.java:310) > =A0 =A0 =A0 =A0at org.apache.cassandra.utils.ByteBufferUtil.readWithLengt= h(ByteBufferUtil.java:267) > : > > nodetool netstats reports: > > Mode: Normal > Not sending any streams. > Not receiving any streams. > Pool Name =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0Active =A0 Pending =A0 = =A0 =A0Completed > Commands =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0n/a =A0 =A0 =A0 = =A0 0 =A0 =A0 =A0 =A0 526207 > Responses =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 n/a =A0 =A0 =A0 =A0= 0 =A0 =A0 =A0 =A01747991