Look like skipping SSTables based on max SSTable timestamp is possible if
your have the partition deletion info:

https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java#L538-L550

But it doesn't say nothing about iterating all cells in a single partition
if having a partition tombstone, I need to dig further




On Sat, Jul 30, 2016 at 2:03 AM, Eric Stevens <migh...@gmail.com> wrote:

> I haven't tested that specifically, but I haven't bumped into any
> particular optimization that allows it to skip reading an sstable where the
> entire relevant partition has been row-tombstoned.  It's possible that
> something like that could happen by examining min/max timestamps on
> sstables, and not reading from any sstable with a partition-level tombstone
> where the max timestamp is less than the timestamp of the partition
> tombstone.  However that presumes that it can have read the tombstones from
> each sstable before it read the occluded data, which I don't think is
> likely.
>
> Such an optimization could be there, but I haven't noticed it if it is,
> though I'm certainly not an expert (more of a well informed novice).  If
> someone wants to set me straight on this point I'd love to know about it.
>
> On Fri, Jul 29, 2016 at 2:37 PM DuyHai Doan <doanduy...@gmail.com> wrote:
>
>> @Eric
>>
>> Very interesting example. But then what is the case of row (should I say
>> partition ?) tombstones ?
>>
>> Suppose that in your example, I issued a DELETE FROM foo WHERE pk='a'
>>
>> With the same SELECT statement than before, would C* be clever enough to
>> skip reading at all the whole partition (let's limit the example to a
>> single SSTable) ?
>>
>> On Fri, Jul 29, 2016 at 7:00 PM, Eric Stevens <migh...@gmail.com> wrote:
>>
>>> > Sai was describing a timeout, not a failure due to the 100 K
>>> tombstone limit from cassandra.yaml. But I still might be missing things
>>> about tombstones.
>>>
>>> The trouble with tombstones is not the tombstones themselves, rather
>>> it's that Cassandra has a lot of deleted data to read through in sstables
>>> in order to satisfy a query.  Although if you range constrain your cluster
>>> key in your query, the read path can optimize that read to start somewhere
>>> near the correct head of your selected data, that is _not_ true for
>>> tombstoned data.
>>>
>>> Consider this exercise:
>>> CREATE TABLE foo (
>>>   pk text,
>>>   ck int,
>>>   PRIMARY KEY ((pk), ck)
>>> )
>>> INSERT INTO foo (pk,ck) VALUES ('a', 1)
>>> ...
>>> INSERT INTO foo (pk,ck) VALUES ('a', 100000)
>>>
>>> $ nodetool flush
>>>
>>> DELETE FROM foo WHERE pk='a' AND ck < 99999
>>>
>>> We've now written a single "tiny" (bytes-wise) range tombstone.
>>>
>>> Now try to select from that table:
>>> SELECT * FROM foo WHERE ck > 50000 LIMIT 1
>>> pk | ck
>>> -- | ------
>>> a  | 100000
>>>
>>> This has to read from the first sstable, skipping over 49999 records
>>> before it can locate the first non-tombstoned cell.
>>>
>>> The problem isn't the size of the tombstone, tombstones themselves are
>>> cheaper (again, bytes-wise) than standard columns because they don't
>>> involve any value for the cell.  The problem is that the read path cannot
>>> anticipate in advance what cells are going to be occluded by the tombstone,
>>> and in order to satisfy the query it needs to read then discard a large
>>> number of deleted cells.
>>>
>>> The reason the thresholds exist in cassandra.yaml is to help guide users
>>> away from performance anti-patterns that come from selects which include a
>>> large number of tombstoned cells.
>>>
>>> On Thu, Jul 28, 2016 at 11:08 PM Alain RODRIGUEZ <arodr...@gmail.com>
>>> wrote:
>>>
>>>> Hi,
>>>>
>>>> @Eric
>>>>
>>>> Large range tombstones can occupy just a few bytes but can occlude
>>>>> millions of records, and have the corresponding performance impact on
>>>>> reads.  It's really not the size of the tombstone on disk that matters, 
>>>>> but
>>>>> the number of records it occludes.
>>>>
>>>>
>>>> Sai was describing a timeout, not a failure due to the 100 K tombstone
>>>> limit from cassandra.yaml. But I still might be missing things about
>>>> tombstones.
>>>>
>>>> The read queries are continuously failing though because of the
>>>>> tombstones. "Request did not complete within rpc_timeout."
>>>>>
>>>>
>>>> So that is what looks weird to me. Reading 220 kb, even holding
>>>> tombstone should probably not take that long... Or am I wrong or missing
>>>> something?
>>>>
>>>> Your talk looks like cool stuff :-).
>>>>
>>>> @Sai
>>>>
>>>> The issues here was that tombstones were not in the SSTable, but rather
>>>>> in the Memtable
>>>>
>>>>
>>>> This sounds weird to me as well, knowing that memory is faster than
>>>> disk and that memtables are mutable data (so less stuff to read from
>>>> there). Flushing might have triggered some compaction, removing tombstones
>>>> though.
>>>>
>>>> This still sounds very weird to me but I am glad you solved your issue
>>>> (temporary at least).
>>>>
>>>> C*heers,
>>>> -----------------------
>>>> Alain Rodriguez - al...@thelastpickle.com
>>>> France
>>>>
>>>> The Last Pickle - Apache Cassandra Consulting
>>>> http://www.thelastpickle.com
>>>>
>>>> 2016-07-29 3:25 GMT+02:00 Eric Stevens <migh...@gmail.com>:
>>>>
>>>>> Tombstones will not get removed even after gc_grace if bloom filters
>>>>> indicate that there is overlapping data with the tombstone's partition in 
>>>>> a
>>>>> different sstable.  This is because compaction can't be certain that the
>>>>> tombstone doesn't overlap data in that other table.  If you're writing to
>>>>> one end of a partition key while deleting off the other end (for example
>>>>> you've created engaged in the queue anti-pattern), your tombstones will
>>>>> essentially never go away.
>>>>>
>>>>>
>>>>>> 220kb worth of tombstones doesn’t seem like enough to worry about.
>>>>>
>>>>>
>>>>> Large range tombstones can occupy just a few bytes but can occlude
>>>>> millions of records, and have the corresponding performance impact on
>>>>> reads.  It's really not the size of the tombstone on disk that matters, 
>>>>> but
>>>>> the number of records it occludes.
>>>>>
>>>>> You must either do a full compaction (while also not writing to the
>>>>> partitions being considered, and after you've forced a cluster-wide flush,
>>>>> and after the tombstones are gc_grace old, and assuming size tiered and 
>>>>> not
>>>>> leveled compaction) to get rid of those tombstones, or probably easier is
>>>>> to do something similar to sstable2json, remove the tombstones by hand,
>>>>> then json2sstable and replace the offending sstable.  Note that you really
>>>>> have to be certain what you're doing here or you'll end up resurrecting
>>>>> deleted records.
>>>>>
>>>>> If these all sound like bad options, it's because they are, and you
>>>>> don't have a lot of options without changing your schema to eventually 
>>>>> stop
>>>>> writing to (and especially reading from) partitions which you also do
>>>>> deletes on.  https://issues.apache.org/jira/browse/CASSANDRA-7019 proposes
>>>>> to offer a better alternative, but it's still in progress.
>>>>>
>>>>> Shameless plug, I'm talking about my company's alternative to
>>>>> tombstones and TTLs at this year's Cassandra Summit:
>>>>> http://myeventagenda.com/sessions/1CBFC920-807D-41C1-942C-8D1A7C10F4FA/5/5#sessionID=165
>>>>>
>>>>>
>>>>> On Thu, Jul 28, 2016 at 11:07 AM sai krishnam raju potturi <
>>>>> pskraj...@gmail.com> wrote:
>>>>>
>>>>>> thanks a lot Alain. That was really great info.
>>>>>>
>>>>>> The issues here was that tombstones were not in the SSTable, but
>>>>>> rather in the Memtable. We had to a nodetool flush, and run a nodetool
>>>>>> compact to get rid of the tombstones, a million of them. The size of the
>>>>>> largest SSTable was actually 48MB.
>>>>>>
>>>>>> This link was helpful in getting the count of tombstones in a
>>>>>> sstable, which was 0 in our case.
>>>>>> https://gist.github.com/JensRantil/063b7c56ca4a8dfe1c50
>>>>>>
>>>>>>     The application team did not have a good model. They are working
>>>>>> on a new datamodel.
>>>>>>
>>>>>> thanks
>>>>>>
>>>>>> On Wed, Jul 27, 2016 at 7:17 PM, Alain RODRIGUEZ <arodr...@gmail.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Hi,
>>>>>>>
>>>>>>> I just released a detailed post about tombstones today that might be
>>>>>>> of some interest for you:
>>>>>>> http://thelastpickle.com/blog/2016/07/27/about-deletes-and-tombstones.html
>>>>>>>
>>>>>>> 220kb worth of tombstones doesn’t seem like enough to worry about.
>>>>>>>
>>>>>>>
>>>>>>> +1
>>>>>>>
>>>>>>> I believe you might be missing some other bigger SSTable having a
>>>>>>> lot of tombstones as well. Finding the biggest sstable and reading the
>>>>>>> tombstone ratio from there might be more relevant.
>>>>>>>
>>>>>>> You also should give a try to: "unchecked_tombstone_compaction" set
>>>>>>> to true rather than tuning other options so aggressively. The "single
>>>>>>> SSTable compaction" section of my post might help you on this issue:
>>>>>>> http://thelastpickle.com/blog/2016/07/27/about-deletes-and-tombstones.html#single-sstable-compaction
>>>>>>>
>>>>>>> Other thoughts:
>>>>>>>
>>>>>>> Also if you use TTLs and timeseries, using TWCS instead of STCS
>>>>>>> could be more efficient evicting tombstones.
>>>>>>>
>>>>>>> we have a columnfamily that has around 1000 rows, with one row is
>>>>>>>> really huge (million columns)
>>>>>>>
>>>>>>>
>>>>>>> I am sorry to say that this model does not look that great.
>>>>>>> Imbalances might become an issue as a few nodes will handle a lot more 
>>>>>>> load
>>>>>>> than the rest of the nodes. Also even if this is getting improved in 
>>>>>>> newer
>>>>>>> versions of Cassandra, wide rows are something you want to avoid while
>>>>>>> using 2.0.14 (which is no longer supported for about a year now). I 
>>>>>>> know it
>>>>>>> is not always easy and never the good time, but maybe should you 
>>>>>>> consider
>>>>>>> upgrading both your model and your version of Cassandra (regardless of 
>>>>>>> the
>>>>>>> fact you manage to solve this issue or not with
>>>>>>> "unchecked_tombstone_compaction").
>>>>>>>
>>>>>>> Good luck,
>>>>>>>
>>>>>>> C*heers,
>>>>>>> -----------------------
>>>>>>> Alain Rodriguez - al...@thelastpickle.com
>>>>>>> France
>>>>>>>
>>>>>>> The Last Pickle - Apache Cassandra Consulting
>>>>>>> http://www.thelastpickle.com
>>>>>>>
>>>>>>> 2016-07-28 0:00 GMT+02:00 sai krishnam raju potturi <
>>>>>>> pskraj...@gmail.com>:
>>>>>>>
>>>>>>>> The read queries are continuously failing though because of the
>>>>>>>> tombstones. "Request did not complete within rpc_timeout."
>>>>>>>>
>>>>>>>> thanks
>>>>>>>>
>>>>>>>>
>>>>>>>> On Wed, Jul 27, 2016 at 5:51 PM, Jeff Jirsa <
>>>>>>>> jeff.ji...@crowdstrike.com> wrote:
>>>>>>>>
>>>>>>>>> 220kb worth of tombstones doesn’t seem like enough to worry about.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> *From: *sai krishnam raju potturi <pskraj...@gmail.com>
>>>>>>>>> *Reply-To: *"user@cassandra.apache.org" <user@cassandra.apache.org
>>>>>>>>> >
>>>>>>>>> *Date: *Wednesday, July 27, 2016 at 2:43 PM
>>>>>>>>> *To: *Cassandra Users <user@cassandra.apache.org>
>>>>>>>>> *Subject: *Re: Re : Purging tombstones from a particular row in
>>>>>>>>> SSTable
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> and also the sstable size in question is like 220 kb in size.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> thanks
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Wed, Jul 27, 2016 at 5:41 PM, sai krishnam raju potturi <
>>>>>>>>> pskraj...@gmail.com> wrote:
>>>>>>>>>
>>>>>>>>> it's set to 1800 Vinay.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>  bloom_filter_fp_chance=0.010000 AND
>>>>>>>>>
>>>>>>>>>   caching='KEYS_ONLY' AND
>>>>>>>>>
>>>>>>>>>   comment='' AND
>>>>>>>>>
>>>>>>>>>   dclocal_read_repair_chance=0.100000 AND
>>>>>>>>>
>>>>>>>>>   gc_grace_seconds=1800 AND
>>>>>>>>>
>>>>>>>>>   index_interval=128 AND
>>>>>>>>>
>>>>>>>>>   read_repair_chance=0.000000 AND
>>>>>>>>>
>>>>>>>>>   replicate_on_write='true' AND
>>>>>>>>>
>>>>>>>>>   populate_io_cache_on_flush='false' AND
>>>>>>>>>
>>>>>>>>>   default_time_to_live=0 AND
>>>>>>>>>
>>>>>>>>>   speculative_retry='99.0PERCENTILE' AND
>>>>>>>>>
>>>>>>>>>   memtable_flush_period_in_ms=0 AND
>>>>>>>>>
>>>>>>>>>   compaction={'min_sstable_size': '1024', 'tombstone_threshold':
>>>>>>>>> '0.01', 'tombstone_compaction_interval': '1800', 'class':
>>>>>>>>> 'SizeTieredCompactionStrategy'} AND
>>>>>>>>>
>>>>>>>>>   compression={'sstable_compression': 'LZ4Compressor'};
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> thanks
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Wed, Jul 27, 2016 at 5:34 PM, Vinay Kumar Chella <
>>>>>>>>> vinaykumar...@gmail.com> wrote:
>>>>>>>>>
>>>>>>>>> What is your GC_grace_seconds set to?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Wed, Jul 27, 2016 at 1:13 PM, sai krishnam raju potturi <
>>>>>>>>> pskraj...@gmail.com> wrote:
>>>>>>>>>
>>>>>>>>> thanks Vinay and DuyHai.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>     we are using verison 2.0.14. I did "user defined compaction"
>>>>>>>>> following the instructions in the below link, The tombstones still 
>>>>>>>>> persist
>>>>>>>>> even after that.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> https://gist.github.com/jeromatron/e238e5795b3e79866b83
>>>>>>>>> <https://urldefense.proofpoint.com/v2/url?u=https-3A__gist.github.com_jeromatron_e238e5795b3e79866b83&d=CwMFaQ&c=08AGY6txKsvMOP6lYkHQpPMRA1U6kqhAwGa8-0QCg3M&r=yfYEBHVkX6l0zImlOIBID0gmhluYPD5Jje-3CtaT3ow&m=-sQ3Vf5bs3z4cO36h_AU-kIhMGVKcb3eCtzIb-fZ1Fc&s=0RQ3r6c0L4vICot8eqpOBKBAuKiKEkoKdmcjLbvBBwY&e=>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Also, we changed the tombstone_compaction_interval : 1800
>>>>>>>>> and tombstone_threshold : 0.1, but it did not help.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> thanks
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Wed, Jul 27, 2016 at 4:05 PM, DuyHai Doan <doanduy...@gmail.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>> This feature is also exposed directly in nodetool from version
>>>>>>>>> Cassandra 3.4
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> nodetool compact --user-defined <SSTable file>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Wed, Jul 27, 2016 at 9:58 PM, Vinay Chella <vche...@netflix.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>> You can run file level compaction using JMX to get rid of
>>>>>>>>> tombstones in one SSTable. Ensure you set GC_Grace_seconds such that
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> current time >= deletion(tombstone time)+ GC_Grace_seconds
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> File level compaction
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> /usr/bin/java -jar cmdline-jmxclient-0.10.3.jar - localhost:
>>>>>>>>>
>>>>>>>>> ​{​
>>>>>>>>>
>>>>>>>>> ​port}
>>>>>>>>>
>>>>>>>>>  org.apache.cassandra.db:type=CompactionManager 
>>>>>>>>> forceUserDefinedCompaction="'${KEYSPACE}','${
>>>>>>>>>
>>>>>>>>> ​SSTABLEFILENAME
>>>>>>>>>
>>>>>>>>> }'""
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Wed, Jul 27, 2016 at 11:59 AM, sai krishnam raju potturi <
>>>>>>>>> pskraj...@gmail.com> wrote:
>>>>>>>>>
>>>>>>>>> hi;
>>>>>>>>>
>>>>>>>>>   we have a columnfamily that has around 1000 rows, with one row
>>>>>>>>> is really huge (million columns). 95% of the row contains tombstones. 
>>>>>>>>> Since
>>>>>>>>> there exists just one SSTable , there is going to be no compaction 
>>>>>>>>> kicked
>>>>>>>>> in. Any way we can get rid of the tombstones in that row?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Userdefined compaction nor nodetool compact had no effect. Any
>>>>>>>>> ideas folks?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> thanks
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>
>>

Reply via email to