Re: Too many tombstones using TTL

2018-01-16 Thread Python_Max
Thanks for a very helpful reply.
Will try to refactor the code accordingly.

On Tue, Jan 16, 2018 at 4:36 PM, Alexander Dejanovski <
a...@thelastpickle.com> wrote:

> I would not plan on deleting data at the row level as you'll end up with a
> lot of tombstones eventually (and you won't even notice them).
> It's not healthy to allow that many tombstones to be read, and while your
> latency may fit your SLA now, it may not in the future.
> Tombstones are going to create a lot of heap pressure and eventually
> trigger long GC pauses, which then tend to affect the whole cluster (a slow
> node is worse than a down node).
>
> You should definitely separate data that is TTLed and data that is not in
> different tables so that you can adjust compaction strategies,
> gc_grace_seconds and read patterns accordingly. I understand that it will
> complexify your code, but it will prevent severe performance issues in
> Cassandra.
>
> Tombstones won't be a problem for repair, they will get repaired as
> classic cells. They negatively affect the read path mostly, and use space
> on disk.
>
> On Tue, Jan 16, 2018 at 2:12 PM Python_Max <python@gmail.com> wrote:
>
>> Hello.
>>
>> I was planning to remove a row (not partition).
>>
>> Most of the tombstones are seen in the use case of geographic grid with
>> X:Y as partition key and object id (timeuuid) as clustering key where
>> objects could be temporary with TTL about 10 hours or fully persistent.
>> When I select all objects in specific X:Y I can even hit 100k (default)
>> limit for some X:Y. I have changed this limit to 500k since 99.9p read
>> latency is < 75ms so I should not (?) care how many tombstones while read
>> latency is fine.
>>
>> Splitting entities to temporary and permanent and using different
>> compaction strategies is an option but it will lead to code duplication and
>> 2x read queries.
>>
>> Is my assumption correct about tombstones are not so big problem as soon
>> as read latency and disk usage are okey? Are tombstones affect repair time
>> (using reaper)?
>>
>> Thanks.
>>
>>
>> On Tue, Jan 16, 2018 at 11:32 AM, Alexander Dejanovski <
>> a...@thelastpickle.com> wrote:
>>
>>> Hi,
>>>
>>> could you be more specific about the deletes you're planning to perform ?
>>> This will end up moving your problem somewhere else as you'll be
>>> generating new tombstones (and if you're planning on deleting rows, be
>>> aware that row level tombstones aren't reported anywhere in the metrics,
>>> logs and query traces).
>>> Currently you can delete your data at the partition level, which will
>>> create a single tombstone that will shadow all your expired (and non
>>> expired) data and is very efficient. The read path is optimized for such
>>> tombstones and the data won't be fully read from disk nor exchanged between
>>> replicas. But that's of course if your use case allows to delete full
>>> partitions.
>>>
>>> We usually model so that we can restrict our reads to live data.
>>> If you're creating time series, your clustering key should include a
>>> timestamp, which you can use to avoid reading expired data. If your TTL is
>>> set to 60 days, you can read only data that is strictly younger than that.
>>> Then you can partition by time ranges, and access exclusively partitions
>>> that have no chance to be expired yet.
>>> Those techniques usually work better with TWCS, but the former could
>>> make you hit a lot of SSTables if your partitions can spread over all time
>>> buckets, so only use TWCS if you can restrict individual reads to up to 4
>>> time windows.
>>>
>>> Cheers,
>>>
>>>
>>> On Tue, Jan 16, 2018 at 10:01 AM Python_Max <python@gmail.com>
>>> wrote:
>>>
>>>> Hi.
>>>>
>>>> Thank you very much for detailed explanation.
>>>> Seems that there is nothing I can do about it except delete records by
>>>> key instead of expiring.
>>>>
>>>>
>>>> On Fri, Jan 12, 2018 at 7:30 PM, Alexander Dejanovski <
>>>> a...@thelastpickle.com> wrote:
>>>>
>>>>> Hi,
>>>>>
>>>>> As DuyHai said, different TTLs could theoretically be set for
>>>>> different cells of the same row. And one TTLed cell could be shadowing
>>>>> another cell that has no TTL (say you forgot to set a TTL and set one
>>>>> afterwards by performing an update), or v

Re: Too many tombstones using TTL

2018-01-16 Thread Python_Max
Hello.

I was planning to remove a row (not partition).

Most of the tombstones are seen in the use case of geographic grid with X:Y
as partition key and object id (timeuuid) as clustering key where objects
could be temporary with TTL about 10 hours or fully persistent.
When I select all objects in specific X:Y I can even hit 100k (default)
limit for some X:Y. I have changed this limit to 500k since 99.9p read
latency is < 75ms so I should not (?) care how many tombstones while read
latency is fine.

Splitting entities to temporary and permanent and using different
compaction strategies is an option but it will lead to code duplication and
2x read queries.

Is my assumption correct about tombstones are not so big problem as soon as
read latency and disk usage are okey? Are tombstones affect repair time
(using reaper)?

Thanks.


On Tue, Jan 16, 2018 at 11:32 AM, Alexander Dejanovski <
a...@thelastpickle.com> wrote:

> Hi,
>
> could you be more specific about the deletes you're planning to perform ?
> This will end up moving your problem somewhere else as you'll be
> generating new tombstones (and if you're planning on deleting rows, be
> aware that row level tombstones aren't reported anywhere in the metrics,
> logs and query traces).
> Currently you can delete your data at the partition level, which will
> create a single tombstone that will shadow all your expired (and non
> expired) data and is very efficient. The read path is optimized for such
> tombstones and the data won't be fully read from disk nor exchanged between
> replicas. But that's of course if your use case allows to delete full
> partitions.
>
> We usually model so that we can restrict our reads to live data.
> If you're creating time series, your clustering key should include a
> timestamp, which you can use to avoid reading expired data. If your TTL is
> set to 60 days, you can read only data that is strictly younger than that.
> Then you can partition by time ranges, and access exclusively partitions
> that have no chance to be expired yet.
> Those techniques usually work better with TWCS, but the former could make
> you hit a lot of SSTables if your partitions can spread over all time
> buckets, so only use TWCS if you can restrict individual reads to up to 4
> time windows.
>
> Cheers,
>
>
> On Tue, Jan 16, 2018 at 10:01 AM Python_Max <python@gmail.com> wrote:
>
>> Hi.
>>
>> Thank you very much for detailed explanation.
>> Seems that there is nothing I can do about it except delete records by
>> key instead of expiring.
>>
>>
>> On Fri, Jan 12, 2018 at 7:30 PM, Alexander Dejanovski <
>> a...@thelastpickle.com> wrote:
>>
>>> Hi,
>>>
>>> As DuyHai said, different TTLs could theoretically be set for different
>>> cells of the same row. And one TTLed cell could be shadowing another cell
>>> that has no TTL (say you forgot to set a TTL and set one afterwards by
>>> performing an update), or vice versa.
>>> One cell could also be missing from a node without Cassandra knowing. So
>>> turning an incomplete row that only has expired cells into a tombstone row
>>> could lead to wrong results being returned at read time : the tombstone row
>>> could potentially shadow a valid live cell from another replica.
>>>
>>> Cassandra needs to retain each TTLed cell and send it to replicas during
>>> reads to cover all possible cases.
>>>
>>>
>>> On Fri, Jan 12, 2018 at 5:28 PM Python_Max <python@gmail.com> wrote:
>>>
>>>> Thank you for response.
>>>>
>>>> I know about the option of setting TTL per column or even per item in
>>>> collection. However in my example entire row has expired, shouldn't
>>>> Cassandra be able to detect this situation and spawn a single tombstone for
>>>> entire row instead of many?
>>>> Is there any reason not doing this except that no one needs it? Is this
>>>> suitable for feature request or improvement?
>>>>
>>>> Thanks.
>>>>
>>>> On Wed, Jan 10, 2018 at 4:52 PM, DuyHai Doan <doanduy...@gmail.com>
>>>> wrote:
>>>>
>>>>> "The question is why Cassandra creates a tombstone for every column
>>>>> instead of single tombstone per row?"
>>>>>
>>>>> --> Simply because technically it is possible to set different TTL
>>>>> value on each column of a CQL row
>>>>>
>>>>> On Wed, Jan 10, 2018 at 2:59 PM, Python_Max <python@gmail.com>
>>>>> wrote:
>>>>>
>>>>>

Re: Too many tombstones using TTL

2018-01-16 Thread Python_Max
Hi.

Thank you very much for detailed explanation.
Seems that there is nothing I can do about it except delete records by key
instead of expiring.


On Fri, Jan 12, 2018 at 7:30 PM, Alexander Dejanovski <
a...@thelastpickle.com> wrote:

> Hi,
>
> As DuyHai said, different TTLs could theoretically be set for different
> cells of the same row. And one TTLed cell could be shadowing another cell
> that has no TTL (say you forgot to set a TTL and set one afterwards by
> performing an update), or vice versa.
> One cell could also be missing from a node without Cassandra knowing. So
> turning an incomplete row that only has expired cells into a tombstone row
> could lead to wrong results being returned at read time : the tombstone row
> could potentially shadow a valid live cell from another replica.
>
> Cassandra needs to retain each TTLed cell and send it to replicas during
> reads to cover all possible cases.
>
>
> On Fri, Jan 12, 2018 at 5:28 PM Python_Max <python@gmail.com> wrote:
>
>> Thank you for response.
>>
>> I know about the option of setting TTL per column or even per item in
>> collection. However in my example entire row has expired, shouldn't
>> Cassandra be able to detect this situation and spawn a single tombstone for
>> entire row instead of many?
>> Is there any reason not doing this except that no one needs it? Is this
>> suitable for feature request or improvement?
>>
>> Thanks.
>>
>> On Wed, Jan 10, 2018 at 4:52 PM, DuyHai Doan <doanduy...@gmail.com>
>> wrote:
>>
>>> "The question is why Cassandra creates a tombstone for every column
>>> instead of single tombstone per row?"
>>>
>>> --> Simply because technically it is possible to set different TTL value
>>> on each column of a CQL row
>>>
>>> On Wed, Jan 10, 2018 at 2:59 PM, Python_Max <python@gmail.com>
>>> wrote:
>>>
>>>> Hello, C* users and experts.
>>>>
>>>> I have (one more) question about tombstones.
>>>>
>>>> Consider the following example:
>>>> cqlsh> create keyspace test_ttl with replication = {'class':
>>>> 'SimpleStrategy', 'replication_factor': '1'}; use test_ttl;
>>>> cqlsh> create table items(a text, b text, c1 text, c2 text, c3 text,
>>>> primary key (a, b));
>>>> cqlsh> insert into items(a,b,c1,c2,c3) values('AAA', 'BBB', 'C111',
>>>> 'C222', 'C333') using ttl 60;
>>>> bash$ nodetool flush
>>>> bash$ sleep 60
>>>> bash$ nodetool compact test_ttl items
>>>> bash$ sstabledump mc-2-big-Data.db
>>>>
>>>> [
>>>>   {
>>>> "partition" : {
>>>>   "key" : [ "AAA" ],
>>>>   "position" : 0
>>>> },
>>>> "rows" : [
>>>>   {
>>>> "type" : "row",
>>>> "position" : 58,
>>>> "clustering" : [ "BBB" ],
>>>> "liveness_info" : { "tstamp" : "2018-01-10T13:29:25.777Z",
>>>> "ttl" : 60, "expires_at" : "2018-01-10T13:30:25Z", "expired" : true },
>>>> "cells" : [
>>>>   { "name" : "c1", "deletion_info" : { "local_delete_time" :
>>>> "2018-01-10T13:29:25Z" }
>>>>   },
>>>>   { "name" : "c2", "deletion_info" : { "local_delete_time" :
>>>> "2018-01-10T13:29:25Z" }
>>>>   },
>>>>   { "name" : "c3", "deletion_info" : { "local_delete_time" :
>>>> "2018-01-10T13:29:25Z" }
>>>>   }
>>>> ]
>>>>   }
>>>> ]
>>>>   }
>>>> ]
>>>>
>>>> The question is why Cassandra creates a tombstone for every column
>>>> instead of single tombstone per row?
>>>>
>>>> In production environment I have a table with ~30 columns and It gives
>>>> me a warning for 30k tombstones and 300 live rows. It is 30 times more then
>>>> it could be.
>>>> Can this behavior be tuned in some way?
>>>>
>>>> Thanks.
>>>>
>>>> --
>>>> Best regards,
>>>> Python_Max.
>>>>
>>>
>>>
>>
>>
>> --
>> Best regards,
>> Python_Max.
>>
>
>
> --
> -
> Alexander Dejanovski
> France
> @alexanderdeja
>
> Consultant
> Apache Cassandra Consulting
> http://www.thelastpickle.com
>



-- 
Best regards,
Python_Max.


Re: Too many tombstones using TTL

2018-01-12 Thread Python_Max
Thank you for response.

I know about the option of setting TTL per column or even per item in
collection. However in my example entire row has expired, shouldn't
Cassandra be able to detect this situation and spawn a single tombstone for
entire row instead of many?
Is there any reason not doing this except that no one needs it? Is this
suitable for feature request or improvement?

Thanks.

On Wed, Jan 10, 2018 at 4:52 PM, DuyHai Doan <doanduy...@gmail.com> wrote:

> "The question is why Cassandra creates a tombstone for every column
> instead of single tombstone per row?"
>
> --> Simply because technically it is possible to set different TTL value
> on each column of a CQL row
>
> On Wed, Jan 10, 2018 at 2:59 PM, Python_Max <python@gmail.com> wrote:
>
>> Hello, C* users and experts.
>>
>> I have (one more) question about tombstones.
>>
>> Consider the following example:
>> cqlsh> create keyspace test_ttl with replication = {'class':
>> 'SimpleStrategy', 'replication_factor': '1'}; use test_ttl;
>> cqlsh> create table items(a text, b text, c1 text, c2 text, c3 text,
>> primary key (a, b));
>> cqlsh> insert into items(a,b,c1,c2,c3) values('AAA', 'BBB', 'C111',
>> 'C222', 'C333') using ttl 60;
>> bash$ nodetool flush
>> bash$ sleep 60
>> bash$ nodetool compact test_ttl items
>> bash$ sstabledump mc-2-big-Data.db
>>
>> [
>>   {
>> "partition" : {
>>   "key" : [ "AAA" ],
>>   "position" : 0
>> },
>> "rows" : [
>>   {
>> "type" : "row",
>> "position" : 58,
>> "clustering" : [ "BBB" ],
>> "liveness_info" : { "tstamp" : "2018-01-10T13:29:25.777Z", "ttl"
>> : 60, "expires_at" : "2018-01-10T13:30:25Z", "expired" : true },
>> "cells" : [
>>   { "name" : "c1", "deletion_info" : { "local_delete_time" :
>> "2018-01-10T13:29:25Z" }
>>   },
>>   { "name" : "c2", "deletion_info" : { "local_delete_time" :
>> "2018-01-10T13:29:25Z" }
>>   },
>>   { "name" : "c3", "deletion_info" : { "local_delete_time" :
>> "2018-01-10T13:29:25Z" }
>>   }
>> ]
>>   }
>> ]
>>   }
>> ]
>>
>> The question is why Cassandra creates a tombstone for every column
>> instead of single tombstone per row?
>>
>> In production environment I have a table with ~30 columns and It gives me
>> a warning for 30k tombstones and 300 live rows. It is 30 times more then it
>> could be.
>> Can this behavior be tuned in some way?
>>
>> Thanks.
>>
>> --
>> Best regards,
>> Python_Max.
>>
>
>


-- 
Best regards,
Python_Max.


Re: sstabledump tries to delete a file

2018-01-12 Thread Python_Max
Done https://issues.apache.org/jira/browse/CASSANDRA-14166

On Wed, Jan 10, 2018 at 8:49 PM, Chris Lohfink <clohfin...@gmail.com> wrote:

> Yes it should be read only, open a jira please. It does look like if the
> fp changed it would rebuild or if your missing. When it builds the table
> metadata from the sstable it can just set the properties to match that of
> the sstable to prevent this.
>
> Chris
>
> On Wed, Jan 10, 2018 at 4:16 AM, Python_Max <python@gmail.com> wrote:
>
>> Hello all.
>>
>> I have an error when trying to dump SSTable (Cassandra 3.11.1):
>>
>> $ sstabledump mc-56801-big-Data.db
>> Exception in thread "main" FSWriteError in /var/lib/cassandra/data/> ace>//mc-56801-big-Summary.db
>> at org.apache.cassandra.io.util.FileUtils.deleteWithConfirm(Fil
>> eUtils.java:142)
>> at org.apache.cassandra.io.util.FileUtils.deleteWithConfirm(Fil
>> eUtils.java:159)
>> at org.apache.cassandra.io.sstable.format.SSTableReader.saveSum
>> mary(SSTableReader.java:935)
>> at org.apache.cassandra.io.sstable.format.SSTableReader.saveSum
>> mary(SSTableReader.java:920)
>> at org.apache.cassandra.io.sstable.format.SSTableReader.load(
>> SSTableReader.java:788)
>> at org.apache.cassandra.io.sstable.format.SSTableReader.load(
>> SSTableReader.java:731)
>> at org.apache.cassandra.io.sstable.format.SSTableReader.open(
>> SSTableReader.java:516)
>> at org.apache.cassandra.io.sstable.format.SSTableReader.openNoV
>> alidation(SSTableReader.java:396)
>> at org.apache.cassandra.tools.SSTableExport.main(SSTableExport.
>> java:191)
>> Caused by: java.nio.file.AccessDeniedException:
>> /var/lib/cassandra/data///mc-56801-big-Summary.db
>> at sun.nio.fs.UnixException.translateToIOException(UnixExceptio
>> n.java:84)
>> at sun.nio.fs.UnixException.rethrowAsIOException(UnixException.
>> java:102)
>> at sun.nio.fs.UnixException.rethrowAsIOException(UnixException.
>> java:107)
>> at sun.nio.fs.UnixFileSystemProvider.implDelete(UnixFileSystemP
>> rovider.java:244)
>> at sun.nio.fs.AbstractFileSystemProvider.delete(AbstractFileSys
>> temProvider.java:103)
>> at java.nio.file.Files.delete(Files.java:1126)
>> at org.apache.cassandra.io.util.FileUtils.deleteWithConfirm(Fil
>> eUtils.java:136)
>> ... 8 more
>>
>> Seems that sstabledump tries to delete and recreate summary file which I
>> think is risky because external modification to files that should be
>> modified only by Cassandra itself can lead to unpredictable behavior.
>> When I copy all related files and change it's owner to myself and run
>> sstabledump in that directory then Summary.db file is recreated but it's
>> md5 is exactly the same as original Summary.db's file.
>>
>> I indeed have changed bloom_filter_fp_chance couple months ago, so I
>> believe that's the reason why SSTableReader wants to recreate summary file.
>>
>> After nodetool scrub an error still happens.
>>
>> I have not found any issues like this in bug tracker.
>> Shouldn't sstabledump be read only?
>>
>> --
>> Best regards,
>> Python_Max.
>>
>
>


-- 
Best regards,
Python_Max.


Re: Deleted data comes back on node decommission

2018-01-10 Thread Python_Max
Thank you all for your help.

I was able to get rid of zombies (at least end users not reporting that
anymore) using nodetool cleanup.
And old SSTables were indeed unable to merge with each other because of
repairedAt > 0, so cassandra stop + sstablerepairedset + cassandra start in
rolling manner did fix the issue.

Thanks.

On Fri, Dec 15, 2017 at 10:47 PM, kurt greaves <k...@instaclustr.com> wrote:

> X==5. I was meant to fill that in...
>
> On 16 Dec. 2017 07:46, "kurt greaves" <k...@instaclustr.com> wrote:
>
>> Yep, if you don't run cleanup on all nodes (except new node) after step
>> x, when you decommissioned node 4 and 5 later on, their tokens will be
>> reclaimed by the previous owner. Suddenly the data in those SSTables is now
>> live again because the token ownership has changed and any data in those
>> SStables will be returned.
>>
>> Remember new nodes only add tokens to the ring, they don't affect other
>> nodes tokens, so if you remove those tokens everything goes back to how it
>> was before those nodes were added.
>>
>> Adding a maker would be incredibly complicated. Plugs not really fit the
>> design of Cassandra. Here it's probably much easier to just follow
>> recommended procedure when adding and removing nodes.
>>
>> On 16 Dec. 2017 01:37, "Python_Max" <python@gmail.com> wrote:
>>
>> Hello, Jeff.
>>
>>
>> Using your hint I was able to reproduce my situation on 5 VMs.
>> Simplified steps are:
>> 1) set up 3-node cluster
>> 2) create keyspace with RF=3 and table with gc_grace_seconds=60,
>> compaction_interval=10 and unchecked_tombstone_compaction=true (to force
>> compaction later)
>> 3) insert 10..20 records with different partition and clustering keys
>> (consistency 'all')
>> 4) 'nodetool flush' on all 3 nodes
>> 5) add 4th node, add 5th node
>> 6) using 'nodetool getendpoints' find key that moved to both 4th and 5th
>> node
>> 7) delete that record from table (consistency 'all')
>> 8) 'nodetool flush' on all 5 nodes, wait gc_grace_seconds, 'nodetool
>> compact' on nodes which responsible for that key, check that key and
>> tombstone gone using sstabledump
>> 9) decommission 5th node, decommission 4th node
>> 10) select data from table where key=key (consistency quorum)
>>
>> And the row is here.
>>
>> It sounds like bug in cassandra but since it is documented here
>> https://docs.datastax.com/en/cassandra/3.0/cassandra/operati
>> ons/opsAddNodeToCluster.html I suppose this counts as feature. It would
>> be better when data which stays in sstable after new node added would have
>> some marker and never returned as result to select query.
>>
>> Thank you very much, Jeff, for pointing me in right direction.
>>
>>
>> On 13.12.17 18:43, Jeff Jirsa wrote:
>>
>>> Did you run cleanup before you shrank the cluster?
>>>
>>>
>> --
>>
>> Best Regards,
>> Python_Max.
>>
>>
>> -
>> To unsubscribe, e-mail: user-unsubscr...@cassandra.apache.org
>> For additional commands, e-mail: user-h...@cassandra.apache.org
>>
>>
>>


-- 
Best regards,
Python_Max.


Too many tombstones using TTL

2018-01-10 Thread Python_Max
Hello, C* users and experts.

I have (one more) question about tombstones.

Consider the following example:
cqlsh> create keyspace test_ttl with replication = {'class':
'SimpleStrategy', 'replication_factor': '1'}; use test_ttl;
cqlsh> create table items(a text, b text, c1 text, c2 text, c3 text,
primary key (a, b));
cqlsh> insert into items(a,b,c1,c2,c3) values('AAA', 'BBB', 'C111', 'C222',
'C333') using ttl 60;
bash$ nodetool flush
bash$ sleep 60
bash$ nodetool compact test_ttl items
bash$ sstabledump mc-2-big-Data.db

[
  {
"partition" : {
  "key" : [ "AAA" ],
  "position" : 0
},
"rows" : [
  {
"type" : "row",
"position" : 58,
"clustering" : [ "BBB" ],
"liveness_info" : { "tstamp" : "2018-01-10T13:29:25.777Z", "ttl" :
60, "expires_at" : "2018-01-10T13:30:25Z", "expired" : true },
"cells" : [
  { "name" : "c1", "deletion_info" : { "local_delete_time" :
"2018-01-10T13:29:25Z" }
  },
  { "name" : "c2", "deletion_info" : { "local_delete_time" :
"2018-01-10T13:29:25Z" }
  },
  { "name" : "c3", "deletion_info" : { "local_delete_time" :
"2018-01-10T13:29:25Z" }
  }
]
  }
]
  }
]

The question is why Cassandra creates a tombstone for every column instead
of single tombstone per row?

In production environment I have a table with ~30 columns and It gives me a
warning for 30k tombstones and 300 live rows. It is 30 times more then it
could be.
Can this behavior be tuned in some way?

Thanks.

-- 
Best regards,
Python_Max.


sstabledump tries to delete a file

2018-01-10 Thread Python_Max
Hello all.

I have an error when trying to dump SSTable (Cassandra 3.11.1):

$ sstabledump mc-56801-big-Data.db
Exception in thread "main" FSWriteError in
/var/lib/cassandra/data///mc-56801-big-Summary.db
at
org.apache.cassandra.io.util.FileUtils.deleteWithConfirm(FileUtils.java:142)
at
org.apache.cassandra.io.util.FileUtils.deleteWithConfirm(FileUtils.java:159)
at
org.apache.cassandra.io.sstable.format.SSTableReader.saveSummary(SSTableReader.java:935)
at
org.apache.cassandra.io.sstable.format.SSTableReader.saveSummary(SSTableReader.java:920)
at
org.apache.cassandra.io.sstable.format.SSTableReader.load(SSTableReader.java:788)
at
org.apache.cassandra.io.sstable.format.SSTableReader.load(SSTableReader.java:731)
at
org.apache.cassandra.io.sstable.format.SSTableReader.open(SSTableReader.java:516)
at
org.apache.cassandra.io.sstable.format.SSTableReader.openNoValidation(SSTableReader.java:396)
at
org.apache.cassandra.tools.SSTableExport.main(SSTableExport.java:191)
Caused by: java.nio.file.AccessDeniedException:
/var/lib/cassandra/data///mc-56801-big-Summary.db
at
sun.nio.fs.UnixException.translateToIOException(UnixException.java:84)
at
sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:102)
at
sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:107)
at
sun.nio.fs.UnixFileSystemProvider.implDelete(UnixFileSystemProvider.java:244)
at
sun.nio.fs.AbstractFileSystemProvider.delete(AbstractFileSystemProvider.java:103)
at java.nio.file.Files.delete(Files.java:1126)
at
org.apache.cassandra.io.util.FileUtils.deleteWithConfirm(FileUtils.java:136)
... 8 more

Seems that sstabledump tries to delete and recreate summary file which I
think is risky because external modification to files that should be
modified only by Cassandra itself can lead to unpredictable behavior.
When I copy all related files and change it's owner to myself and run
sstabledump in that directory then Summary.db file is recreated but it's
md5 is exactly the same as original Summary.db's file.

I indeed have changed bloom_filter_fp_chance couple months ago, so I
believe that's the reason why SSTableReader wants to recreate summary file.

After nodetool scrub an error still happens.

I have not found any issues like this in bug tracker.
Shouldn't sstabledump be read only?

-- 
Best regards,
Python_Max.


Re: Deleted data comes back on node decommission

2017-12-15 Thread Python_Max

Hi, Kurt.

Thank you for response.


Repairs are marked as 'done' without errors in reaper history.

Example of 'wrong order':

* file mc-31384-big-Data.db contains tombstone:

    {
    "type" : "row",
    "position" : 7782,
    "clustering" : [ "9adab970-b46d-11e7-a5cd-a1ba8cfc1426" ],
    "deletion_info" : { "marked_deleted" : 
"2017-10-28T04:51:20.589394Z", "local_delete_time" : 
"2017-10-28T04:51:20Z" },

    "cells" : [ ]
  }

* file mc-31389-big-Data.db contains data:

    {
    "type" : "row",
    "position" : 81317,
    "clustering" : [ "9adab970-b46d-11e7-a5cd-a1ba8cfc1426" ],
    "liveness_info" : { "tstamp" : "2017-10-19T01:34:10.055389Z" },
    "cells" : [...]
  }

Index 31384 is less than 31389 but I'm not sure whether it matters at all.

I assume that data and tombsones are not compacting due to another 
reason: the tokens are not owned by that node anymore and the only way 
to purge such keys is 'nodetool cleanup', isn't it?



On 14.12.17 16:14, kurt greaves wrote:
Are you positive your repairs are completing successfully? Can you 
send through an example of the data in the wrong order? What you're 
saying certainly shouldn't happen, but there's a lot of room for mistakes.


On 14 Dec. 2017 20:13, "Python_Max" <python@gmail.com 
<mailto:python@gmail.com>> wrote:


Thank you for reply.

No, I did not execute 'nodetool cleanup'. Documentation

https://docs.datastax.com/en/cassandra/3.0/cassandra/operations/opsRemoveNode.html

<https://docs.datastax.com/en/cassandra/3.0/cassandra/operations/opsRemoveNode.html>
does not mention that cleanup is required.

Do yo think that extra data which node is not responsible for can
lead to zombie data?


On 13.12.17 18:43, Jeff Jirsa wrote:

Did you run cleanup before you shrank the cluster?


-- 


Best Regards,
Python_Max.


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



--

Best Regards,
Python_Max.



Re: Deleted data comes back on node decommission

2017-12-15 Thread Python_Max

Hello, Jeff.


