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

STAR-1872: Parallelize UCS compactions per output shard #1342

Open
wants to merge 12 commits into
base: main
Choose a base branch
from

Conversation

blambov
Copy link

@blambov blambov commented Oct 9, 2024

This splits compactions that are to produce more than one
output sstable into tasks that can execute in parallel.
Such tasks share a transaction and have combined progress
and observer. Because we cannot mark parts of an sstable
as unneeded, the transaction is only applied when all
tasks have succeeded. This also means that early open
is not supported for such tasks.

At this time the new parallelization mechanism is not taken
into account by the thread allocation scheme, and thus
some levels may take more resources than they should.
Because of this limitation (which should be fixed in the
near future), the new behaviour is off by default.

Also:

  • Adds a flag to combine non-overlapping sets in major
    compactions to reshard data, as major compactions can
    can now be executed as a parallelized operation.

  • Changes SSTable expiration to be done in a separate
    getNextBackgroundCompactions round to improve the
    efficiency of expiration (separate task can run quickly
    and remove the relevant sstables without waiting for
    a compaction to end).

  • Applies small-partition-count correction in
    ShardManager.calculateCombinedDensity.

partCommittedOrAborted();
}

private void partCommittedOrAborted()

Choose a reason for hiding this comment

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

what about passing here the partial transaction as a parameter ?
instead of having only a number partsToCommitOrAbort we could have a reference to all the child transactions and ensure that we don't count the same transaction twice and when we commit all of the children are in the expected state

Copy link
Author

Choose a reason for hiding this comment

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

You can't count the same transaction twice because of the protections in PartialLifecycleTransaction.

There could be some value in knowing which part did not complete, but because we don't have timeouts on these things (and actually can't, as compactions can last days) there's no obvious place to surface that information.

}
else
{
final CompressionMetadata compressionMetadata = getCompressionMetadata();

Choose a reason for hiding this comment

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

are we exercising this branch in the unit tests ?

Copy link
Author

Choose a reason for hiding this comment

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

There's a list of TODOs in UnifiedCompactionStrategy.createAndAddTasks where this is next in line.

return tasks;
}

private <T> List<T> splitSSTablesInShards(Collection<SSTableReader> sstables,

Choose a reason for hiding this comment

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

what about making this method static and writing specific unit tests to cover all of the cases?

@blambov
Copy link
Author

blambov commented Oct 16, 2024

The PR is not yet ready for review.

@blambov blambov force-pushed the STAR-1872 branch 2 times, most recently from 9512132 to 6cc862f Compare October 29, 2024 12:31
This splits compactions that are to produce more than one
output sstable into tasks that can execute in parallel.
Such tasks share a transaction and have combined progress
and observer. Because we cannot mark parts of an sstable
as unneeded, the transaction is only applied when all
tasks have succeeded. This also means that early open
is not supported for such tasks.

At this time the new parallelization mechanism is not taken
into account by the thread allocation scheme, and thus
some levels may take more resources than they should.
Because of this limitation (which should be fixed in the
near future), the new behaviour is off by default.

Also:
- Adds a flag to combine non-overlapping sets in major
  compactions to reshard data, as major compactions can
  can now be executed as a parallelized operation.

- Changes SSTable expiration to be done in a separate
  getNextBackgroundCompactions round to improve the
  efficiency of expiration (separate task can run quickly
  and remove the relevant sstables without waiting for
  a compaction to end).

- Applies small-partition-count correction in
  ShardManager.calculateCombinedDensity.
@blambov
Copy link
Author

blambov commented Oct 29, 2024

The patch is now ready for review.

Copy link

sonarcloud bot commented Nov 7, 2024

@cassci-bot
Copy link

❌ Build ds-cassandra-pr-gate/PR-1342 rejected by Butler


8 new test failure(s) in 16 builds
See build details here


Found 8 new test failures

Test Explanation Branch history Upstream history
...,147,483,647 Modifier 1 Levels 3 Compactors 30] regression 🔴🔵
...positePartitionKeyDataModel{primaryKey=p1, p2}] regression 🔴🔴🔴🔴🔴🔴🔵 🔵🔵🔵🔵🔵🔵🔵
...positePartitionKeyDataModel{primaryKey=p1, p2}] failing 🔴🔴🔴🔴🔴 🔵🔵🔵🔵🔵🔵🔵
...positePartitionKeyDataModel{primaryKey=p1, p2}] regression 🔴🔵🔵🔴🔵🔴🔴 🔵🔵🔵🔵🔵🔵🔵
...positePartitionKeyDataModel{primaryKey=p1, p2}] failing 🔴🔴🔴🔴🔴🔴🔴 🔵🔵🔵🔵🔵🔵🔵
...i.s.c.VectorSiftSmallTest.testMultiSegmentBuild failing 🔴🔴🔴🔴🔴🔴🔴 🔵🔵🔵🔵🔵🔵🔵
...t,wide=false,scenario=POST_BUILD_QUERY] regression 🔴🔵🔵🔵 🔵🔵🔵🔵🔵🔵🔵
...i.s.d.v.VectorCompressionTest.testOpenAiV3Small flaky 🔵🔴🔵🔵 🔵🔵🔵🔵🔵🔵🔵

Found 100 known test failures

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