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).
|
2013-10-16 23:59:46 +02:00
|
|
|
//
|
2011-03-18 23:37:00 +01:00
|
|
|
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
|
|
|
|
// Use of this source code is governed by a BSD-style license that can be
|
|
|
|
// found in the LICENSE file. See the AUTHORS file for names of contributors.
|
|
|
|
|
|
|
|
#include "db/log_reader.h"
|
|
|
|
#include "db/log_writer.h"
|
2013-08-23 17:38:13 +02:00
|
|
|
#include "rocksdb/env.h"
|
2011-03-18 23:37:00 +01:00
|
|
|
#include "util/coding.h"
|
|
|
|
#include "util/crc32c.h"
|
Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env
Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.
Test Plan: Run all existing unit tests.
Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D42321
2015-07-18 01:16:11 +02:00
|
|
|
#include "util/file_reader_writer.h"
|
2011-03-18 23:37:00 +01:00
|
|
|
#include "util/random.h"
|
|
|
|
#include "util/testharness.h"
|
Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env
Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.
Test Plan: Run all existing unit tests.
Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D42321
2015-07-18 01:16:11 +02:00
|
|
|
#include "util/testutil.h"
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2013-10-04 06:49:15 +02:00
|
|
|
namespace rocksdb {
|
2011-03-18 23:37:00 +01:00
|
|
|
namespace log {
|
|
|
|
|
|
|
|
// Construct a string of the specified length made out of the supplied
|
|
|
|
// partial string.
|
|
|
|
static std::string BigString(const std::string& partial_string, size_t n) {
|
|
|
|
std::string result;
|
|
|
|
while (result.size() < n) {
|
|
|
|
result.append(partial_string);
|
|
|
|
}
|
|
|
|
result.resize(n);
|
|
|
|
return result;
|
|
|
|
}
|
|
|
|
|
|
|
|
// Construct a string from a number
|
|
|
|
static std::string NumberString(int n) {
|
|
|
|
char buf[50];
|
|
|
|
snprintf(buf, sizeof(buf), "%d.", n);
|
|
|
|
return std::string(buf);
|
|
|
|
}
|
|
|
|
|
|
|
|
// Return a skewed potentially long string
|
|
|
|
static std::string RandomSkewedString(int i, Random* rnd) {
|
|
|
|
return BigString(NumberString(i), rnd->Skewed(17));
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
class LogTest : public ::testing::TestWithParam<int> {
|
2011-03-18 23:37:00 +01:00
|
|
|
private:
|
|
|
|
class StringSource : public SequentialFile {
|
|
|
|
public:
|
Fix UnmarkEOF for partial blocks
Summary:
Blocks in the transaction log are a fixed size, but the last block in the transaction log file is usually a partial block. When a new record is added after the reader hit the end of the file, a new physical record will be appended to the last block. ReadPhysicalRecord can only read full blocks and assumes that the file position indicator is aligned to the start of a block. If the reader is forced to read further by simply clearing the EOF flag, ReadPhysicalRecord will read a full block starting from somewhere in the middle of a real block, causing it to lose alignment and to have a partial physical record at the end of the read buffer. This will result in length mismatches and checksum failures. When the log file is tailed for replication this will cause the log iterator to become invalid, necessitating the creation of a new iterator which will have to read the log file from scratch.
This diff fixes this issue by reading the remaining portion of the last block we read from. This is done when the reader is forced to read further (UnmarkEOF is called).
Test Plan:
- Added unit tests
- Stress test (with replication). Check dbdir/LOG file for corruptions.
- Test on test tier
Reviewers: emayanke, haobo, dhruba
Reviewed By: haobo
CC: vamsi, sheki, dhruba, kailiu, igor
Differential Revision: https://reviews.facebook.net/D15249
2014-01-27 23:49:10 +01:00
|
|
|
Slice& contents_;
|
2011-03-18 23:37:00 +01:00
|
|
|
bool force_error_;
|
Fix UnmarkEOF for partial blocks
Summary:
Blocks in the transaction log are a fixed size, but the last block in the transaction log file is usually a partial block. When a new record is added after the reader hit the end of the file, a new physical record will be appended to the last block. ReadPhysicalRecord can only read full blocks and assumes that the file position indicator is aligned to the start of a block. If the reader is forced to read further by simply clearing the EOF flag, ReadPhysicalRecord will read a full block starting from somewhere in the middle of a real block, causing it to lose alignment and to have a partial physical record at the end of the read buffer. This will result in length mismatches and checksum failures. When the log file is tailed for replication this will cause the log iterator to become invalid, necessitating the creation of a new iterator which will have to read the log file from scratch.
This diff fixes this issue by reading the remaining portion of the last block we read from. This is done when the reader is forced to read further (UnmarkEOF is called).
Test Plan:
- Added unit tests
- Stress test (with replication). Check dbdir/LOG file for corruptions.
- Test on test tier
Reviewers: emayanke, haobo, dhruba
Reviewed By: haobo
CC: vamsi, sheki, dhruba, kailiu, igor
Differential Revision: https://reviews.facebook.net/D15249
2014-01-27 23:49:10 +01:00
|
|
|
size_t force_error_position_;
|
|
|
|
bool force_eof_;
|
|
|
|
size_t force_eof_position_;
|
2011-03-18 23:37:00 +01:00
|
|
|
bool returned_partial_;
|
Fix UnmarkEOF for partial blocks
Summary:
Blocks in the transaction log are a fixed size, but the last block in the transaction log file is usually a partial block. When a new record is added after the reader hit the end of the file, a new physical record will be appended to the last block. ReadPhysicalRecord can only read full blocks and assumes that the file position indicator is aligned to the start of a block. If the reader is forced to read further by simply clearing the EOF flag, ReadPhysicalRecord will read a full block starting from somewhere in the middle of a real block, causing it to lose alignment and to have a partial physical record at the end of the read buffer. This will result in length mismatches and checksum failures. When the log file is tailed for replication this will cause the log iterator to become invalid, necessitating the creation of a new iterator which will have to read the log file from scratch.
This diff fixes this issue by reading the remaining portion of the last block we read from. This is done when the reader is forced to read further (UnmarkEOF is called).
Test Plan:
- Added unit tests
- Stress test (with replication). Check dbdir/LOG file for corruptions.
- Test on test tier
Reviewers: emayanke, haobo, dhruba
Reviewed By: haobo
CC: vamsi, sheki, dhruba, kailiu, igor
Differential Revision: https://reviews.facebook.net/D15249
2014-01-27 23:49:10 +01:00
|
|
|
explicit StringSource(Slice& contents) :
|
|
|
|
contents_(contents),
|
|
|
|
force_error_(false),
|
|
|
|
force_error_position_(0),
|
|
|
|
force_eof_(false),
|
|
|
|
force_eof_position_(0),
|
|
|
|
returned_partial_(false) { }
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2015-02-26 20:28:41 +01:00
|
|
|
virtual Status Read(size_t n, Slice* result, char* scratch) override {
|
rocksdb: Replace ASSERT* with EXPECT* in functions that does not return void value
Summary:
gtest does not use exceptions to fail a unit test by design, and `ASSERT*`s are implemented using `return`. As a consequence we cannot use `ASSERT*` in a function that does not return `void` value ([[ https://code.google.com/p/googletest/wiki/AdvancedGuide#Assertion_Placement | 1]]), and have to fix our existing code. This diff does this in a generic way, with no manual changes.
In order to detect all existing `ASSERT*` that are used in functions that doesn't return void value, I change the code to generate compile errors for such cases.
In `util/testharness.h` I defined `EXPECT*` assertions, the same way as `ASSERT*`, and redefined `ASSERT*` to return `void`. Then executed:
```lang=bash
% USE_CLANG=1 make all -j55 -k 2> build.log
% perl -naF: -e 'print "-- -number=".$F[1]." ".$F[0]."\n" if /: error:/' \
build.log | xargs -L 1 perl -spi -e 's/ASSERT/EXPECT/g if $. == $number'
% make format
```
After that I reverted back change to `ASSERT*` in `util/testharness.h`. But preserved introduced `EXPECT*`, which is the same as `ASSERT*`. This will be deleted once switched to gtest.
This diff is independent and contains manual changes only in `util/testharness.h`.
Test Plan:
Make sure all tests are passing.
```lang=bash
% USE_CLANG=1 make check
```
Reviewers: igor, lgalanis, sdong, yufei.zhu, rven, meyering
Reviewed By: meyering
Subscribers: dhruba, leveldb
Differential Revision: https://reviews.facebook.net/D33333
2015-03-17 04:52:32 +01:00
|
|
|
EXPECT_TRUE(!returned_partial_) << "must not Read() after eof/error";
|
2011-03-18 23:37:00 +01:00
|
|
|
|
|
|
|
if (force_error_) {
|
Fix UnmarkEOF for partial blocks
Summary:
Blocks in the transaction log are a fixed size, but the last block in the transaction log file is usually a partial block. When a new record is added after the reader hit the end of the file, a new physical record will be appended to the last block. ReadPhysicalRecord can only read full blocks and assumes that the file position indicator is aligned to the start of a block. If the reader is forced to read further by simply clearing the EOF flag, ReadPhysicalRecord will read a full block starting from somewhere in the middle of a real block, causing it to lose alignment and to have a partial physical record at the end of the read buffer. This will result in length mismatches and checksum failures. When the log file is tailed for replication this will cause the log iterator to become invalid, necessitating the creation of a new iterator which will have to read the log file from scratch.
This diff fixes this issue by reading the remaining portion of the last block we read from. This is done when the reader is forced to read further (UnmarkEOF is called).
Test Plan:
- Added unit tests
- Stress test (with replication). Check dbdir/LOG file for corruptions.
- Test on test tier
Reviewers: emayanke, haobo, dhruba
Reviewed By: haobo
CC: vamsi, sheki, dhruba, kailiu, igor
Differential Revision: https://reviews.facebook.net/D15249
2014-01-27 23:49:10 +01:00
|
|
|
if (force_error_position_ >= n) {
|
|
|
|
force_error_position_ -= n;
|
|
|
|
} else {
|
|
|
|
*result = Slice(contents_.data(), force_error_position_);
|
|
|
|
contents_.remove_prefix(force_error_position_);
|
|
|
|
force_error_ = false;
|
|
|
|
returned_partial_ = true;
|
|
|
|
return Status::Corruption("read error");
|
|
|
|
}
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
if (contents_.size() < n) {
|
|
|
|
n = contents_.size();
|
|
|
|
returned_partial_ = true;
|
|
|
|
}
|
Fix UnmarkEOF for partial blocks
Summary:
Blocks in the transaction log are a fixed size, but the last block in the transaction log file is usually a partial block. When a new record is added after the reader hit the end of the file, a new physical record will be appended to the last block. ReadPhysicalRecord can only read full blocks and assumes that the file position indicator is aligned to the start of a block. If the reader is forced to read further by simply clearing the EOF flag, ReadPhysicalRecord will read a full block starting from somewhere in the middle of a real block, causing it to lose alignment and to have a partial physical record at the end of the read buffer. This will result in length mismatches and checksum failures. When the log file is tailed for replication this will cause the log iterator to become invalid, necessitating the creation of a new iterator which will have to read the log file from scratch.
This diff fixes this issue by reading the remaining portion of the last block we read from. This is done when the reader is forced to read further (UnmarkEOF is called).
Test Plan:
- Added unit tests
- Stress test (with replication). Check dbdir/LOG file for corruptions.
- Test on test tier
Reviewers: emayanke, haobo, dhruba
Reviewed By: haobo
CC: vamsi, sheki, dhruba, kailiu, igor
Differential Revision: https://reviews.facebook.net/D15249
2014-01-27 23:49:10 +01:00
|
|
|
|
|
|
|
if (force_eof_) {
|
|
|
|
if (force_eof_position_ >= n) {
|
|
|
|
force_eof_position_ -= n;
|
|
|
|
} else {
|
|
|
|
force_eof_ = false;
|
|
|
|
n = force_eof_position_;
|
|
|
|
returned_partial_ = true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// By using scratch we ensure that caller has control over the
|
|
|
|
// lifetime of result.data()
|
|
|
|
memcpy(scratch, contents_.data(), n);
|
|
|
|
*result = Slice(scratch, n);
|
|
|
|
|
2011-03-18 23:37:00 +01:00
|
|
|
contents_.remove_prefix(n);
|
|
|
|
return Status::OK();
|
|
|
|
}
|
2011-05-21 04:17:43 +02:00
|
|
|
|
2015-02-26 20:28:41 +01:00
|
|
|
virtual Status Skip(uint64_t n) override {
|
2011-05-21 04:17:43 +02:00
|
|
|
if (n > contents_.size()) {
|
|
|
|
contents_.clear();
|
|
|
|
return Status::NotFound("in-memory file skipepd past end");
|
|
|
|
}
|
|
|
|
|
|
|
|
contents_.remove_prefix(n);
|
|
|
|
|
|
|
|
return Status::OK();
|
|
|
|
}
|
2011-03-18 23:37:00 +01:00
|
|
|
};
|
|
|
|
|
|
|
|
class ReportCollector : public Reader::Reporter {
|
|
|
|
public:
|
|
|
|
size_t dropped_bytes_;
|
|
|
|
std::string message_;
|
|
|
|
|
|
|
|
ReportCollector() : dropped_bytes_(0) { }
|
2015-02-26 20:28:41 +01:00
|
|
|
virtual void Corruption(size_t bytes, const Status& status) override {
|
2011-03-18 23:37:00 +01:00
|
|
|
dropped_bytes_ += bytes;
|
|
|
|
message_.append(status.ToString());
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2013-01-20 11:07:13 +01:00
|
|
|
std::string& dest_contents() {
|
2015-08-05 16:33:27 +02:00
|
|
|
auto dest =
|
|
|
|
dynamic_cast<test::StringSink*>(writer_.file()->writable_file());
|
2013-01-20 11:07:13 +01:00
|
|
|
assert(dest);
|
|
|
|
return dest->contents_;
|
|
|
|
}
|
|
|
|
|
|
|
|
const std::string& dest_contents() const {
|
Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env
Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.
Test Plan: Run all existing unit tests.
Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D42321
2015-07-18 01:16:11 +02:00
|
|
|
auto dest =
|
2015-08-05 16:33:27 +02:00
|
|
|
dynamic_cast<const test::StringSink*>(writer_.file()->writable_file());
|
2013-01-20 11:07:13 +01:00
|
|
|
assert(dest);
|
|
|
|
return dest->contents_;
|
|
|
|
}
|
|
|
|
|
|
|
|
void reset_source_contents() {
|
Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env
Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.
Test Plan: Run all existing unit tests.
Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D42321
2015-07-18 01:16:11 +02:00
|
|
|
auto src = dynamic_cast<StringSource*>(reader_.file()->file());
|
2013-01-20 11:07:13 +01:00
|
|
|
assert(src);
|
|
|
|
src->contents_ = dest_contents();
|
|
|
|
}
|
|
|
|
|
Fix UnmarkEOF for partial blocks
Summary:
Blocks in the transaction log are a fixed size, but the last block in the transaction log file is usually a partial block. When a new record is added after the reader hit the end of the file, a new physical record will be appended to the last block. ReadPhysicalRecord can only read full blocks and assumes that the file position indicator is aligned to the start of a block. If the reader is forced to read further by simply clearing the EOF flag, ReadPhysicalRecord will read a full block starting from somewhere in the middle of a real block, causing it to lose alignment and to have a partial physical record at the end of the read buffer. This will result in length mismatches and checksum failures. When the log file is tailed for replication this will cause the log iterator to become invalid, necessitating the creation of a new iterator which will have to read the log file from scratch.
This diff fixes this issue by reading the remaining portion of the last block we read from. This is done when the reader is forced to read further (UnmarkEOF is called).
Test Plan:
- Added unit tests
- Stress test (with replication). Check dbdir/LOG file for corruptions.
- Test on test tier
Reviewers: emayanke, haobo, dhruba
Reviewed By: haobo
CC: vamsi, sheki, dhruba, kailiu, igor
Differential Revision: https://reviews.facebook.net/D15249
2014-01-27 23:49:10 +01:00
|
|
|
Slice reader_contents_;
|
Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env
Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.
Test Plan: Run all existing unit tests.
Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D42321
2015-07-18 01:16:11 +02:00
|
|
|
unique_ptr<WritableFileWriter> dest_holder_;
|
|
|
|
unique_ptr<SequentialFileReader> source_holder_;
|
2011-03-18 23:37:00 +01:00
|
|
|
ReportCollector report_;
|
|
|
|
Writer writer_;
|
|
|
|
Reader reader_;
|
|
|
|
|
2011-05-21 04:17:43 +02:00
|
|
|
// Record metadata for testing initial offset functionality
|
|
|
|
static size_t initial_offset_record_sizes_[];
|
2015-10-19 23:24:05 +02:00
|
|
|
uint64_t initial_offset_last_record_offsets_[4];
|
2011-05-21 04:17:43 +02:00
|
|
|
|
2011-03-18 23:37:00 +01:00
|
|
|
public:
|
Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env
Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.
Test Plan: Run all existing unit tests.
Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D42321
2015-07-18 01:16:11 +02:00
|
|
|
LogTest()
|
|
|
|
: reader_contents_(),
|
2015-10-08 19:07:15 +02:00
|
|
|
dest_holder_(test::GetWritableFileWriter(
|
|
|
|
new test::StringSink(&reader_contents_))),
|
Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env
Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.
Test Plan: Run all existing unit tests.
Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D42321
2015-07-18 01:16:11 +02:00
|
|
|
source_holder_(
|
|
|
|
test::GetSequentialFileReader(new StringSource(reader_contents_))),
|
2015-10-08 19:07:15 +02:00
|
|
|
writer_(std::move(dest_holder_), 123, GetParam()),
|
2015-10-19 23:24:05 +02:00
|
|
|
reader_(NULL, std::move(source_holder_), &report_, true /*checksum*/,
|
|
|
|
0 /*initial_offset*/, 123) {
|
|
|
|
int header_size = GetParam() ? kRecyclableHeaderSize : kHeaderSize;
|
|
|
|
initial_offset_last_record_offsets_[0] = 0;
|
|
|
|
initial_offset_last_record_offsets_[1] = header_size + 10000;
|
|
|
|
initial_offset_last_record_offsets_[2] = 2 * (header_size + 10000);
|
|
|
|
initial_offset_last_record_offsets_[3] = 2 * (header_size + 10000) +
|
|
|
|
(2 * log::kBlockSize - 1000) +
|
|
|
|
3 * header_size;
|
|
|
|
}
|
2011-03-18 23:37:00 +01:00
|
|
|
|
2015-12-11 20:12:03 +01:00
|
|
|
Slice* get_reader_contents() { return &reader_contents_; }
|
|
|
|
|
2011-03-18 23:37:00 +01:00
|
|
|
void Write(const std::string& msg) {
|
|
|
|
writer_.AddRecord(Slice(msg));
|
|
|
|
}
|
|
|
|
|
|
|
|
size_t WrittenBytes() const {
|
2013-01-20 11:07:13 +01:00
|
|
|
return dest_contents().size();
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2015-10-02 21:53:59 +02:00
|
|
|
std::string Read(const WALRecoveryMode wal_recovery_mode =
|
|
|
|
WALRecoveryMode::kTolerateCorruptedTailRecords) {
|
2011-03-18 23:37:00 +01:00
|
|
|
std::string scratch;
|
|
|
|
Slice record;
|
2015-10-02 21:53:59 +02:00
|
|
|
if (reader_.ReadRecord(&record, &scratch, wal_recovery_mode)) {
|
2011-03-18 23:37:00 +01:00
|
|
|
return record.ToString();
|
|
|
|
} else {
|
|
|
|
return "EOF";
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-10-19 19:48:47 +02:00
|
|
|
void IncrementByte(int offset, char delta) {
|
2013-01-20 11:07:13 +01:00
|
|
|
dest_contents()[offset] += delta;
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
void SetByte(int offset, char new_byte) {
|
2013-01-20 11:07:13 +01:00
|
|
|
dest_contents()[offset] = new_byte;
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
void ShrinkSize(int bytes) {
|
2015-08-05 16:33:27 +02:00
|
|
|
auto dest =
|
|
|
|
dynamic_cast<test::StringSink*>(writer_.file()->writable_file());
|
Fix UnmarkEOF for partial blocks
Summary:
Blocks in the transaction log are a fixed size, but the last block in the transaction log file is usually a partial block. When a new record is added after the reader hit the end of the file, a new physical record will be appended to the last block. ReadPhysicalRecord can only read full blocks and assumes that the file position indicator is aligned to the start of a block. If the reader is forced to read further by simply clearing the EOF flag, ReadPhysicalRecord will read a full block starting from somewhere in the middle of a real block, causing it to lose alignment and to have a partial physical record at the end of the read buffer. This will result in length mismatches and checksum failures. When the log file is tailed for replication this will cause the log iterator to become invalid, necessitating the creation of a new iterator which will have to read the log file from scratch.
This diff fixes this issue by reading the remaining portion of the last block we read from. This is done when the reader is forced to read further (UnmarkEOF is called).
Test Plan:
- Added unit tests
- Stress test (with replication). Check dbdir/LOG file for corruptions.
- Test on test tier
Reviewers: emayanke, haobo, dhruba
Reviewed By: haobo
CC: vamsi, sheki, dhruba, kailiu, igor
Differential Revision: https://reviews.facebook.net/D15249
2014-01-27 23:49:10 +01:00
|
|
|
assert(dest);
|
|
|
|
dest->Drop(bytes);
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2015-10-19 23:24:05 +02:00
|
|
|
void FixChecksum(int header_offset, int len, bool recyclable) {
|
2011-03-18 23:37:00 +01:00
|
|
|
// Compute crc of type/len/data
|
2015-10-19 23:24:05 +02:00
|
|
|
int header_size = recyclable ? kRecyclableHeaderSize : kHeaderSize;
|
|
|
|
uint32_t crc = crc32c::Value(&dest_contents()[header_offset + 6],
|
|
|
|
header_size - 6 + len);
|
2011-03-18 23:37:00 +01:00
|
|
|
crc = crc32c::Mask(crc);
|
2013-01-20 11:07:13 +01:00
|
|
|
EncodeFixed32(&dest_contents()[header_offset], crc);
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
Fix UnmarkEOF for partial blocks
Summary:
Blocks in the transaction log are a fixed size, but the last block in the transaction log file is usually a partial block. When a new record is added after the reader hit the end of the file, a new physical record will be appended to the last block. ReadPhysicalRecord can only read full blocks and assumes that the file position indicator is aligned to the start of a block. If the reader is forced to read further by simply clearing the EOF flag, ReadPhysicalRecord will read a full block starting from somewhere in the middle of a real block, causing it to lose alignment and to have a partial physical record at the end of the read buffer. This will result in length mismatches and checksum failures. When the log file is tailed for replication this will cause the log iterator to become invalid, necessitating the creation of a new iterator which will have to read the log file from scratch.
This diff fixes this issue by reading the remaining portion of the last block we read from. This is done when the reader is forced to read further (UnmarkEOF is called).
Test Plan:
- Added unit tests
- Stress test (with replication). Check dbdir/LOG file for corruptions.
- Test on test tier
Reviewers: emayanke, haobo, dhruba
Reviewed By: haobo
CC: vamsi, sheki, dhruba, kailiu, igor
Differential Revision: https://reviews.facebook.net/D15249
2014-01-27 23:49:10 +01:00
|
|
|
void ForceError(size_t position = 0) {
|
Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env
Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.
Test Plan: Run all existing unit tests.
Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D42321
2015-07-18 01:16:11 +02:00
|
|
|
auto src = dynamic_cast<StringSource*>(reader_.file()->file());
|
2013-01-20 11:07:13 +01:00
|
|
|
src->force_error_ = true;
|
Fix UnmarkEOF for partial blocks
Summary:
Blocks in the transaction log are a fixed size, but the last block in the transaction log file is usually a partial block. When a new record is added after the reader hit the end of the file, a new physical record will be appended to the last block. ReadPhysicalRecord can only read full blocks and assumes that the file position indicator is aligned to the start of a block. If the reader is forced to read further by simply clearing the EOF flag, ReadPhysicalRecord will read a full block starting from somewhere in the middle of a real block, causing it to lose alignment and to have a partial physical record at the end of the read buffer. This will result in length mismatches and checksum failures. When the log file is tailed for replication this will cause the log iterator to become invalid, necessitating the creation of a new iterator which will have to read the log file from scratch.
This diff fixes this issue by reading the remaining portion of the last block we read from. This is done when the reader is forced to read further (UnmarkEOF is called).
Test Plan:
- Added unit tests
- Stress test (with replication). Check dbdir/LOG file for corruptions.
- Test on test tier
Reviewers: emayanke, haobo, dhruba
Reviewed By: haobo
CC: vamsi, sheki, dhruba, kailiu, igor
Differential Revision: https://reviews.facebook.net/D15249
2014-01-27 23:49:10 +01:00
|
|
|
src->force_error_position_ = position;
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
size_t DroppedBytes() const {
|
|
|
|
return report_.dropped_bytes_;
|
|
|
|
}
|
|
|
|
|
2011-05-21 04:17:43 +02:00
|
|
|
std::string ReportMessage() const {
|
|
|
|
return report_.message_;
|
|
|
|
}
|
|
|
|
|
Fix UnmarkEOF for partial blocks
Summary:
Blocks in the transaction log are a fixed size, but the last block in the transaction log file is usually a partial block. When a new record is added after the reader hit the end of the file, a new physical record will be appended to the last block. ReadPhysicalRecord can only read full blocks and assumes that the file position indicator is aligned to the start of a block. If the reader is forced to read further by simply clearing the EOF flag, ReadPhysicalRecord will read a full block starting from somewhere in the middle of a real block, causing it to lose alignment and to have a partial physical record at the end of the read buffer. This will result in length mismatches and checksum failures. When the log file is tailed for replication this will cause the log iterator to become invalid, necessitating the creation of a new iterator which will have to read the log file from scratch.
This diff fixes this issue by reading the remaining portion of the last block we read from. This is done when the reader is forced to read further (UnmarkEOF is called).
Test Plan:
- Added unit tests
- Stress test (with replication). Check dbdir/LOG file for corruptions.
- Test on test tier
Reviewers: emayanke, haobo, dhruba
Reviewed By: haobo
CC: vamsi, sheki, dhruba, kailiu, igor
Differential Revision: https://reviews.facebook.net/D15249
2014-01-27 23:49:10 +01:00
|
|
|
void ForceEOF(size_t position = 0) {
|
Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env
Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.
Test Plan: Run all existing unit tests.
Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D42321
2015-07-18 01:16:11 +02:00
|
|
|
auto src = dynamic_cast<StringSource*>(reader_.file()->file());
|
Fix UnmarkEOF for partial blocks
Summary:
Blocks in the transaction log are a fixed size, but the last block in the transaction log file is usually a partial block. When a new record is added after the reader hit the end of the file, a new physical record will be appended to the last block. ReadPhysicalRecord can only read full blocks and assumes that the file position indicator is aligned to the start of a block. If the reader is forced to read further by simply clearing the EOF flag, ReadPhysicalRecord will read a full block starting from somewhere in the middle of a real block, causing it to lose alignment and to have a partial physical record at the end of the read buffer. This will result in length mismatches and checksum failures. When the log file is tailed for replication this will cause the log iterator to become invalid, necessitating the creation of a new iterator which will have to read the log file from scratch.
This diff fixes this issue by reading the remaining portion of the last block we read from. This is done when the reader is forced to read further (UnmarkEOF is called).
Test Plan:
- Added unit tests
- Stress test (with replication). Check dbdir/LOG file for corruptions.
- Test on test tier
Reviewers: emayanke, haobo, dhruba
Reviewed By: haobo
CC: vamsi, sheki, dhruba, kailiu, igor
Differential Revision: https://reviews.facebook.net/D15249
2014-01-27 23:49:10 +01:00
|
|
|
src->force_eof_ = true;
|
|
|
|
src->force_eof_position_ = position;
|
|
|
|
}
|
|
|
|
|
|
|
|
void UnmarkEOF() {
|
Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env
Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.
Test Plan: Run all existing unit tests.
Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D42321
2015-07-18 01:16:11 +02:00
|
|
|
auto src = dynamic_cast<StringSource*>(reader_.file()->file());
|
Fix UnmarkEOF for partial blocks
Summary:
Blocks in the transaction log are a fixed size, but the last block in the transaction log file is usually a partial block. When a new record is added after the reader hit the end of the file, a new physical record will be appended to the last block. ReadPhysicalRecord can only read full blocks and assumes that the file position indicator is aligned to the start of a block. If the reader is forced to read further by simply clearing the EOF flag, ReadPhysicalRecord will read a full block starting from somewhere in the middle of a real block, causing it to lose alignment and to have a partial physical record at the end of the read buffer. This will result in length mismatches and checksum failures. When the log file is tailed for replication this will cause the log iterator to become invalid, necessitating the creation of a new iterator which will have to read the log file from scratch.
This diff fixes this issue by reading the remaining portion of the last block we read from. This is done when the reader is forced to read further (UnmarkEOF is called).
Test Plan:
- Added unit tests
- Stress test (with replication). Check dbdir/LOG file for corruptions.
- Test on test tier
Reviewers: emayanke, haobo, dhruba
Reviewed By: haobo
CC: vamsi, sheki, dhruba, kailiu, igor
Differential Revision: https://reviews.facebook.net/D15249
2014-01-27 23:49:10 +01:00
|
|
|
src->returned_partial_ = false;
|
|
|
|
reader_.UnmarkEOF();
|
|
|
|
}
|
|
|
|
|
|
|
|
bool IsEOF() {
|
|
|
|
return reader_.IsEOF();
|
|
|
|
}
|
|
|
|
|
2011-03-18 23:37:00 +01:00
|
|
|
// Returns OK iff recorded error message contains "msg"
|
|
|
|
std::string MatchError(const std::string& msg) const {
|
|
|
|
if (report_.message_.find(msg) == std::string::npos) {
|
|
|
|
return report_.message_;
|
|
|
|
} else {
|
|
|
|
return "OK";
|
|
|
|
}
|
|
|
|
}
|
2011-05-21 04:17:43 +02:00
|
|
|
|
|
|
|
void WriteInitialOffsetLog() {
|
|
|
|
for (int i = 0; i < 4; i++) {
|
|
|
|
std::string record(initial_offset_record_sizes_[i],
|
|
|
|
static_cast<char>('a' + i));
|
|
|
|
Write(record);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void CheckOffsetPastEndReturnsNoRecords(uint64_t offset_past_end) {
|
|
|
|
WriteInitialOffsetLog();
|
Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env
Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.
Test Plan: Run all existing unit tests.
Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D42321
2015-07-18 01:16:11 +02:00
|
|
|
unique_ptr<SequentialFileReader> file_reader(
|
|
|
|
test::GetSequentialFileReader(new StringSource(reader_contents_)));
|
2013-01-20 11:07:13 +01:00
|
|
|
unique_ptr<Reader> offset_reader(
|
2015-10-08 19:06:16 +02:00
|
|
|
new Reader(NULL, std::move(file_reader), &report_,
|
|
|
|
true /*checksum*/, WrittenBytes() + offset_past_end, 123));
|
2011-05-21 04:17:43 +02:00
|
|
|
Slice record;
|
|
|
|
std::string scratch;
|
|
|
|
ASSERT_TRUE(!offset_reader->ReadRecord(&record, &scratch));
|
|
|
|
}
|
|
|
|
|
|
|
|
void CheckInitialOffsetRecord(uint64_t initial_offset,
|
|
|
|
int expected_record_offset) {
|
|
|
|
WriteInitialOffsetLog();
|
Move rate_limiter, write buffering, most perf context instrumentation and most random kill out of Env
Summary: We want to keep Env a think layer for better portability. Less platform dependent codes should be moved out of Env. In this patch, I create a wrapper of file readers and writers, and put rate limiting, write buffering, as well as most perf context instrumentation and random kill out of Env. It will make it easier to maintain multiple Env in the future.
Test Plan: Run all existing unit tests.
Reviewers: anthony, kradhakrishnan, IslamAbdelRahman, yhchiang, igor
Reviewed By: igor
Subscribers: leveldb, dhruba
Differential Revision: https://reviews.facebook.net/D42321
2015-07-18 01:16:11 +02:00
|
|
|
unique_ptr<SequentialFileReader> file_reader(
|
|
|
|
test::GetSequentialFileReader(new StringSource(reader_contents_)));
|
2015-10-08 19:06:16 +02:00
|
|
|
unique_ptr<Reader> offset_reader(
|
|
|
|
new Reader(NULL, std::move(file_reader), &report_,
|
|
|
|
true /*checksum*/, initial_offset, 123));
|
2011-05-21 04:17:43 +02:00
|
|
|
Slice record;
|
|
|
|
std::string scratch;
|
|
|
|
ASSERT_TRUE(offset_reader->ReadRecord(&record, &scratch));
|
|
|
|
ASSERT_EQ(initial_offset_record_sizes_[expected_record_offset],
|
|
|
|
record.size());
|
|
|
|
ASSERT_EQ(initial_offset_last_record_offsets_[expected_record_offset],
|
|
|
|
offset_reader->LastRecordOffset());
|
|
|
|
ASSERT_EQ((char)('a' + expected_record_offset), record.data()[0]);
|
|
|
|
}
|
|
|
|
|
2011-03-18 23:37:00 +01:00
|
|
|
};
|
|
|
|
|
2011-05-21 04:17:43 +02:00
|
|
|
size_t LogTest::initial_offset_record_sizes_[] =
|
|
|
|
{10000, // Two sizable records in first block
|
|
|
|
10000,
|
|
|
|
2 * log::kBlockSize - 1000, // Span three blocks
|
|
|
|
1};
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, Empty) { ASSERT_EQ("EOF", Read()); }
|
2011-05-21 04:17:43 +02:00
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, ReadWrite) {
|
2011-03-18 23:37:00 +01:00
|
|
|
Write("foo");
|
|
|
|
Write("bar");
|
|
|
|
Write("");
|
|
|
|
Write("xxxx");
|
|
|
|
ASSERT_EQ("foo", Read());
|
|
|
|
ASSERT_EQ("bar", Read());
|
|
|
|
ASSERT_EQ("", Read());
|
|
|
|
ASSERT_EQ("xxxx", Read());
|
|
|
|
ASSERT_EQ("EOF", Read());
|
|
|
|
ASSERT_EQ("EOF", Read()); // Make sure reads at eof work
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, ManyBlocks) {
|
2011-03-18 23:37:00 +01:00
|
|
|
for (int i = 0; i < 100000; i++) {
|
|
|
|
Write(NumberString(i));
|
|
|
|
}
|
|
|
|
for (int i = 0; i < 100000; i++) {
|
|
|
|
ASSERT_EQ(NumberString(i), Read());
|
|
|
|
}
|
|
|
|
ASSERT_EQ("EOF", Read());
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, Fragmentation) {
|
2011-03-18 23:37:00 +01:00
|
|
|
Write("small");
|
|
|
|
Write(BigString("medium", 50000));
|
|
|
|
Write(BigString("large", 100000));
|
|
|
|
ASSERT_EQ("small", Read());
|
|
|
|
ASSERT_EQ(BigString("medium", 50000), Read());
|
|
|
|
ASSERT_EQ(BigString("large", 100000), Read());
|
|
|
|
ASSERT_EQ("EOF", Read());
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, MarginalTrailer) {
|
2011-03-18 23:37:00 +01:00
|
|
|
// Make a trailer that is exactly the same length as an empty record.
|
2015-10-19 23:24:05 +02:00
|
|
|
int header_size = GetParam() ? kRecyclableHeaderSize : kHeaderSize;
|
|
|
|
const int n = kBlockSize - 2 * header_size;
|
2011-03-18 23:37:00 +01:00
|
|
|
Write(BigString("foo", n));
|
2015-10-19 23:24:05 +02:00
|
|
|
ASSERT_EQ((unsigned int)(kBlockSize - header_size), WrittenBytes());
|
2011-03-18 23:37:00 +01:00
|
|
|
Write("");
|
|
|
|
Write("bar");
|
|
|
|
ASSERT_EQ(BigString("foo", n), Read());
|
|
|
|
ASSERT_EQ("", Read());
|
|
|
|
ASSERT_EQ("bar", Read());
|
|
|
|
ASSERT_EQ("EOF", Read());
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, MarginalTrailer2) {
|
2011-05-21 04:17:43 +02:00
|
|
|
// Make a trailer that is exactly the same length as an empty record.
|
2015-10-19 23:24:05 +02:00
|
|
|
int header_size = GetParam() ? kRecyclableHeaderSize : kHeaderSize;
|
|
|
|
const int n = kBlockSize - 2 * header_size;
|
2011-05-21 04:17:43 +02:00
|
|
|
Write(BigString("foo", n));
|
2015-10-19 23:24:05 +02:00
|
|
|
ASSERT_EQ((unsigned int)(kBlockSize - header_size), WrittenBytes());
|
2011-05-21 04:17:43 +02:00
|
|
|
Write("bar");
|
|
|
|
ASSERT_EQ(BigString("foo", n), Read());
|
|
|
|
ASSERT_EQ("bar", Read());
|
|
|
|
ASSERT_EQ("EOF", Read());
|
2012-11-06 21:02:18 +01:00
|
|
|
ASSERT_EQ(0U, DroppedBytes());
|
2011-05-21 04:17:43 +02:00
|
|
|
ASSERT_EQ("", ReportMessage());
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, ShortTrailer) {
|
2015-10-19 23:24:05 +02:00
|
|
|
int header_size = GetParam() ? kRecyclableHeaderSize : kHeaderSize;
|
|
|
|
const int n = kBlockSize - 2 * header_size + 4;
|
2011-03-18 23:37:00 +01:00
|
|
|
Write(BigString("foo", n));
|
2015-10-19 23:24:05 +02:00
|
|
|
ASSERT_EQ((unsigned int)(kBlockSize - header_size + 4), WrittenBytes());
|
2011-03-18 23:37:00 +01:00
|
|
|
Write("");
|
|
|
|
Write("bar");
|
|
|
|
ASSERT_EQ(BigString("foo", n), Read());
|
|
|
|
ASSERT_EQ("", Read());
|
|
|
|
ASSERT_EQ("bar", Read());
|
|
|
|
ASSERT_EQ("EOF", Read());
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, AlignedEof) {
|
2015-10-19 23:24:05 +02:00
|
|
|
int header_size = GetParam() ? kRecyclableHeaderSize : kHeaderSize;
|
|
|
|
const int n = kBlockSize - 2 * header_size + 4;
|
2011-03-18 23:37:00 +01:00
|
|
|
Write(BigString("foo", n));
|
2015-10-19 23:24:05 +02:00
|
|
|
ASSERT_EQ((unsigned int)(kBlockSize - header_size + 4), WrittenBytes());
|
2011-03-18 23:37:00 +01:00
|
|
|
ASSERT_EQ(BigString("foo", n), Read());
|
|
|
|
ASSERT_EQ("EOF", Read());
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, RandomRead) {
|
2011-03-18 23:37:00 +01:00
|
|
|
const int N = 500;
|
|
|
|
Random write_rnd(301);
|
|
|
|
for (int i = 0; i < N; i++) {
|
|
|
|
Write(RandomSkewedString(i, &write_rnd));
|
|
|
|
}
|
|
|
|
Random read_rnd(301);
|
|
|
|
for (int i = 0; i < N; i++) {
|
|
|
|
ASSERT_EQ(RandomSkewedString(i, &read_rnd), Read());
|
|
|
|
}
|
|
|
|
ASSERT_EQ("EOF", Read());
|
|
|
|
}
|
|
|
|
|
|
|
|
// Tests of all the error paths in log_reader.cc follow:
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, ReadError) {
|
2011-03-18 23:37:00 +01:00
|
|
|
Write("foo");
|
|
|
|
ForceError();
|
|
|
|
ASSERT_EQ("EOF", Read());
|
2012-11-06 21:02:18 +01:00
|
|
|
ASSERT_EQ((unsigned int)kBlockSize, DroppedBytes());
|
2011-03-18 23:37:00 +01:00
|
|
|
ASSERT_EQ("OK", MatchError("read error"));
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, BadRecordType) {
|
2011-03-18 23:37:00 +01:00
|
|
|
Write("foo");
|
|
|
|
// Type is stored in header[6]
|
|
|
|
IncrementByte(6, 100);
|
2015-10-19 23:24:05 +02:00
|
|
|
FixChecksum(0, 3, false);
|
2011-03-18 23:37:00 +01:00
|
|
|
ASSERT_EQ("EOF", Read());
|
2012-11-06 21:02:18 +01:00
|
|
|
ASSERT_EQ(3U, DroppedBytes());
|
2011-03-18 23:37:00 +01:00
|
|
|
ASSERT_EQ("OK", MatchError("unknown record type"));
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, TruncatedTrailingRecordIsIgnored) {
|
2011-03-18 23:37:00 +01:00
|
|
|
Write("foo");
|
|
|
|
ShrinkSize(4); // Drop all payload as well as a header byte
|
|
|
|
ASSERT_EQ("EOF", Read());
|
2014-02-28 22:19:47 +01:00
|
|
|
// Truncated last record is ignored, not treated as an error
|
2014-03-14 23:44:35 +01:00
|
|
|
ASSERT_EQ(0U, DroppedBytes());
|
2014-02-28 22:19:47 +01:00
|
|
|
ASSERT_EQ("", ReportMessage());
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, TruncatedTrailingRecordIsNotIgnored) {
|
2015-06-15 21:03:13 +02:00
|
|
|
Write("foo");
|
|
|
|
ShrinkSize(4); // Drop all payload as well as a header byte
|
2015-10-02 21:53:59 +02:00
|
|
|
ASSERT_EQ("EOF", Read(WALRecoveryMode::kAbsoluteConsistency));
|
2015-06-15 21:03:13 +02:00
|
|
|
// Truncated last record is ignored, not treated as an error
|
|
|
|
ASSERT_GT(DroppedBytes(), 0U);
|
|
|
|
ASSERT_EQ("OK", MatchError("Corruption: truncated header"));
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, BadLength) {
|
2015-10-19 23:24:05 +02:00
|
|
|
int header_size = GetParam() ? kRecyclableHeaderSize : kHeaderSize;
|
|
|
|
const int kPayloadSize = kBlockSize - header_size;
|
2014-02-28 22:19:47 +01:00
|
|
|
Write(BigString("bar", kPayloadSize));
|
|
|
|
Write("foo");
|
|
|
|
// Least significant size byte is stored in header[4].
|
|
|
|
IncrementByte(4, 1);
|
2015-12-11 23:17:43 +01:00
|
|
|
if (!GetParam()) {
|
|
|
|
ASSERT_EQ("foo", Read());
|
|
|
|
ASSERT_EQ(kBlockSize, DroppedBytes());
|
|
|
|
ASSERT_EQ("OK", MatchError("bad record length"));
|
|
|
|
} else {
|
|
|
|
ASSERT_EQ("EOF", Read());
|
|
|
|
}
|
2014-02-28 22:19:47 +01:00
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, BadLengthAtEndIsIgnored) {
|
2011-03-18 23:37:00 +01:00
|
|
|
Write("foo");
|
|
|
|
ShrinkSize(1);
|
|
|
|
ASSERT_EQ("EOF", Read());
|
2014-03-14 23:44:35 +01:00
|
|
|
ASSERT_EQ(0U, DroppedBytes());
|
2014-02-28 22:19:47 +01:00
|
|
|
ASSERT_EQ("", ReportMessage());
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, BadLengthAtEndIsNotIgnored) {
|
2015-06-15 21:03:13 +02:00
|
|
|
Write("foo");
|
|
|
|
ShrinkSize(1);
|
2015-10-02 21:53:59 +02:00
|
|
|
ASSERT_EQ("EOF", Read(WALRecoveryMode::kAbsoluteConsistency));
|
2015-06-15 21:03:13 +02:00
|
|
|
ASSERT_GT(DroppedBytes(), 0U);
|
|
|
|
ASSERT_EQ("OK", MatchError("Corruption: truncated header"));
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, ChecksumMismatch) {
|
2015-10-19 23:24:05 +02:00
|
|
|
Write("foooooo");
|
|
|
|
IncrementByte(0, 14);
|
2011-03-18 23:37:00 +01:00
|
|
|
ASSERT_EQ("EOF", Read());
|
2015-12-11 23:17:43 +01:00
|
|
|
if (!GetParam()) {
|
|
|
|
ASSERT_EQ(14U, DroppedBytes());
|
|
|
|
ASSERT_EQ("OK", MatchError("checksum mismatch"));
|
|
|
|
} else {
|
|
|
|
ASSERT_EQ(0U, DroppedBytes());
|
|
|
|
ASSERT_EQ("", ReportMessage());
|
|
|
|
}
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, UnexpectedMiddleType) {
|
2011-03-18 23:37:00 +01:00
|
|
|
Write("foo");
|
2017-10-19 19:48:47 +02:00
|
|
|
SetByte(6, static_cast<char>(GetParam() ? kRecyclableMiddleType : kMiddleType));
|
2015-10-19 23:24:05 +02:00
|
|
|
FixChecksum(0, 3, !!GetParam());
|
2011-03-18 23:37:00 +01:00
|
|
|
ASSERT_EQ("EOF", Read());
|
2012-11-06 21:02:18 +01:00
|
|
|
ASSERT_EQ(3U, DroppedBytes());
|
2011-03-18 23:37:00 +01:00
|
|
|
ASSERT_EQ("OK", MatchError("missing start"));
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, UnexpectedLastType) {
|
2011-03-18 23:37:00 +01:00
|
|
|
Write("foo");
|
2017-10-19 19:48:47 +02:00
|
|
|
SetByte(6, static_cast<char>(GetParam() ? kRecyclableLastType : kLastType));
|
2015-10-19 23:24:05 +02:00
|
|
|
FixChecksum(0, 3, !!GetParam());
|
2011-03-18 23:37:00 +01:00
|
|
|
ASSERT_EQ("EOF", Read());
|
2012-11-06 21:02:18 +01:00
|
|
|
ASSERT_EQ(3U, DroppedBytes());
|
2011-03-18 23:37:00 +01:00
|
|
|
ASSERT_EQ("OK", MatchError("missing start"));
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, UnexpectedFullType) {
|
2011-03-18 23:37:00 +01:00
|
|
|
Write("foo");
|
|
|
|
Write("bar");
|
2017-10-19 19:48:47 +02:00
|
|
|
SetByte(6, static_cast<char>(GetParam() ? kRecyclableFirstType : kFirstType));
|
2015-10-19 23:24:05 +02:00
|
|
|
FixChecksum(0, 3, !!GetParam());
|
2011-03-18 23:37:00 +01:00
|
|
|
ASSERT_EQ("bar", Read());
|
|
|
|
ASSERT_EQ("EOF", Read());
|
2012-11-06 21:02:18 +01:00
|
|
|
ASSERT_EQ(3U, DroppedBytes());
|
2011-03-18 23:37:00 +01:00
|
|
|
ASSERT_EQ("OK", MatchError("partial record without end"));
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, UnexpectedFirstType) {
|
2011-03-18 23:37:00 +01:00
|
|
|
Write("foo");
|
|
|
|
Write(BigString("bar", 100000));
|
2017-10-19 19:48:47 +02:00
|
|
|
SetByte(6, static_cast<char>(GetParam() ? kRecyclableFirstType : kFirstType));
|
2015-10-19 23:24:05 +02:00
|
|
|
FixChecksum(0, 3, !!GetParam());
|
2011-03-18 23:37:00 +01:00
|
|
|
ASSERT_EQ(BigString("bar", 100000), Read());
|
|
|
|
ASSERT_EQ("EOF", Read());
|
2012-11-06 21:02:18 +01:00
|
|
|
ASSERT_EQ(3U, DroppedBytes());
|
2011-03-18 23:37:00 +01:00
|
|
|
ASSERT_EQ("OK", MatchError("partial record without end"));
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, MissingLastIsIgnored) {
|
2014-02-28 22:19:47 +01:00
|
|
|
Write(BigString("bar", kBlockSize));
|
|
|
|
// Remove the LAST block, including header.
|
|
|
|
ShrinkSize(14);
|
|
|
|
ASSERT_EQ("EOF", Read());
|
|
|
|
ASSERT_EQ("", ReportMessage());
|
2014-03-14 23:44:35 +01:00
|
|
|
ASSERT_EQ(0U, DroppedBytes());
|
2014-02-28 22:19:47 +01:00
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, MissingLastIsNotIgnored) {
|
2015-06-15 21:03:13 +02:00
|
|
|
Write(BigString("bar", kBlockSize));
|
|
|
|
// Remove the LAST block, including header.
|
|
|
|
ShrinkSize(14);
|
2015-10-02 21:53:59 +02:00
|
|
|
ASSERT_EQ("EOF", Read(WALRecoveryMode::kAbsoluteConsistency));
|
2015-06-15 21:03:13 +02:00
|
|
|
ASSERT_GT(DroppedBytes(), 0U);
|
|
|
|
ASSERT_EQ("OK", MatchError("Corruption: error reading trailing data"));
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, PartialLastIsIgnored) {
|
2014-02-28 22:19:47 +01:00
|
|
|
Write(BigString("bar", kBlockSize));
|
|
|
|
// Cause a bad record length in the LAST block.
|
|
|
|
ShrinkSize(1);
|
|
|
|
ASSERT_EQ("EOF", Read());
|
|
|
|
ASSERT_EQ("", ReportMessage());
|
2014-03-14 23:44:35 +01:00
|
|
|
ASSERT_EQ(0U, DroppedBytes());
|
2014-02-28 22:19:47 +01:00
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, PartialLastIsNotIgnored) {
|
2015-06-15 21:03:13 +02:00
|
|
|
Write(BigString("bar", kBlockSize));
|
|
|
|
// Cause a bad record length in the LAST block.
|
|
|
|
ShrinkSize(1);
|
2015-10-02 21:53:59 +02:00
|
|
|
ASSERT_EQ("EOF", Read(WALRecoveryMode::kAbsoluteConsistency));
|
2015-06-15 21:03:13 +02:00
|
|
|
ASSERT_GT(DroppedBytes(), 0U);
|
|
|
|
ASSERT_EQ("OK", MatchError(
|
|
|
|
"Corruption: truncated headerCorruption: "
|
|
|
|
"error reading trailing data"));
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, ErrorJoinsRecords) {
|
2011-03-18 23:37:00 +01:00
|
|
|
// Consider two fragmented records:
|
|
|
|
// first(R1) last(R1) first(R2) last(R2)
|
|
|
|
// where the middle two fragments disappear. We do not want
|
|
|
|
// first(R1),last(R2) to get joined and returned as a valid record.
|
|
|
|
|
|
|
|
// Write records that span two blocks
|
|
|
|
Write(BigString("foo", kBlockSize));
|
|
|
|
Write(BigString("bar", kBlockSize));
|
|
|
|
Write("correct");
|
|
|
|
|
|
|
|
// Wipe the middle block
|
2013-03-19 22:53:22 +01:00
|
|
|
for (unsigned int offset = kBlockSize; offset < 2*kBlockSize; offset++) {
|
2011-03-18 23:37:00 +01:00
|
|
|
SetByte(offset, 'x');
|
|
|
|
}
|
|
|
|
|
2015-12-11 23:17:43 +01:00
|
|
|
if (!GetParam()) {
|
|
|
|
ASSERT_EQ("correct", Read());
|
|
|
|
ASSERT_EQ("EOF", Read());
|
|
|
|
size_t dropped = DroppedBytes();
|
|
|
|
ASSERT_LE(dropped, 2 * kBlockSize + 100);
|
|
|
|
ASSERT_GE(dropped, 2 * kBlockSize);
|
|
|
|
} else {
|
|
|
|
ASSERT_EQ("EOF", Read());
|
|
|
|
}
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, ReadStart) { CheckInitialOffsetRecord(0, 0); }
|
2011-05-21 04:17:43 +02:00
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, ReadSecondOneOff) { CheckInitialOffsetRecord(1, 1); }
|
2011-05-21 04:17:43 +02:00
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, ReadSecondTenThousand) { CheckInitialOffsetRecord(10000, 1); }
|
2011-05-21 04:17:43 +02:00
|
|
|
|
2015-10-19 23:24:05 +02:00
|
|
|
TEST_P(LogTest, ReadSecondStart) {
|
|
|
|
int header_size = GetParam() ? kRecyclableHeaderSize : kHeaderSize;
|
|
|
|
CheckInitialOffsetRecord(10000 + header_size, 1);
|
|
|
|
}
|
2011-05-21 04:17:43 +02:00
|
|
|
|
2015-10-19 23:24:05 +02:00
|
|
|
TEST_P(LogTest, ReadThirdOneOff) {
|
|
|
|
int header_size = GetParam() ? kRecyclableHeaderSize : kHeaderSize;
|
|
|
|
CheckInitialOffsetRecord(10000 + header_size + 1, 2);
|
|
|
|
}
|
2011-05-21 04:17:43 +02:00
|
|
|
|
2015-10-19 23:24:05 +02:00
|
|
|
TEST_P(LogTest, ReadThirdStart) {
|
|
|
|
int header_size = GetParam() ? kRecyclableHeaderSize : kHeaderSize;
|
|
|
|
CheckInitialOffsetRecord(20000 + 2 * header_size, 2);
|
|
|
|
}
|
2011-05-21 04:17:43 +02:00
|
|
|
|
2015-10-19 23:24:05 +02:00
|
|
|
TEST_P(LogTest, ReadFourthOneOff) {
|
|
|
|
int header_size = GetParam() ? kRecyclableHeaderSize : kHeaderSize;
|
|
|
|
CheckInitialOffsetRecord(20000 + 2 * header_size + 1, 3);
|
|
|
|
}
|
2011-05-21 04:17:43 +02:00
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, ReadFourthFirstBlockTrailer) {
|
2011-05-21 04:17:43 +02:00
|
|
|
CheckInitialOffsetRecord(log::kBlockSize - 4, 3);
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, ReadFourthMiddleBlock) {
|
2011-05-21 04:17:43 +02:00
|
|
|
CheckInitialOffsetRecord(log::kBlockSize + 1, 3);
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, ReadFourthLastBlock) {
|
2011-05-21 04:17:43 +02:00
|
|
|
CheckInitialOffsetRecord(2 * log::kBlockSize + 1, 3);
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, ReadFourthStart) {
|
2015-10-19 23:24:05 +02:00
|
|
|
int header_size = GetParam() ? kRecyclableHeaderSize : kHeaderSize;
|
2011-05-21 04:17:43 +02:00
|
|
|
CheckInitialOffsetRecord(
|
2015-10-19 23:24:05 +02:00
|
|
|
2 * (header_size + 1000) + (2 * log::kBlockSize - 1000) + 3 * header_size,
|
2011-05-21 04:17:43 +02:00
|
|
|
3);
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, ReadEnd) { CheckOffsetPastEndReturnsNoRecords(0); }
|
2011-05-21 04:17:43 +02:00
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, ReadPastEnd) { CheckOffsetPastEndReturnsNoRecords(5); }
|
2011-05-21 04:17:43 +02:00
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, ClearEofSingleBlock) {
|
Fix UnmarkEOF for partial blocks
Summary:
Blocks in the transaction log are a fixed size, but the last block in the transaction log file is usually a partial block. When a new record is added after the reader hit the end of the file, a new physical record will be appended to the last block. ReadPhysicalRecord can only read full blocks and assumes that the file position indicator is aligned to the start of a block. If the reader is forced to read further by simply clearing the EOF flag, ReadPhysicalRecord will read a full block starting from somewhere in the middle of a real block, causing it to lose alignment and to have a partial physical record at the end of the read buffer. This will result in length mismatches and checksum failures. When the log file is tailed for replication this will cause the log iterator to become invalid, necessitating the creation of a new iterator which will have to read the log file from scratch.
This diff fixes this issue by reading the remaining portion of the last block we read from. This is done when the reader is forced to read further (UnmarkEOF is called).
Test Plan:
- Added unit tests
- Stress test (with replication). Check dbdir/LOG file for corruptions.
- Test on test tier
Reviewers: emayanke, haobo, dhruba
Reviewed By: haobo
CC: vamsi, sheki, dhruba, kailiu, igor
Differential Revision: https://reviews.facebook.net/D15249
2014-01-27 23:49:10 +01:00
|
|
|
Write("foo");
|
|
|
|
Write("bar");
|
2015-10-19 23:24:05 +02:00
|
|
|
int header_size = GetParam() ? kRecyclableHeaderSize : kHeaderSize;
|
|
|
|
ForceEOF(3 + header_size + 2);
|
Fix UnmarkEOF for partial blocks
Summary:
Blocks in the transaction log are a fixed size, but the last block in the transaction log file is usually a partial block. When a new record is added after the reader hit the end of the file, a new physical record will be appended to the last block. ReadPhysicalRecord can only read full blocks and assumes that the file position indicator is aligned to the start of a block. If the reader is forced to read further by simply clearing the EOF flag, ReadPhysicalRecord will read a full block starting from somewhere in the middle of a real block, causing it to lose alignment and to have a partial physical record at the end of the read buffer. This will result in length mismatches and checksum failures. When the log file is tailed for replication this will cause the log iterator to become invalid, necessitating the creation of a new iterator which will have to read the log file from scratch.
This diff fixes this issue by reading the remaining portion of the last block we read from. This is done when the reader is forced to read further (UnmarkEOF is called).
Test Plan:
- Added unit tests
- Stress test (with replication). Check dbdir/LOG file for corruptions.
- Test on test tier
Reviewers: emayanke, haobo, dhruba
Reviewed By: haobo
CC: vamsi, sheki, dhruba, kailiu, igor
Differential Revision: https://reviews.facebook.net/D15249
2014-01-27 23:49:10 +01:00
|
|
|
ASSERT_EQ("foo", Read());
|
|
|
|
UnmarkEOF();
|
|
|
|
ASSERT_EQ("bar", Read());
|
|
|
|
ASSERT_TRUE(IsEOF());
|
|
|
|
ASSERT_EQ("EOF", Read());
|
|
|
|
Write("xxx");
|
|
|
|
UnmarkEOF();
|
|
|
|
ASSERT_EQ("xxx", Read());
|
|
|
|
ASSERT_TRUE(IsEOF());
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, ClearEofMultiBlock) {
|
Fix UnmarkEOF for partial blocks
Summary:
Blocks in the transaction log are a fixed size, but the last block in the transaction log file is usually a partial block. When a new record is added after the reader hit the end of the file, a new physical record will be appended to the last block. ReadPhysicalRecord can only read full blocks and assumes that the file position indicator is aligned to the start of a block. If the reader is forced to read further by simply clearing the EOF flag, ReadPhysicalRecord will read a full block starting from somewhere in the middle of a real block, causing it to lose alignment and to have a partial physical record at the end of the read buffer. This will result in length mismatches and checksum failures. When the log file is tailed for replication this will cause the log iterator to become invalid, necessitating the creation of a new iterator which will have to read the log file from scratch.
This diff fixes this issue by reading the remaining portion of the last block we read from. This is done when the reader is forced to read further (UnmarkEOF is called).
Test Plan:
- Added unit tests
- Stress test (with replication). Check dbdir/LOG file for corruptions.
- Test on test tier
Reviewers: emayanke, haobo, dhruba
Reviewed By: haobo
CC: vamsi, sheki, dhruba, kailiu, igor
Differential Revision: https://reviews.facebook.net/D15249
2014-01-27 23:49:10 +01:00
|
|
|
size_t num_full_blocks = 5;
|
2015-10-19 23:24:05 +02:00
|
|
|
int header_size = GetParam() ? kRecyclableHeaderSize : kHeaderSize;
|
|
|
|
size_t n = (kBlockSize - header_size) * num_full_blocks + 25;
|
Fix UnmarkEOF for partial blocks
Summary:
Blocks in the transaction log are a fixed size, but the last block in the transaction log file is usually a partial block. When a new record is added after the reader hit the end of the file, a new physical record will be appended to the last block. ReadPhysicalRecord can only read full blocks and assumes that the file position indicator is aligned to the start of a block. If the reader is forced to read further by simply clearing the EOF flag, ReadPhysicalRecord will read a full block starting from somewhere in the middle of a real block, causing it to lose alignment and to have a partial physical record at the end of the read buffer. This will result in length mismatches and checksum failures. When the log file is tailed for replication this will cause the log iterator to become invalid, necessitating the creation of a new iterator which will have to read the log file from scratch.
This diff fixes this issue by reading the remaining portion of the last block we read from. This is done when the reader is forced to read further (UnmarkEOF is called).
Test Plan:
- Added unit tests
- Stress test (with replication). Check dbdir/LOG file for corruptions.
- Test on test tier
Reviewers: emayanke, haobo, dhruba
Reviewed By: haobo
CC: vamsi, sheki, dhruba, kailiu, igor
Differential Revision: https://reviews.facebook.net/D15249
2014-01-27 23:49:10 +01:00
|
|
|
Write(BigString("foo", n));
|
|
|
|
Write(BigString("bar", n));
|
2015-10-19 23:24:05 +02:00
|
|
|
ForceEOF(n + num_full_blocks * header_size + header_size + 3);
|
Fix UnmarkEOF for partial blocks
Summary:
Blocks in the transaction log are a fixed size, but the last block in the transaction log file is usually a partial block. When a new record is added after the reader hit the end of the file, a new physical record will be appended to the last block. ReadPhysicalRecord can only read full blocks and assumes that the file position indicator is aligned to the start of a block. If the reader is forced to read further by simply clearing the EOF flag, ReadPhysicalRecord will read a full block starting from somewhere in the middle of a real block, causing it to lose alignment and to have a partial physical record at the end of the read buffer. This will result in length mismatches and checksum failures. When the log file is tailed for replication this will cause the log iterator to become invalid, necessitating the creation of a new iterator which will have to read the log file from scratch.
This diff fixes this issue by reading the remaining portion of the last block we read from. This is done when the reader is forced to read further (UnmarkEOF is called).
Test Plan:
- Added unit tests
- Stress test (with replication). Check dbdir/LOG file for corruptions.
- Test on test tier
Reviewers: emayanke, haobo, dhruba
Reviewed By: haobo
CC: vamsi, sheki, dhruba, kailiu, igor
Differential Revision: https://reviews.facebook.net/D15249
2014-01-27 23:49:10 +01:00
|
|
|
ASSERT_EQ(BigString("foo", n), Read());
|
|
|
|
ASSERT_TRUE(IsEOF());
|
|
|
|
UnmarkEOF();
|
|
|
|
ASSERT_EQ(BigString("bar", n), Read());
|
|
|
|
ASSERT_TRUE(IsEOF());
|
|
|
|
Write(BigString("xxx", n));
|
|
|
|
UnmarkEOF();
|
|
|
|
ASSERT_EQ(BigString("xxx", n), Read());
|
|
|
|
ASSERT_TRUE(IsEOF());
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, ClearEofError) {
|
Fix UnmarkEOF for partial blocks
Summary:
Blocks in the transaction log are a fixed size, but the last block in the transaction log file is usually a partial block. When a new record is added after the reader hit the end of the file, a new physical record will be appended to the last block. ReadPhysicalRecord can only read full blocks and assumes that the file position indicator is aligned to the start of a block. If the reader is forced to read further by simply clearing the EOF flag, ReadPhysicalRecord will read a full block starting from somewhere in the middle of a real block, causing it to lose alignment and to have a partial physical record at the end of the read buffer. This will result in length mismatches and checksum failures. When the log file is tailed for replication this will cause the log iterator to become invalid, necessitating the creation of a new iterator which will have to read the log file from scratch.
This diff fixes this issue by reading the remaining portion of the last block we read from. This is done when the reader is forced to read further (UnmarkEOF is called).
Test Plan:
- Added unit tests
- Stress test (with replication). Check dbdir/LOG file for corruptions.
- Test on test tier
Reviewers: emayanke, haobo, dhruba
Reviewed By: haobo
CC: vamsi, sheki, dhruba, kailiu, igor
Differential Revision: https://reviews.facebook.net/D15249
2014-01-27 23:49:10 +01:00
|
|
|
// If an error occurs during Read() in UnmarkEOF(), the records contained
|
|
|
|
// in the buffer should be returned on subsequent calls of ReadRecord()
|
|
|
|
// until no more full records are left, whereafter ReadRecord() should return
|
|
|
|
// false to indicate that it cannot read any further.
|
|
|
|
|
|
|
|
Write("foo");
|
|
|
|
Write("bar");
|
|
|
|
UnmarkEOF();
|
|
|
|
ASSERT_EQ("foo", Read());
|
|
|
|
ASSERT_TRUE(IsEOF());
|
|
|
|
Write("xxx");
|
|
|
|
ForceError(0);
|
|
|
|
UnmarkEOF();
|
|
|
|
ASSERT_EQ("bar", Read());
|
|
|
|
ASSERT_EQ("EOF", Read());
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
TEST_P(LogTest, ClearEofError2) {
|
Fix UnmarkEOF for partial blocks
Summary:
Blocks in the transaction log are a fixed size, but the last block in the transaction log file is usually a partial block. When a new record is added after the reader hit the end of the file, a new physical record will be appended to the last block. ReadPhysicalRecord can only read full blocks and assumes that the file position indicator is aligned to the start of a block. If the reader is forced to read further by simply clearing the EOF flag, ReadPhysicalRecord will read a full block starting from somewhere in the middle of a real block, causing it to lose alignment and to have a partial physical record at the end of the read buffer. This will result in length mismatches and checksum failures. When the log file is tailed for replication this will cause the log iterator to become invalid, necessitating the creation of a new iterator which will have to read the log file from scratch.
This diff fixes this issue by reading the remaining portion of the last block we read from. This is done when the reader is forced to read further (UnmarkEOF is called).
Test Plan:
- Added unit tests
- Stress test (with replication). Check dbdir/LOG file for corruptions.
- Test on test tier
Reviewers: emayanke, haobo, dhruba
Reviewed By: haobo
CC: vamsi, sheki, dhruba, kailiu, igor
Differential Revision: https://reviews.facebook.net/D15249
2014-01-27 23:49:10 +01:00
|
|
|
Write("foo");
|
|
|
|
Write("bar");
|
|
|
|
UnmarkEOF();
|
|
|
|
ASSERT_EQ("foo", Read());
|
|
|
|
Write("xxx");
|
|
|
|
ForceError(3);
|
|
|
|
UnmarkEOF();
|
|
|
|
ASSERT_EQ("bar", Read());
|
|
|
|
ASSERT_EQ("EOF", Read());
|
|
|
|
ASSERT_EQ(3U, DroppedBytes());
|
|
|
|
ASSERT_EQ("OK", MatchError("read error"));
|
|
|
|
}
|
|
|
|
|
2015-12-11 20:12:03 +01:00
|
|
|
TEST_P(LogTest, Recycle) {
|
|
|
|
if (!GetParam()) {
|
|
|
|
return; // test is only valid for recycled logs
|
|
|
|
}
|
|
|
|
Write("foo");
|
|
|
|
Write("bar");
|
|
|
|
Write("baz");
|
|
|
|
Write("bif");
|
|
|
|
Write("blitz");
|
|
|
|
while (get_reader_contents()->size() < log::kBlockSize * 2) {
|
|
|
|
Write("xxxxxxxxxxxxxxxx");
|
|
|
|
}
|
|
|
|
unique_ptr<WritableFileWriter> dest_holder(test::GetWritableFileWriter(
|
|
|
|
new test::OverwritingStringSink(get_reader_contents())));
|
|
|
|
Writer recycle_writer(std::move(dest_holder), 123, true);
|
|
|
|
recycle_writer.AddRecord(Slice("foooo"));
|
|
|
|
recycle_writer.AddRecord(Slice("bar"));
|
|
|
|
ASSERT_GE(get_reader_contents()->size(), log::kBlockSize * 2);
|
|
|
|
ASSERT_EQ("foooo", Read());
|
|
|
|
ASSERT_EQ("bar", Read());
|
|
|
|
ASSERT_EQ("EOF", Read());
|
|
|
|
}
|
|
|
|
|
2015-10-08 19:07:15 +02:00
|
|
|
INSTANTIATE_TEST_CASE_P(bool, LogTest, ::testing::Values(0, 2));
|
|
|
|
|
2011-10-31 18:22:06 +01:00
|
|
|
} // namespace log
|
2013-10-04 06:49:15 +02:00
|
|
|
} // namespace rocksdb
|
2011-03-18 23:37:00 +01:00
|
|
|
|
|
|
|
int main(int argc, char** argv) {
|
2015-03-17 22:08:00 +01:00
|
|
|
::testing::InitGoogleTest(&argc, argv);
|
|
|
|
return RUN_ALL_TESTS();
|
2011-03-18 23:37:00 +01:00
|
|
|
}
|