Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

CASSANDRA-18397: Unified Compaction Strategy #2287

Closed
wants to merge 27 commits into from

Conversation

blambov
Copy link
Contributor

@blambov blambov commented Apr 20, 2023

Implementation of the Unified Compaction Strategy as described in CEP-26.

The approach is documented in the included UnifiedCompactionStrategy.md.

The patch is best reviewed by individual commits, which split the preparation and enhancements into smaller chunks. The main contribution is in 0715a66.

Copy link
Contributor

@Maxwell-Guo Maxwell-Guo left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

complete the first round

private static final Pattern SCALING_PARAMETER_PATTERN = Pattern.compile("(N)|L(\\d+)|T(\\d+)|([+-]?\\d+)");
private static final String SCALING_PARAMETER_PATTERN_SIMPLIFIED = SCALING_PARAMETER_PATTERN.pattern()
.replaceAll("[()]", "")

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it seems there is an empty line.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed

@@ -516,6 +517,11 @@ public enum CassandraRelevantProperties
TRIGGERS_DIR("cassandra.triggers_dir"),
TRUNCATE_BALLOT_METADATA("cassandra.truncate_ballot_metadata"),
TYPE_UDT_CONFLICT_BEHAVIOR("cassandra.type.udt.conflict_behavior"),
UCS_BASE_SHARD_COUNT("unified_compaction.base_shard_count", "4"),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

do we need to add some descriptions for these UCS configuration?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added a reference to Controller, where they are defined and described.

* Gets the value of a system property as a double.
* @return system property long value if it exists, defaultValue otherwise.
*/
public double getLong(double overrideDefaultValue)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

do we need to change the function name ? the description is get the value as a double but the funtion name is getLong ? this is confusing. May be change to getDouble(double overrideDefaultValue)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, changed.

/**
* Default compaction configuration, used if a table does not specify any.
*/
public ParameterizedClass default_compaction = null;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  1. I think the parameter name should be changed to default_compaction_for_table,or something else.
  2. we should add some detailed description in yaml(But I do not saw any information change in cassandra.yaml in this patch), because we may modify the user's default behavior, which needs to be perceived by the user. In the old way if a use create a table without specify the compaction strategy , the STCS is used, but now the strategy may changed if this paramter has been set to other compaction strategy.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added an entry in cassandra.yaml.

The flag specifies a default value for the compaction parameter. Could you elaborate on why it should be renamed?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The flag specifies a default value for the compaction parameter. Could you elaborate on why it should be renamed?

Actually, it feels like there's no need to change it

*/
default double rangeSpanned(SSTableReader rdr)
{
double reported = rdr.tokenSpaceCoverage();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what about change to
double span = (reported > 0) ? reported : rangeSpanned(rdr.getFirst(), rdr.getLast());

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The full form is slightly easier to read.

@@ -0,0 +1,336 @@
/*
* Copyright DataStax, Inc.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove this

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

@@ -0,0 +1,913 @@
/*
* Copyright DataStax, Inc.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove this

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

List<SSTableReader> allSstables = new ArrayList<>();

List<SSTableReader> sstables = mockSSTables(numSSTables,
// minSstableSizeBytes,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think it is better to remove this line instand of comment out this line

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed

@@ -0,0 +1,295 @@
/*
* Copyright DataStax, Inc.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove this line

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

@@ -0,0 +1,144 @@
/*
* Copyright DataStax, Inc.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove this line

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

Copy link
Contributor

@Gerrrr Gerrrr left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

First pass. I hope to do another pass later this week with more meaningful comments.

while (sum.add(currentRangeWidth).compareTo(perPart) >= 0)
{
BigInteger withinRangeBoundary = perPart.subtract(sum);
left = left.add(withinRangeBoundary);
left = left.add(withinRangeBoundary.multiply(currentRangeFactor));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does this deserve a test that demonstrates the issue?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This came up as a result of a failure of ShardManagerTest.testShardBoundariesWeighted.

Added a more direct test in SplitterTest.testWithWeight.

Collection<Index> indexes,
LifecycleNewTracker lifecycleNewTracker)
{
// FIXME: needs the metadata collector fix
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does this comment refer to 2558dfb?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, comment removed.

Level level = new Level(controller, index, 0, maxSize);
for (SSTableReader candidate : suitable)
{
final double size = shardManager.density(candidate);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: we should probably rename local variables such as maxSize and size to maxDensity and density to avoid confusion.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done


/**
* Groups the sstables passed in into arenas and buckets. This is used by the strategy to determine
* new compactions, and by external tools in CNDB to analyze the strategy decisions.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: javadoc references CNDB

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does CNDB is some thing that developed by datastax ? I have saw this keyword in one of the patchs of sai

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

CNDB is in internal name for DataStax's Astra DB. Removed reference.

final int threshold; // number of SSTables that trigger a compaction
final double min; // min density of sstables for this level
final double max; // max density of sstables for this level
int maxOverlap = -1; // maximum number of overlapping sstables
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: Can you please add a line on why one would want to limit the overlap? What happens if a level/bucket crosses this threshold? Do we not run a compaction in that level? How do we recover?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added a line here and an explanation of the process in getCompactionPick as well as getBuckets.

* shards used by {@link org.apache.cassandra.db.compaction.UnifiedCompactionStrategy} as long as the size of
* the sstable so far is sufficiently large.
* <p/>
* This is class is similar to {@link ShardedMultiWriter} but for flushing. Unfortunately
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Similar to some other multi-writer?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed

@Gerrrr
Copy link
Contributor

Gerrrr commented Jun 26, 2023

Q: does this patch require any changes or additional test cases in dtests?

@Maxwell-Guo
Copy link
Contributor

Maxwell-Guo commented Jun 27, 2023

I think there is no need for a dtest,because compaction is a single node behavior.

But Is it necessary to add a tests that we alter schema from stcs/lcs/twcs to UCS ? Will the alter behavior result meet expectations?

@blambov
Copy link
Contributor Author

blambov commented Jun 28, 2023

There is an in-JVM dtest (UnifiedCompactionDensitiesTest) for the part that can be affected by local ownership.
Another reason I would use a dtest is to check behaviour on shutting down and restarting, but in this case the strategy is stateless and should not need it.

@blambov
Copy link
Contributor Author

blambov commented Jul 18, 2023

Pushed a fix for one flaky test and added a dtest pull request to correct a test for the more precise printout in bytes. The remaining repeated test failures of the corrupted compaction test are due to not treating some kinds of data corruption as such. The fix for this is somewhat invasive and needs to be done in a separate ticket (the problem is not introduced by this patch).

@Maxwell-Guo, do you need any further changes? If not, could you approve the PR?

@Maxwell-Guo
Copy link
Contributor

Hi @blambov I want to do the last round of review tomorrow before +1, is that ok? I've been a little busy these days, sorry

blambov added 19 commits July 18, 2023 23:54
Introduces prettyPrintBinary (with 2^10 multiplier, e.g. MiB) and
prettyPrintDecimal (with 1000 multiplier, e.g. ms) that cover the
full range of double, and a corresponding parsing method that can
read both.
If a compaction strategy splits sstables during flush, this can cause
the cardinality information of an sstable to include
more keys than it actually has, resulting in grossly overestimated
key counts when picking a bloom filter size; the latter can result in
several times larger bloom filters on L1 than they should be.
This is needed to avoid situations where changed local token space
ownership causes SSTables to have their density calculated incorrectly.

This is only calculated and written by UCS writers; when the value is
not available in the file, it is calculated from the covered range.
and switches to UCS for the trie unit test suite
Store targetSSTableSize and apply sqrt(0.5) in calculation.
Copy link
Contributor

@Maxwell-Guo Maxwell-Guo left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

only some small nits ,+1 if final CI results are green

# Default compaction strategy, applied when a table's parameters do not
# specify compaction.
# The default is to use SizeTieredCompactionStrategy, with its default
# threshold of 4 sstables.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The default strategy of STCS here should not only include the lower limit of 4, but also the upper limit of 32, so
what about just modify the description to :
“The default is to use SizeTieredCompactionStrategy with its defaut compaction params.”
It feels more ordinary

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

# threshold of 4 sstables.
# The selected compaction strategy will also apply to system tables.
#
# default_compaction:
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As this is a newly introduced configuration, so I think we should give a more detailed description abouth the usage, like :
class_name should be the different compaction strategy name , parameters sholud be the maps of differen parameters that different compaction strategy used. Then the next is an example

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I started changing this to:

# Default compaction strategy, applied when a table's parameters do not
# specify compaction. A compaction "class_name" must be given, and optionally
# its class-specific configuration parameters under "parameters" as in the
# example below.
# The selected compaction strategy will also apply to system tables.
#
# The default is to use SizeTieredCompactionStrategy, with its default
# compaction parameters.

but then realized this is out of line with other such items in the YAML, which give the configuration by example only. E.g. commitlog_compression, hints_compression are described similarly and go through the same transformation (class among other parameters in the schema to class_name and parameters fields in the YAML).

I'm leaning towards keeping it short and in line with other items, but I'm happy to change it if you prefer.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok,then I think keep same with "commitlog_compression" or "hints_compression" is enough.

@@ -0,0 +1,375 @@
<!--
#
# Copyright DataStax, Inc.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have missed this before ~~~ line 1 to line 3
I think we should modify these lines.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shoot, I thought I changed all these. Fixed now.

I did another search for "DataStax" and nothing license-related appeared.

for (SSTableWriter writer : writers)
if (writer != null)
return writer.getFilename();
return "";
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should file name be "" if all writer is null ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think so. This is used for logging, and reporting no filename with 0B written is sensible.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok

# threshold of 4 sstables.
# The selected compaction strategy will also apply to system tables.
#
# default_compaction:
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok,then I think keep same with "commitlog_compression" or "hints_compression" is enough.

@blambov
Copy link
Contributor Author

blambov commented Jul 20, 2023

CI j11: Repeated tests failing due to CASSANDRA-18676 plus another known flake.
CI j8: Repeated tests failing due to CASSANDRA-18676, plus testPagingWithClustering-.jdk11 and a flake of randomSplitTestVNodesMurmur3Partitioner-compression.jdk11.

Taking another look if the latter two can be caused by this patch.

@blambov
Copy link
Contributor Author

blambov commented Jul 20, 2023

randomSplitTestVNodesMurmur3Partitioner-compression.jdk11 uses only a weight of 1.0 and thus executes exactly the same code as before (as BigInteger.multiply(new BigInteger(1)) boils down to copying).
testPagingWithClustering-.jdk11 didn't fail in previous runs and in this case it's not running with UCS. I can't think of a way that it could be related.

I'm going to commit the patch and continue with CASSANDRA-18676.

@blambov
Copy link
Contributor Author

blambov commented Jul 20, 2023

Committed as 957eca2.

@blambov blambov closed this Jul 20, 2023
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants