2016-02-10 00:12:00 +01:00
|
|
|
// Copyright (c) 2011-present, Facebook, Inc. All rights reserved.
|
2017-07-16 01:03:42 +02:00
|
|
|
// This source code is licensed under both the GPLv2 (found in the
|
|
|
|
// COPYING file in the root directory) and Apache 2.0 License
|
|
|
|
// (found in the LICENSE.Apache file in the root directory).
|
RangeSync not to sync last 1MB of the file
Summary:
From other ones' investigation:
"sync_file_range() behavior highly depends on kernel version and filesystem.
xfs does neighbor page flushing outside of the specified ranges. For example, sync_file_range(fd, 8192, 16384) does not only trigger flushing page #3 to #4, but also flushing many more dirty pages (i.e. up to page#16)... Ranges of the sync_file_range() should be far enough from write() offset (at least 1MB)."
Test Plan: make all check
Reviewers: igor, rven, kradhakrishnan, yhchiang, IslamAbdelRahman, anthony
Reviewed By: anthony
Subscribers: yoshinorim, MarkCallaghan, sumeet, domas, dhruba, leveldb, ljin
Differential Revision: https://reviews.facebook.net/D15807
2015-07-20 23:46:15 +02:00
|
|
|
//
|
2017-02-18 20:54:49 +01:00
|
|
|
#include <algorithm>
|
|
|
|
#include <vector>
|
2021-01-29 07:08:46 +01:00
|
|
|
|
Using existing crc32c checksum in checksum handoff for Manifest and WAL (#8412)
Summary:
In PR https://github.com/facebook/rocksdb/issues/7523 , checksum handoff is introduced in RocksDB for WAL, Manifest, and SST files. When user enable checksum handoff for a certain type of file, before the data is written to the lower layer storage system, we calculate the checksum (crc32c) of each piece of data and pass the checksum down with the data, such that data verification can be down by the lower layer storage system if it has the capability. However, it cannot cover the whole lifetime of the data in the memory and also it potentially introduces extra checksum calculation overhead.
In this PR, we introduce a new interface in WritableFileWriter::Append, which allows the caller be able to pass the data and the checksum (crc32c) together. In this way, WritableFileWriter can directly use the pass-in checksum (crc32c) to generate the checksum of data being passed down to the storage system. It saves the calculation overhead and achieves higher protection coverage. When a new checksum is added with the data, we use Crc32cCombine https://github.com/facebook/rocksdb/issues/8305 to combine the existing checksum and the new checksum. To avoid the segmenting of data by rate-limiter before it is stored, rate-limiter is called enough times to accumulate enough credits for a certain write. This design only support Manifest and WAL which use log_writer in the current stage.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/8412
Test Plan: make check, add new testing cases.
Reviewed By: anand1976
Differential Revision: D29151545
Pulled By: zhichao-cao
fbshipit-source-id: 75e2278c5126cfd58393c67b1efd18dcc7a30772
2021-06-25 09:46:33 +02:00
|
|
|
#include "db/db_test_util.h"
|
2021-03-10 05:10:51 +01:00
|
|
|
#include "env/mock_env.h"
|
|
|
|
#include "file/line_file_reader.h"
|
2019-09-16 19:31:27 +02:00
|
|
|
#include "file/random_access_file_reader.h"
|
Using existing crc32c checksum in checksum handoff for Manifest and WAL (#8412)
Summary:
In PR https://github.com/facebook/rocksdb/issues/7523 , checksum handoff is introduced in RocksDB for WAL, Manifest, and SST files. When user enable checksum handoff for a certain type of file, before the data is written to the lower layer storage system, we calculate the checksum (crc32c) of each piece of data and pass the checksum down with the data, such that data verification can be down by the lower layer storage system if it has the capability. However, it cannot cover the whole lifetime of the data in the memory and also it potentially introduces extra checksum calculation overhead.
In this PR, we introduce a new interface in WritableFileWriter::Append, which allows the caller be able to pass the data and the checksum (crc32c) together. In this way, WritableFileWriter can directly use the pass-in checksum (crc32c) to generate the checksum of data being passed down to the storage system. It saves the calculation overhead and achieves higher protection coverage. When a new checksum is added with the data, we use Crc32cCombine https://github.com/facebook/rocksdb/issues/8305 to combine the existing checksum and the new checksum. To avoid the segmenting of data by rate-limiter before it is stored, rate-limiter is called enough times to accumulate enough credits for a certain write. This design only support Manifest and WAL which use log_writer in the current stage.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/8412
Test Plan: make check, add new testing cases.
Reviewed By: anand1976
Differential Revision: D29151545
Pulled By: zhichao-cao
fbshipit-source-id: 75e2278c5126cfd58393c67b1efd18dcc7a30772
2021-06-25 09:46:33 +02:00
|
|
|
#include "file/read_write_util.h"
|
2019-09-16 19:31:27 +02:00
|
|
|
#include "file/readahead_raf.h"
|
|
|
|
#include "file/sequence_file_reader.h"
|
|
|
|
#include "file/writable_file_writer.h"
|
2021-01-29 07:08:46 +01:00
|
|
|
#include "rocksdb/file_system.h"
|
2019-05-30 20:21:38 +02:00
|
|
|
#include "test_util/testharness.h"
|
|
|
|
#include "test_util/testutil.h"
|
Using existing crc32c checksum in checksum handoff for Manifest and WAL (#8412)
Summary:
In PR https://github.com/facebook/rocksdb/issues/7523 , checksum handoff is introduced in RocksDB for WAL, Manifest, and SST files. When user enable checksum handoff for a certain type of file, before the data is written to the lower layer storage system, we calculate the checksum (crc32c) of each piece of data and pass the checksum down with the data, such that data verification can be down by the lower layer storage system if it has the capability. However, it cannot cover the whole lifetime of the data in the memory and also it potentially introduces extra checksum calculation overhead.
In this PR, we introduce a new interface in WritableFileWriter::Append, which allows the caller be able to pass the data and the checksum (crc32c) together. In this way, WritableFileWriter can directly use the pass-in checksum (crc32c) to generate the checksum of data being passed down to the storage system. It saves the calculation overhead and achieves higher protection coverage. When a new checksum is added with the data, we use Crc32cCombine https://github.com/facebook/rocksdb/issues/8305 to combine the existing checksum and the new checksum. To avoid the segmenting of data by rate-limiter before it is stored, rate-limiter is called enough times to accumulate enough credits for a certain write. This design only support Manifest and WAL which use log_writer in the current stage.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/8412
Test Plan: make check, add new testing cases.
Reviewed By: anand1976
Differential Revision: D29151545
Pulled By: zhichao-cao
fbshipit-source-id: 75e2278c5126cfd58393c67b1efd18dcc7a30772
2021-06-25 09:46:33 +02:00
|
|
|
#include "util/crc32c.h"
|
2019-05-31 02:39:43 +02:00
|
|
|
#include "util/random.h"
|
Using existing crc32c checksum in checksum handoff for Manifest and WAL (#8412)
Summary:
In PR https://github.com/facebook/rocksdb/issues/7523 , checksum handoff is introduced in RocksDB for WAL, Manifest, and SST files. When user enable checksum handoff for a certain type of file, before the data is written to the lower layer storage system, we calculate the checksum (crc32c) of each piece of data and pass the checksum down with the data, such that data verification can be down by the lower layer storage system if it has the capability. However, it cannot cover the whole lifetime of the data in the memory and also it potentially introduces extra checksum calculation overhead.
In this PR, we introduce a new interface in WritableFileWriter::Append, which allows the caller be able to pass the data and the checksum (crc32c) together. In this way, WritableFileWriter can directly use the pass-in checksum (crc32c) to generate the checksum of data being passed down to the storage system. It saves the calculation overhead and achieves higher protection coverage. When a new checksum is added with the data, we use Crc32cCombine https://github.com/facebook/rocksdb/issues/8305 to combine the existing checksum and the new checksum. To avoid the segmenting of data by rate-limiter before it is stored, rate-limiter is called enough times to accumulate enough credits for a certain write. This design only support Manifest and WAL which use log_writer in the current stage.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/8412
Test Plan: make check, add new testing cases.
Reviewed By: anand1976
Differential Revision: D29151545
Pulled By: zhichao-cao
fbshipit-source-id: 75e2278c5126cfd58393c67b1efd18dcc7a30772
2021-06-25 09:46:33 +02:00
|
|
|
#include "utilities/fault_injection_fs.h"
|
RangeSync not to sync last 1MB of the file
Summary:
From other ones' investigation:
"sync_file_range() behavior highly depends on kernel version and filesystem.
xfs does neighbor page flushing outside of the specified ranges. For example, sync_file_range(fd, 8192, 16384) does not only trigger flushing page #3 to #4, but also flushing many more dirty pages (i.e. up to page#16)... Ranges of the sync_file_range() should be far enough from write() offset (at least 1MB)."
Test Plan: make all check
Reviewers: igor, rven, kradhakrishnan, yhchiang, IslamAbdelRahman, anthony
Reviewed By: anthony
Subscribers: yoshinorim, MarkCallaghan, sumeet, domas, dhruba, leveldb, ljin
Differential Revision: https://reviews.facebook.net/D15807
2015-07-20 23:46:15 +02:00
|
|
|
|
2020-02-20 21:07:53 +01:00
|
|
|
namespace ROCKSDB_NAMESPACE {
|
RangeSync not to sync last 1MB of the file
Summary:
From other ones' investigation:
"sync_file_range() behavior highly depends on kernel version and filesystem.
xfs does neighbor page flushing outside of the specified ranges. For example, sync_file_range(fd, 8192, 16384) does not only trigger flushing page #3 to #4, but also flushing many more dirty pages (i.e. up to page#16)... Ranges of the sync_file_range() should be far enough from write() offset (at least 1MB)."
Test Plan: make all check
Reviewers: igor, rven, kradhakrishnan, yhchiang, IslamAbdelRahman, anthony
Reviewed By: anthony
Subscribers: yoshinorim, MarkCallaghan, sumeet, domas, dhruba, leveldb, ljin
Differential Revision: https://reviews.facebook.net/D15807
2015-07-20 23:46:15 +02:00
|
|
|
|
|
|
|
class WritableFileWriterTest : public testing::Test {};
|
|
|
|
|
2022-01-28 04:29:17 +01:00
|
|
|
constexpr uint32_t kMb = static_cast<uint32_t>(1) << 20;
|
RangeSync not to sync last 1MB of the file
Summary:
From other ones' investigation:
"sync_file_range() behavior highly depends on kernel version and filesystem.
xfs does neighbor page flushing outside of the specified ranges. For example, sync_file_range(fd, 8192, 16384) does not only trigger flushing page #3 to #4, but also flushing many more dirty pages (i.e. up to page#16)... Ranges of the sync_file_range() should be far enough from write() offset (at least 1MB)."
Test Plan: make all check
Reviewers: igor, rven, kradhakrishnan, yhchiang, IslamAbdelRahman, anthony
Reviewed By: anthony
Subscribers: yoshinorim, MarkCallaghan, sumeet, domas, dhruba, leveldb, ljin
Differential Revision: https://reviews.facebook.net/D15807
2015-07-20 23:46:15 +02:00
|
|
|
|
|
|
|
TEST_F(WritableFileWriterTest, RangeSync) {
|
2021-01-29 07:08:46 +01:00
|
|
|
class FakeWF : public FSWritableFile {
|
RangeSync not to sync last 1MB of the file
Summary:
From other ones' investigation:
"sync_file_range() behavior highly depends on kernel version and filesystem.
xfs does neighbor page flushing outside of the specified ranges. For example, sync_file_range(fd, 8192, 16384) does not only trigger flushing page #3 to #4, but also flushing many more dirty pages (i.e. up to page#16)... Ranges of the sync_file_range() should be far enough from write() offset (at least 1MB)."
Test Plan: make all check
Reviewers: igor, rven, kradhakrishnan, yhchiang, IslamAbdelRahman, anthony
Reviewed By: anthony
Subscribers: yoshinorim, MarkCallaghan, sumeet, domas, dhruba, leveldb, ljin
Differential Revision: https://reviews.facebook.net/D15807
2015-07-20 23:46:15 +02:00
|
|
|
public:
|
|
|
|
explicit FakeWF() : size_(0), last_synced_(0) {}
|
2019-02-14 22:52:47 +01:00
|
|
|
~FakeWF() override {}
|
RangeSync not to sync last 1MB of the file
Summary:
From other ones' investigation:
"sync_file_range() behavior highly depends on kernel version and filesystem.
xfs does neighbor page flushing outside of the specified ranges. For example, sync_file_range(fd, 8192, 16384) does not only trigger flushing page #3 to #4, but also flushing many more dirty pages (i.e. up to page#16)... Ranges of the sync_file_range() should be far enough from write() offset (at least 1MB)."
Test Plan: make all check
Reviewers: igor, rven, kradhakrishnan, yhchiang, IslamAbdelRahman, anthony
Reviewed By: anthony
Subscribers: yoshinorim, MarkCallaghan, sumeet, domas, dhruba, leveldb, ljin
Differential Revision: https://reviews.facebook.net/D15807
2015-07-20 23:46:15 +02:00
|
|
|
|
2021-01-29 07:08:46 +01:00
|
|
|
using FSWritableFile::Append;
|
|
|
|
IOStatus Append(const Slice& data, const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
RangeSync not to sync last 1MB of the file
Summary:
From other ones' investigation:
"sync_file_range() behavior highly depends on kernel version and filesystem.
xfs does neighbor page flushing outside of the specified ranges. For example, sync_file_range(fd, 8192, 16384) does not only trigger flushing page #3 to #4, but also flushing many more dirty pages (i.e. up to page#16)... Ranges of the sync_file_range() should be far enough from write() offset (at least 1MB)."
Test Plan: make all check
Reviewers: igor, rven, kradhakrishnan, yhchiang, IslamAbdelRahman, anthony
Reviewed By: anthony
Subscribers: yoshinorim, MarkCallaghan, sumeet, domas, dhruba, leveldb, ljin
Differential Revision: https://reviews.facebook.net/D15807
2015-07-20 23:46:15 +02:00
|
|
|
size_ += data.size();
|
2021-01-29 07:08:46 +01:00
|
|
|
return IOStatus::OK();
|
|
|
|
}
|
|
|
|
IOStatus Truncate(uint64_t /*size*/, const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
|
|
|
return IOStatus::OK();
|
RangeSync not to sync last 1MB of the file
Summary:
From other ones' investigation:
"sync_file_range() behavior highly depends on kernel version and filesystem.
xfs does neighbor page flushing outside of the specified ranges. For example, sync_file_range(fd, 8192, 16384) does not only trigger flushing page #3 to #4, but also flushing many more dirty pages (i.e. up to page#16)... Ranges of the sync_file_range() should be far enough from write() offset (at least 1MB)."
Test Plan: make all check
Reviewers: igor, rven, kradhakrishnan, yhchiang, IslamAbdelRahman, anthony
Reviewed By: anthony
Subscribers: yoshinorim, MarkCallaghan, sumeet, domas, dhruba, leveldb, ljin
Differential Revision: https://reviews.facebook.net/D15807
2015-07-20 23:46:15 +02:00
|
|
|
}
|
2021-01-29 07:08:46 +01:00
|
|
|
IOStatus Close(const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
RangeSync not to sync last 1MB of the file
Summary:
From other ones' investigation:
"sync_file_range() behavior highly depends on kernel version and filesystem.
xfs does neighbor page flushing outside of the specified ranges. For example, sync_file_range(fd, 8192, 16384) does not only trigger flushing page #3 to #4, but also flushing many more dirty pages (i.e. up to page#16)... Ranges of the sync_file_range() should be far enough from write() offset (at least 1MB)."
Test Plan: make all check
Reviewers: igor, rven, kradhakrishnan, yhchiang, IslamAbdelRahman, anthony
Reviewed By: anthony
Subscribers: yoshinorim, MarkCallaghan, sumeet, domas, dhruba, leveldb, ljin
Differential Revision: https://reviews.facebook.net/D15807
2015-07-20 23:46:15 +02:00
|
|
|
EXPECT_GE(size_, last_synced_ + kMb);
|
|
|
|
EXPECT_LT(size_, last_synced_ + 2 * kMb);
|
|
|
|
// Make sure random writes generated enough writes.
|
|
|
|
EXPECT_GT(size_, 10 * kMb);
|
2021-01-29 07:08:46 +01:00
|
|
|
return IOStatus::OK();
|
|
|
|
}
|
|
|
|
IOStatus Flush(const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
|
|
|
return IOStatus::OK();
|
|
|
|
}
|
|
|
|
IOStatus Sync(const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
|
|
|
return IOStatus::OK();
|
|
|
|
}
|
|
|
|
IOStatus Fsync(const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
|
|
|
return IOStatus::OK();
|
RangeSync not to sync last 1MB of the file
Summary:
From other ones' investigation:
"sync_file_range() behavior highly depends on kernel version and filesystem.
xfs does neighbor page flushing outside of the specified ranges. For example, sync_file_range(fd, 8192, 16384) does not only trigger flushing page #3 to #4, but also flushing many more dirty pages (i.e. up to page#16)... Ranges of the sync_file_range() should be far enough from write() offset (at least 1MB)."
Test Plan: make all check
Reviewers: igor, rven, kradhakrishnan, yhchiang, IslamAbdelRahman, anthony
Reviewed By: anthony
Subscribers: yoshinorim, MarkCallaghan, sumeet, domas, dhruba, leveldb, ljin
Differential Revision: https://reviews.facebook.net/D15807
2015-07-20 23:46:15 +02:00
|
|
|
}
|
2018-03-05 22:08:17 +01:00
|
|
|
void SetIOPriority(Env::IOPriority /*pri*/) override {}
|
2021-01-29 07:08:46 +01:00
|
|
|
uint64_t GetFileSize(const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
|
|
|
return size_;
|
|
|
|
}
|
2018-03-05 22:08:17 +01:00
|
|
|
void GetPreallocationStatus(size_t* /*block_size*/,
|
|
|
|
size_t* /*last_allocated_block*/) override {}
|
|
|
|
size_t GetUniqueId(char* /*id*/, size_t /*max_size*/) const override {
|
|
|
|
return 0;
|
|
|
|
}
|
2021-01-29 07:08:46 +01:00
|
|
|
IOStatus InvalidateCache(size_t /*offset*/, size_t /*length*/) override {
|
|
|
|
return IOStatus::OK();
|
RangeSync not to sync last 1MB of the file
Summary:
From other ones' investigation:
"sync_file_range() behavior highly depends on kernel version and filesystem.
xfs does neighbor page flushing outside of the specified ranges. For example, sync_file_range(fd, 8192, 16384) does not only trigger flushing page #3 to #4, but also flushing many more dirty pages (i.e. up to page#16)... Ranges of the sync_file_range() should be far enough from write() offset (at least 1MB)."
Test Plan: make all check
Reviewers: igor, rven, kradhakrishnan, yhchiang, IslamAbdelRahman, anthony
Reviewed By: anthony
Subscribers: yoshinorim, MarkCallaghan, sumeet, domas, dhruba, leveldb, ljin
Differential Revision: https://reviews.facebook.net/D15807
2015-07-20 23:46:15 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
protected:
|
2021-01-29 07:08:46 +01:00
|
|
|
IOStatus Allocate(uint64_t /*offset*/, uint64_t /*len*/,
|
|
|
|
const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
|
|
|
return IOStatus::OK();
|
2018-03-05 22:08:17 +01:00
|
|
|
}
|
2021-01-29 07:08:46 +01:00
|
|
|
IOStatus RangeSync(uint64_t offset, uint64_t nbytes,
|
|
|
|
const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
RangeSync not to sync last 1MB of the file
Summary:
From other ones' investigation:
"sync_file_range() behavior highly depends on kernel version and filesystem.
xfs does neighbor page flushing outside of the specified ranges. For example, sync_file_range(fd, 8192, 16384) does not only trigger flushing page #3 to #4, but also flushing many more dirty pages (i.e. up to page#16)... Ranges of the sync_file_range() should be far enough from write() offset (at least 1MB)."
Test Plan: make all check
Reviewers: igor, rven, kradhakrishnan, yhchiang, IslamAbdelRahman, anthony
Reviewed By: anthony
Subscribers: yoshinorim, MarkCallaghan, sumeet, domas, dhruba, leveldb, ljin
Differential Revision: https://reviews.facebook.net/D15807
2015-07-20 23:46:15 +02:00
|
|
|
EXPECT_EQ(offset % 4096, 0u);
|
|
|
|
EXPECT_EQ(nbytes % 4096, 0u);
|
|
|
|
|
|
|
|
EXPECT_EQ(offset, last_synced_);
|
|
|
|
last_synced_ = offset + nbytes;
|
|
|
|
EXPECT_GE(size_, last_synced_ + kMb);
|
|
|
|
if (size_ > 2 * kMb) {
|
|
|
|
EXPECT_LT(size_, last_synced_ + 2 * kMb);
|
|
|
|
}
|
2021-01-29 07:08:46 +01:00
|
|
|
return IOStatus::OK();
|
RangeSync not to sync last 1MB of the file
Summary:
From other ones' investigation:
"sync_file_range() behavior highly depends on kernel version and filesystem.
xfs does neighbor page flushing outside of the specified ranges. For example, sync_file_range(fd, 8192, 16384) does not only trigger flushing page #3 to #4, but also flushing many more dirty pages (i.e. up to page#16)... Ranges of the sync_file_range() should be far enough from write() offset (at least 1MB)."
Test Plan: make all check
Reviewers: igor, rven, kradhakrishnan, yhchiang, IslamAbdelRahman, anthony
Reviewed By: anthony
Subscribers: yoshinorim, MarkCallaghan, sumeet, domas, dhruba, leveldb, ljin
Differential Revision: https://reviews.facebook.net/D15807
2015-07-20 23:46:15 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
uint64_t size_;
|
|
|
|
uint64_t last_synced_;
|
|
|
|
};
|
|
|
|
|
|
|
|
EnvOptions env_options;
|
|
|
|
env_options.bytes_per_sync = kMb;
|
2018-11-09 20:17:34 +01:00
|
|
|
std::unique_ptr<FakeWF> wf(new FakeWF);
|
|
|
|
std::unique_ptr<WritableFileWriter> writer(
|
2021-01-29 07:08:46 +01:00
|
|
|
new WritableFileWriter(std::move(wf), "" /* don't care */, env_options));
|
RangeSync not to sync last 1MB of the file
Summary:
From other ones' investigation:
"sync_file_range() behavior highly depends on kernel version and filesystem.
xfs does neighbor page flushing outside of the specified ranges. For example, sync_file_range(fd, 8192, 16384) does not only trigger flushing page #3 to #4, but also flushing many more dirty pages (i.e. up to page#16)... Ranges of the sync_file_range() should be far enough from write() offset (at least 1MB)."
Test Plan: make all check
Reviewers: igor, rven, kradhakrishnan, yhchiang, IslamAbdelRahman, anthony
Reviewed By: anthony
Subscribers: yoshinorim, MarkCallaghan, sumeet, domas, dhruba, leveldb, ljin
Differential Revision: https://reviews.facebook.net/D15807
2015-07-20 23:46:15 +02:00
|
|
|
Random r(301);
|
2020-09-29 01:03:43 +02:00
|
|
|
Status s;
|
RangeSync not to sync last 1MB of the file
Summary:
From other ones' investigation:
"sync_file_range() behavior highly depends on kernel version and filesystem.
xfs does neighbor page flushing outside of the specified ranges. For example, sync_file_range(fd, 8192, 16384) does not only trigger flushing page #3 to #4, but also flushing many more dirty pages (i.e. up to page#16)... Ranges of the sync_file_range() should be far enough from write() offset (at least 1MB)."
Test Plan: make all check
Reviewers: igor, rven, kradhakrishnan, yhchiang, IslamAbdelRahman, anthony
Reviewed By: anthony
Subscribers: yoshinorim, MarkCallaghan, sumeet, domas, dhruba, leveldb, ljin
Differential Revision: https://reviews.facebook.net/D15807
2015-07-20 23:46:15 +02:00
|
|
|
std::unique_ptr<char[]> large_buf(new char[10 * kMb]);
|
|
|
|
for (int i = 0; i < 1000; i++) {
|
|
|
|
int skew_limit = (i < 700) ? 10 : 15;
|
|
|
|
uint32_t num = r.Skewed(skew_limit) * 100 + r.Uniform(100);
|
2020-09-29 01:03:43 +02:00
|
|
|
s = writer->Append(Slice(large_buf.get(), num));
|
|
|
|
ASSERT_OK(s);
|
RangeSync not to sync last 1MB of the file
Summary:
From other ones' investigation:
"sync_file_range() behavior highly depends on kernel version and filesystem.
xfs does neighbor page flushing outside of the specified ranges. For example, sync_file_range(fd, 8192, 16384) does not only trigger flushing page #3 to #4, but also flushing many more dirty pages (i.e. up to page#16)... Ranges of the sync_file_range() should be far enough from write() offset (at least 1MB)."
Test Plan: make all check
Reviewers: igor, rven, kradhakrishnan, yhchiang, IslamAbdelRahman, anthony
Reviewed By: anthony
Subscribers: yoshinorim, MarkCallaghan, sumeet, domas, dhruba, leveldb, ljin
Differential Revision: https://reviews.facebook.net/D15807
2015-07-20 23:46:15 +02:00
|
|
|
|
|
|
|
// Flush in a chance of 1/10.
|
|
|
|
if (r.Uniform(10) == 0) {
|
2020-09-29 01:03:43 +02:00
|
|
|
s = writer->Flush();
|
|
|
|
ASSERT_OK(s);
|
RangeSync not to sync last 1MB of the file
Summary:
From other ones' investigation:
"sync_file_range() behavior highly depends on kernel version and filesystem.
xfs does neighbor page flushing outside of the specified ranges. For example, sync_file_range(fd, 8192, 16384) does not only trigger flushing page #3 to #4, but also flushing many more dirty pages (i.e. up to page#16)... Ranges of the sync_file_range() should be far enough from write() offset (at least 1MB)."
Test Plan: make all check
Reviewers: igor, rven, kradhakrishnan, yhchiang, IslamAbdelRahman, anthony
Reviewed By: anthony
Subscribers: yoshinorim, MarkCallaghan, sumeet, domas, dhruba, leveldb, ljin
Differential Revision: https://reviews.facebook.net/D15807
2015-07-20 23:46:15 +02:00
|
|
|
}
|
|
|
|
}
|
2020-09-29 01:03:43 +02:00
|
|
|
s = writer->Close();
|
|
|
|
ASSERT_OK(s);
|
RangeSync not to sync last 1MB of the file
Summary:
From other ones' investigation:
"sync_file_range() behavior highly depends on kernel version and filesystem.
xfs does neighbor page flushing outside of the specified ranges. For example, sync_file_range(fd, 8192, 16384) does not only trigger flushing page #3 to #4, but also flushing many more dirty pages (i.e. up to page#16)... Ranges of the sync_file_range() should be far enough from write() offset (at least 1MB)."
Test Plan: make all check
Reviewers: igor, rven, kradhakrishnan, yhchiang, IslamAbdelRahman, anthony
Reviewed By: anthony
Subscribers: yoshinorim, MarkCallaghan, sumeet, domas, dhruba, leveldb, ljin
Differential Revision: https://reviews.facebook.net/D15807
2015-07-20 23:46:15 +02:00
|
|
|
}
|
2015-10-28 05:04:00 +01:00
|
|
|
|
2017-06-13 13:34:51 +02:00
|
|
|
TEST_F(WritableFileWriterTest, IncrementalBuffer) {
|
2021-01-29 07:08:46 +01:00
|
|
|
class FakeWF : public FSWritableFile {
|
2017-06-13 13:34:51 +02:00
|
|
|
public:
|
|
|
|
explicit FakeWF(std::string* _file_data, bool _use_direct_io,
|
|
|
|
bool _no_flush)
|
|
|
|
: file_data_(_file_data),
|
|
|
|
use_direct_io_(_use_direct_io),
|
|
|
|
no_flush_(_no_flush) {}
|
2019-02-14 22:52:47 +01:00
|
|
|
~FakeWF() override {}
|
2017-06-13 13:34:51 +02:00
|
|
|
|
2021-01-29 07:08:46 +01:00
|
|
|
using FSWritableFile::Append;
|
|
|
|
IOStatus Append(const Slice& data, const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
2017-06-13 13:34:51 +02:00
|
|
|
file_data_->append(data.data(), data.size());
|
|
|
|
size_ += data.size();
|
2021-01-29 07:08:46 +01:00
|
|
|
return IOStatus::OK();
|
2017-06-13 13:34:51 +02:00
|
|
|
}
|
2021-01-29 07:08:46 +01:00
|
|
|
using FSWritableFile::PositionedAppend;
|
|
|
|
IOStatus PositionedAppend(const Slice& data, uint64_t pos,
|
|
|
|
const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
2017-06-13 13:34:51 +02:00
|
|
|
EXPECT_TRUE(pos % 512 == 0);
|
|
|
|
EXPECT_TRUE(data.size() % 512 == 0);
|
|
|
|
file_data_->resize(pos);
|
|
|
|
file_data_->append(data.data(), data.size());
|
|
|
|
size_ += data.size();
|
2021-01-29 07:08:46 +01:00
|
|
|
return IOStatus::OK();
|
2017-06-13 13:34:51 +02:00
|
|
|
}
|
|
|
|
|
2021-01-29 07:08:46 +01:00
|
|
|
IOStatus Truncate(uint64_t size, const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
2017-06-13 13:34:51 +02:00
|
|
|
file_data_->resize(size);
|
2021-01-29 07:08:46 +01:00
|
|
|
return IOStatus::OK();
|
|
|
|
}
|
|
|
|
IOStatus Close(const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
|
|
|
return IOStatus::OK();
|
|
|
|
}
|
|
|
|
IOStatus Flush(const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
|
|
|
return IOStatus::OK();
|
|
|
|
}
|
|
|
|
IOStatus Sync(const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
|
|
|
return IOStatus::OK();
|
|
|
|
}
|
|
|
|
IOStatus Fsync(const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
|
|
|
return IOStatus::OK();
|
2017-06-13 13:34:51 +02:00
|
|
|
}
|
2018-03-05 22:08:17 +01:00
|
|
|
void SetIOPriority(Env::IOPriority /*pri*/) override {}
|
2021-01-29 07:08:46 +01:00
|
|
|
uint64_t GetFileSize(const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
|
|
|
return size_;
|
|
|
|
}
|
2018-03-05 22:08:17 +01:00
|
|
|
void GetPreallocationStatus(size_t* /*block_size*/,
|
|
|
|
size_t* /*last_allocated_block*/) override {}
|
|
|
|
size_t GetUniqueId(char* /*id*/, size_t /*max_size*/) const override {
|
|
|
|
return 0;
|
|
|
|
}
|
2021-01-29 07:08:46 +01:00
|
|
|
IOStatus InvalidateCache(size_t /*offset*/, size_t /*length*/) override {
|
|
|
|
return IOStatus::OK();
|
2017-06-13 13:34:51 +02:00
|
|
|
}
|
|
|
|
bool use_direct_io() const override { return use_direct_io_; }
|
|
|
|
|
|
|
|
std::string* file_data_;
|
|
|
|
bool use_direct_io_;
|
|
|
|
bool no_flush_;
|
|
|
|
size_t size_ = 0;
|
|
|
|
};
|
|
|
|
|
|
|
|
Random r(301);
|
|
|
|
const int kNumAttempts = 50;
|
|
|
|
for (int attempt = 0; attempt < kNumAttempts; attempt++) {
|
|
|
|
bool no_flush = (attempt % 3 == 0);
|
|
|
|
EnvOptions env_options;
|
|
|
|
env_options.writable_file_max_buffer_size =
|
|
|
|
(attempt < kNumAttempts / 2) ? 512 * 1024 : 700 * 1024;
|
|
|
|
std::string actual;
|
2018-11-09 20:17:34 +01:00
|
|
|
std::unique_ptr<FakeWF> wf(new FakeWF(&actual,
|
2017-07-27 06:02:53 +02:00
|
|
|
#ifndef ROCKSDB_LITE
|
2018-11-09 20:17:34 +01:00
|
|
|
attempt % 2 == 1,
|
2017-07-27 06:02:53 +02:00
|
|
|
#else
|
2018-11-09 20:17:34 +01:00
|
|
|
false,
|
2017-07-27 06:02:53 +02:00
|
|
|
#endif
|
2018-11-09 20:17:34 +01:00
|
|
|
no_flush));
|
2021-01-29 07:08:46 +01:00
|
|
|
std::unique_ptr<WritableFileWriter> writer(new WritableFileWriter(
|
|
|
|
std::move(wf), "" /* don't care */, env_options));
|
2017-06-13 13:34:51 +02:00
|
|
|
|
|
|
|
std::string target;
|
|
|
|
for (int i = 0; i < 20; i++) {
|
|
|
|
uint32_t num = r.Skewed(16) * 100 + r.Uniform(100);
|
2020-07-09 23:33:42 +02:00
|
|
|
std::string random_string = r.RandomString(num);
|
2020-12-24 01:54:05 +01:00
|
|
|
ASSERT_OK(writer->Append(Slice(random_string.c_str(), num)));
|
2017-06-13 13:34:51 +02:00
|
|
|
target.append(random_string.c_str(), num);
|
|
|
|
|
|
|
|
// In some attempts, flush in a chance of 1/10.
|
|
|
|
if (!no_flush && r.Uniform(10) == 0) {
|
2020-12-24 01:54:05 +01:00
|
|
|
ASSERT_OK(writer->Flush());
|
2017-06-13 13:34:51 +02:00
|
|
|
}
|
|
|
|
}
|
2020-12-24 01:54:05 +01:00
|
|
|
ASSERT_OK(writer->Flush());
|
|
|
|
ASSERT_OK(writer->Close());
|
2017-06-13 13:34:51 +02:00
|
|
|
ASSERT_EQ(target.size(), actual.size());
|
|
|
|
ASSERT_EQ(target, actual);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-01-28 04:29:17 +01:00
|
|
|
TEST_F(WritableFileWriterTest, BufferWithZeroCapacityDirectIO) {
|
|
|
|
EnvOptions env_opts;
|
|
|
|
env_opts.use_direct_writes = true;
|
|
|
|
env_opts.writable_file_max_buffer_size = 0;
|
|
|
|
{
|
|
|
|
std::unique_ptr<WritableFileWriter> writer;
|
|
|
|
const Status s =
|
|
|
|
WritableFileWriter::Create(FileSystem::Default(), /*fname=*/"dont_care",
|
|
|
|
FileOptions(env_opts), &writer,
|
|
|
|
/*dbg=*/nullptr);
|
|
|
|
ASSERT_TRUE(s.IsInvalidArgument());
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
Using existing crc32c checksum in checksum handoff for Manifest and WAL (#8412)
Summary:
In PR https://github.com/facebook/rocksdb/issues/7523 , checksum handoff is introduced in RocksDB for WAL, Manifest, and SST files. When user enable checksum handoff for a certain type of file, before the data is written to the lower layer storage system, we calculate the checksum (crc32c) of each piece of data and pass the checksum down with the data, such that data verification can be down by the lower layer storage system if it has the capability. However, it cannot cover the whole lifetime of the data in the memory and also it potentially introduces extra checksum calculation overhead.
In this PR, we introduce a new interface in WritableFileWriter::Append, which allows the caller be able to pass the data and the checksum (crc32c) together. In this way, WritableFileWriter can directly use the pass-in checksum (crc32c) to generate the checksum of data being passed down to the storage system. It saves the calculation overhead and achieves higher protection coverage. When a new checksum is added with the data, we use Crc32cCombine https://github.com/facebook/rocksdb/issues/8305 to combine the existing checksum and the new checksum. To avoid the segmenting of data by rate-limiter before it is stored, rate-limiter is called enough times to accumulate enough credits for a certain write. This design only support Manifest and WAL which use log_writer in the current stage.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/8412
Test Plan: make check, add new testing cases.
Reviewed By: anand1976
Differential Revision: D29151545
Pulled By: zhichao-cao
fbshipit-source-id: 75e2278c5126cfd58393c67b1efd18dcc7a30772
2021-06-25 09:46:33 +02:00
|
|
|
class DBWritableFileWriterTest : public DBTestBase {
|
|
|
|
public:
|
|
|
|
DBWritableFileWriterTest()
|
2021-07-23 17:37:27 +02:00
|
|
|
: DBTestBase("db_secondary_cache_test", /*env_do_fsync=*/true) {
|
Using existing crc32c checksum in checksum handoff for Manifest and WAL (#8412)
Summary:
In PR https://github.com/facebook/rocksdb/issues/7523 , checksum handoff is introduced in RocksDB for WAL, Manifest, and SST files. When user enable checksum handoff for a certain type of file, before the data is written to the lower layer storage system, we calculate the checksum (crc32c) of each piece of data and pass the checksum down with the data, such that data verification can be down by the lower layer storage system if it has the capability. However, it cannot cover the whole lifetime of the data in the memory and also it potentially introduces extra checksum calculation overhead.
In this PR, we introduce a new interface in WritableFileWriter::Append, which allows the caller be able to pass the data and the checksum (crc32c) together. In this way, WritableFileWriter can directly use the pass-in checksum (crc32c) to generate the checksum of data being passed down to the storage system. It saves the calculation overhead and achieves higher protection coverage. When a new checksum is added with the data, we use Crc32cCombine https://github.com/facebook/rocksdb/issues/8305 to combine the existing checksum and the new checksum. To avoid the segmenting of data by rate-limiter before it is stored, rate-limiter is called enough times to accumulate enough credits for a certain write. This design only support Manifest and WAL which use log_writer in the current stage.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/8412
Test Plan: make check, add new testing cases.
Reviewed By: anand1976
Differential Revision: D29151545
Pulled By: zhichao-cao
fbshipit-source-id: 75e2278c5126cfd58393c67b1efd18dcc7a30772
2021-06-25 09:46:33 +02:00
|
|
|
fault_fs_.reset(new FaultInjectionTestFS(env_->GetFileSystem()));
|
|
|
|
fault_env_.reset(new CompositeEnvWrapper(env_, fault_fs_));
|
|
|
|
}
|
|
|
|
|
|
|
|
std::shared_ptr<FaultInjectionTestFS> fault_fs_;
|
|
|
|
std::unique_ptr<Env> fault_env_;
|
|
|
|
};
|
|
|
|
|
|
|
|
TEST_F(DBWritableFileWriterTest, AppendWithChecksum) {
|
|
|
|
FileOptions file_options = FileOptions();
|
|
|
|
Options options = GetDefaultOptions();
|
|
|
|
options.create_if_missing = true;
|
|
|
|
DestroyAndReopen(options);
|
2022-01-28 04:29:17 +01:00
|
|
|
std::string fname = dbname_ + "/test_file";
|
Using existing crc32c checksum in checksum handoff for Manifest and WAL (#8412)
Summary:
In PR https://github.com/facebook/rocksdb/issues/7523 , checksum handoff is introduced in RocksDB for WAL, Manifest, and SST files. When user enable checksum handoff for a certain type of file, before the data is written to the lower layer storage system, we calculate the checksum (crc32c) of each piece of data and pass the checksum down with the data, such that data verification can be down by the lower layer storage system if it has the capability. However, it cannot cover the whole lifetime of the data in the memory and also it potentially introduces extra checksum calculation overhead.
In this PR, we introduce a new interface in WritableFileWriter::Append, which allows the caller be able to pass the data and the checksum (crc32c) together. In this way, WritableFileWriter can directly use the pass-in checksum (crc32c) to generate the checksum of data being passed down to the storage system. It saves the calculation overhead and achieves higher protection coverage. When a new checksum is added with the data, we use Crc32cCombine https://github.com/facebook/rocksdb/issues/8305 to combine the existing checksum and the new checksum. To avoid the segmenting of data by rate-limiter before it is stored, rate-limiter is called enough times to accumulate enough credits for a certain write. This design only support Manifest and WAL which use log_writer in the current stage.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/8412
Test Plan: make check, add new testing cases.
Reviewed By: anand1976
Differential Revision: D29151545
Pulled By: zhichao-cao
fbshipit-source-id: 75e2278c5126cfd58393c67b1efd18dcc7a30772
2021-06-25 09:46:33 +02:00
|
|
|
std::unique_ptr<FSWritableFile> writable_file_ptr;
|
|
|
|
ASSERT_OK(fault_fs_->NewWritableFile(fname, file_options, &writable_file_ptr,
|
|
|
|
/*dbg*/ nullptr));
|
|
|
|
std::unique_ptr<TestFSWritableFile> file;
|
|
|
|
file.reset(new TestFSWritableFile(
|
|
|
|
fname, file_options, std::move(writable_file_ptr), fault_fs_.get()));
|
|
|
|
std::unique_ptr<WritableFileWriter> file_writer;
|
|
|
|
ImmutableOptions ioptions(options);
|
|
|
|
file_writer.reset(new WritableFileWriter(
|
|
|
|
std::move(file), fname, file_options, SystemClock::Default().get(),
|
|
|
|
nullptr, ioptions.stats, ioptions.listeners,
|
|
|
|
ioptions.file_checksum_gen_factory.get(), true, true));
|
|
|
|
|
|
|
|
Random rnd(301);
|
|
|
|
std::string data = rnd.RandomString(1000);
|
|
|
|
uint32_t data_crc32c = crc32c::Value(data.c_str(), data.size());
|
|
|
|
fault_fs_->SetChecksumHandoffFuncType(ChecksumType::kCRC32c);
|
|
|
|
|
|
|
|
ASSERT_OK(file_writer->Append(Slice(data.c_str()), data_crc32c));
|
|
|
|
ASSERT_OK(file_writer->Flush());
|
|
|
|
Random size_r(47);
|
|
|
|
for (int i = 0; i < 2000; i++) {
|
|
|
|
data = rnd.RandomString((static_cast<int>(size_r.Next()) % 10000));
|
|
|
|
data_crc32c = crc32c::Value(data.c_str(), data.size());
|
|
|
|
ASSERT_OK(file_writer->Append(Slice(data.c_str()), data_crc32c));
|
|
|
|
|
|
|
|
data = rnd.RandomString((static_cast<int>(size_r.Next()) % 97));
|
|
|
|
ASSERT_OK(file_writer->Append(Slice(data.c_str())));
|
|
|
|
ASSERT_OK(file_writer->Flush());
|
|
|
|
}
|
|
|
|
ASSERT_OK(file_writer->Close());
|
|
|
|
Destroy(options);
|
|
|
|
}
|
|
|
|
|
|
|
|
TEST_F(DBWritableFileWriterTest, AppendVerifyNoChecksum) {
|
|
|
|
FileOptions file_options = FileOptions();
|
|
|
|
Options options = GetDefaultOptions();
|
|
|
|
options.create_if_missing = true;
|
|
|
|
DestroyAndReopen(options);
|
2022-01-28 04:29:17 +01:00
|
|
|
std::string fname = dbname_ + "/test_file";
|
Using existing crc32c checksum in checksum handoff for Manifest and WAL (#8412)
Summary:
In PR https://github.com/facebook/rocksdb/issues/7523 , checksum handoff is introduced in RocksDB for WAL, Manifest, and SST files. When user enable checksum handoff for a certain type of file, before the data is written to the lower layer storage system, we calculate the checksum (crc32c) of each piece of data and pass the checksum down with the data, such that data verification can be down by the lower layer storage system if it has the capability. However, it cannot cover the whole lifetime of the data in the memory and also it potentially introduces extra checksum calculation overhead.
In this PR, we introduce a new interface in WritableFileWriter::Append, which allows the caller be able to pass the data and the checksum (crc32c) together. In this way, WritableFileWriter can directly use the pass-in checksum (crc32c) to generate the checksum of data being passed down to the storage system. It saves the calculation overhead and achieves higher protection coverage. When a new checksum is added with the data, we use Crc32cCombine https://github.com/facebook/rocksdb/issues/8305 to combine the existing checksum and the new checksum. To avoid the segmenting of data by rate-limiter before it is stored, rate-limiter is called enough times to accumulate enough credits for a certain write. This design only support Manifest and WAL which use log_writer in the current stage.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/8412
Test Plan: make check, add new testing cases.
Reviewed By: anand1976
Differential Revision: D29151545
Pulled By: zhichao-cao
fbshipit-source-id: 75e2278c5126cfd58393c67b1efd18dcc7a30772
2021-06-25 09:46:33 +02:00
|
|
|
std::unique_ptr<FSWritableFile> writable_file_ptr;
|
|
|
|
ASSERT_OK(fault_fs_->NewWritableFile(fname, file_options, &writable_file_ptr,
|
|
|
|
/*dbg*/ nullptr));
|
|
|
|
std::unique_ptr<TestFSWritableFile> file;
|
|
|
|
file.reset(new TestFSWritableFile(
|
|
|
|
fname, file_options, std::move(writable_file_ptr), fault_fs_.get()));
|
|
|
|
std::unique_ptr<WritableFileWriter> file_writer;
|
|
|
|
ImmutableOptions ioptions(options);
|
|
|
|
// Enable checksum handoff for this file, but do not enable buffer checksum.
|
|
|
|
// So Append with checksum logic will not be triggered
|
|
|
|
file_writer.reset(new WritableFileWriter(
|
|
|
|
std::move(file), fname, file_options, SystemClock::Default().get(),
|
|
|
|
nullptr, ioptions.stats, ioptions.listeners,
|
|
|
|
ioptions.file_checksum_gen_factory.get(), true, false));
|
|
|
|
|
|
|
|
Random rnd(301);
|
|
|
|
std::string data = rnd.RandomString(1000);
|
|
|
|
uint32_t data_crc32c = crc32c::Value(data.c_str(), data.size());
|
|
|
|
fault_fs_->SetChecksumHandoffFuncType(ChecksumType::kCRC32c);
|
|
|
|
|
|
|
|
ASSERT_OK(file_writer->Append(Slice(data.c_str()), data_crc32c));
|
|
|
|
ASSERT_OK(file_writer->Flush());
|
|
|
|
Random size_r(47);
|
|
|
|
for (int i = 0; i < 1000; i++) {
|
|
|
|
data = rnd.RandomString((static_cast<int>(size_r.Next()) % 10000));
|
|
|
|
data_crc32c = crc32c::Value(data.c_str(), data.size());
|
|
|
|
ASSERT_OK(file_writer->Append(Slice(data.c_str()), data_crc32c));
|
|
|
|
|
|
|
|
data = rnd.RandomString((static_cast<int>(size_r.Next()) % 97));
|
|
|
|
ASSERT_OK(file_writer->Append(Slice(data.c_str())));
|
|
|
|
ASSERT_OK(file_writer->Flush());
|
|
|
|
}
|
|
|
|
ASSERT_OK(file_writer->Close());
|
|
|
|
Destroy(options);
|
|
|
|
}
|
|
|
|
|
|
|
|
TEST_F(DBWritableFileWriterTest, AppendWithChecksumRateLimiter) {
|
|
|
|
FileOptions file_options = FileOptions();
|
|
|
|
file_options.rate_limiter = nullptr;
|
|
|
|
Options options = GetDefaultOptions();
|
|
|
|
options.create_if_missing = true;
|
|
|
|
DestroyAndReopen(options);
|
2022-01-28 04:29:17 +01:00
|
|
|
std::string fname = dbname_ + "/test_file";
|
Using existing crc32c checksum in checksum handoff for Manifest and WAL (#8412)
Summary:
In PR https://github.com/facebook/rocksdb/issues/7523 , checksum handoff is introduced in RocksDB for WAL, Manifest, and SST files. When user enable checksum handoff for a certain type of file, before the data is written to the lower layer storage system, we calculate the checksum (crc32c) of each piece of data and pass the checksum down with the data, such that data verification can be down by the lower layer storage system if it has the capability. However, it cannot cover the whole lifetime of the data in the memory and also it potentially introduces extra checksum calculation overhead.
In this PR, we introduce a new interface in WritableFileWriter::Append, which allows the caller be able to pass the data and the checksum (crc32c) together. In this way, WritableFileWriter can directly use the pass-in checksum (crc32c) to generate the checksum of data being passed down to the storage system. It saves the calculation overhead and achieves higher protection coverage. When a new checksum is added with the data, we use Crc32cCombine https://github.com/facebook/rocksdb/issues/8305 to combine the existing checksum and the new checksum. To avoid the segmenting of data by rate-limiter before it is stored, rate-limiter is called enough times to accumulate enough credits for a certain write. This design only support Manifest and WAL which use log_writer in the current stage.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/8412
Test Plan: make check, add new testing cases.
Reviewed By: anand1976
Differential Revision: D29151545
Pulled By: zhichao-cao
fbshipit-source-id: 75e2278c5126cfd58393c67b1efd18dcc7a30772
2021-06-25 09:46:33 +02:00
|
|
|
std::unique_ptr<FSWritableFile> writable_file_ptr;
|
|
|
|
ASSERT_OK(fault_fs_->NewWritableFile(fname, file_options, &writable_file_ptr,
|
|
|
|
/*dbg*/ nullptr));
|
|
|
|
std::unique_ptr<TestFSWritableFile> file;
|
|
|
|
file.reset(new TestFSWritableFile(
|
|
|
|
fname, file_options, std::move(writable_file_ptr), fault_fs_.get()));
|
|
|
|
std::unique_ptr<WritableFileWriter> file_writer;
|
|
|
|
ImmutableOptions ioptions(options);
|
|
|
|
// Enable checksum handoff for this file, but do not enable buffer checksum.
|
|
|
|
// So Append with checksum logic will not be triggered
|
|
|
|
file_writer.reset(new WritableFileWriter(
|
|
|
|
std::move(file), fname, file_options, SystemClock::Default().get(),
|
|
|
|
nullptr, ioptions.stats, ioptions.listeners,
|
|
|
|
ioptions.file_checksum_gen_factory.get(), true, true));
|
|
|
|
fault_fs_->SetChecksumHandoffFuncType(ChecksumType::kCRC32c);
|
|
|
|
|
|
|
|
Random rnd(301);
|
|
|
|
std::string data;
|
|
|
|
uint32_t data_crc32c;
|
|
|
|
uint64_t start = fault_env_->NowMicros();
|
|
|
|
Random size_r(47);
|
|
|
|
uint64_t bytes_written = 0;
|
|
|
|
for (int i = 0; i < 100; i++) {
|
|
|
|
data = rnd.RandomString((static_cast<int>(size_r.Next()) % 10000));
|
|
|
|
data_crc32c = crc32c::Value(data.c_str(), data.size());
|
|
|
|
ASSERT_OK(file_writer->Append(Slice(data.c_str()), data_crc32c));
|
|
|
|
bytes_written += static_cast<uint64_t>(data.size());
|
|
|
|
|
|
|
|
data = rnd.RandomString((static_cast<int>(size_r.Next()) % 97));
|
|
|
|
ASSERT_OK(file_writer->Append(Slice(data.c_str())));
|
|
|
|
ASSERT_OK(file_writer->Flush());
|
|
|
|
bytes_written += static_cast<uint64_t>(data.size());
|
|
|
|
}
|
|
|
|
uint64_t elapsed = fault_env_->NowMicros() - start;
|
|
|
|
double raw_rate = bytes_written * 1000000.0 / elapsed;
|
|
|
|
ASSERT_OK(file_writer->Close());
|
|
|
|
|
|
|
|
// Set the rate-limiter
|
|
|
|
FileOptions file_options1 = FileOptions();
|
|
|
|
file_options1.rate_limiter =
|
|
|
|
NewGenericRateLimiter(static_cast<int64_t>(0.5 * raw_rate));
|
2022-01-28 04:29:17 +01:00
|
|
|
fname = dbname_ + "/test_file_1";
|
Using existing crc32c checksum in checksum handoff for Manifest and WAL (#8412)
Summary:
In PR https://github.com/facebook/rocksdb/issues/7523 , checksum handoff is introduced in RocksDB for WAL, Manifest, and SST files. When user enable checksum handoff for a certain type of file, before the data is written to the lower layer storage system, we calculate the checksum (crc32c) of each piece of data and pass the checksum down with the data, such that data verification can be down by the lower layer storage system if it has the capability. However, it cannot cover the whole lifetime of the data in the memory and also it potentially introduces extra checksum calculation overhead.
In this PR, we introduce a new interface in WritableFileWriter::Append, which allows the caller be able to pass the data and the checksum (crc32c) together. In this way, WritableFileWriter can directly use the pass-in checksum (crc32c) to generate the checksum of data being passed down to the storage system. It saves the calculation overhead and achieves higher protection coverage. When a new checksum is added with the data, we use Crc32cCombine https://github.com/facebook/rocksdb/issues/8305 to combine the existing checksum and the new checksum. To avoid the segmenting of data by rate-limiter before it is stored, rate-limiter is called enough times to accumulate enough credits for a certain write. This design only support Manifest and WAL which use log_writer in the current stage.
Pull Request resolved: https://github.com/facebook/rocksdb/pull/8412
Test Plan: make check, add new testing cases.
Reviewed By: anand1976
Differential Revision: D29151545
Pulled By: zhichao-cao
fbshipit-source-id: 75e2278c5126cfd58393c67b1efd18dcc7a30772
2021-06-25 09:46:33 +02:00
|
|
|
std::unique_ptr<FSWritableFile> writable_file_ptr1;
|
|
|
|
ASSERT_OK(fault_fs_->NewWritableFile(fname, file_options1,
|
|
|
|
&writable_file_ptr1,
|
|
|
|
/*dbg*/ nullptr));
|
|
|
|
file.reset(new TestFSWritableFile(
|
|
|
|
fname, file_options1, std::move(writable_file_ptr1), fault_fs_.get()));
|
|
|
|
// Enable checksum handoff for this file, but do not enable buffer checksum.
|
|
|
|
// So Append with checksum logic will not be triggered
|
|
|
|
file_writer.reset(new WritableFileWriter(
|
|
|
|
std::move(file), fname, file_options1, SystemClock::Default().get(),
|
|
|
|
nullptr, ioptions.stats, ioptions.listeners,
|
|
|
|
ioptions.file_checksum_gen_factory.get(), true, true));
|
|
|
|
|
|
|
|
for (int i = 0; i < 1000; i++) {
|
|
|
|
data = rnd.RandomString((static_cast<int>(size_r.Next()) % 10000));
|
|
|
|
data_crc32c = crc32c::Value(data.c_str(), data.size());
|
|
|
|
ASSERT_OK(file_writer->Append(Slice(data.c_str()), data_crc32c));
|
|
|
|
|
|
|
|
data = rnd.RandomString((static_cast<int>(size_r.Next()) % 97));
|
|
|
|
ASSERT_OK(file_writer->Append(Slice(data.c_str())));
|
|
|
|
ASSERT_OK(file_writer->Flush());
|
|
|
|
}
|
|
|
|
ASSERT_OK(file_writer->Close());
|
|
|
|
if (file_options1.rate_limiter != nullptr) {
|
|
|
|
delete file_options1.rate_limiter;
|
|
|
|
}
|
|
|
|
|
|
|
|
Destroy(options);
|
|
|
|
}
|
|
|
|
|
2017-02-16 19:25:06 +01:00
|
|
|
#ifndef ROCKSDB_LITE
|
2015-10-28 05:04:00 +01:00
|
|
|
TEST_F(WritableFileWriterTest, AppendStatusReturn) {
|
2021-01-29 07:08:46 +01:00
|
|
|
class FakeWF : public FSWritableFile {
|
2015-10-28 05:04:00 +01:00
|
|
|
public:
|
2016-12-22 21:51:29 +01:00
|
|
|
explicit FakeWF() : use_direct_io_(false), io_error_(false) {}
|
2015-10-28 05:04:00 +01:00
|
|
|
|
2019-02-14 22:52:47 +01:00
|
|
|
bool use_direct_io() const override { return use_direct_io_; }
|
2021-01-29 07:08:46 +01:00
|
|
|
|
|
|
|
using FSWritableFile::Append;
|
|
|
|
IOStatus Append(const Slice& /*data*/, const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
2015-10-28 05:04:00 +01:00
|
|
|
if (io_error_) {
|
2021-01-29 07:08:46 +01:00
|
|
|
return IOStatus::IOError("Fake IO error");
|
2015-10-28 05:04:00 +01:00
|
|
|
}
|
2021-01-29 07:08:46 +01:00
|
|
|
return IOStatus::OK();
|
2015-10-28 05:04:00 +01:00
|
|
|
}
|
2021-01-29 07:08:46 +01:00
|
|
|
using FSWritableFile::PositionedAppend;
|
|
|
|
IOStatus PositionedAppend(const Slice& /*data*/, uint64_t,
|
|
|
|
const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
2015-10-28 05:04:00 +01:00
|
|
|
if (io_error_) {
|
2021-01-29 07:08:46 +01:00
|
|
|
return IOStatus::IOError("Fake IO error");
|
2015-10-28 05:04:00 +01:00
|
|
|
}
|
2021-01-29 07:08:46 +01:00
|
|
|
return IOStatus::OK();
|
|
|
|
}
|
|
|
|
IOStatus Close(const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
|
|
|
return IOStatus::OK();
|
|
|
|
}
|
|
|
|
IOStatus Flush(const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
|
|
|
return IOStatus::OK();
|
|
|
|
}
|
|
|
|
IOStatus Sync(const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
|
|
|
return IOStatus::OK();
|
2015-10-28 05:04:00 +01:00
|
|
|
}
|
2017-01-13 21:01:08 +01:00
|
|
|
void Setuse_direct_io(bool val) { use_direct_io_ = val; }
|
2015-10-28 05:04:00 +01:00
|
|
|
void SetIOError(bool val) { io_error_ = val; }
|
|
|
|
|
|
|
|
protected:
|
2016-12-22 21:51:29 +01:00
|
|
|
bool use_direct_io_;
|
2015-10-28 05:04:00 +01:00
|
|
|
bool io_error_;
|
|
|
|
};
|
2018-11-09 20:17:34 +01:00
|
|
|
std::unique_ptr<FakeWF> wf(new FakeWF());
|
2017-01-13 21:01:08 +01:00
|
|
|
wf->Setuse_direct_io(true);
|
2018-11-09 20:17:34 +01:00
|
|
|
std::unique_ptr<WritableFileWriter> writer(
|
2021-01-29 07:08:46 +01:00
|
|
|
new WritableFileWriter(std::move(wf), "" /* don't care */, EnvOptions()));
|
2015-10-28 05:04:00 +01:00
|
|
|
|
|
|
|
ASSERT_OK(writer->Append(std::string(2 * kMb, 'a')));
|
|
|
|
|
|
|
|
// Next call to WritableFile::Append() should fail
|
2021-01-29 07:08:46 +01:00
|
|
|
FakeWF* fwf = static_cast<FakeWF*>(writer->writable_file());
|
|
|
|
fwf->SetIOError(true);
|
2015-10-28 05:04:00 +01:00
|
|
|
ASSERT_NOK(writer->Append(std::string(2 * kMb, 'b')));
|
|
|
|
}
|
2017-02-16 19:25:06 +01:00
|
|
|
#endif
|
2015-10-28 05:04:00 +01:00
|
|
|
|
2017-02-18 20:54:49 +01:00
|
|
|
class ReadaheadRandomAccessFileTest
|
|
|
|
: public testing::Test,
|
|
|
|
public testing::WithParamInterface<size_t> {
|
|
|
|
public:
|
|
|
|
static std::vector<size_t> GetReadaheadSizeList() {
|
|
|
|
return {1lu << 12, 1lu << 16};
|
|
|
|
}
|
2019-02-14 22:52:47 +01:00
|
|
|
void SetUp() override {
|
2017-02-18 20:54:49 +01:00
|
|
|
readahead_size_ = GetParam();
|
|
|
|
scratch_.reset(new char[2 * readahead_size_]);
|
|
|
|
ResetSourceStr();
|
|
|
|
}
|
|
|
|
ReadaheadRandomAccessFileTest() : control_contents_() {}
|
|
|
|
std::string Read(uint64_t offset, size_t n) {
|
|
|
|
Slice result;
|
2021-01-05 00:59:52 +01:00
|
|
|
Status s = test_read_holder_->Read(offset, n, IOOptions(), &result,
|
|
|
|
scratch_.get(), nullptr);
|
2020-09-29 01:03:43 +02:00
|
|
|
EXPECT_TRUE(s.ok() || s.IsInvalidArgument());
|
2017-02-18 20:54:49 +01:00
|
|
|
return std::string(result.data(), result.size());
|
|
|
|
}
|
|
|
|
void ResetSourceStr(const std::string& str = "") {
|
2021-01-05 00:59:52 +01:00
|
|
|
std::unique_ptr<FSWritableFile> sink(
|
|
|
|
new test::StringSink(&control_contents_));
|
|
|
|
std::unique_ptr<WritableFileWriter> write_holder(new WritableFileWriter(
|
|
|
|
std::move(sink), "" /* don't care */, FileOptions()));
|
2020-09-29 01:03:43 +02:00
|
|
|
Status s = write_holder->Append(Slice(str));
|
|
|
|
EXPECT_OK(s);
|
|
|
|
s = write_holder->Flush();
|
|
|
|
EXPECT_OK(s);
|
2021-01-05 00:59:52 +01:00
|
|
|
std::unique_ptr<FSRandomAccessFile> read_holder(
|
2017-02-18 20:54:49 +01:00
|
|
|
new test::StringSource(control_contents_));
|
|
|
|
test_read_holder_ =
|
|
|
|
NewReadaheadRandomAccessFile(std::move(read_holder), readahead_size_);
|
|
|
|
}
|
|
|
|
size_t GetReadaheadSize() const { return readahead_size_; }
|
|
|
|
|
|
|
|
private:
|
|
|
|
size_t readahead_size_;
|
|
|
|
Slice control_contents_;
|
2021-01-05 00:59:52 +01:00
|
|
|
std::unique_ptr<FSRandomAccessFile> test_read_holder_;
|
2017-02-18 20:54:49 +01:00
|
|
|
std::unique_ptr<char[]> scratch_;
|
|
|
|
};
|
|
|
|
|
2020-01-31 08:08:56 +01:00
|
|
|
TEST_P(ReadaheadRandomAccessFileTest, EmptySourceStr) {
|
2017-02-18 20:54:49 +01:00
|
|
|
ASSERT_EQ("", Read(0, 1));
|
|
|
|
ASSERT_EQ("", Read(0, 0));
|
|
|
|
ASSERT_EQ("", Read(13, 13));
|
|
|
|
}
|
|
|
|
|
2020-01-31 08:08:56 +01:00
|
|
|
TEST_P(ReadaheadRandomAccessFileTest, SourceStrLenLessThanReadaheadSize) {
|
2017-02-18 20:54:49 +01:00
|
|
|
std::string str = "abcdefghijklmnopqrs";
|
|
|
|
ResetSourceStr(str);
|
|
|
|
ASSERT_EQ(str.substr(3, 4), Read(3, 4));
|
|
|
|
ASSERT_EQ(str.substr(0, 3), Read(0, 3));
|
|
|
|
ASSERT_EQ(str, Read(0, str.size()));
|
|
|
|
ASSERT_EQ(str.substr(7, std::min(static_cast<int>(str.size()) - 7, 30)),
|
|
|
|
Read(7, 30));
|
|
|
|
ASSERT_EQ("", Read(100, 100));
|
|
|
|
}
|
|
|
|
|
2020-01-31 08:08:56 +01:00
|
|
|
TEST_P(ReadaheadRandomAccessFileTest, SourceStrLenGreaterThanReadaheadSize) {
|
2017-02-18 20:54:49 +01:00
|
|
|
Random rng(42);
|
|
|
|
for (int k = 0; k < 100; ++k) {
|
|
|
|
size_t strLen = k * GetReadaheadSize() +
|
|
|
|
rng.Uniform(static_cast<int>(GetReadaheadSize()));
|
2020-07-09 23:33:42 +02:00
|
|
|
std::string str = rng.HumanReadableString(static_cast<int>(strLen));
|
2017-02-18 20:54:49 +01:00
|
|
|
ResetSourceStr(str);
|
|
|
|
for (int test = 1; test <= 100; ++test) {
|
|
|
|
size_t offset = rng.Uniform(static_cast<int>(strLen));
|
|
|
|
size_t n = rng.Uniform(static_cast<int>(GetReadaheadSize()));
|
2019-07-17 03:18:07 +02:00
|
|
|
ASSERT_EQ(str.substr(offset, std::min(n, strLen - offset)),
|
2017-02-18 20:54:49 +01:00
|
|
|
Read(offset, n));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-01-31 08:08:56 +01:00
|
|
|
TEST_P(ReadaheadRandomAccessFileTest, ReadExceedsReadaheadSize) {
|
2017-02-18 20:54:49 +01:00
|
|
|
Random rng(7);
|
|
|
|
size_t strLen = 4 * GetReadaheadSize() +
|
|
|
|
rng.Uniform(static_cast<int>(GetReadaheadSize()));
|
2020-07-09 23:33:42 +02:00
|
|
|
std::string str = rng.HumanReadableString(static_cast<int>(strLen));
|
2017-02-18 20:54:49 +01:00
|
|
|
ResetSourceStr(str);
|
|
|
|
for (int test = 1; test <= 100; ++test) {
|
|
|
|
size_t offset = rng.Uniform(static_cast<int>(strLen));
|
|
|
|
size_t n =
|
|
|
|
GetReadaheadSize() + rng.Uniform(static_cast<int>(GetReadaheadSize()));
|
2019-07-17 03:18:07 +02:00
|
|
|
ASSERT_EQ(str.substr(offset, std::min(n, strLen - offset)),
|
2017-02-18 20:54:49 +01:00
|
|
|
Read(offset, n));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-06-04 00:53:09 +02:00
|
|
|
INSTANTIATE_TEST_CASE_P(
|
2020-01-31 08:08:56 +01:00
|
|
|
EmptySourceStr, ReadaheadRandomAccessFileTest,
|
2017-02-18 20:54:49 +01:00
|
|
|
::testing::ValuesIn(ReadaheadRandomAccessFileTest::GetReadaheadSizeList()));
|
2020-06-04 00:53:09 +02:00
|
|
|
INSTANTIATE_TEST_CASE_P(
|
2020-01-31 08:08:56 +01:00
|
|
|
SourceStrLenLessThanReadaheadSize, ReadaheadRandomAccessFileTest,
|
2017-02-18 20:54:49 +01:00
|
|
|
::testing::ValuesIn(ReadaheadRandomAccessFileTest::GetReadaheadSizeList()));
|
2020-06-04 00:53:09 +02:00
|
|
|
INSTANTIATE_TEST_CASE_P(
|
2020-01-31 08:08:56 +01:00
|
|
|
SourceStrLenGreaterThanReadaheadSize, ReadaheadRandomAccessFileTest,
|
2017-02-18 20:54:49 +01:00
|
|
|
::testing::ValuesIn(ReadaheadRandomAccessFileTest::GetReadaheadSizeList()));
|
2020-06-04 00:53:09 +02:00
|
|
|
INSTANTIATE_TEST_CASE_P(
|
2020-01-31 08:08:56 +01:00
|
|
|
ReadExceedsReadaheadSize, ReadaheadRandomAccessFileTest,
|
2017-02-18 20:54:49 +01:00
|
|
|
::testing::ValuesIn(ReadaheadRandomAccessFileTest::GetReadaheadSizeList()));
|
|
|
|
|
2019-07-17 03:18:07 +02:00
|
|
|
class ReadaheadSequentialFileTest : public testing::Test,
|
|
|
|
public testing::WithParamInterface<size_t> {
|
|
|
|
public:
|
|
|
|
static std::vector<size_t> GetReadaheadSizeList() {
|
2019-07-18 02:01:30 +02:00
|
|
|
return {1lu << 8, 1lu << 12, 1lu << 16, 1lu << 18};
|
2019-07-17 03:18:07 +02:00
|
|
|
}
|
|
|
|
void SetUp() override {
|
|
|
|
readahead_size_ = GetParam();
|
|
|
|
scratch_.reset(new char[2 * readahead_size_]);
|
|
|
|
ResetSourceStr();
|
|
|
|
}
|
|
|
|
ReadaheadSequentialFileTest() {}
|
|
|
|
std::string Read(size_t n) {
|
|
|
|
Slice result;
|
2020-09-29 01:03:43 +02:00
|
|
|
Status s = test_read_holder_->Read(n, &result, scratch_.get());
|
|
|
|
EXPECT_TRUE(s.ok() || s.IsInvalidArgument());
|
2019-07-17 03:18:07 +02:00
|
|
|
return std::string(result.data(), result.size());
|
|
|
|
}
|
|
|
|
void Skip(size_t n) { test_read_holder_->Skip(n); }
|
|
|
|
void ResetSourceStr(const std::string& str = "") {
|
2021-01-05 00:59:52 +01:00
|
|
|
auto read_holder = std::unique_ptr<FSSequentialFile>(
|
2020-02-08 00:16:29 +01:00
|
|
|
new test::SeqStringSource(str, &seq_read_count_));
|
2021-01-05 00:59:52 +01:00
|
|
|
test_read_holder_.reset(new SequentialFileReader(std::move(read_holder),
|
|
|
|
"test", readahead_size_));
|
2019-07-17 03:18:07 +02:00
|
|
|
}
|
|
|
|
size_t GetReadaheadSize() const { return readahead_size_; }
|
|
|
|
|
|
|
|
private:
|
|
|
|
size_t readahead_size_;
|
|
|
|
std::unique_ptr<SequentialFileReader> test_read_holder_;
|
|
|
|
std::unique_ptr<char[]> scratch_;
|
2020-02-08 00:16:29 +01:00
|
|
|
std::atomic<int> seq_read_count_;
|
2019-07-17 03:18:07 +02:00
|
|
|
};
|
|
|
|
|
2020-01-31 08:08:56 +01:00
|
|
|
TEST_P(ReadaheadSequentialFileTest, EmptySourceStr) {
|
2019-07-17 03:18:07 +02:00
|
|
|
ASSERT_EQ("", Read(0));
|
|
|
|
ASSERT_EQ("", Read(1));
|
|
|
|
ASSERT_EQ("", Read(13));
|
|
|
|
}
|
|
|
|
|
2020-01-31 08:08:56 +01:00
|
|
|
TEST_P(ReadaheadSequentialFileTest, SourceStrLenLessThanReadaheadSize) {
|
2019-07-17 03:18:07 +02:00
|
|
|
std::string str = "abcdefghijklmnopqrs";
|
|
|
|
ResetSourceStr(str);
|
|
|
|
ASSERT_EQ(str.substr(0, 3), Read(3));
|
|
|
|
ASSERT_EQ(str.substr(3, 1), Read(1));
|
|
|
|
ASSERT_EQ(str.substr(4), Read(str.size()));
|
|
|
|
ASSERT_EQ("", Read(100));
|
|
|
|
}
|
|
|
|
|
2020-01-31 08:08:56 +01:00
|
|
|
TEST_P(ReadaheadSequentialFileTest, SourceStrLenGreaterThanReadaheadSize) {
|
2019-07-17 03:18:07 +02:00
|
|
|
Random rng(42);
|
|
|
|
for (int s = 0; s < 1; ++s) {
|
|
|
|
for (int k = 0; k < 100; ++k) {
|
|
|
|
size_t strLen = k * GetReadaheadSize() +
|
|
|
|
rng.Uniform(static_cast<int>(GetReadaheadSize()));
|
2020-07-09 23:33:42 +02:00
|
|
|
std::string str = rng.HumanReadableString(static_cast<int>(strLen));
|
2019-07-17 03:18:07 +02:00
|
|
|
ResetSourceStr(str);
|
|
|
|
size_t offset = 0;
|
|
|
|
for (int test = 1; test <= 100; ++test) {
|
|
|
|
size_t n = rng.Uniform(static_cast<int>(GetReadaheadSize()));
|
|
|
|
if (s && test % 2) {
|
|
|
|
Skip(n);
|
|
|
|
} else {
|
|
|
|
ASSERT_EQ(str.substr(offset, std::min(n, strLen - offset)), Read(n));
|
|
|
|
}
|
|
|
|
offset = std::min(offset + n, strLen);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-01-31 08:08:56 +01:00
|
|
|
TEST_P(ReadaheadSequentialFileTest, ReadExceedsReadaheadSize) {
|
2019-07-17 03:18:07 +02:00
|
|
|
Random rng(42);
|
|
|
|
for (int s = 0; s < 1; ++s) {
|
|
|
|
for (int k = 0; k < 100; ++k) {
|
|
|
|
size_t strLen = k * GetReadaheadSize() +
|
|
|
|
rng.Uniform(static_cast<int>(GetReadaheadSize()));
|
2020-07-09 23:33:42 +02:00
|
|
|
std::string str = rng.HumanReadableString(static_cast<int>(strLen));
|
2019-07-17 03:18:07 +02:00
|
|
|
ResetSourceStr(str);
|
|
|
|
size_t offset = 0;
|
|
|
|
for (int test = 1; test <= 100; ++test) {
|
|
|
|
size_t n = GetReadaheadSize() +
|
|
|
|
rng.Uniform(static_cast<int>(GetReadaheadSize()));
|
|
|
|
if (s && test % 2) {
|
|
|
|
Skip(n);
|
|
|
|
} else {
|
|
|
|
ASSERT_EQ(str.substr(offset, std::min(n, strLen - offset)), Read(n));
|
|
|
|
}
|
|
|
|
offset = std::min(offset + n, strLen);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-06-04 00:53:09 +02:00
|
|
|
INSTANTIATE_TEST_CASE_P(
|
2020-01-31 08:08:56 +01:00
|
|
|
EmptySourceStr, ReadaheadSequentialFileTest,
|
2019-07-17 03:18:07 +02:00
|
|
|
::testing::ValuesIn(ReadaheadSequentialFileTest::GetReadaheadSizeList()));
|
2020-06-04 00:53:09 +02:00
|
|
|
INSTANTIATE_TEST_CASE_P(
|
2020-01-31 08:08:56 +01:00
|
|
|
SourceStrLenLessThanReadaheadSize, ReadaheadSequentialFileTest,
|
2019-07-17 03:18:07 +02:00
|
|
|
::testing::ValuesIn(ReadaheadSequentialFileTest::GetReadaheadSizeList()));
|
2020-06-04 00:53:09 +02:00
|
|
|
INSTANTIATE_TEST_CASE_P(
|
2020-01-31 08:08:56 +01:00
|
|
|
SourceStrLenGreaterThanReadaheadSize, ReadaheadSequentialFileTest,
|
2019-07-17 03:18:07 +02:00
|
|
|
::testing::ValuesIn(ReadaheadSequentialFileTest::GetReadaheadSizeList()));
|
2020-06-04 00:53:09 +02:00
|
|
|
INSTANTIATE_TEST_CASE_P(
|
2020-01-31 08:08:56 +01:00
|
|
|
ReadExceedsReadaheadSize, ReadaheadSequentialFileTest,
|
2019-07-17 03:18:07 +02:00
|
|
|
::testing::ValuesIn(ReadaheadSequentialFileTest::GetReadaheadSizeList()));
|
2021-03-10 05:10:51 +01:00
|
|
|
|
|
|
|
namespace {
|
|
|
|
std::string GenerateLine(int n) {
|
|
|
|
std::string rv;
|
|
|
|
// Multiples of 17 characters per line, for likely bad buffer alignment
|
|
|
|
for (int i = 0; i < n; ++i) {
|
|
|
|
rv.push_back(static_cast<char>('0' + (i % 10)));
|
|
|
|
rv.append("xxxxxxxxxxxxxxxx");
|
|
|
|
}
|
|
|
|
return rv;
|
|
|
|
}
|
|
|
|
} // namespace
|
|
|
|
|
|
|
|
TEST(LineFileReaderTest, LineFileReaderTest) {
|
|
|
|
const int nlines = 1000;
|
|
|
|
|
2021-09-21 17:53:03 +02:00
|
|
|
std::unique_ptr<Env> mem_env(MockEnv::Create(Env::Default()));
|
2021-03-10 05:10:51 +01:00
|
|
|
std::shared_ptr<FileSystem> fs = mem_env->GetFileSystem();
|
|
|
|
// Create an input file
|
|
|
|
{
|
|
|
|
std::unique_ptr<FSWritableFile> file;
|
|
|
|
ASSERT_OK(
|
|
|
|
fs->NewWritableFile("testfile", FileOptions(), &file, /*dbg*/ nullptr));
|
|
|
|
|
|
|
|
for (int i = 0; i < nlines; ++i) {
|
|
|
|
std::string line = GenerateLine(i);
|
|
|
|
line.push_back('\n');
|
|
|
|
ASSERT_OK(file->Append(line, IOOptions(), /*dbg*/ nullptr));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Verify with no I/O errors
|
|
|
|
{
|
|
|
|
std::unique_ptr<LineFileReader> reader;
|
|
|
|
ASSERT_OK(LineFileReader::Create(fs, "testfile", FileOptions(), &reader,
|
|
|
|
nullptr));
|
|
|
|
std::string line;
|
|
|
|
int count = 0;
|
|
|
|
while (reader->ReadLine(&line)) {
|
|
|
|
ASSERT_EQ(line, GenerateLine(count));
|
|
|
|
++count;
|
|
|
|
ASSERT_EQ(static_cast<int>(reader->GetLineNumber()), count);
|
|
|
|
}
|
|
|
|
ASSERT_OK(reader->GetStatus());
|
|
|
|
ASSERT_EQ(count, nlines);
|
|
|
|
ASSERT_EQ(static_cast<int>(reader->GetLineNumber()), count);
|
|
|
|
// And still
|
|
|
|
ASSERT_FALSE(reader->ReadLine(&line));
|
|
|
|
ASSERT_OK(reader->GetStatus());
|
|
|
|
ASSERT_EQ(static_cast<int>(reader->GetLineNumber()), count);
|
|
|
|
}
|
|
|
|
|
|
|
|
// Verify with injected I/O error
|
|
|
|
{
|
|
|
|
std::unique_ptr<LineFileReader> reader;
|
|
|
|
ASSERT_OK(LineFileReader::Create(fs, "testfile", FileOptions(), &reader,
|
|
|
|
nullptr));
|
|
|
|
std::string line;
|
|
|
|
int count = 0;
|
|
|
|
// Read part way through the file
|
|
|
|
while (count < nlines / 4) {
|
|
|
|
ASSERT_TRUE(reader->ReadLine(&line));
|
|
|
|
ASSERT_EQ(line, GenerateLine(count));
|
|
|
|
++count;
|
|
|
|
ASSERT_EQ(static_cast<int>(reader->GetLineNumber()), count);
|
|
|
|
}
|
|
|
|
ASSERT_OK(reader->GetStatus());
|
|
|
|
|
|
|
|
// Inject error
|
|
|
|
int callback_count = 0;
|
|
|
|
SyncPoint::GetInstance()->SetCallBack(
|
|
|
|
"MemFile::Read:IOStatus", [&](void* arg) {
|
|
|
|
IOStatus* status = static_cast<IOStatus*>(arg);
|
|
|
|
*status = IOStatus::Corruption("test");
|
|
|
|
++callback_count;
|
|
|
|
});
|
|
|
|
SyncPoint::GetInstance()->EnableProcessing();
|
|
|
|
|
|
|
|
while (reader->ReadLine(&line)) {
|
|
|
|
ASSERT_EQ(line, GenerateLine(count));
|
|
|
|
++count;
|
|
|
|
ASSERT_EQ(static_cast<int>(reader->GetLineNumber()), count);
|
|
|
|
}
|
|
|
|
ASSERT_TRUE(reader->GetStatus().IsCorruption());
|
|
|
|
ASSERT_LT(count, nlines / 2);
|
|
|
|
ASSERT_EQ(callback_count, 1);
|
|
|
|
|
|
|
|
// Still get error & no retry
|
|
|
|
ASSERT_FALSE(reader->ReadLine(&line));
|
|
|
|
ASSERT_TRUE(reader->GetStatus().IsCorruption());
|
|
|
|
ASSERT_EQ(callback_count, 1);
|
|
|
|
|
|
|
|
SyncPoint::GetInstance()->DisableProcessing();
|
|
|
|
SyncPoint::GetInstance()->ClearAllCallBacks();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-11-19 02:09:54 +01:00
|
|
|
#ifndef ROCKSDB_LITE
|
|
|
|
class IOErrorEventListener : public EventListener {
|
|
|
|
public:
|
|
|
|
IOErrorEventListener() { notify_error_.store(0); }
|
|
|
|
|
|
|
|
void OnIOError(const IOErrorInfo& io_error_info) override {
|
|
|
|
notify_error_++;
|
|
|
|
EXPECT_FALSE(io_error_info.file_path.empty());
|
|
|
|
EXPECT_FALSE(io_error_info.io_status.ok());
|
|
|
|
}
|
|
|
|
|
|
|
|
size_t NotifyErrorCount() { return notify_error_; }
|
|
|
|
|
|
|
|
bool ShouldBeNotifiedOnFileIO() override { return true; }
|
|
|
|
|
|
|
|
private:
|
|
|
|
std::atomic<size_t> notify_error_;
|
|
|
|
};
|
|
|
|
|
|
|
|
TEST_F(DBWritableFileWriterTest, IOErrorNotification) {
|
|
|
|
class FakeWF : public FSWritableFile {
|
|
|
|
public:
|
|
|
|
explicit FakeWF() : io_error_(false) {
|
|
|
|
file_append_errors_.store(0);
|
|
|
|
file_flush_errors_.store(0);
|
|
|
|
}
|
|
|
|
|
|
|
|
using FSWritableFile::Append;
|
|
|
|
IOStatus Append(const Slice& /*data*/, const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
|
|
|
if (io_error_) {
|
|
|
|
file_append_errors_++;
|
|
|
|
return IOStatus::IOError("Fake IO error");
|
|
|
|
}
|
|
|
|
return IOStatus::OK();
|
|
|
|
}
|
|
|
|
|
|
|
|
using FSWritableFile::PositionedAppend;
|
|
|
|
IOStatus PositionedAppend(const Slice& /*data*/, uint64_t,
|
|
|
|
const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
|
|
|
if (io_error_) {
|
|
|
|
return IOStatus::IOError("Fake IO error");
|
|
|
|
}
|
|
|
|
return IOStatus::OK();
|
|
|
|
}
|
|
|
|
IOStatus Close(const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
|
|
|
return IOStatus::OK();
|
|
|
|
}
|
|
|
|
IOStatus Flush(const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
|
|
|
if (io_error_) {
|
|
|
|
file_flush_errors_++;
|
|
|
|
return IOStatus::IOError("Fake IO error");
|
|
|
|
}
|
|
|
|
return IOStatus::OK();
|
|
|
|
}
|
|
|
|
IOStatus Sync(const IOOptions& /*options*/,
|
|
|
|
IODebugContext* /*dbg*/) override {
|
|
|
|
return IOStatus::OK();
|
|
|
|
}
|
|
|
|
|
|
|
|
void SetIOError(bool val) { io_error_ = val; }
|
|
|
|
|
|
|
|
void CheckCounters(int file_append_errors, int file_flush_errors) {
|
|
|
|
ASSERT_EQ(file_append_errors, file_append_errors_);
|
|
|
|
ASSERT_EQ(file_flush_errors_, file_flush_errors);
|
|
|
|
}
|
|
|
|
|
|
|
|
protected:
|
|
|
|
bool io_error_;
|
|
|
|
std::atomic<size_t> file_append_errors_;
|
|
|
|
std::atomic<size_t> file_flush_errors_;
|
|
|
|
};
|
|
|
|
|
|
|
|
FileOptions file_options = FileOptions();
|
|
|
|
Options options = GetDefaultOptions();
|
|
|
|
options.create_if_missing = true;
|
|
|
|
IOErrorEventListener* listener = new IOErrorEventListener();
|
|
|
|
options.listeners.emplace_back(listener);
|
|
|
|
|
|
|
|
DestroyAndReopen(options);
|
|
|
|
ImmutableOptions ioptions(options);
|
|
|
|
|
2022-01-28 04:29:17 +01:00
|
|
|
std::string fname = dbname_ + "/test_file";
|
2021-11-19 02:09:54 +01:00
|
|
|
std::unique_ptr<FakeWF> writable_file_ptr(new FakeWF);
|
|
|
|
|
|
|
|
std::unique_ptr<WritableFileWriter> file_writer;
|
|
|
|
writable_file_ptr->SetIOError(true);
|
|
|
|
|
|
|
|
file_writer.reset(new WritableFileWriter(
|
|
|
|
std::move(writable_file_ptr), fname, file_options,
|
|
|
|
SystemClock::Default().get(), nullptr, ioptions.stats, ioptions.listeners,
|
|
|
|
ioptions.file_checksum_gen_factory.get(), true, true));
|
|
|
|
|
|
|
|
FakeWF* fwf = static_cast<FakeWF*>(file_writer->writable_file());
|
|
|
|
|
|
|
|
fwf->SetIOError(true);
|
|
|
|
ASSERT_NOK(file_writer->Append(std::string(2 * kMb, 'a')));
|
|
|
|
fwf->CheckCounters(1, 0);
|
|
|
|
ASSERT_EQ(listener->NotifyErrorCount(), 1);
|
|
|
|
|
|
|
|
fwf->SetIOError(true);
|
|
|
|
ASSERT_NOK(file_writer->Flush());
|
|
|
|
fwf->CheckCounters(1, 1);
|
|
|
|
ASSERT_EQ(listener->NotifyErrorCount(), 2);
|
|
|
|
|
|
|
|
/* No error generation */
|
|
|
|
fwf->SetIOError(false);
|
|
|
|
ASSERT_OK(file_writer->Append(std::string(2 * kMb, 'b')));
|
|
|
|
ASSERT_EQ(listener->NotifyErrorCount(), 2);
|
|
|
|
fwf->CheckCounters(1, 1);
|
|
|
|
}
|
|
|
|
#endif // ROCKSDB_LITE
|
2020-02-20 21:07:53 +01:00
|
|
|
} // namespace ROCKSDB_NAMESPACE
|
RangeSync not to sync last 1MB of the file
Summary:
From other ones' investigation:
"sync_file_range() behavior highly depends on kernel version and filesystem.
xfs does neighbor page flushing outside of the specified ranges. For example, sync_file_range(fd, 8192, 16384) does not only trigger flushing page #3 to #4, but also flushing many more dirty pages (i.e. up to page#16)... Ranges of the sync_file_range() should be far enough from write() offset (at least 1MB)."
Test Plan: make all check
Reviewers: igor, rven, kradhakrishnan, yhchiang, IslamAbdelRahman, anthony
Reviewed By: anthony
Subscribers: yoshinorim, MarkCallaghan, sumeet, domas, dhruba, leveldb, ljin
Differential Revision: https://reviews.facebook.net/D15807
2015-07-20 23:46:15 +02:00
|
|
|
|
|
|
|
int main(int argc, char** argv) {
|
|
|
|
::testing::InitGoogleTest(&argc, argv);
|
|
|
|
return RUN_ALL_TESTS();
|
|
|
|
}
|