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

[IngestExternalFile] Allow ingesting overlapping files #5539

Closed

Conversation

igorcanadi
Copy link
Collaborator

Currently IngestExternalFile() fails when its input files' ranges overlap. This condition doesn't need to hold for files that are to be ingested in L0, though.

This commit allows overlapping files and forces their target level to L0.

Additionally, ingest job's completion is logged to EventLogger, analogous to flush and compaction jobs.

Copy link
Contributor

@riversand963 riversand963 left a comment

Choose a reason for hiding this comment

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

Thanks @igorcanadi for the PR. The change itself mostly LGTM except two questions.
On the one hand, allowing multiple files to be ingested to L0 can potentially cause write stalls to occur faster due to the number of L0 increasing. Can you advise on how to properly use the new interface?
On the other hand, if the files have overlapping keys, will it be possible to ingest them one-by-one? Or you want them to have the same seqno?

@@ -250,6 +255,13 @@ void ExternalSstFileIngestionJob::UpdateStats() {
uint64_t total_keys = 0;
uint64_t total_l0_files = 0;
uint64_t total_time = env_->NowMicros() - job_start_time_;

auto stream = event_logger_->Log();
Copy link
Contributor

Choose a reason for hiding this comment

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

Maybe specify the type of stream explicitly? We prefer to use auto when the type is easily known from the context.

@riversand963
Copy link
Contributor

The Travis test ExternalSSTFileTest.AddList also fails.

@igorcanadi
Copy link
Collaborator Author

On the one hand, allowing multiple files to be ingested to L0 can potentially cause write stalls to occur faster due to the number of L0 increasing. Can you advise on how to properly use the new interface?

In my case I'm using the interface in a bulk load scenario, where compactions are disabled until the very end, when we execute one giant compaction. I agree it is tricky to ingest files to level 0, but I don't think that's the reason to not allow it.

On the other hand, if the files have overlapping keys, will it be possible to ingest them one-by-one? Or you want them to have the same seqno?

It would be possible to ingest them one-by-one. However, I'm using this in the context of RocksDB-Cloud (https://github.com/rockset/rocksdb-cloud), where each write to MANIFEST requires one write to cloud storage (S3). As such, adding N files one-by-one adds significant single-threaded latency and slows down my ingest. In my case I do not care about the seqno.

The Travis test ExternalSSTFileTest.AddList also fails.

Will take a look.

@igorcanadi igorcanadi force-pushed the allow_overlapping_ingest branch from 05df5e6 to 243cef3 Compare July 23, 2019 23:21
@igorcanadi
Copy link
Collaborator Author

@riversand963 I fixed the tests and addressed your comment.

@riversand963
Copy link
Contributor

Hi @igorcanadi . I chatted with @siying offline. We are concerned about one issue.
If we allow ingestion of overlapping key ranges to level0, two internal keys in two SSTs may be exactly the same. This is due to current implementation of file ingestion. It is preferable to make different versions of the same user key have different seqno in order to minimize confusion. I understand that your use case is not interested in seqno, but are you willing to handle this in your PR?

@igorcanadi
Copy link
Collaborator Author

@riversand963 That makes sense. I have to admit I'm not familiar with how we assign seqnos to ingested files -- Would you mind shortly explaining what would be the best way to address the issue?

@petermattis
Copy link
Contributor

Hi @igorcanadi . I chatted with @siying offline. We are concerned about one issue.
If we allow ingestion of overlapping key ranges to level0, two internal keys in two SSTs may be exactly the same. This is due to current implementation of file ingestion. It is preferable to make different versions of the same user key have different seqno in order to minimize confusion. I understand that your use case is not interested in seqno, but are you willing to handle this in your PR?

Right now there is a single sequence number assigned to all tables, but it wouldn't be difficult to give each ingested sstable a different global sequence number.

@siying
Copy link
Contributor

siying commented Jul 26, 2019

@igorcanadi igorcanadi force-pushed the allow_overlapping_ingest branch from 243cef3 to 947fef2 Compare July 29, 2019 23:12
@igorcanadi
Copy link
Collaborator Author

@siying @riversand963 I updated the commit to assign different sequence numbers to different files when the ranges overlap.

@ajkr
Copy link
Contributor

ajkr commented Aug 12, 2019

Will this be merged? We found that even ingesting non-overlapping files to L0 in a single IngestExternalFile() causes an assertion failure (test case is https://gist.github.com/ajkr/2874c8348ff4731cd10a3bea505d720d). One of the invariants on L0 files is the seqnum of a newer L0 file has to be strictly greater than the seqnums of an older L0 file. So if multiple files are ingested to L0 with the same seqnum, the invariant is violated. I'm sure there are other ways to fix this besides this PR (like relaxing the invariant), but this PR could also solve the problem.

@ajkr
Copy link
Contributor

ajkr commented Sep 3, 2019

I think this is a good change that fixes two issues (atomically ingesting multiple overlapping files to a DB and atomically ingesting multiple non-overlapping files to L0). Is anybody available to review?

@siying
Copy link
Contributor

siying commented Sep 10, 2019

@riversand963 do you remember the status to this PR?

@riversand963
Copy link
Contributor

@igorcanadi sorry for the delay in code review. Would you mind rebasing this PR to latest master? Overall LGTM. Will take another look and land if everything looks good.

Currently IngestExternalFile() fails when its input files' ranges overlap. This condition doesn't need to hold for files that are to be ingested in L0, though.

This commit allows overlapping files and forces their target level to L0.

Additionally, ingest job's completion is logged to EventLogger, analogous to flush and compaction jobs.
@igorcanadi igorcanadi force-pushed the allow_overlapping_ingest branch from 947fef2 to 6cf74b0 Compare September 13, 2019 12:53
@igorcanadi
Copy link
Collaborator Author

@riversand963 Rebased.

Copy link
Contributor

@facebook-github-bot facebook-github-bot left a comment

Choose a reason for hiding this comment

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

@riversand963 has imported this pull request. If you are a Facebook employee, you can view this diff on Phabricator.

@facebook-github-bot
Copy link
Contributor

@igorcanadi has updated the pull request. Re-import the pull request

Copy link
Contributor

@riversand963 riversand963 left a comment

Choose a reason for hiding this comment

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

LGTM. I also adjusted format and made two small changes.
Thanks @igorcanadi for the contribution!

Copy link
Contributor

@facebook-github-bot facebook-github-bot left a comment

Choose a reason for hiding this comment

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

@riversand963 has imported this pull request. If you are a Facebook employee, you can view this diff on Phabricator.

@facebook-github-bot
Copy link
Contributor

@riversand963 merged this pull request in 9763135.

ajkr added a commit to ajkr/rocksdb that referenced this pull request Sep 26, 2019
This is necessary to revert
cockroachdb/cockroach#39562 to support RocksDB
built without `-DNDEBUG`. Consecutive files ingested to L0 may be
assigned the same seqnum, and this PR updates the LSM validation to
allow that.

There is a longer-term fix in
facebook#5539. We should take that
later, maybe after the 19.2 release.

Test Plan: ran some tests that previously failed with corruption error
like `ENABLE_ROCKSDB_ASSERTIONS=1 make
PKG=github.com/cockroachdb/cockroach/pkg/ccl/backupccl testrace`;
verified they succeed now.
ajkr added a commit to ajkr/rocksdb that referenced this pull request Sep 27, 2019
This is necessary to revert
cockroachdb/cockroach#39562 to support RocksDB
built without `-DNDEBUG`. Consecutive files ingested to L0 may be
assigned the same seqnum, and this PR updates the LSM validation to
allow that.

There is a longer-term fix in
facebook#5539. We should take that
later, maybe after the 19.2 release.

Test Plan: ran some tests that previously failed with corruption error
like `ENABLE_ROCKSDB_ASSERTIONS=1 make
PKG=github.com/cockroachdb/cockroach/pkg/ccl/backupccl testrace`;
verified they succeed now.
ajkr added a commit to cockroachdb/rocksdb that referenced this pull request Sep 27, 2019
This is necessary to revert
cockroachdb/cockroach#39562 to support RocksDB
built without `-DNDEBUG`. Consecutive files ingested to L0 may be
assigned the same seqnum, and this PR updates the LSM validation to
allow that.

There is a longer-term fix in
facebook#5539. We should take that
later, maybe after the 19.2 release.

Test Plan: ran some tests that previously failed with corruption error
like `ENABLE_ROCKSDB_ASSERTIONS=1 make
PKG=github.com/cockroachdb/cockroach/pkg/ccl/backupccl testrace`;
verified they succeed now.
merryChris pushed a commit to merryChris/rocksdb that referenced this pull request Nov 18, 2019
Summary:
Currently IngestExternalFile() fails when its input files' ranges overlap. This condition doesn't need to hold for files that are to be ingested in L0, though.

This commit allows overlapping files and forces their target level to L0.

Additionally, ingest job's completion is logged to EventLogger, analogous to flush and compaction jobs.
Pull Request resolved: facebook#5539

Differential Revision: D17370660

Pulled By: riversand963

fbshipit-source-id: 749a3899b17d1be267a5afd5b0a99d96b38ab2f3
assert(!!consumed_seqno_count ==
!!ingestion_jobs[i].ConsumedSequenceNumbersCount());
consumed_seqno_count +=
ingestion_jobs[i].ConsumedSequenceNumbersCount();
Copy link

@fredzqm fredzqm Mar 24, 2021

Choose a reason for hiding this comment

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

This line makes debug build and production build different. Is this a bug?

        consumed_seqno_count +=
            ingestion_jobs[i].ConsumedSequenceNumbersCount();

We managed to hit the assertion here in tests due to some but not all sstable file overlap with db.

        assert(!!consumed_seqno_count ==
               !!ingestion_jobs[i].ConsumedSequenceNumbersCount());

Copy link
Contributor

Choose a reason for hiding this comment

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

What is the !! supposed to do? That is a new operator to me...

Copy link

Choose a reason for hiding this comment

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

I think !! force converts a integer to a boolean. 0 => 0, others => 1

Copy link

Choose a reason for hiding this comment

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

I also posted the question here: https://groups.google.com/g/rocksdb/c/g5LACzCltWI
Thanks in advance!

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.

8 participants