Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 64AE0200AE4 for ; Fri, 24 Jun 2016 12:04:18 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 6373F160A62; Fri, 24 Jun 2016 10:04:18 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 6FD2C160A38 for ; Fri, 24 Jun 2016 12:04:17 +0200 (CEST) Received: (qmail 19067 invoked by uid 500); 24 Jun 2016 10:04:16 -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 19038 invoked by uid 99); 24 Jun 2016 10:04:16 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 24 Jun 2016 10:04:16 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id 57FB32C1F62 for ; Fri, 24 Jun 2016 10:04:16 +0000 (UTC) Date: Fri, 24 Jun 2016 10:04:16 +0000 (UTC) From: "Branimir Lambov (JIRA)" To: commits@cassandra.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (CASSANDRA-11993) Cannot read Snappy compressed tables with 3.6 MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Fri, 24 Jun 2016 10:04:18 -0000 [ https://issues.apache.org/jira/browse/CASSANDRA-11993?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15348098#comment-15348098 ] Branimir Lambov commented on CASSANDRA-11993: --------------------------------------------- You are right, whether or not OFF_HEAP is the right thing to do, this does become inconsistent with the rest of the machinery. Either choice would provide correctness, and I do not want to change it in other places in the code without some testing to show that this change makes sense everywhere. Updated patch here: |[trunk, compressor-preferred|https://github.com/blambov/cassandra/tree/11993-preferred]|[utests|http://cassci.datastax.com/view/Dev/view/blambov/job/blambov-11993-preferred-testall/]|[dtests|http://cassci.datastax.com/view/Dev/view/blambov/job/blambov-11993-preferred-dtest/]| > Cannot read Snappy compressed tables with 3.6 > --------------------------------------------- > > Key: CASSANDRA-11993 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11993 > Project: Cassandra > Issue Type: Bug > Reporter: Nimi Wariboko Jr. > Assignee: Branimir Lambov > Fix For: 3.6 > > > After upgrading to 3.6, I can no longer read/compact sstables compressed with snappy compression. The memtable_allocation_type makes no difference both offheap_buffers and heap_buffers cause the errors. > {code} > WARN [SharedPool-Worker-5] 2016-06-10 15:45:18,731 AbstractLocalAwareExecutorService.java:169 - Uncaught exception on thread Thread[SharedPool-Worker-5,5,main]: {} > org.xerial.snappy.SnappyError: [NOT_A_DIRECT_BUFFER] destination is not a direct buffer > at org.xerial.snappy.Snappy.uncompress(Snappy.java:509) ~[snappy-java-1.1.1.7.jar:na] > at org.apache.cassandra.io.compress.SnappyCompressor.uncompress(SnappyCompressor.java:102) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.io.util.CompressedSegmentedFile$Mmap.readChunk(CompressedSegmentedFile.java:323) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.cache.ChunkCache.load(ChunkCache.java:137) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.cache.ChunkCache.load(ChunkCache.java:19) ~[apache-cassandra-3.6.jar:3.6] > at com.github.benmanes.caffeine.cache.BoundedLocalCache$BoundedLocalLoadingCache.lambda$new$0(BoundedLocalCache.java:2949) ~[caffeine-2.2.6.jar:na] > at com.github.benmanes.caffeine.cache.BoundedLocalCache.lambda$doComputeIfAbsent$15(BoundedLocalCache.java:1807) ~[caffeine-2.2.6.jar:na] > at java.util.concurrent.ConcurrentHashMap.compute(ConcurrentHashMap.java:1853) ~[na:1.8.0_66] > at com.github.benmanes.caffeine.cache.BoundedLocalCache.doComputeIfAbsent(BoundedLocalCache.java:1805) ~[caffeine-2.2.6.jar:na] > at com.github.benmanes.caffeine.cache.BoundedLocalCache.computeIfAbsent(BoundedLocalCache.java:1788) ~[caffeine-2.2.6.jar:na] > at com.github.benmanes.caffeine.cache.LocalCache.computeIfAbsent(LocalCache.java:97) ~[caffeine-2.2.6.jar:na] > at com.github.benmanes.caffeine.cache.LocalLoadingCache.get(LocalLoadingCache.java:66) ~[caffeine-2.2.6.jar:na] > at org.apache.cassandra.cache.ChunkCache$CachingRebufferer.rebuffer(ChunkCache.java:215) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.cache.ChunkCache$CachingRebufferer.rebuffer(ChunkCache.java:193) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.io.util.RandomAccessReader.reBufferAt(RandomAccessReader.java:78) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.io.util.RandomAccessReader.seek(RandomAccessReader.java:220) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.io.util.SegmentedFile.createReader(SegmentedFile.java:138) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.io.sstable.format.SSTableReader.getFileDataInput(SSTableReader.java:1779) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.db.columniterator.AbstractSSTableIterator.(AbstractSSTableIterator.java:103) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.db.columniterator.SSTableIterator.(SSTableIterator.java:44) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(BigTableReader.java:72) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(BigTableReader.java:65) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.db.rows.UnfilteredRowIteratorWithLowerBound.initializeIterator(UnfilteredRowIteratorWithLowerBound.java:85) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.maybeInit(LazilyInitializedUnfilteredRowIterator.java:48) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:99) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.db.rows.UnfilteredRowIteratorWithLowerBound.computeNext(UnfilteredRowIteratorWithLowerBound.java:94) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.db.rows.UnfilteredRowIteratorWithLowerBound.computeNext(UnfilteredRowIteratorWithLowerBound.java:26) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:374) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.utils.MergeIterator$ManyToOne.advance(MergeIterator.java:186) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:155) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:419) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:279) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:112) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:133) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:89) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.serialize(UnfilteredRowIteratorSerializer.java:79) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.db.partitions.UnfilteredPartitionIterators$Serializer.serialize(UnfilteredPartitionIterators.java:294) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.db.ReadResponse$LocalDataResponse.build(ReadResponse.java:134) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.db.ReadResponse$LocalDataResponse.(ReadResponse.java:127) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.db.ReadResponse$LocalDataResponse.(ReadResponse.java:123) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.db.ReadResponse.createDataResponse(ReadResponse.java:65) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.db.ReadCommand.createResponse(ReadCommand.java:292) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.db.ReadCommandVerbHandler.doVerb(ReadCommandVerbHandler.java:50) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:64) ~[apache-cassandra-3.6.jar:3.6] > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_66] > at org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164) ~[apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:136) [apache-cassandra-3.6.jar:3.6] > at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) [apache-cassandra-3.6.jar:3.6] > at java.lang.Thread.run(Thread.java:745) [na:1.8.0_66] > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)