You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by ad...@apache.org on 2017/08/24 03:57:10 UTC

[1/2] kudu git commit: Give more context on errors reading cfiles

Repository: kudu
Updated Branches:
  refs/heads/master f33a13c12 -> b37bde72c


Give more context on errors reading cfiles

Recently, a user reported an error loading a tablet in which the only
reported message was "bad magic". This wasn't useful for pinpointing the
id of the bad block or what might have happened to cause the problem.

This patch adds more context info in such circumstances: we now include
DebugString output for "bad magic" errors as well as the block ID in all
cases.

The unit test is updated to check that block IDs show up in all
Corruption status messages.

Change-Id: I0bda5688a020032c512235ee574cb3e53c7872af
Reviewed-on: http://gerrit.cloudera.org:8080/7620
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Kudu Jenkins


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/dbfb92f3
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/dbfb92f3
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/dbfb92f3

Branch: refs/heads/master
Commit: dbfb92f3ada84f68ed3f05dd0c9ea24fca9d334e
Parents: f33a13c
Author: Todd Lipcon <to...@apache.org>
Authored: Tue Aug 8 15:02:05 2017 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Thu Aug 24 03:34:40 2017 +0000

----------------------------------------------------------------------
 src/kudu/cfile/cfile-test.cc   |  1 +
 src/kudu/cfile/cfile_reader.cc | 55 +++++++++++++++++++++++++------------
 src/kudu/cfile/index_btree.cc  |  8 +++++-
 src/kudu/tablet/deltafile.cc   |  7 ++++-
 4 files changed, 52 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/dbfb92f3/src/kudu/cfile/cfile-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/cfile-test.cc b/src/kudu/cfile/cfile-test.cc
index 4ab605e..8525d4d 100644
--- a/src/kudu/cfile/cfile-test.cc
+++ b/src/kudu/cfile/cfile-test.cc
@@ -889,6 +889,7 @@ TEST_P(TestCFileBothCacheTypes, TestDataCorruption) {
     for (uint8_t flip = 0; flip < 8; flip++) {
       Status s = CorruptAndReadBlock(id, i, flip);
       ASSERT_TRUE(s.IsCorruption());
+      ASSERT_STR_MATCHES(s.ToString(), "block [0-9]+");
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/dbfb92f3/src/kudu/cfile/cfile_reader.cc
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/cfile_reader.cc b/src/kudu/cfile/cfile_reader.cc
index 86c72ca..6311b75 100644
--- a/src/kudu/cfile/cfile_reader.cc
+++ b/src/kudu/cfile/cfile_reader.cc
@@ -104,12 +104,13 @@ static Status ParseMagicAndLength(const Slice &data,
   } else if (memcmp(kMagicStringV2, data.data(), kMagicLength) == 0) {
     version = 2;
   } else {
-    return Status::Corruption("bad magic");
+    return Status::Corruption("bad CFile header magic", data.ToDebugString());
   }
 
   uint32_t len = DecodeFixed32(data.data() + kMagicLength);
   if (len > kMaxHeaderFooterPBSize) {
-    return Status::Corruption("invalid data size");
+    return Status::Corruption("invalid data size for header",
+                              std::to_string(len));
   }
 
   *cfile_version = version;
@@ -190,7 +191,10 @@ Status CFileReader::InitOnce() {
 }
 
 Status CFileReader::Init() {
-  return init_once_.Init(&CFileReader::InitOnce, this);
+  RETURN_NOT_OK_PREPEND(init_once_.Init(&CFileReader::InitOnce, this),
+                        Substitute("failed to init CFileReader for block $0",
+                                   block_id().ToString()));
+  return Status::OK();
 }
 
 Status CFileReader::ReadAndParseHeader() {
@@ -203,13 +207,15 @@ Status CFileReader::ReadAndParseHeader() {
   // proper protobuf header.
   uint8_t mal_scratch[kMagicAndLengthSize];
   Slice mal(mal_scratch, kMagicAndLengthSize);
-  RETURN_NOT_OK(block_->Read(0, &mal));
+  RETURN_NOT_OK_PREPEND(block_->Read(0, &mal),
+                        "failed to read CFile pre-header");
   uint32_t header_size;
-  RETURN_NOT_OK(ParseMagicAndLength(mal, &cfile_version_, &header_size));
+  RETURN_NOT_OK_PREPEND(ParseMagicAndLength(mal, &cfile_version_, &header_size),
+                        "failed to parse CFile pre-header");
 
   // Quick check to ensure the header size is reasonable.
   if (header_size >= file_size_ - kMagicAndLengthSize) {
-    return Status::Corruption("invalid header size");
+    return Status::Corruption("invalid CFile header size", std::to_string(header_size));
   }
 
   // Setup the data slices.
@@ -233,7 +239,8 @@ Status CFileReader::ReadAndParseHeader() {
   // Parse the protobuf header.
   header_.reset(new CFileHeaderPB());
   if (!header_->ParseFromArray(header.data(), header.size())) {
-    return Status::Corruption("Invalid cfile pb header");
+    return Status::Corruption("invalid cfile pb header",
+                              header.ToDebugString());
   }
 
   VLOG(2) << "Read header: " << SecureDebugString(*header_);
@@ -258,7 +265,9 @@ Status CFileReader::ReadAndParseFooter() {
 
   // Quick check to ensure the footer size is reasonable.
   if (footer_size >= file_size_ - kMagicAndLengthSize) {
-    return Status::Corruption("invalid footer size");
+    return Status::Corruption(Substitute(
+        "invalid CFile footer size $0 in block of size $1",
+        footer_size, file_size_));
   }
 
   uint8_t footer_scratch[footer_size];
@@ -279,7 +288,7 @@ Status CFileReader::ReadAndParseFooter() {
   // incompatible_features flag tells us if a checksum exists at all.
   footer_.reset(new CFileFooterPB());
   if (!footer_->ParseFromArray(footer.data(), footer.size())) {
-    return Status::Corruption("Invalid cfile pb footer");
+    return Status::Corruption("invalid cfile pb footer", footer.ToDebugString());
   }
 
   // Verify the footer checksum if needed.
@@ -290,7 +299,8 @@ Status CFileReader::ReadAndParseFooter() {
 
   // Verify if the compression codec is available.
   if (footer_->compression() != NO_COMPRESSION) {
-    RETURN_NOT_OK(GetCompressionCodec(footer_->compression(), &codec_));
+    RETURN_NOT_OK_PREPEND(GetCompressionCodec(footer_->compression(), &codec_),
+                          "failed to load CFile compression codec");
   }
 
   VLOG(2) << "Read footer: " << SecureDebugString(*footer_);
@@ -434,8 +444,9 @@ Status CFileReader::ReadBlock(const BlockPointer &ptr, CacheControl cache_contro
 
   uint32_t data_size = ptr.size();
   if (has_checksums()) {
-    if (kChecksumSize > data_size) {
-      return Status::Corruption("invalid data size");
+    if (PREDICT_FALSE(kChecksumSize > data_size)) {
+      return Status::Corruption("invalid data size for block pointer",
+                                ptr.ToString());
     }
     data_size -= kChecksumSize;
   }
@@ -459,10 +470,14 @@ Status CFileReader::ReadBlock(const BlockPointer &ptr, CacheControl cache_contro
   if (has_checksums() && FLAGS_cfile_verify_checksums) {
     results.push_back(checksum);
   }
-  RETURN_NOT_OK(block_->ReadV(ptr.offset(), &results));
+  RETURN_NOT_OK_PREPEND(block_->ReadV(ptr.offset(), &results),
+                        Substitute("failed to read CFile block $0 at $1",
+                                   block_id().ToString(), ptr.ToString()));
 
   if (has_checksums() && FLAGS_cfile_verify_checksums) {
-    RETURN_NOT_OK(VerifyChecksum({ block }, checksum));
+    RETURN_NOT_OK_PREPEND(VerifyChecksum({ block }, checksum),
+                          Substitute("checksum error on CFile block $0 at $1",
+                                     block_id().ToString(), ptr.ToString()));
   }
 
   // Decompress the block
@@ -827,10 +842,13 @@ Status CFileIterator::PrepareForNewSeek() {
 
     // Cache the dictionary for performance
     RETURN_NOT_OK_PREPEND(reader_->ReadBlock(bp, CFileReader::CACHE_BLOCK, &dict_block_handle_),
-                          "Couldn't read dictionary block");
+                          "couldn't read dictionary block");
 
     dict_decoder_.reset(new BinaryPlainBlockDecoder(dict_block_handle_.data()));
-    RETURN_NOT_OK_PREPEND(dict_decoder_->ParseHeader(), "Couldn't parse dictionary block header");
+    RETURN_NOT_OK_PREPEND(dict_decoder_->ParseHeader(),
+                          Substitute("couldn't parse dictionary block header in block $0 ($1)",
+                                     reader_->block_id().ToString(),
+                                     bp.ToString()));
   }
 
   seeked_ = nullptr;
@@ -886,7 +904,10 @@ Status CFileIterator::ReadCurrentDataBlock(const IndexTreeIterator &idx_iter,
   BlockDecoder *bd;
   RETURN_NOT_OK(reader_->type_encoding_info()->CreateBlockDecoder(&bd, data_block, this));
   prep_block->dblk_.reset(bd);
-  RETURN_NOT_OK(prep_block->dblk_->ParseHeader());
+  RETURN_NOT_OK_PREPEND(prep_block->dblk_->ParseHeader(),
+                        Substitute("unable to decode data block header in block $0 ($1)",
+                                   reader_->block_id().ToString(),
+                                   prep_block->dblk_ptr_.ToString()));
 
   // For nullable blocks, we filled in the row count from the null information above,
   // since the data block decoder only knows about the non-null values.

http://git-wip-us.apache.org/repos/asf/kudu/blob/dbfb92f3/src/kudu/cfile/index_btree.cc
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/index_btree.cc b/src/kudu/cfile/index_btree.cc
index d83932f..e47c679 100644
--- a/src/kudu/cfile/index_btree.cc
+++ b/src/kudu/cfile/index_btree.cc
@@ -26,11 +26,14 @@
 #include "kudu/cfile/cfile_writer.h"
 #include "kudu/cfile/cfile_util.h"
 #include "kudu/cfile/index_btree.h"
+#include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/debug-util.h"
 #include "kudu/util/make_shared.h"
 
 using std::vector;
 
+using strings::Substitute;
+
 namespace kudu {
 namespace cfile {
 
@@ -278,7 +281,10 @@ Status IndexTreeIterator::LoadBlock(const BlockPointer &block, int depth) {
   seeked->block_ptr = block;
 
   // Parse the new block.
-  RETURN_NOT_OK(seeked->reader.Parse(seeked->data.data()));
+  RETURN_NOT_OK_PREPEND(seeked->reader.Parse(seeked->data.data()),
+                        Substitute("failed to parse index block in block $0 at $1",
+                                   reader_->block_id().ToString(),
+                                   block.ToString()));
 
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/dbfb92f3/src/kudu/tablet/deltafile.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tablet/deltafile.cc b/src/kudu/tablet/deltafile.cc
index 9a822fa..3bfdc96 100644
--- a/src/kudu/tablet/deltafile.cc
+++ b/src/kudu/tablet/deltafile.cc
@@ -43,6 +43,7 @@
 #include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/mathlimits.h"
 #include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/strings/substitute.h"
 #include "kudu/tablet/mutation.h"
 #include "kudu/tablet/mvcc.h"
 #include "kudu/tablet/tablet.pb.h"
@@ -66,6 +67,7 @@ using std::shared_ptr;
 using std::string;
 using std::unique_ptr;
 using std::vector;
+using strings::Substitute;
 
 namespace kudu {
 
@@ -459,7 +461,10 @@ Status DeltaFileIterator::ReadCurrentBlockOntoQueue() {
 
   // Decode the block.
   pdb->decoder_.reset(new BinaryPlainBlockDecoder(pdb->block_.data()));
-  RETURN_NOT_OK(pdb->decoder_->ParseHeader());
+  RETURN_NOT_OK_PREPEND(pdb->decoder_->ParseHeader(),
+                        Substitute("unable to decode data block header in delta block $0 ($1)",
+                                   dfr_->cfile_reader()->block_id().ToString(),
+                                   dblk_ptr.ToString()));
 
   RETURN_NOT_OK(GetFirstRowIndexInCurrentBlock(&pdb->first_updated_idx_));
   RETURN_NOT_OK(GetLastRowIndexInDecodedBlock(*pdb->decoder_, &pdb->last_updated_idx_));


[2/2] kudu git commit: log block manager: use unsigned int for next_block_id_

Posted by ad...@apache.org.
log block manager: use unsigned int for next_block_id_

KUDU-1538 introduced 'next_block_id_' to keep track of unique block
ID that should be used for block creation. Currently, it is defined
as int64_t. However, it could be updated based on the value of
'max_block_id' which is uint64_t. Since block IDs are defined as
uint64_t both on disk (fs.proto) and in memory (block_id.h), it
makes more sense to treat 'next_block_id_' as uint64_t rather than
to convert it correctly to int64_t everywhere.

This patch changes the type of 'next_block_id_' to uint64_t to
avoid overflow due to conversion of unsigned int to int, which
can result in the reuse of an existing block ID. It does not add
a standalone test case because the failure is most likely to occur
in specific test-only scenarios.

Change-Id: Ib315b20719ef529331304df5c56c4242902524d4
Reviewed-on: http://gerrit.cloudera.org:8080/7796
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Reviewed-by: Todd Lipcon <to...@apache.org>
Tested-by: Kudu Jenkins


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/b37bde72
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/b37bde72
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/b37bde72

Branch: refs/heads/master
Commit: b37bde72c47370293827a306532d9be15b1c5f40
Parents: dbfb92f
Author: hahao <ha...@cloudera.com>
Authored: Wed Aug 23 17:35:02 2017 -0700
Committer: Adar Dembo <ad...@cloudera.com>
Committed: Thu Aug 24 03:56:47 2017 +0000

----------------------------------------------------------------------
 src/kudu/fs/file_block_manager.h | 2 +-
 src/kudu/fs/log_block_manager.h  | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/b37bde72/src/kudu/fs/file_block_manager.h
----------------------------------------------------------------------
diff --git a/src/kudu/fs/file_block_manager.h b/src/kudu/fs/file_block_manager.h
index f62c900..6c6cc81 100644
--- a/src/kudu/fs/file_block_manager.h
+++ b/src/kudu/fs/file_block_manager.h
@@ -127,7 +127,7 @@ class FileBlockManager : public BlockManager {
 
   // For generating block IDs.
   ThreadSafeRandom rand_;
-  AtomicInt<int64_t> next_block_id_;
+  AtomicInt<uint64_t> next_block_id_;
 
   // Protects 'dirty_dirs_'.
   mutable simple_spinlock lock_;

http://git-wip-us.apache.org/repos/asf/kudu/blob/b37bde72/src/kudu/fs/log_block_manager.h
----------------------------------------------------------------------
diff --git a/src/kudu/fs/log_block_manager.h b/src/kudu/fs/log_block_manager.h
index 1474cee..0407841 100644
--- a/src/kudu/fs/log_block_manager.h
+++ b/src/kudu/fs/log_block_manager.h
@@ -408,7 +408,7 @@ class LogBlockManager : public BlockManager {
   ObjectIdGenerator oid_generator_;
 
   // For generating block IDs.
-  AtomicInt<int64_t> next_block_id_;
+  AtomicInt<uint64_t> next_block_id_;
 
   // Metrics for the block manager.
   //