cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Andrew S (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (CASSANDRA-8089) Invalid tombstone warnings / exceptions
Date Fri, 10 Oct 2014 09:03:34 GMT

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

Andrew S edited comment on CASSANDRA-8089 at 10/10/14 9:03 AM:
---------------------------------------------------------------

We are also getting the following exception on one of the servers after manually flushing
(using nodetool) another table with similar structure.

2014-10-10T07:02:46.402+0000 host9 ERROR [CompactionExecutor:2] 2014-10-10 09:02:46,396 CassandraDaemon.java:166
- Exception in thread Thread[CompactionExecutor:2,1,main] java.lang.IllegalStateException:
Unable to compute ceiling for max when histogram overflowed at org.apache.cassandra.utils.EstimatedHistogram.mean(EstimatedHistogram.java:203)
~[apache-cassandra-2.1.0.jar:2.1.0] at org.apache.cassandra.io.sstable.metadata.StatsMetadata.getEstimatedDroppableTombstoneRatio(StatsMetadata.java:98)
~[apache-cassandra-2.1.0.jar:2.1.0] at org.apache.cassandra.io.sstable.SSTableReader.getEstimatedDroppableTombstoneRatio(SSTableReader.java:1805)
~[apache-cassandra-2.1.0.jar:2.1.0] at org.apache.cassandra.db.compaction.AbstractCompactionStrategy.worthDroppingTombstones(AbstractCompactionStrategy.java:297)
~[apache-cassandra-2.1.0.jar:2.1.0] at org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy.getNextBackgroundSSTables(SizeTieredCompactionStrategy.java:106)
~[apache-cassandra-2.1.0.jar:2.1.0] at org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy.getNextBackgroundTask(SizeTieredCompactionStrategy.java:267)
~[apache-cassandra-2.1.0.jar:2.1.0] at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:229)
~[apache-cassandra-2.1.0.jar:2.1.0] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
~[na:1.7.0_51] at java.util.concurrent.FutureTask.run(FutureTask.java:262) ~[na:1.7.0_51]
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) ~[na:1.7.0_51]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_51]
at java.lang.Thread.run(Thread.java:744) [na:1.7.0_51]


was (Author: andrews):
We are also getting the following exception on one of the servers after flushing another table
with similar structure.

2014-10-10T07:02:46.402+0000 su8 ERROR [CompactionExecutor:2] 2014-10-10 09:02:46,396 CassandraDaemon.java:166
- Exception in thread Thread[CompactionExecutor:2,1,main] java.lang.IllegalStateException:
Unable to compute ceiling for max when histogram overflowed at org.apache.cassandra.utils.EstimatedHistogram.mean(EstimatedHistogram.java:203)
~[apache-cassandra-2.1.0.jar:2.1.0] at org.apache.cassandra.io.sstable.metadata.StatsMetadata.getEstimatedDroppableTombstoneRatio(StatsMetadata.java:98)
~[apache-cassandra-2.1.0.jar:2.1.0] at org.apache.cassandra.io.sstable.SSTableReader.getEstimatedDroppableTombstoneRatio(SSTableReader.java:1805)
~[apache-cassandra-2.1.0.jar:2.1.0] at org.apache.cassandra.db.compaction.AbstractCompactionStrategy.worthDroppingTombstones(AbstractCompactionStrategy.java:297)
~[apache-cassandra-2.1.0.jar:2.1.0] at org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy.getNextBackgroundSSTables(SizeTieredCompactionStrategy.java:106)
~[apache-cassandra-2.1.0.jar:2.1.0] at org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy.getNextBackgroundTask(SizeTieredCompactionStrategy.java:267)
~[apache-cassandra-2.1.0.jar:2.1.0] at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:229)
~[apache-cassandra-2.1.0.jar:2.1.0] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
~[na:1.7.0_51] at java.util.concurrent.FutureTask.run(FutureTask.java:262) ~[na:1.7.0_51]
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) ~[na:1.7.0_51]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_51]
at java.lang.Thread.run(Thread.java:744) [na:1.7.0_51]

> Invalid tombstone warnings / exceptions
> ---------------------------------------
>
>                 Key: CASSANDRA-8089
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-8089
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>         Environment: Cassandra 2.1.0
> Debian 7.6, 3.2.0-4-amd64 GNU/Linux
> java version "1.7.0_51"
> Java(TM) SE Runtime Environment (build 1.7.0_51-b13)
> Java HotSpot(TM) 64-Bit Server VM (build 24.51-b03, mixed mode)
>            Reporter: Andrew S
>
> Hey,
> We are having a strange issue with tombstone warnings which look like this:
> {code}
> WARN  12:28:42 Read 129 live and 4113 tombstoned cells in XXX.xxx (see tombstone_warn_threshold).
500 columns was requested, slices=[31660a4e-4f94-11e4-ac1d-53f244a29642-0a8073aa-4f9f-11e4-87c7-5b3e253389d8:!],
delInfo={deletedAt=-9223372036854775808, localDeletion=2147483647}
> {code}
> What is strange is that the row requested should not contain any tombstones as we never
delete data from that row. (We do delete data from other row in the same column family)
> To debug the issue we have dumped the data for this row using sstable2json and the result
does not contain any tombstones. (We have done this on all nodes having the data and all sstables
containing the key)
> {code}
> ./sstable2json /var/lib/cassandra/data/XXX/xxx/XXX-xxx-ka-81524-Data.db -k "xxxxxxxxxxx"
> {code}
> We are getting the warnings after issuing a simple query:
> {code}
> select count(*) from xxx where key = 'xxxxx' and aggregate='xxxxx';
> {code}
> There are only ~500 cells but it issues a warning about scanning 1700 tombstones.
> We are very worried about this because for some of the queries we are hitting TombstoneOverwhelmingException
for no obvious reason.
> Here is the table definiion:
> {code}
> CREATE TABLE "Xxxx".xxx (
>     key text,
>     aggregate text,
>     t timeuuid,
>     ..... {date fields }
>     PRIMARY KEY ((key, aggregate), t)
> ) WITH CLUSTERING ORDER BY (t ASC)
>     AND bloom_filter_fp_chance = 0.01
>     AND caching = '{"keys":"ALL", "rows_per_partition":"NONE"}'
>     AND comment = 'we love cassandra'
>     AND compaction = {'min_threshold': '6', 'class': 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy',
'max_threshold': '32'}
>     AND compression = {'sstable_compression': 'org.apache.cassandra.io.compress.SnappyCompressor'}
>     AND dclocal_read_repair_chance = 0.0
>     AND default_time_to_live = 0
>     AND gc_grace_seconds = 3600
>     AND max_index_interval = 2048
>     AND memtable_flush_period_in_ms = 0
>     AND min_index_interval = 128
>     AND read_repair_chance = 0.1
>     AND speculative_retry = '99.0PERCENTILE';
> {code}
> Do you have any ideas how can we debug this further?
> Thanks,
> Andrew



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

Mime
View raw message