Return-Path: X-Original-To: apmail-cassandra-commits-archive@www.apache.org Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 64D1610BB3 for ; Tue, 10 Feb 2015 18:24:18 +0000 (UTC) Received: (qmail 91595 invoked by uid 500); 10 Feb 2015 18:24:13 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 91559 invoked by uid 500); 10 Feb 2015 18:24:13 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 91547 invoked by uid 99); 10 Feb 2015 18:24:13 -0000 Received: from arcas.apache.org (HELO arcas.apache.org) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 10 Feb 2015 18:24:13 +0000 Date: Tue, 10 Feb 2015 18:24:13 +0000 (UTC) From: "Marcus Eriksson (JIRA)" To: commits@cassandra.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (CASSANDRA-8719) Using thrift HSHA with offheap_objects appears to corrupt data MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/CASSANDRA-8719?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14314587#comment-14314587 ] Marcus Eriksson commented on CASSANDRA-8719: -------------------------------------------- I'll dig some more, but I suspect I will need [~xedin] or [~benedict] help here > Using thrift HSHA with offheap_objects appears to corrupt data > -------------------------------------------------------------- > > Key: CASSANDRA-8719 > URL: https://issues.apache.org/jira/browse/CASSANDRA-8719 > Project: Cassandra > Issue Type: Bug > Components: Core > Reporter: Randy Fradin > Assignee: Marcus Eriksson > Fix For: 2.1.3 > > Attachments: repro8719.sh > > > Copying my comment from CASSANDRA-6285 to a new issue since that issue is long closed and I'm not sure if they are related... > I am getting this exception using Thrift HSHA in 2.1.0: > {quote} > INFO [CompactionExecutor:8] 2015-01-26 13:32:51,818 CompactionTask.java (line 138) Compacting [SSTableReader(path='/tmp/cass_test/cassandra/TestCassandra/data/test_ks/test_cf-1c45da40a58911e4826751fbbc77b187/test_ks-test_cf-ka-2-Data.db'), SSTableReader(path='/tmp/cass_test/cassandra/TestCassandra/data/test_ks/test_cf-1c45da40a58911e4826751fbbc77b187/test_ks-test_cf-ka-1-Data.db')] > INFO [CompactionExecutor:8] 2015-01-26 13:32:51,890 ColumnFamilyStore.java (line 856) Enqueuing flush of compactions_in_progress: 212 (0%) on-heap, 20 (0%) off-heap > INFO [MemtableFlushWriter:8] 2015-01-26 13:32:51,892 Memtable.java (line 326) Writing Memtable-compactions_in_progress@1155018639(0 serialized bytes, 1 ops, 0%/0% of on/off-heap limit) > INFO [MemtableFlushWriter:8] 2015-01-26 13:32:51,896 Memtable.java (line 360) Completed flushing /tmp/cass_test/cassandra/TestCassandra/data/system/compactions_in_progress-55080ab05d9c388690a4acb25fe1f77b/system-compactions_in_progress-ka-2-Data.db (42 bytes) for commitlog position ReplayPosition(segmentId=1422296630707, position=430226) > ERROR [CompactionExecutor:8] 2015-01-26 13:32:51,906 CassandraDaemon.java (line 166) Exception in thread Thread[CompactionExecutor:8,1,RMI Runtime] > java.lang.RuntimeException: Last written key DecoratedKey(131206587314004820534098544948237170809, 800100010000000c62617463685f6d7574617465000000) >= current key DecoratedKey(14775611966645399672119169777260659240, 726f776b65793030385f31343232323937313537353835) writing into /tmp/cass_test/cassandra/TestCassandra/data/test_ks/test_cf-1c45da40a58911e4826751fbbc77b187/test_ks-test_cf-tmp-ka-3-Data.db > at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:172) ~[apache-cassandra-2.1.0.jar:2.1.0] > at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:196) ~[apache-cassandra-2.1.0.jar:2.1.0] > at org.apache.cassandra.io.sstable.SSTableRewriter.append(SSTableRewriter.java:110) ~[apache-cassandra-2.1.0.jar:2.1.0] > at org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:177) ~[apache-cassandra-2.1.0.jar:2.1.0] > at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48) ~[apache-cassandra-2.1.0.jar:2.1.0] > at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) ~[apache-cassandra-2.1.0.jar:2.1.0] > at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:74) ~[apache-cassandra-2.1.0.jar:2.1.0] > at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59) ~[apache-cassandra-2.1.0.jar:2.1.0] > at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:235) ~[apache-cassandra-2.1.0.jar:2.1.0] > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) ~[na:1.7.0_40] > at java.util.concurrent.FutureTask.run(FutureTask.java:262) ~[na:1.7.0_40] > at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) ~[na:1.7.0_40] > at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_40] > at java.lang.Thread.run(Thread.java:724) [na:1.7.0_40] > {quote} > I don't think it's caused by CASSANDRA-8211, because it happens during the first compaction that takes place between the first 2 SSTables to get flushed from an initially empty column family. > Also, I've only been able to reproduce it when using both *hsha* for the rpc server and *offheap_objects* for memtable allocation. If I switch either to sync or to offheap_buffers or heap_buffers then I cannot reproduce the problem. Also under the same circumstances I'm pretty sure I've seen incorrect data being returned to a client multiget_slice request before any SSTables had been flushed yet, so I presume this is corruption that happens before any flush/compaction takes place. > nodetool scrub yielded these errors: > {quote} > INFO [CompactionExecutor:9] 2015-01-26 13:48:01,512 OutputHandler.java (line 42) Scrubbing SSTableReader(path='/tmp/cass_test/cassandra/TestCassandra/data/test_ks/test_cf-1c45da40a58911e4826751fbbc77b187/test_ks-test_cf-ka-2-Data.db') (168780 bytes) > INFO [CompactionExecutor:10] 2015-01-26 13:48:01,512 OutputHandler.java (line 42) Scrubbing SSTableReader(path='/tmp/cass_test/cassandra/TestCassandra/data/test_ks/test_cf-1c45da40a58911e4826751fbbc77b187/test_ks-test_cf-ka-1-Data.db') (135024 bytes) > WARN [CompactionExecutor:9] 2015-01-26 13:48:01,531 OutputHandler.java (line 52) Out of order row detected (DecoratedKey(14775611966645399672119169777260659240, 726f776b65793030385f31343232323937313537353835) found after DecoratedKey(131206587314004820534098544948237170809, 800100010000000c62617463685f6d7574617465000000)) > WARN [CompactionExecutor:9] 2015-01-26 13:48:01,534 OutputHandler.java (line 57) Error reading row (stacktrace follows): > java.lang.RuntimeException: Last written key DecoratedKey(131206587314004820534098544948237170809, 800100010000000c62617463685f6d7574617465000000) >= current key DecoratedKey(131206587314004820534098544948237170809, 800100010000000c62617463685f6d7574617465000000) writing into /tmp/cass_test/cassandra/TestCassandra/data/test_ks/test_cf-1c45da40a58911e4826751fbbc77b187/test_ks-test_cf-tmp-ka-4-Data.db > at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:172) ~[apache-cassandra-2.1.0.jar:2.1.0] > at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:196) ~[apache-cassandra-2.1.0.jar:2.1.0] > at org.apache.cassandra.io.sstable.SSTableRewriter.append(SSTableRewriter.java:110) ~[apache-cassandra-2.1.0.jar:2.1.0] > at org.apache.cassandra.io.sstable.SSTableRewriter.tryAppend(SSTableRewriter.java:141) ~[apache-cassandra-2.1.0.jar:2.1.0] > at org.apache.cassandra.db.compaction.Scrubber.scrub(Scrubber.java:186) ~[apache-cassandra-2.1.0.jar:2.1.0] > at org.apache.cassandra.db.compaction.CompactionManager.scrubOne(CompactionManager.java:592) [apache-cassandra-2.1.0.jar:2.1.0] > at org.apache.cassandra.db.compaction.CompactionManager.access$300(CompactionManager.java:100) [apache-cassandra-2.1.0.jar:2.1.0] > at org.apache.cassandra.db.compaction.CompactionManager$3.execute(CompactionManager.java:315) [apache-cassandra-2.1.0.jar:2.1.0] > at org.apache.cassandra.db.compaction.CompactionManager$2.call(CompactionManager.java:270) [apache-cassandra-2.1.0.jar:2.1.0] > at java.util.concurrent.FutureTask.run(FutureTask.java:262) [na:1.7.0_40] > at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) [na:1.7.0_40] > at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_40] > at java.lang.Thread.run(Thread.java:724) [na:1.7.0_40] > WARN [CompactionExecutor:9] 2015-01-26 13:48:01,534 OutputHandler.java (line 52) Row starting at position 25342 is unreadable; skipping to next > WARN [CompactionExecutor:10] 2015-01-26 13:48:01,534 OutputHandler.java (line 52) Out of order row detected (DecoratedKey(29459452031265566667651334397450214244, 726f776b65793030355f31343232323936393033323837) found after DecoratedKey(131206587314004820534098544948237170809, 800100010000000c62617463685f6d7574617465000000)) > etc... > {quote} > I've able to reliably reproduce by creating a new empty table, writing a few partitions containing a few rows each, forcing a flush using nodetool, writing some more, and flushing again, and letting the first compaction happen (using LCS). If I read before flushing, sometimes I will get zero rows back (with quorum between writes and reads), or corrupt bytes, though sometimes reading would yield the correct results. I haven't been able to narrow down yet the cases when the data comes back corrupt vs not, though in the same test case (one set of of writes) the results of a read appear to be consistent (successive reads are either all correct or all incorrect). -- This message was sent by Atlassian JIRA (v6.3.4#6332)