Using your hint I was able to reproduce my situation on 5 VMs.
Simplified steps are:
1) set up 3-node cluster
2) create keyspace with RF=3 and table with gc_grace_seconds=60, 
compaction_interval=10 and unchecked_tombstone_compaction=true (to force 
compaction later)
3) insert 10..20 records with different partition and clustering keys 
(consistency 'all')

4) 'nodetool flush' on all 3 nodes
5) add 4th node, add 5th node
6) using 'nodetool getendpoints' find key that moved to both 4th and 5th 
node

7) delete that record from table (consistency 'all')
8) 'nodetool flush' on all 5 nodes, wait gc_grace_seconds, 'nodetool 
compact' on nodes which responsible for that key, check that key and 
tombstone gone using sstabledump

9) decommission 5th node, decommission 4th node
10) select data from table where key=key (consistency quorum)

And the row is here.

It sounds like bug in cassandra but since it is documented here 
https://docs.datastax.com/en/cassandra/3.0/cassandra/operations/opsAddNodeToCluster.html 
I suppose this counts as feature. It would be better when data which 
stays in sstable after new node added would have some marker and never 
returned as result to select query.


Thank you very much, Jeff, for pointing me in right direction.

On 13.12.17 18:43, Jeff Jirsa wrote:

Did you run cleanup before you shrank the cluster?



--

Best Regards,
Python_Max.


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



Re: Deleted data comes back on node decommission

2017-12-14 Thread Python_Max

Thank you for reply.

No, I did not execute 'nodetool cleanup'. Documentation 
https://docs.datastax.com/en/cassandra/3.0/cassandra/operations/opsRemoveNode.html 
does not mention that cleanup is required.


Do yo think that extra data which node is not responsible for can lead 
to zombie data?



On 13.12.17 18:43, Jeff Jirsa wrote:

Did you run cleanup before you shrank the cluster?



--

Best Regards,
Python_Max.


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



Deleted data comes back on node decommission

2017-12-13 Thread Python_Max

Hello.

I have a situation similar to 
https://issues.apache.org/jira/browse/CASSANDRA-13153 except mine 
cassandra is 3.11.1 (that issue should be fixed according to jira).


Cluster consist of 40 nodes which I have to shrink to 25 more powerful 
nodes moving less powerful out from the cluster.


SizeTieredCompactionStrategy is used for target table with default 
compaction options and default gc_grace_seconds (10 days).


Full sequential repair executed every day using 
https://github.com/thelastpickle/cassandra-reaper


Before reaper was involved the default (incremental) 'nodetool repair' 
was used.



There are couple of problems that I observe.

1. Compaction.

There are rows in target table which deleted long time ago 
(gc_grace_seconds passed) but they are not compacting. I tried 'nodetool 
compact' and 'nodetool repair -full -seq' with same outcome: sstables 
recreated but that rows still there (I used sstabledump to detect this 
state).


Some of that rows have tombstone and data in wrong order: the data 
located at more recent sstable by index but relative tombstone is 
located in previous sstable (shouldn't it be opposite?). CQL select does 
not return that rows (correct behaviour) until node decommission.


sstables with non-compacted rows have very old repairedAt value (about 
10 days before the first deleted row in sstable which should have been 
compacted long time ago).



2. Streaming.

When moving the node out of cluster 'nodetool decommission' is used. 
After streaming complete some old rows that was not compacted earlier is 
back to life in shrinked cluster.


CQL select does return that rows as alive until running full sequential 
repair using cassandra-reaper.


As workaround I tried to shut down a node and using 'nodetool 
removenode' in case the node itself is streaming wrong data on 
decommission but that did not work either (deleted data is back to life).



Is this a known issue?


PS: I have not tried 'nodetool scrub' yet nor dropping repairedAt for 
affected sstables.



--

Best Regards,
Python_Max.


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