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

[persist] Thread chunks of data from consolidation to the batch builder #29577

Merged
merged 9 commits into from
Sep 19, 2024

Conversation

bkirwi
Copy link
Contributor

@bkirwi bkirwi commented Sep 16, 2024

  • Pass around structured data in the batch builder, instead of just the flat columnar records.
  • Allow passing batches of data into the batch builder (which unlike individual updates can contain structured data).
  • Estimate the size of individual rows in compaction and use that to influence the generated batch size.

The upshot is that it should now be possible to go from input to output in compaction without converting from codec-encoded data to structured data or vice-versa; the process can work in terms of structured data from top to bottom.

Motivation

Subtask for MaterializeInc/database-issues#7411.

Tips for reviewer

Chunking up compaction output is a little tricky, and I will probably want to make some changes in a followup. It's all behind a flag for now of course!

Checklist

  • This PR has adequate test coverage / QA involvement has been duly considered. (trigger-ci for additional test/nightly runs)
  • This PR has an associated up-to-date design doc, is a design doc (template), or is sufficiently small to not require a design.
  • If this PR evolves an existing $T ⇔ Proto$T mapping (possibly in a backwards-incompatible way), then it is tagged with a T-proto label.
  • If this PR will require changes to cloud orchestration or tests, there is a companion cloud PR to account for those changes that is tagged with the release-blocker label (example).
  • If this PR includes major user-facing behavior changes, I have pinged the relevant PM to schedule a changelog post.

@bkirwi bkirwi force-pushed the batch-many branch 2 times, most recently from 0ec24a0 to 1045da9 Compare September 17, 2024 16:04
@bkirwi bkirwi changed the title [wip] [persist] Pass sets of updates to the batch builder [wip] [persist] Thread chunks of data from consolidation to the batch builder Sep 17, 2024
@bkirwi bkirwi changed the title [wip] [persist] Thread chunks of data from consolidation to the batch builder [persist] Thread chunks of data from consolidation to the batch builder Sep 17, 2024
@bkirwi bkirwi marked this pull request as ready for review September 17, 2024 21:52
@bkirwi bkirwi requested a review from a team as a code owner September 17, 2024 21:52
@bkirwi
Copy link
Contributor Author

bkirwi commented Sep 18, 2024

In case interesting, here is my thinking for the batch building estimates:

  • It estimates arrow-size and not parquet-size, because arrow is easier to estimate, and because in the cases where they're radically different parquet is likely to be smaller thanks to things like compression.
  • It estimates size based on ArrowOrd because it was straightforward to implement and ought to produce results similar to the old method.
  • It makes compaction responsible for sizing parts instead of the batch builder - compaction needs to chunk up data anyways, and making it try and generate the right size of chunks up front avoids copies.

The remaining issue is that consolidation may in some circumstances produce chunks that are too small (if. eg. we don't have enough data downloaded yet). This is the sort of tuning I plan to tackle in a followup. The latest version of the PR will concat multiple chunks together to get one of the right size. I think normally we'll have just one chunk and not need to do any extra work, but I've added a metric so I can monitor how much copying is happening to see if it's worth tuning consolidation to output larger batches.

Copy link
Member

@ParkMyCar ParkMyCar left a comment

Choose a reason for hiding this comment

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

Overall LGTM! Just a couple of thoughts

src/persist-client/src/batch.rs Outdated Show resolved Hide resolved
src/persist-client/src/internal/compact.rs Show resolved Hide resolved
src/persist-client/src/iter.rs Show resolved Hide resolved
src/persist-client/src/iter.rs Show resolved Hide resolved
Comment on lines 705 to 707
// Keep a running estimate of the size left in the budget, returning None after it's
// exhausted. Note that we can't use take_while here... it's important that we pass
// on every value we receive from the iterator.
Copy link
Member

Choose a reason for hiding this comment

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

I might misunderstsand what this comment means, but I'm pretty sure we stop returning elements once the budget == 0? Here's an example from the Rust Playground

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, that's correct! The idea of the second half of this comment is that it would be bad for us to pull an element from the consolidating iter and not foward it along, so we can't use take_while. (Which drops the first non-matching element.) I'll rephrase.

src/persist/src/indexed/encoding.rs Show resolved Hide resolved
// This is a noop if there are no such updates.
// TODO: consider moving the individual updates to BatchBuilder?
let previous = self.buffer.drain();
self.flush_part(previous).await;
Copy link
Member

Choose a reason for hiding this comment

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

What's the rationale for always flushing the previous parts? It seems like it might be nice to flush the current updates with the previous into a single blob?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

What's the rationale for always flushing the previous parts?

Simplest thing to do, basically... we only hit this case if someone was mixing the two types of calls, which never happens in practice.

It seems like it might be nice to flush the current updates with the previous into a single blob?

There's some risk that it outputs a blob that's larger than our target size. (So we could make it conditional, but for the above reason I'm not inclined to put much effort in!.)

src/persist-client/src/batch.rs Outdated Show resolved Hide resolved
src/persist-types/src/arrow.rs Outdated Show resolved Hide resolved
src/persist-types/src/arrow.rs Outdated Show resolved Hide resolved
src/persist-client/src/internal/compact.rs Show resolved Hide resolved
Copy link

shepherdlybot bot commented Sep 19, 2024

Risk Score:82 / 100 Bug Hotspots:0 Resilience Coverage:66%

Mitigations

Completing required mitigations increases Resilience Coverage.

  • (Required) Code Review 🔍 Detected
  • (Required) Feature Flag
  • (Required) Integration Test
  • (Required) Observability
  • (Required) QA Review
  • (Required) Run Nightly Tests
  • Unit Test
Risk Summary:

This pull request carries a high risk score of 82, driven by predictors such as the average age of files, cognitive complexity within files, and the delta of executable lines. Historically, PRs with these predictors are 102% more likely to cause a bug than the repository baseline. Notably, the observed bug trend in the repository is decreasing.

Note: The risk score is not based on semantic analysis but on historical predictors of bug occurrence in the repository. The attributes above were deemed the strongest predictors based on that history. Predictors and the score may change as the PR evolves in code, time, and review activity.

@bkirwi
Copy link
Contributor Author

bkirwi commented Sep 19, 2024

Thanks all! Think I've addressed all the blockers, but if anything doesn't look right I'm happy to take it in a followup.

This is a vestige of back when we had two sets of schemas in compaction,
which was weird and is long gone. Enforce that all the data in a batch
has the same schema.
Following the similar method on columnar records.
If this is more than a small faction of bytes, it's worth tuning
consolidation to return larger chunks of output.
@bkirwi bkirwi requested a review from a team as a code owner September 19, 2024 18:34
@bkirwi
Copy link
Contributor Author

bkirwi commented Sep 19, 2024

Ran into a test failure after rebasing on main, and I think I've figured it out.

The test_builtin_connection_alterations_are_preserved_across_restarts test is failing with a SIGSEGV, apparently a stack overflow. This is extra likely to be a stack overflow because @def- just found and fixed it in bf8dcd9.

However, the fix was reverted by @jkosh44 in #29593. I don't see a rationale in the PR desc - Joe, was that intentional and/or would the test change to unrevert?

I've re-applied Dennis's fix on this branch, and it does seem to get the test passing again.

@jkosh44
Copy link
Contributor

jkosh44 commented Sep 19, 2024

However, the fix was reverted by @jkosh44 in #29593. I don't see a rationale in the PR desc - Joe, was that intentional and/or would the test change to unrevert?

I think I understand what happened.

  1. The PR Revert "catalog: Combine epoch and deploy generation" #29433 did two things, revert fa2c417 and add new commit bf8dcd9.
  2. I reverted Revert "catalog: Combine epoch and deploy generation" #29433, which un-reverted fa2c417 but reverted bf8dcd9.

So, no, it was not intentional. Please unrevert bf8dcd9.

@bkirwi bkirwi merged commit 0cebb4a into MaterializeInc:main Sep 19, 2024
84 checks passed
@bkirwi
Copy link
Contributor Author

bkirwi commented Sep 19, 2024

Ah, that'll do it! Thanks for chasing that down.

@github-actions github-actions bot locked and limited conversation to collaborators Sep 19, 2024
@def-
Copy link
Contributor

def- commented Sep 19, 2024

Sorry about that! I had to mix that in to get cargo test green

@bkirwi
Copy link
Contributor Author

bkirwi commented Sep 19, 2024

No need to apologize -- far from it! I was in fact very happy to find out that someone had already figured out a fix for the weird test failure I was hitting...

Sign up for free to subscribe to this conversation on GitHub. Already have an account? Sign in.
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants