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

feat(storage): enable static compaction group #3453

Merged
merged 21 commits into from
Jul 4, 2022

Conversation

zwang28
Copy link
Contributor

@zwang28 zwang28 commented Jun 24, 2022

I hereby agree to the terms of the Singularity Data, Inc. Contributor License Agreement.

What's changed and what's your intention?

  • Enables e2e static compaction group feature. There're 2 groups for
    • materialized view and materialized source.
    • internal state and source.
  • Assert table ids given to commit_epoch all have been registered to compaction group.

Checklist

  • I have written necessary docs and comments
  • I have added necessary unit tests and integration tests
  • All checks passed in ./risedev check (or alias, ./risedev c)

Refer to a related PR or issue link (optional)

#2065

@zwang28 zwang28 force-pushed the wangzheng/enable_compaction_group branch from 1176775 to 93a7d7b Compare June 24, 2022 08:18
@zwang28 zwang28 force-pushed the wangzheng/enable_compaction_group branch from 93a7d7b to 90d191c Compare June 24, 2022 08:33
@codecov
Copy link

codecov bot commented Jun 24, 2022

Codecov Report

Merging #3453 (bac0835) into main (86c2105) will increase coverage by 0.01%.
The diff coverage is 74.08%.

@@            Coverage Diff             @@
##             main    #3453      +/-   ##
==========================================
+ Coverage   74.29%   74.31%   +0.01%     
==========================================
  Files         773      773              
  Lines      109551   109868     +317     
==========================================
+ Hits        81391    81646     +255     
- Misses      28160    28222      +62     
Flag Coverage Δ
rust 74.31% <74.08%> (+0.01%) ⬆️

Flags with carried forward coverage won't be shown. Click here to find out more.

Impacted Files Coverage Δ
src/bench/ss_bench/main.rs 0.65% <0.00%> (ø)
src/compute/src/rpc/service/stream_service.rs 0.00% <0.00%> (ø)
src/compute/src/server.rs 0.00% <0.00%> (ø)
src/ctl/src/cmd_impl/hummock/sst_dump.rs 0.00% <0.00%> (ø)
src/meta/src/hummock/error.rs 2.56% <ø> (ø)
...rc/storage/hummock_sdk/src/compaction_group/mod.rs 100.00% <ø> (+34.61%) ⬆️
src/storage/src/hummock/test_utils.rs 83.91% <0.00%> (-1.80%) ⬇️
src/storage/src/monitor/monitored_store.rs 1.74% <0.00%> (-0.08%) ⬇️
src/storage/src/store_impl.rs 7.54% <0.00%> (-0.46%) ⬇️
src/stream/src/task/stream_manager.rs 0.00% <0.00%> (ø)
... and 35 more

📣 Codecov can now indicate which changes are the most critical in Pull Requests. Learn more

/// Tries to get from local cache
pub async fn try_get_compaction_group_id(&self, prefix: Prefix) -> Option<CompactionGroupId> {
async fn try_get_compaction_group_id(&self, prefix: Prefix) -> Option<CompactionGroupId> {
Copy link
Contributor

Choose a reason for hiding this comment

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

Do not async RwLock because any RPC request will block other read request.

Copy link
Contributor

Choose a reason for hiding this comment

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

suggest parking_lot::RwLock to reduce lock wait.

Copy link
Contributor

Choose a reason for hiding this comment

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

        let mut guard = self.inner.read();
        if let Some(compaction_group_id) = guard.get(&prefix) {
            return Ok(Some(compaction_group_id));
        }
       drop(guard);
        let compaction_groups = self
            .hummock_meta_client
            .get_compaction_groups()
            .await
            .map_err(HummockError::meta_error)?;
                let mut guard = self.inner.write();
        guard.set_index(compaction_groups);
        Ok(guard.get(&prefix))

Copy link
Contributor Author

@zwang28 zwang28 Jun 25, 2022

Choose a reason for hiding this comment

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

Then we also need to avoid duplicate RPCs. It's doable.

I used the current approach because I think cache update is rare, which is only required when new table is created.

Copy link
Contributor

Choose a reason for hiding this comment

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

need to avoid duplicate RPCs

in this case get_compaction_groups is a read operation , why we need to avoid duplicate RPCs (to reduce read rpc ?)

Copy link
Contributor

Choose a reason for hiding this comment

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

No. This RPC is light and cost little CPU of meta-service. So we do not need to avoid duplicate RPCs

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

…uld not be unregistered from compaction group.
@zwang28 zwang28 marked this pull request as ready for review June 26, 2022 09:26
@zwang28 zwang28 marked this pull request as draft June 27, 2022 04:53
# Conflicts:
#	src/meta/src/stream/source_manager.rs
#	src/storage/src/hummock/compactor_tests.rs
@zwang28 zwang28 marked this pull request as ready for review June 27, 2022 08:52
@zwang28 zwang28 requested a review from hzxa21 June 27, 2022 08:52
# Conflicts:
#	src/storage/src/hummock/state_store.rs
#[async_trait::async_trait]
impl CompactionGroupClient for CompactionGroupClientImpl {
/// Tries to get from local cache first,then from meta service
async fn get_compaction_group_id(
Copy link
Collaborator

Choose a reason for hiding this comment

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

If shared buffer is already split by compaction group, I have a feeling that we will need to know more than just a compaction group id, for example, the compaction group configuration.

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 so, hope that CompactionGroupClient can fetch more info when rpc, (link configuration) , in order to support ttl and so on

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 complete compaction group info is already given to CompactionGroupClient, but CompactionGroupClient only store part of it for now.
I think you can modify it as need later @Li0k .

Copy link
Collaborator

@hzxa21 hzxa21 left a comment

Choose a reason for hiding this comment

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

All my comments are concerns on future PRs and not related to "static" compaction group. You can merge this PR first if you want.

@zwang28
Copy link
Contributor Author

zwang28 commented Jun 28, 2022

@Little-Wallace suggests to update CN's table->group mapping in a more timely manner, rather than current lazy manner. It makes sense to me. For example, we can update this mapping in CN when CN is process conf change barrier.

Also there is an unstable failure parallel e2e test in main. Would only merge this PR after it has been fixed.

@zwang28
Copy link
Contributor Author

zwang28 commented Jun 28, 2022

For example, we can update this mapping in CN when CN is process conf change barrier.

After some attempts, I now suggest to keep the lazy update approach for now. Because to implement aforementioned more timely approach:
- We need update mapping in LocalStreamManager::send_barrier. It is not async, while update RPC requires async.
- Alternatively it's possible to piggyback the mapping in barrier payload, thus async update RPC is avoided. But I prefer not to complicate barrier.

@zwang28
Copy link
Contributor Author

zwang28 commented Jun 28, 2022

The rpc deduplication implementation is flawed. Will fix before merge.
We update cache in the beginning of build_actors. Because it can avoid RPC duplication better.

@zwang28 zwang28 force-pushed the wangzheng/enable_compaction_group branch from 040f551 to cab82cb Compare June 30, 2022 03:46
# Conflicts:
#	src/meta/src/hummock/hummock_manager.rs
#	src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs
@zwang28 zwang28 force-pushed the wangzheng/enable_compaction_group branch from c0d5fbd to 1859a3d Compare June 30, 2022 08:58
@zwang28
Copy link
Contributor Author

zwang28 commented Jun 30, 2022

A summary of changes since last review @Li0k @Little-Wallace @hzxa21 :

  • CN's compaction group cache is updated in the beginning of build_actors each time, rather than lazily updated when queried.
  • Remove prefix concept in compaction group. Use table id directly.
  • CompactionGroupClient is quite simple now.

Copy link
Contributor

@Li0k Li0k left a comment

Choose a reason for hiding this comment

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

Thanks for the PR. LGTM!

# Conflicts:
#	src/ctl/src/cmd_impl/hummock/sst_dump.rs
#	src/meta/src/hummock/compaction_group/manager.rs
#	src/meta/src/hummock/compaction_group/mod.rs
#	src/storage/src/hummock/compactor_tests.rs
#	src/storage/src/hummock/state_store.rs
self.compaction_group_manager
.register_table_fragments(&table_fragments, &table_properties)
.await?;

// In the second stage, each [`WorkerNode`] builds local actors and connect them with
// channels.
for (node_id, actors) in node_actors {
Copy link
Collaborator

Choose a reason for hiding this comment

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

Do we need to unregister_table_fragments if BuildActorsRequest fails?

Copy link
Contributor Author

@zwang28 zwang28 Jul 1, 2022

Choose a reason for hiding this comment

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

purge_stale_members ensures stale data is eventually cleaned up on each restart.
We can unregister_table_fragments/unregister_source explicitly to clean up more timely. I didn't have it here, in order to avoid introducing more error handling code here.
Let me add it.

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

@zwang28 zwang28 merged commit dc165bb into main Jul 4, 2022
@zwang28 zwang28 deleted the wangzheng/enable_compaction_group branch July 4, 2022 04:24
nasnoisaac pushed a commit to nasnoisaac/risingwave that referenced this pull request Aug 9, 2022
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

4 participants