You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2018/08/07 17:18:23 UTC

[4/5] impala git commit: Print DebugString() when hitting DCHECK in BufferPool::Client

Print DebugString() when hitting DCHECK in BufferPool::Client

To aid in debugging IMPALA-7402, print a bit more info when hitting the
DCHECK.

Change-Id: I334f323520f2174817c3400e660ce53efb10b16e
Reviewed-on: http://gerrit.cloudera.org:8080/11133
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/master
Commit: 75a11d41f714a897b32d5df3217296dbd3fb88a6
Parents: a8d7a50
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Mon Aug 6 15:03:15 2018 -0700
Committer: Impala Public Jenkins <im...@cloudera.com>
Committed: Tue Aug 7 08:48:05 2018 +0000

----------------------------------------------------------------------
 be/src/runtime/bufferpool/buffer-pool-internal.h | 10 +++++++---
 be/src/runtime/bufferpool/buffer-pool.cc         | 12 ++++++++----
 2 files changed, 15 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/75a11d41/be/src/runtime/bufferpool/buffer-pool-internal.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/bufferpool/buffer-pool-internal.h b/be/src/runtime/bufferpool/buffer-pool-internal.h
index 64c92e5..4cd8cf9 100644
--- a/be/src/runtime/bufferpool/buffer-pool-internal.h
+++ b/be/src/runtime/bufferpool/buffer-pool-internal.h
@@ -289,22 +289,23 @@ class BufferPool::Client {
     return pinned_pages_.size() < num_pages_;
   }
 
+  /// Print debugging info about the state of the client. Caller must not hold 'lock_'.
   std::string DebugString();
 
  private:
   // Check consistency of client, DCHECK if inconsistent. 'lock_' must be held.
   void DCheckConsistency() {
-    DCHECK_GE(buffers_allocated_bytes_, 0);
+    DCHECK_GE(buffers_allocated_bytes_, 0) << DebugStringLocked();
     pinned_pages_.DCheckConsistency();
     dirty_unpinned_pages_.DCheckConsistency();
     in_flight_write_pages_.DCheckConsistency();
     DCHECK_LE(pinned_pages_.size() + dirty_unpinned_pages_.size()
             + in_flight_write_pages_.size(),
-        num_pages_);
+        num_pages_) << DebugStringLocked();
     // Check that we flushed enough pages to disk given our eviction policy.
     DCHECK_GE(reservation_.GetReservation(), buffers_allocated_bytes_
             + pinned_pages_.bytes() + dirty_unpinned_pages_.bytes()
-            + in_flight_write_pages_.bytes());
+            + in_flight_write_pages_.bytes()) << DebugStringLocked();
   }
 
   /// Must be called once before allocating or reclaiming a buffer of 'len'. Ensures that
@@ -330,6 +331,9 @@ class BufferPool::Client {
   Status StartMoveEvictedToPinned(
       boost::unique_lock<boost::mutex>* client_lock, ClientHandle* client, Page* page);
 
+  /// Same as DebugString() except the caller must already hold 'lock_'.
+  std::string DebugStringLocked();
+
   /// The buffer pool that owns the client.
   BufferPool* const pool_;
 

http://git-wip-us.apache.org/repos/asf/impala/blob/75a11d41/be/src/runtime/bufferpool/buffer-pool.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/bufferpool/buffer-pool.cc b/be/src/runtime/bufferpool/buffer-pool.cc
index 6b51ed3..7169bab 100644
--- a/be/src/runtime/bufferpool/buffer-pool.cc
+++ b/be/src/runtime/bufferpool/buffer-pool.cc
@@ -630,7 +630,7 @@ Status BufferPool::Client::CleanPages(unique_lock<mutex>* client_lock, int64_t l
   // Wait until enough writes have finished so that we can make the allocation without
   // violating the eviction policy. I.e. so that other clients can immediately get the
   // memory they're entitled to without waiting for this client's write to complete.
-  DCHECK_GE(in_flight_write_pages_.bytes(), min_bytes_to_write);
+  DCHECK_GE(in_flight_write_pages_.bytes(), min_bytes_to_write) << DebugStringLocked();
   while (dirty_unpinned_pages_.bytes() + in_flight_write_pages_.bytes()
       > target_dirty_bytes) {
     SCOPED_TIMER(counters().write_wait_time);
@@ -641,8 +641,8 @@ Status BufferPool::Client::CleanPages(unique_lock<mutex>* client_lock, int64_t l
 }
 
 void BufferPool::Client::WriteDirtyPagesAsync(int64_t min_bytes_to_write) {
-  DCHECK_GE(min_bytes_to_write, 0);
-  DCHECK_LE(min_bytes_to_write, dirty_unpinned_pages_.bytes());
+  DCHECK_GE(min_bytes_to_write, 0) << DebugStringLocked();
+  DCHECK_LE(min_bytes_to_write, dirty_unpinned_pages_.bytes()) << DebugStringLocked();
   if (file_group_ == NULL) {
     // Spilling disabled - there should be no unpinned pages to write.
     DCHECK_EQ(0, min_bytes_to_write);
@@ -698,7 +698,7 @@ void BufferPool::Client::WriteCompleteCallback(Page* page, const Status& write_s
 #endif
   {
     unique_lock<mutex> cl(lock_);
-    DCHECK(in_flight_write_pages_.Contains(page));
+    DCHECK(in_flight_write_pages_.Contains(page)) << DebugStringLocked();
     // The status should always be propagated.
     // TODO: if we add cancellation support to TmpFileMgr, consider cancellation path.
     if (!write_status.ok()) write_status_.MergeStatus(write_status);
@@ -732,6 +732,10 @@ void BufferPool::Client::WaitForAllWrites() {
 
 string BufferPool::Client::DebugString() {
   lock_guard<mutex> lock(lock_);
+  return DebugStringLocked();
+}
+
+string BufferPool::Client::DebugStringLocked() {
   stringstream ss;
   ss << Substitute("<BufferPool::Client> $0 name: $1 write_status: $2 "
                    "buffers allocated $3 num_pages: $4 pinned_bytes: $5 "