cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Alex Petrov (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (CASSANDRA-12832) SASI index corruption on too many overflow items
Date Tue, 08 Nov 2016 11:25:58 GMT

    [ https://issues.apache.org/jira/browse/CASSANDRA-12832?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15647274#comment-15647274
] 

Alex Petrov edited comment on CASSANDRA-12832 at 11/8/16 11:25 AM:
-------------------------------------------------------------------

[~xedin] would you be able to take a short look? Current behaviour (corrupting index file)
is certainly incorrect. Question is what exactly do to.. I've switched from assertion error
to logging an item. Problem here is that the returned results are now incorrect (we'll return
only 8 items we wrote into the overflow trailer).

Alternatively, we could make overflow trailer size configurable and/or dynamic. For example,
if we notice than over 8 items were inserted, we write an additional block of 1024 bytes filled
with overflow items. Or we skip writing alltogether and opt-out for filtering (probably the
least desired option). Maybe you have a better idea on how to deal with that.

|[trunk|https://github.com/ifesdjeen/cassandra/tree/12832-trunk]|[dtest|https://cassci.datastax.com/view/Dev/view/ifesdjeen/job/ifesdjeen-12832-trunk-testall/]|[utest|https://cassci.datastax.com/view/Dev/view/ifesdjeen/job/ifesdjeen-12832-trunk-testall/]|


was (Author: ifesdjeen):
[~xedin] would you be able to take a short look? Current behaviour (corrupting index file)
is certainly incorrect. Question is what exactly do to.. I've switched from assertion error
to logging an item. Problem here is that the returned results are now incorrect (we'll return
only 8 items we have seen so far).

Alternatively, we could make overflow trailer size configurable and/or dynamic. For example,
if we notice than over 8 items were inserted, we write an additional block of 1024 bytes filled
with overflow items. Or we skip writing alltogether and opt-out for filtering (probably the
least desired option). Maybe you have a better idea on how to deal with that.

|[trunk|https://github.com/ifesdjeen/cassandra/tree/12832-trunk]|[dtest|https://cassci.datastax.com/view/Dev/view/ifesdjeen/job/ifesdjeen-12832-trunk-testall/]|[utest|https://cassci.datastax.com/view/Dev/view/ifesdjeen/job/ifesdjeen-12832-trunk-testall/]|

> SASI index corruption on too many overflow items
> ------------------------------------------------
>
>                 Key: CASSANDRA-12832
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-12832
>             Project: Cassandra
>          Issue Type: Bug
>          Components: sasi
>            Reporter: Alex Petrov
>            Assignee: Alex Petrov
>
> When SASI index has too many overflow items, it currently writes a corrupted index file:
> {code}
> java.lang.AssertionError: cannot have more than 8 overflow collisions per leaf, but had:
15
>         at org.apache.cassandra.index.sasi.disk.AbstractTokenTreeBuilder$Leaf.createOverflowEntry(AbstractTokenTreeBuilder.java:357)
~[main/:na]
>         at org.apache.cassandra.index.sasi.disk.AbstractTokenTreeBuilder$Leaf.createEntry(AbstractTokenTreeBuilder.java:346)
~[main/:na]
>         at org.apache.cassandra.index.sasi.disk.DynamicTokenTreeBuilder$DynamicLeaf.serializeData(DynamicTokenTreeBuilder.java:180)
~[main/:na]
>         at org.apache.cassandra.index.sasi.disk.AbstractTokenTreeBuilder$Leaf.serialize(AbstractTokenTreeBuilder.java:306)
~[main/:na]
>         at org.apache.cassandra.index.sasi.disk.AbstractTokenTreeBuilder.write(AbstractTokenTreeBuilder.java:90)
~[main/:na]
>         at org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder$MutableDataBlock.flushAndClear(OnDiskIndexBuilder.java:629)
~[main/:na]
>         at org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder$MutableLevel.flush(OnDiskIndexBuilder.java:446)
~[main/:na]
>         at org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder$MutableLevel.finalFlush(OnDiskIndexBuilder.java:451)
~[main/:na]
>         at org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder.finish(OnDiskIndexBuilder.java:296)
~[main/:na]
>         at org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder.finish(OnDiskIndexBuilder.java:258)
~[main/:na]
>         at org.apache.cassandra.index.sasi.disk.OnDiskIndexBuilder.finish(OnDiskIndexBuilder.java:241)
~[main/:na]
>         at org.apache.cassandra.index.sasi.disk.PerSSTableIndexWriter$Index.lambda$scheduleSegmentFlush$0(PerSSTableIndexWriter.java:267)
~[main/:na]
>         at org.apache.cassandra.index.sasi.disk.PerSSTableIndexWriter$Index.lambda$complete$1(PerSSTableIndexWriter.java:296)
~[main/:na]
>         at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_91]
>         at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_91]
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
~[na:1.8.0_91]
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
[na:1.8.0_91]
>         at java.lang.Thread.run(Thread.java:745) [na:1.8.0_91]
> ERROR [MemtableFlushWriter:4] 2016-10-23 23:17:19,920 DataTracker.java:168 - Can't open
index file at ...., skipping.
> java.lang.IllegalArgumentException: position: -524200, limit: 12288
>         at org.apache.cassandra.index.sasi.utils.MappedBuffer.position(MappedBuffer.java:106)
~[main/:na]
>         at org.apache.cassandra.index.sasi.disk.OnDiskIndex.<init>(OnDiskIndex.java:155)
~[main/:na]
>         at org.apache.cassandra.index.sasi.SSTableIndex.<init>(SSTableIndex.java:62)
~[main/:na]
>         at org.apache.cassandra.index.sasi.conf.DataTracker.getIndexes(DataTracker.java:150)
[main/:na]
>         at org.apache.cassandra.index.sasi.conf.DataTracker.update(DataTracker.java:69)
[main/:na]
>         at org.apache.cassandra.index.sasi.conf.ColumnIndex.update(ColumnIndex.java:147)
[main/:na]
>         at org.apache.cassandra.index.sasi.SASIIndex.handleNotification(SASIIndex.java:320)
[main/:na]
>         at org.apache.cassandra.db.lifecycle.Tracker.notifyAdded(Tracker.java:421) [main/:na]
>         at org.apache.cassandra.db.lifecycle.Tracker.replaceFlushed(Tracker.java:356)
[main/:na]
>         at org.apache.cassandra.db.compaction.CompactionStrategyManager.replaceFlushed(CompactionStrategyManager.java:317)
[main/:na]
>         at org.apache.cassandra.db.ColumnFamilyStore.replaceFlushed(ColumnFamilyStore.java:1569)
[main/:na]
>         at org.apache.cassandra.db.ColumnFamilyStore$Flush.flushMemtable(ColumnFamilyStore.java:1197)
[main/:na]
>         at org.apache.cassandra.db.ColumnFamilyStore$Flush.run(ColumnFamilyStore.java:1100)
[main/:na]
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
[na:1.8.0_91]
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
[na:1.8.0_91]
>         at java.lang.Thread.run(Thread.java:745) [na:1.8.0_91]
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message