Re: Alternate major compaction

2013-07-17 Thread Radim Kolar

Dne 16.7.2013 20:45, Robert Coli napsal(a):
On Fri, Jul 12, 2013 at 2:28 AM, Radim Kolar h...@filez.com 
mailto:h...@filez.com wrote:


with some very little work (less then 10 KB of code) is possible
to have online sstable splitter and exported this functionality
over JMX.


Are you volunteering? If so, I'd totally dig using the patch! :D

I do not work for free.


Re: Alternate major compaction

2013-07-16 Thread Robert Coli
On Fri, Jul 12, 2013 at 2:28 AM, Radim Kolar h...@filez.com wrote:

 with some very little work (less then 10 KB of code) is possible to have
 online sstable splitter and exported this functionality over JMX.


Are you volunteering? If so, I'd totally dig using the patch! :D

=Rob


Re: Alternate major compaction

2013-07-12 Thread Radim Kolar
with some very little work (less then 10 KB of code) is possible to have 
online sstable splitter and exported this functionality over JMX.


Re: Alternate major compaction

2013-07-12 Thread Robert Coli
On Thu, Jul 11, 2013 at 9:43 PM, Takenori Sato ts...@cloudian.com wrote:

 I made the repository public. Now you can checkout from here.

 https://github.com/cloudian/support-tools

 checksstablegarbage is the tool.

 Enjoy, and any feedback is welcome.


Thanks very much, useful tool!

Out of curiousity, what does writesstablekeys do that the upstream tool
sstablekeys does not?

=Rob


Re: Alternate major compaction

2013-07-12 Thread Takenori Sato
It's light. Without -v option, you can even run it against just a SSTable
file without needing the whole Cassandra installation.

- Takenori


On Sat, Jul 13, 2013 at 6:18 AM, Robert Coli rc...@eventbrite.com wrote:

 On Thu, Jul 11, 2013 at 9:43 PM, Takenori Sato ts...@cloudian.com wrote:

 I made the repository public. Now you can checkout from here.

 https://github.com/cloudian/support-tools

 checksstablegarbage is the tool.

 Enjoy, and any feedback is welcome.


 Thanks very much, useful tool!

 Out of curiousity, what does writesstablekeys do that the upstream tool
 sstablekeys does not?

 =Rob



Alternate major compaction

2013-07-11 Thread Tomàs Núnez
Hi

About a year ago, we did a major compaction in our cassandra cluster (a
n00b mistake, I know), and since then we've had huge sstables that never
get compacted, and we were condemned to repeat the major compaction process
every once in a while (we are using SizeTieredCompaction strategy, and
we've not avaluated yet LeveledCompaction, because it has its downsides,
and we've had no time to test all of them in our environment).

I was trying to find a way to solve this situation (that is, do something
like a major compaction that writes small sstables, not huge as major
compaction does), and I couldn't find it in the documentation. I tried
cleanup and scrub/upgradesstables, but they don't do that (as documentation
states). Then I tried deleting all data in a node and then bootstrapping it
(or nodetool rebuild-ing it), hoping that this way the sstables would get
cleaned from deleted records and updates. But the deleted node just copied
the sstables from another node as they were, cleaning nothing.

So I tried a new approach: I switched the sstable compaction strategy
(SizeTiered to Leveled), forcing the sstables to be rewritten from scratch,
and then switching it back (Leveled to SizeTiered). It took a while (but so
do the major compaction process) and it worked, I have smaller sstables,
and I've regained a lot of disk space.

I'm happy with the results, but it doesn't seem a orthodox way of
cleaning the sstables. What do you think, is it something wrong or crazy?
Is there a different way to achieve the same thing?

Let's put an example:
Suppose you have a write-only columnfamily (no updates and no deletes, so
no need for LeveledCompaction, because SizeTiered works perfectly and
requires less I/O) and you mistakenly run a major compaction on it. After a
few months you need more space and you delete half the data, and you find
out that you're not freeing half the disk space, because most of those
records were in the major compacted sstables. How can you free the disk
space? Waiting will do you no good, because the huge sstable won't get
compacted anytime soon. You can run another major compaction, but that
would just postpone the real problem. Then you can switch compaction
strategy and switch it back, as I just did. Is there any other way?

