You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by Python_Max <py...@gmail.com> on 2017/12/13 12:49:10 UTC

Deleted data comes back on node decommission

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-unsubscribe@cassandra.apache.org
For additional commands, e-mail: user-help@cassandra.apache.org


Re: Deleted data comes back on node decommission

Posted by Python_Max <py...@gmail.com>.
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 <ku...@instaclustr.com> wrote:

> X==5. I was meant to fill that in...
>
> On 16 Dec. 2017 07:46, "kurt greaves" <ku...@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" <py...@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-unsubscribe@cassandra.apache.org
>> For additional commands, e-mail: user-help@cassandra.apache.org
>>
>>
>>


-- 
Best regards,
Python_Max.

Re: Deleted data comes back on node decommission

Posted by kurt greaves <ku...@instaclustr.com>.
X==5. I was meant to fill that in...

On 16 Dec. 2017 07:46, "kurt greaves" <ku...@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" <py...@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-unsubscribe@cassandra.apache.org
> For additional commands, e-mail: user-help@cassandra.apache.org
>
>
>

Re: Deleted data comes back on node decommission

Posted by kurt greaves <ku...@instaclustr.com>.
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" <py...@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-unsubscribe@cassandra.apache.org
For additional commands, e-mail: user-help@cassandra.apache.org

Re: Deleted data comes back on node decommission

Posted by Python_Max <py...@gmail.com>.
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-unsubscribe@cassandra.apache.org
For additional commands, e-mail: user-help@cassandra.apache.org


Re: Deleted data comes back on node decommission

Posted by Alexander Dejanovski <al...@thelastpickle.com>.
Hi Max,

I don't know if it's related to your issue but on a side note, if you
decide to use Reaper (and use full repairs, not incremental ones), but mix
that with "nodetool repair", you'll end up with 2 pools of SSTables that
cannot get compacted together.
Reaper uses subrange repair which doesn't mark SSTables are repaired (no
anticompaction is performed, repairedAt remains at 0), while using nodetool
in full and incremental modes will perform anticompaction.

SSTables with repairedAt > 0 cannot be compacted with SSTables with
repairedAt = 0.

Bottom line is that if you want your SSTables to be compacted together
naturally, you have to run repairs either exclusively through Reaper or
exclusively through nodetool.
If you decide to use Reaper exclusively, you have to revert the repairedAt
value to 0 for all sstables on all nodes, using sstablerepairedset
<https://docs.datastax.com/en/cassandra/2.1/cassandra/tools/toolsSSTableRepairedSet.html>
.

Cheers,

On Fri, Dec 15, 2017 at 4:57 PM Jeff Jirsa <jj...@gmail.com> wrote:

> The generation (integer id in file names) doesn’t matter for ordering like
> this
>
> It matters in schema tables for addition of new columns/types, but it’s
> irrelevant for normal tables - you could do a user defined compaction on
> 31384 right now and it’d be rewritten as-is (minus purgable data) with the
> new highest generation, even though it’s all old data.
>
>
> --
> Jeff Jirsa
>
>
> On Dec 15, 2017, at 6:55 AM, Python_Max <py...@gmail.com> wrote:
>
> 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" <py...@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
>> 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-unsubscribe@cassandra.apache.org
>> For additional commands, e-mail: user-help@cassandra.apache.org
>> happen
>
>
> --
>
> Best Regards,
> Python_Max.
>
> --
-----------------
Alexander Dejanovski
France
@alexanderdeja

Consultant
Apache Cassandra Consulting
http://www.thelastpickle.com

Re: Deleted data comes back on node decommission

Posted by Jeff Jirsa <jj...@gmail.com>.
The generation (integer id in file names) doesn’t matter for ordering like this 

It matters in schema tables for addition of new columns/types, but it’s irrelevant for normal tables - you could do a user defined compaction on 31384 right now and it’d be rewritten as-is (minus purgable data) with the new highest generation, even though it’s all old data. 


-- 
Jeff Jirsa


> On Dec 15, 2017, at 6:55 AM, Python_Max <py...@gmail.com> wrote:
> 
> 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" <py...@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 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-unsubscribe@cassandra.apache.org
>>> For additional commands, e-mail: user-help@cassandra.apache.org
>>> happen
> 
> -- 
> 
> Best Regards,
> Python_Max.

Re: Deleted data comes back on node decommission

Posted by Python_Max <py...@gmail.com>.
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.max@gmail.com 
> <ma...@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-unsubscribe@cassandra.apache.org
>     <ma...@cassandra.apache.org>
>     For additional commands, e-mail: user-help@cassandra.apache.org
>     <ma...@cassandra.apache.org>
>     happen 
>

-- 

Best Regards,
Python_Max.


Re: Deleted data comes back on node decommission

Posted by kurt greaves <ku...@instaclustr.com>.
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" <py...@gmail.com> wrote:

> Thank you for reply.
>
> No, I did not execute 'nodetool cleanup'. Documentation
> https://docs.datastax.com/en/cassandra/3.0/cassandra/operati
> ons/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-unsubscribe@cassandra.apache.org
> For additional commands, e-mail: user-help@cassandra.apache.org
> happen

Re: Deleted data comes back on node decommission

Posted by Python_Max <py...@gmail.com>.
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-unsubscribe@cassandra.apache.org
For additional commands, e-mail: user-help@cassandra.apache.org


Re: Deleted data comes back on node decommission

Posted by Jeff Jirsa <jj...@gmail.com>.
Did you run cleanup before you shrank the cluster? 

-- 
Jeff Jirsa


> On Dec 13, 2017, at 4:49 AM, Python_Max <py...@gmail.com> wrote:
> 
> 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-unsubscribe@cassandra.apache.org
> For additional commands, e-mail: user-help@cassandra.apache.org
> 

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