Bloom filter settings have not changed, they are default. In the table
settings  bloom_filter_fp_chance = 0.01. Should I increase it?

DESC TABLE "PerBoxEventSeriesEventIds"

CREATE TABLE "EventsKeyspace"."PerBoxEventSeriesEventIds" (
    key blob,
    column1 text,
    value blob,
    PRIMARY KEY (key, column1)
) WITH COMPACT STORAGE
    AND CLUSTERING ORDER BY (column1 ASC)
    AND bloom_filter_fp_chance = 0.01
    AND caching = '{"keys":"ALL", "rows_per_partition":"NONE"}'
    AND comment = ''
    AND compaction = {'min_threshold': '4', 'enabled': 'True',
'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 = 864000
    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 = 'NONE';

-- 
Darya Melentsova

2018-02-21 12:06 GMT+05:00 Jeff Jirsa <jji...@gmail.com>:
> Your bloom filter settings look broken. Did you set the FP ratio to 0? If so 
> that’s a bad idea and we should have stopped you from doing it.
>
>
> --
> Jeff Jirsa
>
>
>> On Feb 20, 2018, at 11:01 PM, Дарья Меленцова <ifire...@gmail.com> wrote:
>>
>> Hello.
>>
>> Could you help me with LEAK DETECTED error while minor compaction process?
>>
>> There is a table with a lot of small record 6.6*10^9 (mapping
>> (eventId, boxId) -> cellId)).
>> Minor compaction starts and then fails on 99% done with an error:
>>
>> Stacktrace
>> ERROR [Reference-Reaper:1] 2018-02-05 10:06:17,032 Ref.java:207 - LEAK
>> DETECTED: a reference
>> (org.apache.cassandra.utils.concurrent.Ref$State@1ca1bf87) to class
>> org.apache.cassandra.io.util.MmappedSegmentedFile$Cleanup@308695651:/storage1/cassandra_events/data/EventsKeyspace/PerBoxEventSeriesEvents-41847c3049a211e6af50b9221207cca8/tmplink-lb-102593-big-Index.db
>> was not released before the reference was garbage collected
>> ERROR [Reference-Reaper:1] 2018-02-05 10:06:17,033 Ref.java:207 - LEAK
>> DETECTED: a reference
>> (org.apache.cassandra.utils.concurrent.Ref$State@1659d4f7) to class
>> org.apache.cassandra.utils.concurrent.WrappedSharedCloseable$1@1398495320:[Memory@[0..dc),
>> Memory@[0..898)] was not released before the reference was garbage
>> collected
>> ERROR [Reference-Reaper:1] 2018-02-05 10:06:17,033 Ref.java:207 - LEAK
>> DETECTED: a reference
>> (org.apache.cassandra.utils.concurrent.Ref$State@42978833) to class
>> org.apache.cassandra.utils.concurrent.WrappedSharedCloseable$1@1648504648:[[OffHeapBitSet]]
>> was not released before the reference was garbage collected
>> ERROR [Reference-Reaper:1] 2018-02-05 10:06:17,033 Ref.java:207 - LEAK
>> DETECTED: a reference
>> (org.apache.cassandra.utils.concurrent.Ref$State@3a64a19b) to class
>> org.apache.cassandra.io.sstable.format.SSTableReader$DescriptorTypeTidy@863282967:/storage1/cassandra_events/data/EventsKeyspace/PerBoxEventSeriesEvents-41847c3049a211e6af50b9221207cca8/tmplink-lb-102593-big
>> was not released before the reference was garbage collected
>> ERROR [Reference-Reaper:1] 2018-02-05 10:06:17,033 Ref.java:207 - LEAK
>> DETECTED: a reference
>> (org.apache.cassandra.utils.concurrent.Ref$State@4ddc775a) to class
>> org.apache.cassandra.io.util.CompressedPoolingSegmentedFile$Cleanup@1041709510:/storage1/cassandra_events/data/EventsKeyspace/PerBoxEventSeriesEvents-41847c3049a211e6af50b9221207cca8/tmplink-lb-102593-big-Data.db
>> was not released before the reference was garbage collected
>>
>> I have tried increase max heap size (8GB -> 16GB), but got the same error.
>> How can I resolve the issue?
>>
>>
>> Cassandra parameters and the problem table
>> Cassandra v 2.2.9
>> MAX_HEAP_SIZE="16G"
>> java version "1.8.0_121"
>>
>> compaction = {'min_threshold': '4', 'enabled': 'True', 'class':
>> 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy',
>> 'max_threshold': '32'}
>> compression = {'sstable_compression':
>> 'org.apache.cassandra.io.compress.SnappyCompressor'}
>>
>> nodetoole tablestats
>> Read Count: 1454605
>>        Read Latency: 2.0174777647540054 ms.
>>        Write Count: 12034909
>>        Write Latency: 0.044917336558174224 ms.
>>        Pending Flushes: 0
>>                Table: PerBoxEventSeriesEventIds
>>                SSTable count: 20
>>                Space used (live): 885969527458
>>                Space used (total): 885981801994
>>                Space used by snapshots (total): 0
>>                Off heap memory used (total): 19706226232
>>                SSTable Compression Ratio: 0.5722091068132875
>>                Number of keys (estimate): 6614724684
>>                Memtable cell count: 375796
>>                Memtable data size: 31073510
>>                Memtable off heap memory used: 0
>>                Memtable switch count: 30
>>                Local read count: 1454605
>>                Local read latency: NaN ms
>>                Local write count: 12034909
>>                Local write latency: NaN ms
>>                Pending flushes: 0
>>                Bloom filter false positives: 0
>>                Bloom filter false ratio: 0.00000
>>                Bloom filter space used: -4075791744
>>                Bloom filter off heap memory used: 17399044576
>>                Index summary off heap memory used: 2091833184
>>                Compression metadata off heap memory used: 215348472
>>                Compacted partition minimum bytes: 104
>>                Compacted partition maximum bytes: 149
>>                Compacted partition mean bytes: 149
>>                Average live cells per slice (last five minutes): NaN
>>                Maximum live cells per slice (last five minutes): 0
>>                Average tombstones per slice (last five minutes): NaN
>>                Maximum tombstones per slice (last five minutes): 0
>>
>> Thank You
>> Darya Melentsova
>>
>> email: ifire...@gmail.com
>>
>> ---------------------------------------------------------------------
>> To unsubscribe, e-mail: user-unsubscr...@cassandra.apache.org
>> For additional commands, e-mail: user-h...@cassandra.apache.org
>>
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: user-unsubscr...@cassandra.apache.org
> For additional commands, e-mail: user-h...@cassandra.apache.org
>

---------------------------------------------------------------------
To unsubscribe, e-mail: user-unsubscr...@cassandra.apache.org
For additional commands, e-mail: user-h...@cassandra.apache.org

Reply via email to