-- 
[image: Groupalia] http://es.groupalia.com/
www.groupalia.com http://es.groupalia.com/Tomàs NúñezIT-SysprodTel. + 34
93 159 31 00 Fax. + 34 93 396 18 52Llull, 95-97, 2º planta, 08005
BarcelonaSkype:
tomas.nunez.groupaliatomas.nu...@groupalia.comnombre.apell...@groupalia.com[image:
Twitter] Twitter http://twitter.com/#%21/groupaliaes[image: Twitter]
 Facebook https://www.facebook.com/GroupaliaEspana[image: Twitter]
 Linkedin http://www.linkedin.com/company/groupalia
linkedin.pngtwitter.pngfacebook.pnggroupalia.jpg

Re: Alternate major compaction

2013-07-11 Thread srmore
Thanks Takenori,
Looks like the tool provides some good info that people can use. It would
be great if you can share it with the community.



On Thu, Jul 11, 2013 at 6:51 AM, Takenori Sato ts...@cloudian.com wrote:

 Hi,

 I think it is a common headache for users running a large Cassandra
 cluster in production.


 Running a major compaction is not the only cause, but more. For example, I
 see two typical scenario.

 1. backup use case
 2. active wide row

 In the case of 1, say, one data is removed a year later. This means,
 tombstone on the row is 1 year away from the original row. To remove an
 expired row entirely, a compaction set has to include all the rows. So,
 when do the original, 1 year old row, and the tombstoned row are included
 in a compaction set? It is likely to take one year.

 In the case of 2, such an active wide row exists in most of sstable files.
 And it typically contains many expired columns. But none of them wouldn't
 be removed entirely because a compaction set practically do not include all
 the row fragments.


 Btw, there is a very convenient MBean API is available. It is
 CompactionManager's forceUserDefinedCompaction. You can invoke a minor
 compaction on a file set you define. So the question is how to find an
 optimal set of sstable files.

 Then, I wrote a tool to check garbage, and print outs some useful
 information to find such an optimal set.

 Here's a simple log output.

 # /opt/cassandra/bin/checksstablegarbage -e 
 /cassandra_data/UserData/Test5_BLOB-hc-4-Data.db
 [Keyspace, ColumnFamily, gcGraceSeconds(gcBefore)] = [UserData, Test5_BLOB, 
 300(1373504071)]
 ===
 ROW_KEY, TOTAL_SIZE, COMPACTED_SIZE, TOMBSTONED, EXPIRED, 
 REMAINNING_SSTABLE_FILES
 ===
 hello5/100.txt.1373502926003, 40, 40, YES, YES, Test5_BLOB-hc-3-Data.db
 ---
 TOTAL, 40, 40
 ===

 REMAINNING_SSTABLE_FILES means any other sstable files that contain the
 respective row. So, the following is an optimal set.

 # /opt/cassandra/bin/checksstablegarbage -e 
 /cassandra_data/UserData/Test5_BLOB-hc-4-Data.db 
 /cassandra_data/UserData/Test5_BLOB-hc-3-Data.db
 [Keyspace, ColumnFamily, gcGraceSeconds(gcBefore)] = [UserData, Test5_BLOB, 
 300(1373504131)]
 ===
 ROW_KEY, TOTAL_SIZE, COMPACTED_SIZE, TOMBSTONED, EXPIRED, 
 REMAINNING_SSTABLE_FILES
 ===
 hello5/100.txt.1373502926003, 223, 0, YES, YES
 ---
 TOTAL, 223, 0
 ===

 This tool relies on SSTableReader and an aggregation iterator as Cassandra
 does in compaction. I was considering to share this with the community. So
 let me know if anyone is interested.

 Ah, note that it is based on 1.0.7. So I will need to check and update for
 newer versions.

 Thanks,
 Takenori


 On Thu, Jul 11, 2013 at 6:46 PM, Tomàs Núnez tomas.nu...@groupalia.comwrote:

 Hi

 About a year ago, we did a major compaction in our cassandra cluster (a
 n00b mistake, I know), and since then we've had huge sstables that never
 get compacted, and we were condemned to repeat the major compaction process
 every once in a while (we are using SizeTieredCompaction strategy, and
 we've not avaluated yet LeveledCompaction, because it has its downsides,
 and we've had no time to test all of them in our environment).

 I was trying to find a way to solve this situation (that is, do something
 like a major compaction that writes small sstables, not huge as major
 compaction does), and I couldn't find it in the documentation. I tried
 cleanup and scrub/upgradesstables, but they don't do that (as documentation
 states). Then I tried deleting all data in a node and then bootstrapping it
 (or nodetool rebuild-ing it), hoping that this way the sstables would get
 cleaned from deleted records and updates. But the deleted node just copied
 the sstables from another node as they were, cleaning nothing.

 So I tried a new approach: I switched the sstable compaction strategy
 (SizeTiered to Leveled), forcing the sstables to be rewritten from scratch,
 and then switching it back (Leveled to SizeTiered). It took a while (but so
 do the major compaction process) and it worked, I have smaller sstables,
 and I've regained a lot of disk space.

 I'm happy with the results, but it doesn't seem a orthodox way of
 cleaning the sstables. What do you think, is it something wrong or crazy?
 Is there a different way to achieve the same thing?

 Let's put an example:
 Suppose you have a 

