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

Staging compaction 1st draft #7442

Merged
merged 2 commits into from
Feb 22, 2024
Merged

Conversation

itaiad200
Copy link
Contributor

Proposal for #2092

Copy link

github-actions bot commented Feb 6, 2024

No linked issues found. Please add the corresponding issues in the pull request description.
Use GitHub automation to close the issue when a PR is merged

@itaiad200 itaiad200 added the exclude-changelog PR description should not be included in next release changelog label Feb 6, 2024
Copy link

github-actions bot commented Feb 6, 2024

E2E Test Results - DynamoDB Local - Local Block Adapter

10 passed

Copy link
Contributor

@arielshaqed arielshaqed left a comment

Choose a reason for hiding this comment

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

This is beautiful and fits in very nicely with the existing code in Graveler. Yes, please!

We might add a section explicitly stating the advantages by counting the operations needed with and without compaction.

There might be other cases where the structure of the `Branch` entity impacts
the performance of reading from the staging area, for example when the
number of sealed tokens is large (N)[^1] and reading a missing entry requires
reading from all N sealed tokens.
Copy link
Contributor

Choose a reason for hiding this comment

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

Also I'm putting together a separate proposal for partitioning the staging area. It is much simpler and more efficient to partition only forwards in time and not change anything already written. So we will want to compact staging as a way to reduce read load on already-written parts, by removing KV partitions entirely!

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'll love to read that proposal. I'm having hard time understanding how compacting partitions work 😅

based on the number of sealed tokens, although not necessarily a priority
for the first version. We can probably avoid the need to query the kv-store
to retrieve that information (except for the service startups) by caching
the information in the Sensor.
Copy link
Contributor

Choose a reason for hiding this comment

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

It is hard to cache the number of deletions locally in a cluster of lakeFSes. One option might be for each lakeFS instance to select a unique ID and store its deletions count on KV some key. Writes are immediately uncontended, and write load can be reduced by batching the increment for a short while.

Comment on lines +23 to +26
There might be other cases where the structure of the `Branch` entity impacts
the performance of reading from the staging area, for example when the
number of sealed tokens is large (N)[^1] and reading a missing entry requires
reading from all N sealed tokens.
Copy link
Contributor

Choose a reason for hiding this comment

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

As far as I know multiple sealed tokens case is rare.
Is there an open open performance issue where we found more than one token that cause performance issue?

Copy link
Contributor

Choose a reason for hiding this comment

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

Nope. Well, I did see this when experimenting with very many concurrent merges. But there are other reasons why those are slow. Still, once I get to split staging tokens this might become more common.
Definitely something for later, agree with @itaiad200 that initially it least we should trigger this due to multiple deletions -- it will definitely help there immediately.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It's indeed rare as every sealed token represents a fail operation on the branch, so I don't think it has a big impact.

- Preserve lakeFS consistency guarantees for all scenarios.
- A more efficient way to read from branches with a large number of
tombstones (i.e. Fix the issue mentioned above).
- Preserve lakeFS performance for other scenarios.[^2]
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 that "other scenarios" can cause also be accessing the blockstore. Iterate over metarange can cause slowdown while we requrire to download blocks (unless they are cached). The currently iterator uses next for both uncommitted and committed so I assume we can say that we have the same tombstone issue while we scan over committed data if the staging/uncommtited entries are full of changes too and we "pay" in performance while scan data that we will served but still scan 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.

I don't think it's true. The CompactedMetaRange will not contain just the uncommitted entries. It will contain a snapshot of the branch at the time of the compaction, i.e. as if commit ran at the same time. Meaning, tombstones will not be part of the CompactedMetaRange as we do not store deletions in a commit. So we can't have the same tombstone issue.
However, you're pointing on a different potential problem. We're replacing load from the kv-store with load on the object store.

Comment on lines 155 to 160
### Commit Flow

The commit flow is slightly affected by the compaction process. If
compaction never happened, the commit flow is the same as today. If a
compaction happened, apply the changes to the compacted metarange instead of
the HEAD commit metarange.
Copy link
Contributor

Choose a reason for hiding this comment

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

Guess we will need to address "empty commit" differently in the case of compaction metarange.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Why are you calling out empty commit specifically? The view of the staging remains the same. You look at staging token + sealed tokens + compaction (if available) and see if they present any changes to the commit.

