You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by clohfink <gi...@git.apache.org> on 2018/08/18 06:19:15 UTC

[GitHub] cassandra pull request #252: Reduce heap pressure during compactions for CAS...

GitHub user clohfink opened a pull request:

    https://github.com/apache/cassandra/pull/252

    Reduce heap pressure during compactions for CASSANDRA-14654

    

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/clohfink/cassandra compaction_allocs

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/cassandra/pull/252.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #252
    
----
commit 38e79ff150bde4f2333f507dd31d892fc400b30a
Author: Chris Lohfink <cl...@...>
Date:   2018-08-18T02:35:17Z

    Reduce heap pressure during compactions for CASSANDRA-14654

----


---

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


[GitHub] cassandra pull request #252: Reduce heap pressure during compactions for CAS...

Posted by krummas <gi...@git.apache.org>.
Github user krummas commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/252#discussion_r224459241
  
    --- Diff: src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java ---
    @@ -131,16 +132,20 @@ public RowIndexEntry append(UnfilteredRowIterator partition)
         {
             // we do this before appending to ensure we can resetAndTruncate() safely if the append fails
             DecoratedKey key = partition.partitionKey();
    -        maybeReopenEarly(key);
    +        if (preemptiveOpenInterval != Long.MAX_VALUE)
    --- End diff --
    
    this is checked in `maybeReopenEarly` - no need to check here as well


---

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


[GitHub] cassandra pull request #252: Reduce heap pressure during compactions for CAS...

Posted by krummas <gi...@git.apache.org>.
Github user krummas commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/252#discussion_r224695370
  
    --- Diff: src/java/org/apache/cassandra/config/Config.java ---
    @@ -238,6 +238,7 @@
         public int hints_flush_period_in_ms = 10000;
         public int max_hints_file_size_in_mb = 128;
         public ParameterizedClass hints_compression;
    +    public boolean invalidate_cache_on_compaction = true;
    --- End diff --
    
    this should be volatile for hotprops


---

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


[GitHub] cassandra pull request #252: Reduce heap pressure during compactions for CAS...

Posted by krummas <gi...@git.apache.org>.
Github user krummas commented on a diff in the pull request:

    https://github.com/apache/cassandra/pull/252#discussion_r224696746
  
    --- Diff: src/java/org/apache/cassandra/config/Config.java ---
    @@ -238,6 +238,7 @@
         public int hints_flush_period_in_ms = 10000;
         public int max_hints_file_size_in_mb = 128;
         public ParameterizedClass hints_compression;
    +    public boolean invalidate_cache_on_compaction = true;
    --- End diff --
    
    The biggest performance impact of setting this to `false` is probably not the fact that we don't invalidate the cache anymore, but that we don't warm the cache up for the new sstables - maybe we should rename the property? (warm_keycache_on_compaction or migrate_keycache_on_compaction?) Or at least add a comment about it? And it should probably say "keycache" instead of "cache"


---

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