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 056E42050 for ; Mon, 25 Apr 2011 21:25:37 +0000 (UTC) Received: (qmail 10777 invoked by uid 500); 25 Apr 2011 21:25:35 -0000 Delivered-To: apmail-cassandra-user-archive@cassandra.apache.org Received: (qmail 10687 invoked by uid 500); 25 Apr 2011 21:25:35 -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 10679 invoked by uid 99); 25 Apr 2011 21:25:34 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 25 Apr 2011 21:25:34 +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 sanjeev@locomatix.com designates 209.85.216.44 as permitted sender) Received: from [209.85.216.44] (HELO mail-qw0-f44.google.com) (209.85.216.44) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 25 Apr 2011 21:25:30 +0000 Received: by qwc23 with SMTP id 23so14075qwc.31 for ; Mon, 25 Apr 2011 14:25:09 -0700 (PDT) Received: by 10.224.208.197 with SMTP id gd5mr3315040qab.77.1303766708948; Mon, 25 Apr 2011 14:25:08 -0700 (PDT) References: From: Sanjeev Kulkarni In-Reply-To: Mime-Version: 1.0 (iPhone Mail 8F190) Date: Mon, 25 Apr 2011 14:25:01 -0700 Message-ID: <-9169902028367561485@unknownmsgid> Subject: Re: 0.7.4 Bad sstables? To: "user@cassandra.apache.org" Content-Type: text/plain; charset=ISO-8859-1 I pepper my objects based on a hash so without reading the row I cant tell how big it is. Thanks! Sent from my iPhone On Apr 25, 2011, at 10:08 AM, Jonathan Ellis wrote: > Was it on a "large" row? (> in_memory_compaction_limit?) > > I'm starting to suspect that LazilyCompactedRow is computing row size > incorrectly in some cases. > > On Mon, Apr 25, 2011 at 11:47 AM, Terje Marthinussen > wrote: >> I have been hunting similar looking corruptions, especially in the hints >> column family, but I believe it occurs somewhere while compacting. >> I looked in greater detail on one sstable and the row length was longer than >> the actual data in the row, and as far as I could see, either the length was >> wrong or the row was missing data as there was was no extra data in the row >> after the last column. >> This was however on a somewhat aging dataset, so suspected it could be >> related to 2376. >> >> Playing around with 0.8 at the moment and not seen it there yet.... (bet it >> will show up tomorrow once I wrote that.. :)) >> Terje >> >> On Tue, Apr 26, 2011 at 12:44 AM, Sanjeev Kulkarni >> wrote: >>> >>> Hi Sylvain, >>> I started it from 0.7.4 with the patch 2376. No upgrade. >>> Thanks! >>> >>> On Mon, Apr 25, 2011 at 7:48 AM, Sylvain Lebresne >>> wrote: >>>> >>>> Hi Sanjeev, >>>> >>>> What's the story of the cluster ? Did you started with 0.7.4, or is it >>>> upgraded from >>>> some earlier version ? >>>> >>>> On Mon, Apr 25, 2011 at 5:54 AM, Sanjeev Kulkarni >>>> wrote: >>>>> Hey guys, >>>>> Running a one node cassandra server with version 0.7.4 patched >>>>> with https://issues.apache.org/jira/browse/CASSANDRA-2376 >>>>> The system was running fine for a couple of days when we started >>>>> noticing >>>>> something strange with cassandra. I stopped all applications and >>>>> restarted >>>>> cassandra. And then did a scrub. During scrub, I noticed these in the >>>>> logs >>>>> WARN [CompactionExecutor:1] 2011-04-24 23:37:07,561 >>>>> CompactionManager.java >>>>> (line 607) Non-fatal error reading row (stacktrace follows) >>>>> java.io.IOError: java.io.IOException: Impossible row size >>>>> 1516029079813320210 >>>>> at >>>>> >>>>> org.apache.cassandra.db.CompactionManager.doScrub(CompactionManager.java:589) >>>>> at >>>>> >>>>> org.apache.cassandra.db.CompactionManager.access$600(CompactionManager.java:56) >>>>> at >>>>> >>>>> org.apache.cassandra.db.CompactionManager$3.call(CompactionManager.java:195) >>>>> 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) >>>>> Caused by: java.io.IOException: Impossible row size 1516029079813320210 >>>>> ... 8 more >>>>> INFO [CompactionExecutor:1] 2011-04-24 23:37:07,640 >>>>> CompactionManager.java >>>>> (line 613) Retrying from row index; data is -1768177699 bytes starting >>>>> at >>>>> 2626524914 >>>>> WARN [CompactionExecutor:1] 2011-04-24 23:37:07,641 >>>>> CompactionManager.java >>>>> (line 633) Retry failed too. Skipping to next row (retry's stacktrace >>>>> follows) >>>>> java.io.IOError: java.io.EOFException: bloom filter claims to be >>>>> 1868982636 >>>>> bytes, longer than entire row size -1768177699 at >>>>> >>>>> org.apache.cassandra.io.sstable.SSTableIdentityIterator.(SSTableIdentityIterator.java:117) >>>>> at >>>>> >>>>> org.apache.cassandra.db.CompactionManager.doScrub(CompactionManager.java:618) >>>>> at >>>>> >>>>> org.apache.cassandra.db.CompactionManager.access$600(CompactionManager.java:56) >>>>> at >>>>> >>>>> org.apache.cassandra.db.CompactionManager$3.call(CompactionManager.java:195) >>>>> 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) >>>>> Caused by: java.io.EOFException: bloom filter claims to be 1868982636 >>>>> bytes, >>>>> longer than entire row size -1768177699 at >>>>> >>>>> org.apache.cassandra.io.sstable.IndexHelper.defreezeBloomFilter(IndexHelper.java:116) >>>>> at >>>>> >>>>> org.apache.cassandra.io.sstable.SSTableIdentityIterator.(SSTableIdentityIterator.java:87) >>>>> ... 8 more >>>>> WARN [CompactionExecutor:1] 2011-04-24 23:37:16,545 >>>>> CompactionManager.java >>>>> (line 607) Non-fatal error reading row (stacktrace follows) >>>>> java.io.IOError: java.io.EOFException >>>>> at >>>>> >>>>> org.apache.cassandra.io.sstable.SSTableIdentityIterator.next(SSTableIdentityIterator.java:144) >>>>> at >>>>> >>>>> org.apache.cassandra.io.sstable.SSTableIdentityIterator.next(SSTableIdentityIterator.java:40) >>>>> at >>>>> >>>>> org.apache.commons.collections.iterators.CollatingIterator.set(CollatingIterator.java:284) >>>>> at >>>>> >>>>> org.apache.commons.collections.iterators.CollatingIterator.least(CollatingIterator.java:326) >>>>> at >>>>> >>>>> org.apache.commons.collections.iterators.CollatingIterator.next(CollatingIterator.java:230) >>>>> at >>>>> >>>>> org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterator.java:68) >>>>> at >>>>> >>>>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:136) >>>>> at >>>>> >>>>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:131) >>>>> at >>>>> com.google.common.collect.Iterators$7.computeNext(Iterators.java:604) >>>>> at >>>>> >>>>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:136) >>>>> at >>>>> >>>>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:131) >>>>> at >>>>> >>>>> org.apache.cassandra.db.ColumnIndexer.serializeInternal(ColumnIndexer.java:76) >>>>> at >>>>> org.apache.cassandra.db.ColumnIndexer.serialize(ColumnIndexer.java:50) >>>>> at >>>>> >>>>> org.apache.cassandra.io.LazilyCompactedRow.(LazilyCompactedRow.java:90) >>>>> at >>>>> >>>>> org.apache.cassandra.db.CompactionManager.getCompactedRow(CompactionManager.java:778) >>>>> at >>>>> >>>>> org.apache.cassandra.db.CompactionManager.doScrub(CompactionManager.java:591) >>>>> at >>>>> >>>>> org.apache.cassandra.db.CompactionManager.access$600(CompactionManager.java:56) >>>>> at >>>>> >>>>> org.apache.cassandra.db.CompactionManager$3.call(CompactionManager.java:195) >>>>> 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) >>>>> Caused by: 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(BufferedRandomAccessFile.java:270) >>>>> at >>>>> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:317) >>>>> at >>>>> >>>>> org.apache.cassandra.utils.ByteBufferUtil.readWithLength(ByteBufferUtil.java:273) >>>>> at >>>>> >>>>> org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:94) >>>>> at >>>>> >>>>> org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:35) >>>>> at >>>>> >>>>> org.apache.cassandra.io.sstable.SSTableIdentityIterator.next(SSTableIdentityIterator.java:140) >>>>> ... 22 more >>>>> INFO [CompactionExecutor:1] 2011-04-24 23:37:16,561 >>>>> CompactionManager.java >>>>> (line 613) Retrying from row index; data is 78540539 bytes starting at >>>>> 2229643127 >>>>> >>>>> And then when i restarted the readers, i get the following crash >>>>> ERROR [ReadStage:24] 2011-04-24 23:43:05,658 >>>>> AbstractCassandraDaemon.java >>>>> (line 112) Fatal exception in thread Thread[ReadStage:24,5,main] >>>>> java.lang.AssertionError: mmap segment underflow; remaining is >>>>> 791462117 but >>>>> 1970433058 requested >>>>> at >>>>> >>>>> org.apache.cassandra.io.util.MappedFileDataInput.readBytes(MappedFileDataInput.java:119) >>>>> at >>>>> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:317) >>>>> at >>>>> >>>>> org.apache.cassandra.utils.ByteBufferUtil.readWithLength(ByteBufferUtil.java:273) >>>>> 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.columniterator.IndexedSliceReader$IndexedBlockFetcher.getNextBlock(IndexedSliceReader.java:181) >>>>> at >>>>> >>>>> org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNext(IndexedSliceReader.java:121) >>>>> at >>>>> >>>>> org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNext(IndexedSliceReader.java:49) >>>>> at >>>>> >>>>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:136) >>>>> at >>>>> >>>>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:131) >>>>> at >>>>> >>>>> org.apache.cassandra.db.columniterator.SSTableSliceIterator.hasNext(SSTableSliceIterator.java:108) >>>>> at >>>>> >>>>> org.apache.commons.collections.iterators.CollatingIterator.set(CollatingIterator.java:283) >>>>> at >>>>> >>>>> org.apache.commons.collections.iterators.CollatingIterator.least(CollatingIterator.java:326) >>>>> at >>>>> >>>>> org.apache.commons.collections.iterators.CollatingIterator.next(CollatingIterator.java:230) >>>>> at >>>>> >>>>> org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterator.java:68) >>>>> at >>>>> >>>>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:136) >>>>> at >>>>> >>>>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:131) >>>>> at >>>>> >>>>> org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns(SliceQueryFilter.java:116) >>>>> at >>>>> >>>>> org.apache.cassandra.db.filter.QueryFilter.collectCollatedColumns(QueryFilter.java:130) >>>>> at >>>>> >>>>> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1368) >>>>> at >>>>> >>>>> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1245) >>>>> at >>>>> >>>>> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1173) >>>>> at org.apache.cassandra.db.Table.getRow(Table.java:333) >>>>> at >>>>> >>>>> org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:63) >>>>> at >>>>> >>>>> org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:453) >>>>> at >>>>> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30) >>>>> 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) >>>>> >>>>> Any ideas? >>>>> Thanks! >>> >> >> > > > > -- > Jonathan Ellis > Project Chair, Apache Cassandra > co-founder of DataStax, the source for professional Cassandra support > http://www.datastax.com