Re: Alternate major compaction

2013-07-11 Thread Brian Tarbox
Perhaps I should already know this but why is running a major compaction
considered so bad?  We're running 1.1.6.

Thanks.


On Thu, Jul 11, 2013 at 7:51 AM, Takenori Sato ts...@cloudian.com wrote:

 Hi,

 I think it is a common headache for users running a large Cassandra
 cluster in production.


 Running a major compaction is not the only cause, but more. For example, I
 see two typical scenario.

 1. backup use case
 2. active wide row

 In the case of 1, say, one data is removed a year later. This means,
 tombstone on the row is 1 year away from the original row. To remove an
 expired row entirely, a compaction set has to include all the rows. So,
 when do the original, 1 year old row, and the tombstoned row are included
 in a compaction set? It is likely to take one year.

 In the case of 2, such an active wide row exists in most of sstable files.
 And it typically contains many expired columns. But none of them wouldn't
 be removed entirely because a compaction set practically do not include all
 the row fragments.


 Btw, there is a very convenient MBean API is available. It is
 CompactionManager's forceUserDefinedCompaction. You can invoke a minor
 compaction on a file set you define. So the question is how to find an
 optimal set of sstable files.

 Then, I wrote a tool to check garbage, and print outs some useful
 information to find such an optimal set.

 Here's a simple log output.

 # /opt/cassandra/bin/checksstablegarbage -e 
 /cassandra_data/UserData/Test5_BLOB-hc-4-Data.db
 [Keyspace, ColumnFamily, gcGraceSeconds(gcBefore)] = [UserData, Test5_BLOB, 
 300(1373504071)]
 ===
 ROW_KEY, TOTAL_SIZE, COMPACTED_SIZE, TOMBSTONED, EXPIRED, 
 REMAINNING_SSTABLE_FILES
 ===
 hello5/100.txt.1373502926003, 40, 40, YES, YES, Test5_BLOB-hc-3-Data.db
 ---
 TOTAL, 40, 40
 ===

 REMAINNING_SSTABLE_FILES means any other sstable files that contain the
 respective row. So, the following is an optimal set.

 # /opt/cassandra/bin/checksstablegarbage -e 
 /cassandra_data/UserData/Test5_BLOB-hc-4-Data.db 
 /cassandra_data/UserData/Test5_BLOB-hc-3-Data.db
 [Keyspace, ColumnFamily, gcGraceSeconds(gcBefore)] = [UserData, Test5_BLOB, 
 300(1373504131)]
 ===
 ROW_KEY, TOTAL_SIZE, COMPACTED_SIZE, TOMBSTONED, EXPIRED, 
 REMAINNING_SSTABLE_FILES
 ===
 hello5/100.txt.1373502926003, 223, 0, YES, YES
 ---
 TOTAL, 223, 0
 ===

 This tool relies on SSTableReader and an aggregation iterator as Cassandra
 does in compaction. I was considering to share this with the community. So
 let me know if anyone is interested.

 Ah, note that it is based on 1.0.7. So I will need to check and update for
 newer versions.

 Thanks,
 Takenori


 On Thu, Jul 11, 2013 at 6:46 PM, Tomàs Núnez tomas.nu...@groupalia.comwrote:

 Hi

 About a year ago, we did a major compaction in our cassandra cluster (a
 n00b mistake, I know), and since then we've had huge sstables that never
 get compacted, and we were condemned to repeat the major compaction process
 every once in a while (we are using SizeTieredCompaction strategy, and
 we've not avaluated yet LeveledCompaction, because it has its downsides,
 and we've had no time to test all of them in our environment).

 I was trying to find a way to solve this situation (that is, do something
 like a major compaction that writes small sstables, not huge as major
 compaction does), and I couldn't find it in the documentation. I tried
 cleanup and scrub/upgradesstables, but they don't do that (as documentation
 states). Then I tried deleting all data in a node and then bootstrapping it
 (or nodetool rebuild-ing it), hoping that this way the sstables would get
 cleaned from deleted records and updates. But the deleted node just copied
 the sstables from another node as they were, cleaning nothing.

 So I tried a new approach: I switched the sstable compaction strategy
 (SizeTiered to Leveled), forcing the sstables to be rewritten from scratch,
 and then switching it back (Leveled to SizeTiered). It took a while (but so
 do the major compaction process) and it worked, I have smaller sstables,
 and I've regained a lot of disk space.

 I'm happy with the results, but it doesn't seem a orthodox way of
 cleaning the sstables. What do you think, is it something wrong or crazy?
 Is there a different way to achieve the same thing?

 Let's put an example:
 Suppose you have a write-only 

Re: Alternate major compaction

2013-07-11 Thread Michael Theroux
Information is only deleted from Cassandra during a compaction.  Using 
SizeTieredCompaction, compaction only occurs when a number of similarly sized 
sstables are combined into a new sstable.  

When you perform a major compaction, all sstables are combined into one, very 
large, sstable.  As a result, any tombstoned data in that large sstable will 
only be removed when a number of very large sstable exists.  This means 
tombstoned data maybe trapped in that sstable for a very long time (or 
indefinitely depending on your usecase).

-Mike

On Jul 11, 2013, at 9:31 AM, Brian Tarbox wrote:

 Perhaps I should already know this but why is running a major compaction 
 considered so bad?  We're running 1.1.6.
 
 Thanks.
 
 
 On Thu, Jul 11, 2013 at 7:51 AM, Takenori Sato ts...@cloudian.com wrote:
 Hi,
 
 I think it is a common headache for users running a large Cassandra cluster 
 in production.
 
 
 Running a major compaction is not the only cause, but more. For example, I 
 see two typical scenario.
 
 1. backup use case
 2. active wide row
 
 In the case of 1, say, one data is removed a year later. This means, 
 tombstone on the row is 1 year away from the original row. To remove an 
 expired row entirely, a compaction set has to include all the rows. So, when 
 do the original, 1 year old row, and the tombstoned row are included in a 
 compaction set? It is likely to take one year.
 
 In the case of 2, such an active wide row exists in most of sstable files. 
 And it typically contains many expired columns. But none of them wouldn't be 
 removed entirely because a compaction set practically do not include all the 
 row fragments.
 
 
 Btw, there is a very convenient MBean API is available. It is 
 CompactionManager's forceUserDefinedCompaction. You can invoke a minor 
 compaction on a file set you define. So the question is how to find an 
 optimal set of sstable files.
 
 Then, I wrote a tool to check garbage, and print outs some useful information 
 to find such an optimal set.
 
 Here's a simple log output.
 
 # /opt/cassandra/bin/checksstablegarbage -e 
 /cassandra_data/UserData/Test5_BLOB-hc-4-Data.db
 [Keyspace, ColumnFamily, gcGraceSeconds(gcBefore)] = [UserData, Test5_BLOB, 
 300(1373504071)]
 ===
 ROW_KEY, TOTAL_SIZE, COMPACTED_SIZE, TOMBSTONED, EXPIRED, 
 REMAINNING_SSTABLE_FILES
 ===
 hello5/100.txt.1373502926003, 40, 40, YES, YES, Test5_BLOB-hc-3-Data.db
 ---
 TOTAL, 40, 40
 ===
 REMAINNING_SSTABLE_FILES means any other sstable files that contain the 
 respective row. So, the following is an optimal set.
 
 # /opt/cassandra/bin/checksstablegarbage -e 
 /cassandra_data/UserData/Test5_BLOB-hc-4-Data.db 
 /cassandra_data/UserData/Test5_BLOB-hc-3-Data.db 
 [Keyspace, ColumnFamily, gcGraceSeconds(gcBefore)] = [UserData, Test5_BLOB, 
 300(1373504131)]
 ===
 ROW_KEY, TOTAL_SIZE, COMPACTED_SIZE, TOMBSTONED, EXPIRED, 
 REMAINNING_SSTABLE_FILES
 ===
 hello5/100.txt.1373502926003, 223, 0, YES, YES
 ---
 TOTAL, 223, 0
 ===
 This tool relies on SSTableReader and an aggregation iterator as Cassandra 
 does in compaction. I was considering to share this with the community. So 
 let me know if anyone is interested.
 
 Ah, note that it is based on 1.0.7. So I will need to check and update for 
 newer versions.
 
 Thanks,
 Takenori
 
 
 On Thu, Jul 11, 2013 at 6:46 PM, Tomàs Núnez tomas.nu...@groupalia.com 
 wrote:
 Hi
 
 About a year ago, we did a major compaction in our cassandra cluster (a n00b 
 mistake, I know), and since then we've had huge sstables that never get 
 compacted, and we were condemned to repeat the major compaction process every 
 once in a while (we are using SizeTieredCompaction strategy, and we've not 
 avaluated yet LeveledCompaction, because it has its downsides, and we've had 
 no time to test all of them in our environment).
 
 I was trying to find a way to solve this situation (that is, do something 
 like a major compaction that writes small sstables, not huge as major 
 compaction does), and I couldn't find it in the documentation. I tried 
 cleanup and scrub/upgradesstables, but they don't do that (as documentation 
 states). Then I tried deleting all data in a node and then bootstrapping it 
 (or nodetool rebuild-ing it), hoping that this way the sstables would get 
 cleaned from deleted records and updates. But the deleted node just copied 
 the 

Re: Alternate major compaction

2013-07-11 Thread Robert Coli
On Thu, Jul 11, 2013 at 2:46 AM, Tomàs Núnez tomas.nu...@groupalia.comwrote:

 Hi

 About a year ago, we did a major compaction in our cassandra cluster (a
 n00b mistake, I know), and since then we've had huge sstables that never
 get compacted, and we were condemned to repeat the major compaction process
 every once in a while (we are using SizeTieredCompaction strategy, and
 we've not avaluated yet LeveledCompaction, because it has its downsides,
 and we've had no time to test all of them in our environment).

 I was trying to find a way to solve this situation (that is, do something
 like a major compaction that writes small sstables, not huge as major
 compaction does), and I couldn't find it in the documentation.


https://github.com/pcmanus/cassandra/tree/sstable_split

1) run sstable_split on One Big SSTable (being careful to avoid name
collisions if done with node running)
2) stop node
3) remove One Big SSTable
4) start node

This approach is significantly more i/o efficient than your online
solution, but does require a node restart and messing around directly with
SSTables. Your online solution is clever!

If you choose to use this tool, please let us know the result. With some
feedback, pcmanus (Sylvain) is likely to merge it into Cassandra as a
useful tool for dealing with for example this situation.

=Rob


Re: Alternate major compaction

2013-07-11 Thread Takenori Sato
Hi,

I made the repository public. Now you can checkout from here.

https://github.com/cloudian/support-tools

checksstablegarbage is the tool.

Enjoy, and any feedback is welcome.

Thanks,
- Takenori


On Thu, Jul 11, 2013 at 10:12 PM, srmore comom...@gmail.com wrote:

 Thanks Takenori,
 Looks like the tool provides some good info that people can use. It would
 be great if you can share it with the community.




 On Thu, Jul 11, 2013 at 6:51 AM, Takenori Sato ts...@cloudian.com wrote:

 Hi,

 I think it is a common headache for users running a large Cassandra
 cluster in production.


 Running a major compaction is not the only cause, but more. For example,
 I see two typical scenario.

 1. backup use case
 2. active wide row

 In the case of 1, say, one data is removed a year later. This means,
 tombstone on the row is 1 year away from the original row. To remove an
 expired row entirely, a compaction set has to include all the rows. So,
 when do the original, 1 year old row, and the tombstoned row are included
 in a compaction set? It is likely to take one year.

 In the case of 2, such an active wide row exists in most of sstable
 files. And it typically contains many expired columns. But none of them
 wouldn't be removed entirely because a compaction set practically do not
 include all the row fragments.


 Btw, there is a very convenient MBean API is available. It is
 CompactionManager's forceUserDefinedCompaction. You can invoke a minor
 compaction on a file set you define. So the question is how to find an
 optimal set of sstable files.

 Then, I wrote a tool to check garbage, and print outs some useful
 information to find such an optimal set.

 Here's a simple log output.

 # /opt/cassandra/bin/checksstablegarbage -e 
 /cassandra_data/UserData/Test5_BLOB-hc-4-Data.db
 [Keyspace, ColumnFamily, gcGraceSeconds(gcBefore)] = [UserData, Test5_BLOB, 
 300(1373504071)]
 ===
 ROW_KEY, TOTAL_SIZE, COMPACTED_SIZE, TOMBSTONED, EXPIRED, 
 REMAINNING_SSTABLE_FILES
 ===
 hello5/100.txt.1373502926003, 40, 40, YES, YES, Test5_BLOB-hc-3-Data.db
 ---
 TOTAL, 40, 40
 ===

 REMAINNING_SSTABLE_FILES means any other sstable files that contain the
 respective row. So, the following is an optimal set.

 # /opt/cassandra/bin/checksstablegarbage -e 
 /cassandra_data/UserData/Test5_BLOB-hc-4-Data.db 
 /cassandra_data/UserData/Test5_BLOB-hc-3-Data.db
 [Keyspace, ColumnFamily, gcGraceSeconds(gcBefore)] = [UserData, Test5_BLOB, 
 300(1373504131)]
 ===
 ROW_KEY, TOTAL_SIZE, COMPACTED_SIZE, TOMBSTONED, EXPIRED, 
 REMAINNING_SSTABLE_FILES
 ===
 hello5/100.txt.1373502926003, 223, 0, YES, YES
 ---
 TOTAL, 223, 0
 ===

 This tool relies on SSTableReader and an aggregation iterator as
 Cassandra does in compaction. I was considering to share this with the
 community. So let me know if anyone is interested.

 Ah, note that it is based on 1.0.7. So I will need to check and update
 for newer versions.

 Thanks,
 Takenori


 On Thu, Jul 11, 2013 at 6:46 PM, Tomàs Núnez 
 tomas.nu...@groupalia.comwrote:

 Hi

 About a year ago, we did a major compaction in our cassandra cluster (a
 n00b mistake, I know), and since then we've had huge sstables that never
 get compacted, and we were condemned to repeat the major compaction process
 every once in a while (we are using SizeTieredCompaction strategy, and
 we've not avaluated yet LeveledCompaction, because it has its downsides,
 and we've had no time to test all of them in our environment).

 I was trying to find a way to solve this situation (that is, do
 something like a major compaction that writes small sstables, not huge as
 major compaction does), and I couldn't find it in the documentation. I
 tried cleanup and scrub/upgradesstables, but they don't do that (as
 documentation states). Then I tried deleting all data in a node and then
 bootstrapping it (or nodetool rebuild-ing it), hoping that this way the
 sstables would get cleaned from deleted records and updates. But the
 deleted node just copied the sstables from another node as they were,
 cleaning nothing.

 So I tried a new approach: I switched the sstable compaction strategy
 (SizeTiered to Leveled), forcing the sstables to be rewritten from scratch,
 and then switching it back (Leveled to SizeTiered). It took a while (but so
 do the major compaction process) and it worked, I have smaller sstables,