Copy link
Contributor

Choose a reason for hiding this comment

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

Wait... we are overloading the word "commit" here and I think that it leads to some confusion. AFAICT we just need to set AllowEmpty Let me see if I understand how compaction works. I think we are talking about a scenario like this one: I add many (100_000) objects staging, and delete them without committing.

Now I have a staging area with 100_000 tombstones1. A few seconds later I want to compact. This is not a whole commit! We generate a new metarange using committed.Commit -- this is below Graveler. We told it to AllowEmpty. It runs over the previous base metarange (branch.CompactedMetaRange or, if we don't have, the metarange of branch.CommitID and applies all the tombstones. Nothing happens, so it generates the exact same ranges and metaranges as before. When we close the MetaRangeWriter we get the exact same metarange ID. It only deletes the staging token and the sealed tokens, the base metarange ID stays.

So AFAICS this is the same flow with no changes. The "no empty commits" rule just means we need to set AllowEmpty here.

Footnotes

  1. Reminder: we cannot delete the key from the staging area, because the original uncommitted write might have overwritten a previous object. We never scan backwards, and doing so on deletions would make them expensive.

Copy link
Contributor

@arielshaqed arielshaqed left a comment

Choose a reason for hiding this comment

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

Just replies to @nopcoder , I think this design is a small, compact, and useful change that will help resolve a current issue users have.

Comment on lines +23 to +26
There might be other cases where the structure of the `Branch` entity impacts
the performance of reading from the staging area, for example when the
number of sealed tokens is large (N)[^1] and reading a missing entry requires
reading from all N sealed tokens.
Copy link
Contributor

Choose a reason for hiding this comment

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

Nope. Well, I did see this when experimenting with very many concurrent merges. But there are other reasons why those are slow. Still, once I get to split staging tokens this might become more common.
Definitely something for later, agree with @itaiad200 that initially it least we should trigger this due to multiple deletions -- it will definitely help there immediately.

Comment on lines 155 to 160
### Commit Flow

The commit flow is slightly affected by the compaction process. If
compaction never happened, the commit flow is the same as today. If a
compaction happened, apply the changes to the compacted metarange instead of
the HEAD commit metarange.
Copy link
Contributor

Choose a reason for hiding this comment

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

Wait... we are overloading the word "commit" here and I think that it leads to some confusion. AFAICT we just need to set AllowEmpty Let me see if I understand how compaction works. I think we are talking about a scenario like this one: I add many (100_000) objects staging, and delete them without committing.

Now I have a staging area with 100_000 tombstones1. A few seconds later I want to compact. This is not a whole commit! We generate a new metarange using committed.Commit -- this is below Graveler. We told it to AllowEmpty. It runs over the previous base metarange (branch.CompactedMetaRange or, if we don't have, the metarange of branch.CommitID and applies all the tombstones. Nothing happens, so it generates the exact same ranges and metaranges as before. When we close the MetaRangeWriter we get the exact same metarange ID. It only deletes the staging token and the sealed tokens, the base metarange ID stays.

So AFAICS this is the same flow with no changes. The "no empty commits" rule just means we need to set AllowEmpty here.

Footnotes

  1. Reminder: we cannot delete the key from the staging area, because the original uncommitted write might have overwritten a previous object. We never scan backwards, and doing so on deletions would make them expensive.

@ozkatz
Copy link
Collaborator

ozkatz commented Feb 18, 2024

Return of the (terribly named) partial commit!

Not to be confused, of course, with partial commit.

Copy link
Contributor

@guy-har guy-har left a comment

Choose a reason for hiding this comment

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

Loved the design, very clear and convincing.
I commented only on three areas where I'm missing some additional information
Thanks!!!

Comment on lines +66 to +67
information on writes to the staging area. It will decide when to compact a
certain branch based on the number of deleted entries to its staging area,
Copy link
Contributor

Choose a reason for hiding this comment

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

Just a thought according to "based on the ..."
Do we have any statistics on cases that do bulk delete? IIRC there are spark jobs that do a bunch of deletes. working by a fix number of deleted entries may result in many compaction during a bulk delete.
Another example I can think of is moving a directory (that exists on staging), doing "commits" along the way may result in commits that each of them is "heavier than the only commit that would've been done otherwise.

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 don't have any statistics.

Comment on lines +81 to +85
### Compactor

Upon deciding to compact, the Sensor will trigger the Compactor. The
Compactor will perform an operation very similar to Commit. Before starting
to compact, it will atomically:
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 the "garbage", this compactor may create many "dangling" ranges. Do have any estimation to how this would effect or if it might be an 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.

It will increase the number of dangling commits significantly, let's say by an order of a magnitude? We'll need to handle that at some point in some GC process.

Comment on lines +141 to +144
1. Read from the staging token.
2. Read from the sealed tokens (in order).
3. If a CompactedMetaRangeID exists, read the 2-way diff between the compacted
metarange and the CommitID's metarange.
Copy link
Contributor

Choose a reason for hiding this comment

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

There still is a step I think I'm missing, what do I do with the data from the sealed (and staging) tokens and the data from the diff. There is some kind of "uncommitted merge" that we need to do.

Copy link
Contributor

Choose a reason for hiding this comment

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

+1 on that

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 already exists today. Assume the following simple branch scenario today:

StagingToken: {A:{foo1:barA}}
SealedTokens: [{B:{foo1:barB,foo2:gooB}} , {C:{foo1:barC,foo2:gooC}}]
Commit: [{foo1:barCommitted,foo2:gooCommitted,,foo3:jooCommitted}]

Reading foo1,foo2,foo3 from this branch yields barA,gooB,jooCommitted. For reading from the uncommitted area, you just skip the last layer with reading the commit.
With compaction the same algo holds, but you need to diff the CompactedMetaRange with the commit for the changes that are already manifested in the compaction

number of sealed tokens is large (N)[^1] and reading a missing entry requires
reading from all N sealed tokens.

## Goals
Copy link
Contributor

Choose a reason for hiding this comment

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

I don't see any reference to visibility into this internal feature.
To me this is parallel to having a GC, I wanna know if it worked, when it runs, how much time did it take etc.
If you agree can you add a small section about the visibility requirements?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

As a user, why do you want to know if compaction worked? And how effective was it? You only care about it when performance degrades and you need to drill down further. I guess most users will never hear of that.
GC is different. It's an external process I need to manage by myself (or pay someone to do it) and it results in storage costs.
We should have metrics for it just like with any other flow in Graveler.

Copy link
Contributor

Choose a reason for hiding this comment

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

First, sorry for not being clear - I meant GC in JVM not lakeFS.
And this is super internal but, we must have some visibility into the performance of compaction isolated from the rest.
The reason is because the sensor would evolve over time into different directions with heuristics / tradeoffs / configuration.
Having visibility is important.

I think that this should be mentioned as part of the design, otherwise this can easily be missed and "shipped later" or done with an after thought.

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 prometheus metrics are more than enough for the sophisticated lakeFS user to tweak what needs to be tweaked. What new metrics would you like me to call out in this design?

Copy link
Contributor

Choose a reason for hiding this comment

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

compaction successful runs number, compaction failures number or total compaction runs.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

So a single event compaction_run {success:bool}

Comment on lines +155 to +159
### Commit Flow

The commit flow is slightly affected by the compaction process. If
compaction never happened, the commit flow is the same as today. If a
compaction happened, apply the changes to the compacted metarange instead of
Copy link
Contributor

Choose a reason for hiding this comment

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

What if a compaction happened during a commit? will this fail the commit? If that is the case, are we fine with compaction failing commits?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Updated.

@itaiad200 itaiad200 added the minor-change Used for PRs that don't require issue attached label Feb 22, 2024
@itaiad200 itaiad200 merged commit 3fdd98d into master Feb 22, 2024
36 of 37 checks passed
@itaiad200 itaiad200 deleted the 2092-staging-compaction-proposal branch February 22, 2024 11:33
@guy-har guy-har mentioned this pull request Feb 22, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area/KV Improvements to the KV store implementation exclude-changelog PR description should not be included in next release changelog improvement minor-change Used for PRs that don't require issue attached proposal
Projects
None yet
Development

Successfully merging this pull request may close these issues.

6 participants