Re: how to increase compaction rate?

2012-03-13 Thread Thorsten von Eicken
On 3/12/2012 6:52 AM, Brandon Williams wrote:
 On Mon, Mar 12, 2012 at 4:44 AM, aaron morton aa...@thelastpickle.com wrote:
 I don't understand why I
 don't get multiple concurrent compactions running, that's what would
 make the biggest performance difference.

 concurrent_compactors
 Controls how many concurrent compactions to run, by default it's the number
 of cores on the machine.
I'm on a quad-core machine so not setting concurrent_compactors should
not be a limiting factor...
 With leveled compaction, I don't think you get any concurrency because
 it has to compact an entire level, and it can't proceed to the next
 level without completing the one before it.

 In short, if you want maximum throughput, stick with size tiered.
I switched the CFs to tiered compaction and I still get no concurrency
for the same CF. I now have two compactions running concurrently but
always for different CFs. I've briefly seen a third for one of the small
CFs, so it's willing to run more than two concurrently. Looks like I
have to wait for a few days for all the compactions to complete. Talk
about compaction hell!


 -Brandon



Re: how to increase compaction rate?

2012-03-13 Thread Viktor Jevdokimov
After loosing one node we had to repair, CFs was on leveled compaction.
For one CF each node had about 7GB of data.
Running a repair without primary range switch ended up some nodes exhausted
to about 60-100GB of 5MB sstables for that CF (a lot of files).
After switching back from leveled to tiered we ended up completely blocked
compactions on all nodes since this CF were compacting forever.
On one node a major compaction for that CF is CPU bound and may run with
unlimited compaction speed for 4-7 days at maximum 1MB/s rate, finally
compacting to 3GB of data (some data is deleted by TTL, some merged).

What we did to speedup this process to return all exhausted nodes into
normal state faster:
We have created a 6 temporary virtual single Cassandra nodes with 2 CPU
cores and 8GB RAM.
Stopped completely a compaction for CF on a production node.
Leveled sstables from this production node was divided into 6 ranges and
copied into 6 temporary empty nodes.
On each node we ran a major compaction to compact just 1/6 of data, about
10-14GB. It took 1-2 hours to compact them into 1GB of data.
Then all 6 sstables was copied into one of 6 nodes for a major compaction,
finally getting expected 3GB sstable.
Stopping production node, deleting files that was copied, returning
compacted (may need renaming) and node is back to normal.

Using separate nodes we saved original production nodes time not to compact
exhausted CF forever, blocking compactions for other CFs. With 6 separate
nodes we have compacted 2 productions nodes a day, so maybe it took the
same time, but production nodes were free for regular compactions for other
CFs.

After back to normal for our use case we stick to tiered compaction with a
major compaction nightly.
With our insertion/TTL deletion rates a leveled compaction is a nightmare,
even if amount of data is not very huge, just a few GBs/node.

2012/3/13 Thorsten von Eicken t...@rightscale.com

 On 3/12/2012 6:52 AM, Brandon Williams wrote:
  On Mon, Mar 12, 2012 at 4:44 AM, aaron morton aa...@thelastpickle.com
 wrote:
  I don't understand why I
  don't get multiple concurrent compactions running, that's what would
  make the biggest performance difference.
 
  concurrent_compactors
  Controls how many concurrent compactions to run, by default it's the
 number
  of cores on the machine.
 I'm on a quad-core machine so not setting concurrent_compactors should
 not be a limiting factor...
  With leveled compaction, I don't think you get any concurrency because
  it has to compact an entire level, and it can't proceed to the next
  level without completing the one before it.
 
  In short, if you want maximum throughput, stick with size tiered.
 I switched the CFs to tiered compaction and I still get no concurrency
 for the same CF. I now have two compactions running concurrently but
 always for different CFs. I've briefly seen a third for one of the small
 CFs, so it's willing to run more than two concurrently. Looks like I
 have to wait for a few days for all the compactions to complete. Talk
 about compaction hell!

 
  -Brandon
 



Re: how to increase compaction rate?

2012-03-13 Thread Thorsten von Eicken
On 3/13/2012 4:13 PM, Viktor Jevdokimov wrote:
 What we did to speedup this process to return all exhausted nodes into
 normal state faster:
 We have created a 6 temporary virtual single Cassandra nodes with 2
 CPU cores and 8GB RAM.
 Stopped completely a compaction for CF on a production node.
 Leveled sstables from this production node was divided into 6 ranges
 and copied into 6 temporary empty nodes.
 On each node we ran a major compaction to compact just 1/6 of data,
 about 10-14GB. It took 1-2 hours to compact them into 1GB of data.
 Then all 6 sstables was copied into one of 6 nodes for a major
 compaction, finally getting expected 3GB sstable.
 Stopping production node, deleting files that was copied, returning
 compacted (may need renaming) and node is back to normal.

 Using separate nodes we saved original production nodes time not to
 compact exhausted CF forever, blocking compactions for other CFs. With
 6 separate nodes we have compacted 2 productions nodes a day, so maybe
 it took the same time, but production nodes were free for regular
 compactions for other CFs.
Yikes, that's quite the ordeal, but I totally get why you had to go
there. Cassandra seems to work well within some use-case bounds and
lacks the sophistication to handle others well. I've been wondering
about the way I use it, which is to hold the last N days of logs and
corresponding index. This means that every day I make a zillion inserts
and a corresponding zillion of deletes for the data inserted N days ago.
The way the compaction works this is horrible. The data is essentially
immutable until it's deleted, yet it's copied a whole bunch of times. In
addition, it takes forever for the deletion tombstones to meet the
original data in a compaction and actually compact it away. I've also
run into the zillions of files problem with level compaction you did. I
ended up with over 30k SSTables for ~1TB of data. At that point the
compaction just ceases to make progress. And starting cassandra takes
30 minutes just for it to open all the SSTables and when done 12GB of
memory are used. Better algorithms and some tools will be needed for all
this to just work. But then, we're also just at V1.0.8...
TvE


Re: how to increase compaction rate?

2012-03-13 Thread Edward Capriolo
On Tue, Mar 13, 2012 at 11:32 PM, Thorsten von Eicken
t...@rightscale.com wrote:
 On 3/13/2012 4:13 PM, Viktor Jevdokimov wrote:
 What we did to speedup this process to return all exhausted nodes into
 normal state faster:
 We have created a 6 temporary virtual single Cassandra nodes with 2
 CPU cores and 8GB RAM.
 Stopped completely a compaction for CF on a production node.
 Leveled sstables from this production node was divided into 6 ranges
 and copied into 6 temporary empty nodes.
 On each node we ran a major compaction to compact just 1/6 of data,
 about 10-14GB. It took 1-2 hours to compact them into 1GB of data.
 Then all 6 sstables was copied into one of 6 nodes for a major
 compaction, finally getting expected 3GB sstable.
 Stopping production node, deleting files that was copied, returning
 compacted (may need renaming) and node is back to normal.

 Using separate nodes we saved original production nodes time not to
 compact exhausted CF forever, blocking compactions for other CFs. With
 6 separate nodes we have compacted 2 productions nodes a day, so maybe
 it took the same time, but production nodes were free for regular
 compactions for other CFs.
 Yikes, that's quite the ordeal, but I totally get why you had to go
 there. Cassandra seems to work well within some use-case bounds and
 lacks the sophistication to handle others well. I've been wondering
 about the way I use it, which is to hold the last N days of logs and
 corresponding index. This means that every day I make a zillion inserts
 and a corresponding zillion of deletes for the data inserted N days ago.
 The way the compaction works this is horrible. The data is essentially
 immutable until it's deleted, yet it's copied a whole bunch of times. In
 addition, it takes forever for the deletion tombstones to meet the
 original data in a compaction and actually compact it away. I've also
 run into the zillions of files problem with level compaction you did. I
 ended up with over 30k SSTables for ~1TB of data. At that point the
 compaction just ceases to make progress. And starting cassandra takes
30 minutes just for it to open all the SSTables and when done 12GB of
 memory are used. Better algorithms and some tools will be needed for all
 this to just work. But then, we're also just at V1.0.8...
    TvE

You are correct to say that the way Cassandra works it is not idea for
a dataset where you completely delete and re add the entire dataset
each day. In fact that may be one of the worst use cases for
Cassandra. this has to do with the structured log format and with the
tombstones and grace period. Maybe you can set a lower base.

LevelDB is new and not as common in the wild as the Sized Tiered.
Again it works the way it works. Google must think it is brilliant
after all they invented it.

For a 1TB of data your 12GB is used by bloom filters. Again this is
just a fact of life. Bloom filters are their to make negative lookups
faster. Maybe you can lower the bloom filter sizes and the index
interval. This should use less memory and help the system start up
faster respectively.

But nodes stuffed with a trillion keys may not be optimal for many
reasons. In out case we want a high portion of the data set in memory.
So a 1TB node might need say 256 GB ram :) We opt for more smaller
boxes.


Re: how to increase compaction rate?

2012-03-12 Thread Thorsten von Eicken
On 3/11/2012 9:17 PM, Peter Schuller wrote:
 multithreaded_compaction: false
 Set to true.
I did try that. I didn't see it go any faster. The cpu load was lower,
which I assumed meant fewer bytes/sec being compressed
(SnappyCompressor). I didn't see multiple compactions in parallel.
Nodetool compactionstats behaved strange and instead of showing
individual compactions with a %-complete it showed a running count of
total bytes compacted. (Darn, I don't have the output of that anymore in
my terminal buffer.) It just didn't look good to me. Are you positive
that it is faster with leveled compaction? I don't understand why I
don't get multiple concurrent compactions running, that's what would
make the biggest performance difference. Is the compaction parallelism
perhaps only across multiple CFs? That would explain what I see.

Thorsten


Re: how to increase compaction rate?

2012-03-12 Thread aaron morton
 I don't understand why I
 don't get multiple concurrent compactions running, that's what would
 make the biggest performance difference.
concurrent_compactors
Controls how many concurrent compactions to run, by default it's the number of 
cores on the machine. 

If you are not CPU bound check iostats 
(http://spyced.blogspot.co.nz/2010/01/linux-performance-basics.html_) 

Cheers
-
Aaron Morton
Freelance Developer
@aaronmorton
http://www.thelastpickle.com

On 12/03/2012, at 7:21 PM, Thorsten von Eicken wrote:

 On 3/11/2012 9:17 PM, Peter Schuller wrote:
 multithreaded_compaction: false
 Set to true.
 I did try that. I didn't see it go any faster. The cpu load was lower,
 which I assumed meant fewer bytes/sec being compressed
 (SnappyCompressor). I didn't see multiple compactions in parallel.
 Nodetool compactionstats behaved strange and instead of showing
 individual compactions with a %-complete it showed a running count of
 total bytes compacted. (Darn, I don't have the output of that anymore in
 my terminal buffer.) It just didn't look good to me. Are you positive
 that it is faster with leveled compaction? I don't understand why I
 don't get multiple concurrent compactions running, that's what would
 make the biggest performance difference. Is the compaction parallelism
 perhaps only across multiple CFs? That would explain what I see.
 
 Thorsten



Re: how to increase compaction rate?

2012-03-12 Thread Brandon Williams
On Mon, Mar 12, 2012 at 4:44 AM, aaron morton aa...@thelastpickle.com wrote:
 I don't understand why I
 don't get multiple concurrent compactions running, that's what would
 make the biggest performance difference.

 concurrent_compactors
 Controls how many concurrent compactions to run, by default it's the number
 of cores on the machine.

With leveled compaction, I don't think you get any concurrency because
it has to compact an entire level, and it can't proceed to the next
level without completing the one before it.

In short, if you want maximum throughput, stick with size tiered.

-Brandon


how to increase compaction rate?

2012-03-11 Thread Thorsten von Eicken
I'm having difficulties with leveled compaction, it's not making fast
enough progress. I'm on a quad-core box and it only does one compaction
at a time. Cassandra version: 1.0.6. Here's nodetool compaction stats:

# nodetool -h localhost compactionstats
pending tasks: 2568
  compaction typekeyspace   column family bytes
compacted bytes total  progress
   Compactionrslog_productionreq_text
4974195   314597326 1.58%

The number of pending tasks decreases extremely slowly. In the log, I
can see it perform 3-4 compactions but the number of tasks only
decreases by one. (I turned clients off and even disabled thrift to
ensure this is not because of writes happening at the same time.) In the
log, I see nicely paired Compacting... Compacted... lines after each
other, it doesn't look like there's ever more than one compaction
running at a time. I have 3 cpus sitting idle. My cassandra.yaml has:

snapshot_before_compaction: false
column_index_size_in_kb: 128
in_memory_compaction_limit_in_mb: 64
multithreaded_compaction: false
compaction_throughput_mb_per_sec: 16
compaction_preheat_key_cache: true

I've issued a nodetool -h localhost setcompactionthroughput 100, which
didn't seem to make a difference. Here are some sample log lines:

 INFO [CompactionExecutor:117] 2012-03-12 02:54:43,963
CompactionTask.java (line 113)
Compacting
[SSTableReader(path='/mnt/ebs/data/rslog_production/req_text-hc-753342-Data.db')]
 INFO [CompactionExecutor:117] 2012-03-12 02:54:47,793
CompactionTask.java (line 218)
Compacted to
[/mnt/ebs/data/rslog_production/req_text-hc-753992-Data.db,]. 
30,198,523 to 30,197,052 (~99% of original) bytes for 39,269 keys at
7.519100MB/s.  Time: 3,830ms.
 INFO [CompactionExecutor:119] 2012-03-12 02:54:47,795
CompactionTask.java (line 113)
Compacting
[SSTableReader(path='/mnt/ebs/data/rslog_production/req_text-hc-753933-Data.db')]
 INFO [CompactionExecutor:119] 2012-03-12 02:54:51,731
CompactionTask.java (line 218)
Compacted to
[/mnt/ebs/data/rslog_production/req_text-hc-753994-Data.db,]. 
31,462,495 to 31,462,495 (~100% of original) bytes for 40,267 keys at
7.625152MB/s.  Time: 3,935ms.
 INFO [CompactionExecutor:119] 2012-03-12 02:54:51,734
CompactionTask.java (line 113)
Compacting
[SSTableReader(path='/mnt/ebs/data/rslog_production/req_text-hc-753343-Data.db')]
 INFO [CompactionExecutor:119] 2012-03-12 02:54:56,093
CompactionTask.java (line 218)
Compacted to
[/mnt/ebs/data/rslog_production/req_text-hc-753996-Data.db,]. 
32,643,675 to 32,643,958 (~100% of original) bytes for 57,473 keys at
7.141937MB/s.  Time: 4,359ms.
 INFO [CompactionExecutor:118] 2012-03-12 02:54:56,095
CompactionTask.java (line 113)
Compacting
[SSTableReader(path='/mnt/ebs/data/rslog_production/req_text-hc-753934-Data.db')]
 INFO [CompactionExecutor:118] 2012-03-12 02:54:59,635
CompactionTask.java (line 218)
Compacted to
[/mnt/ebs/data/rslog_production/req_text-hc-753998-Data.db,]. 
30,709,285 to 30,709,285 (~100% of original) bytes for 32,172 keys at
8.275404MB/s.  Time: 3,539ms.
 INFO [CompactionExecutor:118] 2012-03-12 02:54:59,638
CompactionTask.java (line 113)
Compacting
[SSTableReader(path='/mnt/ebs/data/rslog_production/req_text-hc-753344-Data.db')]

I recently added a second node to the ring and, in what I suspect is
related, I can't get it to have data transferred (RF=1). Somewhere I
read that the compaction executor does the data streaming? I'm wondering
whether those tasks are all queued. nodetool ring:

#nodetool -h localhost ring
Address DC  RackStatus State   Load   
OwnsToken
  
85070591730234615865843651857942052865
10.102.37.168   datacenter1 rack1   Up Normal  811.87 GB  
50.00%  0
10.80.161.101   datacenter1 rack1   Up Normal  1.08 MB
50.00%  85070591730234615865843651857942052865

Last thing I attempted here is to move the empty node from ...864 to ...865:

# nodetool -h localhost move 85070591730234615865843651857942052865
 INFO 19:59:19,625 Moving /10.80.161.101 from
85070591730234615865843651857942052864 to
85070591730234615865843651857942052865.
 INFO 19:59:19,628 Sleeping 3 ms before start streaming/fetching ranges.
 INFO 19:59:49,639 MOVING: fetching new ranges and streaming old ranges
 INFO 19:59:52,680 Finished streaming session 97489049918693 from
/10.102.37.168
 INFO 19:59:52,681 Enqueuing flush of
Memtable-LocationInfo@227137515(36/45 serialized/live bytes, 1 ops)
 INFO 19:59:52,682 Writing Memtable-LocationInfo@227137515(36/45
serialized/live bytes, 1 ops)
 INFO 19:59:52,706 Completed flushing
/mnt/ebs/data/system/LocationInfo-hc-19-Data.db (87 bytes)
 INFO 19:59:52,708 Node /10.80.161.101 state jump to normal

I'm pretty stumped at this point... Any pointers to what to do or what I
may have done wrong?
Thanks!
Thorsten



Re: how to increase compaction rate?

2012-03-11 Thread Peter Schuller
 multithreaded_compaction: false

Set to true.

-- 
/ Peter Schuller (@scode, http://worldmodscode.wordpress.com)