From commits-return-6164-archive-asf-public=cust-asf.ponee.io@kudu.apache.org Sat Jun 23 00:28:20 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id CF32B180647 for ; Sat, 23 Jun 2018 00:28:19 +0200 (CEST) Received: (qmail 52394 invoked by uid 500); 22 Jun 2018 22:28:18 -0000 Mailing-List: contact commits-help@kudu.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@kudu.apache.org Delivered-To: mailing list commits@kudu.apache.org Received: (qmail 52385 invoked by uid 99); 22 Jun 2018 22:28:18 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 22 Jun 2018 22:28:18 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id CC5FBE0614; Fri, 22 Jun 2018 22:28:18 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: danburkert@apache.org To: commits@kudu.apache.org Date: Fri, 22 Jun 2018 22:28:18 -0000 Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: [1/3] kudu git commit: KUDU-2260: Log block manager should handle null bytes in metadata on crash Repository: kudu Updated Branches: refs/heads/master c7918e87b -> f6e8fe6c6 KUDU-2260: Log block manager should handle null bytes in metadata on crash On ext4 with data=ordered (the default), it's possible for a write to persist an increase to the filesize without persisting the actual data. In this case, the file will contain null bytes at the end. In the LBM, we considered this case to be corruption if there were enough null bytes (>= 8) for a PB container record length and length checksum. However, it's safe to call this an incomplete record and truncate the file at the end of the last complete record. This patch changes the P container reader code to return Status::Incomplete if it encounters this situation. This will cause the LBM to repair the container metadata appropriately. Two regression tests, at the PB container file and LBM layers, are included. Change-Id: I0af5c9dbbe28afe7a179595ad20392b99cde2a1b Reviewed-on: http://gerrit.cloudera.org:8080/10777 Tested-by: Kudu Jenkins Reviewed-by: Adar Dembo Project: http://git-wip-us.apache.org/repos/asf/kudu/repo Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/009cfe23 Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/009cfe23 Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/009cfe23 Branch: refs/heads/master Commit: 009cfe23ab37e8c8ac5949f365ab424f34a4f520 Parents: c7918e8 Author: Will Berkeley Authored: Tue Jun 19 17:13:02 2018 -0700 Committer: Will Berkeley Committed: Fri Jun 22 17:58:01 2018 +0000 ---------------------------------------------------------------------- src/kudu/consensus/log_util.cc | 27 ---------------- src/kudu/fs/log_block_manager-test.cc | 50 +++++++++++++++++------------- src/kudu/util/pb_util-test.cc | 34 ++++++++++++++++++++ src/kudu/util/pb_util.cc | 43 +++++++++++++++++++++++-- src/kudu/util/slice.cc | 20 ++++++++++++ src/kudu/util/slice.h | 11 +++++++ 6 files changed, 134 insertions(+), 51 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/kudu/blob/009cfe23/src/kudu/consensus/log_util.cc ---------------------------------------------------------------------- diff --git a/src/kudu/consensus/log_util.cc b/src/kudu/consensus/log_util.cc index 07fba47..0f0fd23 100644 --- a/src/kudu/consensus/log_util.cc +++ b/src/kudu/consensus/log_util.cc @@ -454,33 +454,6 @@ Status ReadableLogSegment::ReadHeaderMagicAndHeaderLength(uint32_t *len) { return Status::OK(); } -namespace { - -// We don't run TSAN on this function because it makes it really slow and causes some -// test timeouts. This is only used on local buffers anyway, so we don't lose much -// by not checking it. -ATTRIBUTE_NO_SANITIZE_THREAD -bool IsAllZeros(const Slice& s) { - // Walk a pointer through the slice instead of using s[i] - // since this is way faster in debug mode builds. We also do some - // manual unrolling for the same purpose. - const uint8_t* p = &s[0]; - int rem = s.size(); - - while (rem >= 8) { - if (UNALIGNED_LOAD64(p) != 0) return false; - rem -= 8; - p += 8; - } - - while (rem > 0) { - if (*p++ != '\0') return false; - rem--; - } - return true; -} -} // anonymous namespace - Status ReadableLogSegment::ParseHeaderMagicAndHeaderLength(const Slice &data, uint32_t *parsed_len) { RETURN_NOT_OK_PREPEND(data.check_size(kLogSegmentHeaderMagicAndHeaderLength), http://git-wip-us.apache.org/repos/asf/kudu/blob/009cfe23/src/kudu/fs/log_block_manager-test.cc ---------------------------------------------------------------------- diff --git a/src/kudu/fs/log_block_manager-test.cc b/src/kudu/fs/log_block_manager-test.cc index d93d8aa..3dc79e8 100644 --- a/src/kudu/fs/log_block_manager-test.cc +++ b/src/kudu/fs/log_block_manager-test.cc @@ -622,33 +622,39 @@ TEST_F(LogBlockManagerTest, TestMetadataTruncation) { uint64_t good_meta_size; ASSERT_OK(env_->GetFileSize(metadata_path, &good_meta_size)); - // First, add an extra byte to the end of the metadata file. This makes the - // trailing "record" of the metadata file corrupt, but doesn't cause data + // First, add extra null bytes to the end of the metadata file. This makes + // the trailing "record" of the metadata file corrupt, but doesn't cause data // loss. The result is that the container will automatically truncate the // metadata file back to its correct size. - { - RWFileOptions opts; - opts.mode = Env::OPEN_EXISTING; - unique_ptr file; - ASSERT_OK(env_->NewRWFile(opts, metadata_path, &file)); - ASSERT_OK(file->Truncate(good_meta_size + 1)); - } - + // We'll do this with 1, 8, and 128 extra bytes-- the first case is too few + // bytes to be a valid record, while the second is too few but is enough for + // a data length and its checksum, and the third is too long for a record. + // The 8- and 128-byte cases are regression tests for KUDU-2260. uint64_t cur_meta_size; - ASSERT_OK(env_->GetFileSize(metadata_path, &cur_meta_size)); - ASSERT_EQ(good_meta_size + 1, cur_meta_size); + for (const auto num_bytes : {1, 8, 128}) { + { + RWFileOptions opts; + opts.mode = Env::OPEN_EXISTING; + unique_ptr file; + ASSERT_OK(env_->NewRWFile(opts, metadata_path, &file)); + ASSERT_OK(file->Truncate(good_meta_size + num_bytes)); + } - // Reopen the metadata file. We will still see all of our blocks. The size of - // the metadata file will be restored back to its previous value. - ASSERT_OK(ReopenBlockManager()); - ASSERT_OK(bm_->GetAllBlockIds(&block_ids)); - ASSERT_EQ(4, block_ids.size()); - ASSERT_OK(bm_->OpenBlock(last_block_id, &block)); - ASSERT_OK(block->Close()); + ASSERT_OK(env_->GetFileSize(metadata_path, &cur_meta_size)); + ASSERT_EQ(good_meta_size + num_bytes, cur_meta_size); - // Check that the file was truncated back to its previous size by the system. - ASSERT_OK(env_->GetFileSize(metadata_path, &cur_meta_size)); - ASSERT_EQ(good_meta_size, cur_meta_size); + // Reopen the metadata file. We will still see all of our blocks. The size of + // the metadata file will be restored back to its previous value. + ASSERT_OK(ReopenBlockManager()); + ASSERT_OK(bm_->GetAllBlockIds(&block_ids)); + ASSERT_EQ(4, block_ids.size()); + ASSERT_OK(bm_->OpenBlock(last_block_id, &block)); + ASSERT_OK(block->Close()); + + // Check that the file was truncated back to its previous size by the system. + ASSERT_OK(env_->GetFileSize(metadata_path, &cur_meta_size)); + ASSERT_EQ(good_meta_size, cur_meta_size); + } // Delete the first block we created. This necessitates writing to the // metadata file of the originally-written container, since we append a http://git-wip-us.apache.org/repos/asf/kudu/blob/009cfe23/src/kudu/util/pb_util-test.cc ---------------------------------------------------------------------- diff --git a/src/kudu/util/pb_util-test.cc b/src/kudu/util/pb_util-test.cc index ddead1b..a942e9a 100644 --- a/src/kudu/util/pb_util-test.cc +++ b/src/kudu/util/pb_util-test.cc @@ -363,6 +363,40 @@ TEST_P(TestPBContainerVersions, TestPartialRecord) { ASSERT_OK(pb_file.Close()); } +// KUDU-2260: Test handling extra null bytes at the end of file. This can +// occur, for example, on ext4 in default data=ordered mode when a write +// increases the filesize but the system crashes before the actual data is +// persisted. +TEST_P(TestPBContainerVersions, TestExtraNullBytes) { + ASSERT_OK(CreateKnownGoodContainerFileWithVersion(version_)); + uint64_t known_good_size; + ASSERT_OK(env_->GetFileSize(path_, &known_good_size)); + for (const auto extra_bytes : {1, 8, 128}) { + ASSERT_OK(TruncateFile(path_, known_good_size + extra_bytes)); + + unique_ptr file; + ASSERT_OK(env_->NewRandomAccessFile(path_, &file)); + ReadablePBContainerFile pb_file(std::move(file)); + ASSERT_OK(pb_file.Open()); + ProtoContainerTestPB test_pb; + // Read the first good PB. Trouble starts at the second. + ASSERT_OK(pb_file.ReadNextPB(&test_pb)); + Status s = pb_file.ReadNextPB(&test_pb); + // Loop to verify that the same response is repeatably returned. + for (int i = 0; i < 2; i++) { + ASSERT_TRUE(version_ == 1 ? s.IsCorruption() : s.IsIncomplete()) << s.ToString(); + if (extra_bytes < 8) { + ASSERT_STR_CONTAINS(s.ToString(), "File size not large enough to be valid"); + } else if (version_ == 1) { + ASSERT_STR_CONTAINS(s.ToString(), "Length and data checksum does not match"); + } else { + ASSERT_STR_CONTAINS(s.ToString(), "rest of file is NULL bytes"); + } + } + ASSERT_OK(pb_file.Close()); + } +} + // Test that it is possible to append after a partial write if we truncate the // partial record. This is only fully supported in V2+. TEST_P(TestPBContainerVersions, TestAppendAfterPartialWrite) { http://git-wip-us.apache.org/repos/asf/kudu/blob/009cfe23/src/kudu/util/pb_util.cc ---------------------------------------------------------------------- diff --git a/src/kudu/util/pb_util.cc b/src/kudu/util/pb_util.cc index a943f68..1c072d6 100644 --- a/src/kudu/util/pb_util.cc +++ b/src/kudu/util/pb_util.cc @@ -23,6 +23,7 @@ #include "kudu/util/pb_util.h" +#include #include #include #include @@ -31,7 +32,6 @@ #include #include #include -#include #include #include @@ -55,8 +55,8 @@ #include "kudu/gutil/strings/escaping.h" #include "kudu/gutil/strings/fastmem.h" #include "kudu/gutil/strings/substitute.h" -#include "kudu/util/coding-inl.h" #include "kudu/util/coding.h" +#include "kudu/util/coding-inl.h" #include "kudu/util/crc.h" #include "kudu/util/debug/sanitizer_scopes.h" #include "kudu/util/debug/trace_event.h" @@ -253,6 +253,33 @@ Status CacheFileSize(ReadableFileType* reader, return Status::OK(); } +template +Status RestOfFileIsAllZeros(ReadableFileType* reader, + uint64_t filesize, + uint64_t offset, + bool* all_zeros) { + DCHECK(reader); + DCHECK_GE(filesize, offset); + DCHECK(all_zeros); + constexpr uint64_t max_to_read = 4 * 1024 * 1024; // 4 MiB. + faststring buf; + while (true) { + uint64_t to_read = std::min(max_to_read, filesize - offset); + if (to_read == 0) { + break; + } + buf.resize(to_read); + RETURN_NOT_OK(reader->Read(offset, Slice(buf))); + offset += to_read; + if (!IsAllZeros(buf)) { + *all_zeros = false; + return Status::OK(); + } + } + *all_zeros = true; + return Status::OK(); +} + // Read and parse a message of the specified format at the given offset in the // format documented in pb_util.h. 'offset' is an in-out parameter and will be // updated with the new offset on success. On failure, 'offset' is not modified. @@ -283,6 +310,18 @@ Status ReadPBStartingAt(ReadableFileType* reader, int version, // Versions >= 2 have an individual checksum for the data length. if (version >= 2) { + // KUDU-2260: If the length and checksum data are all 0's, and the rest of + // the file is all 0's, then it's an incomplete record, not corruption. + // This can happen e.g. on ext4 in the default data=ordered mode, when the + // filesize metadata is updated but the new data is not persisted. + // See https://plus.google.com/+KentonVarda/posts/JDwHfAiLGNQ. + if (IsAllZeros(length_and_cksum_buf)) { + bool all_zeros; + RETURN_NOT_OK(RestOfFileIsAllZeros(reader, file_size, tmp_offset, &all_zeros)); + if (all_zeros) { + return Status::Incomplete("incomplete write of PB: rest of file is NULL bytes"); + } + } Slice length_checksum(length_and_cksum_buf.data() + sizeof(uint32_t), kPBContainerChecksumLen); RETURN_NOT_OK_PREPEND(ParseAndCompareChecksum(length_checksum.data(), { length }), CHECKSUM_ERR_MSG("Data length checksum does not match", http://git-wip-us.apache.org/repos/asf/kudu/blob/009cfe23/src/kudu/util/slice.cc ---------------------------------------------------------------------- diff --git a/src/kudu/util/slice.cc b/src/kudu/util/slice.cc index 5d672ae..775d54a 100644 --- a/src/kudu/util/slice.cc +++ b/src/kudu/util/slice.cc @@ -74,4 +74,24 @@ std::string Slice::ToDebugString(size_t max_len) const { return ret; } +bool IsAllZeros(const Slice& s) { + // Walk a pointer through the slice instead of using s[i] + // since this is way faster in debug mode builds. We also do some + // manual unrolling for the same purpose. + const uint8_t* p = &s[0]; + int rem = s.size(); + + while (rem >= 8) { + if (UNALIGNED_LOAD64(p) != 0) return false; + rem -= 8; + p += 8; + } + + while (rem > 0) { + if (*p++ != '\0') return false; + rem--; + } + return true; +} + } // namespace kudu http://git-wip-us.apache.org/repos/asf/kudu/blob/009cfe23/src/kudu/util/slice.h ---------------------------------------------------------------------- diff --git a/src/kudu/util/slice.h b/src/kudu/util/slice.h index 72ad305..d34c744 100644 --- a/src/kudu/util/slice.h +++ b/src/kudu/util/slice.h @@ -23,6 +23,9 @@ #include "kudu/gutil/strings/stringpiece.h" #include "kudu/util/faststring.h" #endif +#ifdef KUDU_HEADERS_NO_STUBS +#include "kudu/gutil/port.h" +#endif #include "kudu/util/kudu_export.h" namespace kudu { @@ -295,6 +298,14 @@ inline int Slice::compare(const Slice& b) const { return r; } +// We don't run TSAN on this function because it makes it really slow and causes some +// test timeouts. This is only used on local buffers anyway, so we don't lose much +// by not checking it. +#ifdef KUDU_HEADERS_NO_STUBS +ATTRIBUTE_NO_SANITIZE_THREAD +#endif +bool IsAllZeros(const Slice& s); + /// @brief STL map whose keys are Slices. /// /// An example of usage: