You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by jo...@apache.org on 2019/04/14 21:34:32 UTC

[impala] 02/02: IMPALA-8322: Add periodic dirty check of done_ in ThreadTokenAvailableCb

This is an automated email from the ASF dual-hosted git repository.

joemcdonnell pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit 8ec17b7cdffbd82ce7b3e652edc2530df083eeab
Author: Joe McDonnell <jo...@cloudera.com>
AuthorDate: Fri Apr 5 16:16:13 2019 -0700

    IMPALA-8322: Add periodic dirty check of done_ in ThreadTokenAvailableCb
    
    When HdfsScanNode is cancelled or hits an error, SetDoneInternal() holds
    HdfsScanNode::lock_ while it runs RequestContext::Cancel(), which can
    wait on IO threads to complete outstanding IOs. This can cause a cascade
    of blocked threads that causes Prepare() to take a significant time and
    cause datastream sender timeouts.
    
    The specific scenario seen has this set of threads:
    Thread 1: A DiskIoMgr thread is blocked on IO in hdfsOpenFile() or
      hdfsRead(), holding HdfsFileReader::lock_.
    Thread 2: An HDFS scanner thread is blocked in
      HdfsScanNode::SetDoneInternal() -> RequestContext::Cancel()
      -> ScanRange::CancelInternal(), waiting on HdfsFileReader::lock_.
      It is holding HdfsScanNode::lock_.
    Thread 3: A thread in ThreadResourceMgr::DestroyPool() -> (a few layers)
      -> HdfsScanNode::ThreadTokenAvailableCb() is blocked waiting on
      HdfsScanNode::lock_ while holding ThreadResourceMgr::lock_.
    Thread 4: A thread in FragmentInstanceState::Prepare()
      -> RuntimeState::Init() -> ThreadResourceMgr::CreatePool() is blocked
      waiting on ThreadResourceMgr::lock_.
    
    When Prepare() takes a significant time, datastream senders will time out
    waiting for the datastream receivers to start up. This causes failed
    queries. S3 has higher latencies for IO and does not have file handle
    caching, so S3 is more susceptible to this issue than other platforms.
    
    This changes HdfsScanNode::ThreadTokenAvailableCb() to periodically do a
    dirty check of HdfsScanNode::done_ when waiting to acquire the lock. This
    avoids the blocking experienced by Thread 3 in the example above.
    
    Testing:
     - Ran tests on normal HDFS and repeatedly on S3
    
    Change-Id: I4881a3e5bfda64e8d60af95ad13b450cf7f8c130
    Reviewed-on: http://gerrit.cloudera.org:8080/12968
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/common/names.h              |  1 +
 be/src/exec/hdfs-scan-node.cc      | 48 ++++++++++++++++++++++++--------------
 be/src/exec/hdfs-scan-node.h       | 17 ++++++++------
 be/src/runtime/io/request-ranges.h | 16 +++++++------
 4 files changed, 51 insertions(+), 31 deletions(-)

diff --git a/be/src/common/names.h b/be/src/common/names.h
index e116673..54eca6f 100644
--- a/be/src/common/names.h
+++ b/be/src/common/names.h
@@ -138,6 +138,7 @@ using boost::thread_group;
 
 #ifdef BOOST_THREAD_MUTEX_HPP
 using boost::mutex;
+using boost::timed_mutex;
 using boost::try_mutex;
 #endif
 
diff --git a/be/src/exec/hdfs-scan-node.cc b/be/src/exec/hdfs-scan-node.cc
index bcf0d71..66b7a8b 100644
--- a/be/src/exec/hdfs-scan-node.cc
+++ b/be/src/exec/hdfs-scan-node.cc
@@ -95,7 +95,7 @@ Status HdfsScanNode::GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos
       // the non-ok status represents the error in ValidateScanRange() or describes
       // the unsupported compression formats. For such non-CANCELLED cases, the status
       // returned by IssueInitialScanRanges() takes precedence.
-      unique_lock<mutex> l(lock_);
+      unique_lock<timed_mutex> l(lock_);
       if (status.IsCancelled() && !status_.ok()) return status_;
       return status;
     }
@@ -108,7 +108,7 @@ Status HdfsScanNode::GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos
 
   Status status = GetNextInternal(state, row_batch, eos);
   if (!status.ok() || *eos) {
-    unique_lock<mutex> l(lock_);
+    unique_lock<timed_mutex> l(lock_);
     lock_guard<SpinLock> l2(file_type_counts_lock_);
     StopAndFinalizeCounters();
   }
@@ -153,7 +153,7 @@ Status HdfsScanNode::GetNextInternal(
   // The RowBatchQueue was shutdown either because all scan ranges are complete or a
   // scanner thread encountered an error.  Check status_ to distinguish those cases.
   *eos = true;
-  unique_lock<mutex> l(lock_);
+  unique_lock<timed_mutex> l(lock_);
   return status_;
 }
 
@@ -208,7 +208,7 @@ void HdfsScanNode::RangeComplete(const THdfsFileFormat::type& file_type,
 }
 
 void HdfsScanNode::TransferToScanNodePool(MemPool* pool) {
-  unique_lock<mutex> l(lock_);
+  unique_lock<timed_mutex> l(lock_);
   HdfsScanNodeBase::TransferToScanNodePool(pool);
 }
 
@@ -250,8 +250,8 @@ int64_t HdfsScanNode::EstimateScannerThreadMemConsumption() const {
   return est_total_bytes;
 }
 
-void HdfsScanNode::ReturnReservationFromScannerThread(const unique_lock<mutex>& lock,
-    int64_t bytes) {
+void HdfsScanNode::ReturnReservationFromScannerThread(
+    const unique_lock<timed_mutex>& lock, int64_t bytes) {
   DCHECK(lock.mutex() == &lock_ && lock.owns_lock());
   // Release as much memory as possible. Must hold onto the minimum reservation, though.
   Status status = buffer_pool_client()->DecreaseReservationTo(
@@ -289,16 +289,28 @@ void HdfsScanNode::ThreadTokenAvailableCb(ThreadResourcePool* pool) {
   while (true) {
     // The lock must be given up between loops in order to give writers to done_,
     // all_ranges_started_ etc. a chance to grab the lock.
+    // IMPALA-8322: Another thread can hold this lock for significant periods of time
+    // if the scan node is being cancelled. Since this function can be called while
+    // holding other locks that can block other threads (e.g. ThreadResourceMgr::lock_),
+    // avoid blocking unnecessarily. There are two remedies. First, we do a check of
+    // done() to try to avoid acquiring the lock_, as there is nothing to do if
+    // the scan node is done. Second, this uses a timeout of 10 milliseconds when
+    // acquiring the lock_ to allow a periodic check of done(). The 10 millisecond
+    // timeout is arbitrary.
     // TODO: This still leans heavily on starvation-free locks, come up with a more
-    // correct way to communicate between this method and ScannerThreadHelper
-    unique_lock<mutex> lock(lock_);
+    // correct way to communicate between this method and ScannerThread().
+    if (done()) break;
+    unique_lock<timed_mutex> lock(lock_, boost::chrono::milliseconds(10));
+    if (!lock.owns_lock()) {
+      continue;
+    }
 
     const int64_t num_active_scanner_threads = thread_state_.GetNumActive();
     const bool first_thread = num_active_scanner_threads == 0;
     const int64_t est_mem = thread_state_.estimated_per_thread_mem();
     const int64_t scanner_thread_reservation = resource_profile_.min_reservation;
     // Cases 1, 2, 3.
-    if (done_ || all_ranges_started_ ||
+    if (done() || all_ranges_started_ ||
         num_active_scanner_threads >= progress_.remaining()) {
       break;
     }
@@ -380,7 +392,7 @@ void HdfsScanNode::ScannerThread(bool first_thread, int64_t scanner_thread_reser
     filter_ctxs.push_back(filter);
   }
 
-  while (!done_) {
+  while (!done()) {
     // Prevent memory accumulating across scan ranges.
     expr_results_pool.Clear();
     // Check if we have enough thread tokens to keep using this optional thread. This
@@ -401,7 +413,7 @@ void HdfsScanNode::ScannerThread(bool first_thread, int64_t scanner_thread_reser
     ScanRange* scan_range;
     Status status = StartNextScanRange(&scanner_thread_reservation, &scan_range);
     if (!status.ok()) {
-      unique_lock<mutex> l(lock_);
+      unique_lock<timed_mutex> l(lock_);
       // If there was already an error, the main thread will do the cleanup
       if (!status_.ok()) break;
 
@@ -424,7 +436,7 @@ void HdfsScanNode::ScannerThread(bool first_thread, int64_t scanner_thread_reser
     }
 
     if (scan_range == nullptr && remaining_scan_range_submissions == 0) {
-      unique_lock<mutex> l(lock_);
+      unique_lock<timed_mutex> l(lock_);
       // All ranges have been queued and DiskIoMgr has no more new ranges for this scan
       // node to process. This means that every range is either done or being processed by
       // another thread.
@@ -445,7 +457,7 @@ void HdfsScanNode::ScannerThread(bool first_thread, int64_t scanner_thread_reser
   }
 
   {
-    unique_lock<mutex> l(lock_);
+    unique_lock<timed_mutex> l(lock_);
     ReturnReservationFromScannerThread(l, scanner_thread_reservation);
   }
   for (auto& ctx: filter_ctxs) ctx.expr_eval->Close(runtime_state_);
@@ -545,21 +557,23 @@ void HdfsScanNode::ProcessSplit(const vector<FilterContext>& filter_ctxs,
 
 void HdfsScanNode::SetDoneInternal(const Status& status) {
   // If the scan node is already in the done state, do nothing.
-  if (done_) return;
+  if (done()) return;
   DCHECK(status_.ok());
-  done_ = true;
+  done_.Store(true);
   if (!status.ok()) status_ = status;
+  // TODO: Cancelling the RequestContext will wait on in-flight IO. We should
+  //       investigate dropping the lock_ or restructuring this cancel.
   if (reader_context_ != nullptr) reader_context_->Cancel();
   thread_state_.Shutdown();
 }
 
 void HdfsScanNode::SetDone() {
-  unique_lock<mutex> l(lock_);
+  unique_lock<timed_mutex> l(lock_);
   SetDoneInternal(status_);
 }
 
 void HdfsScanNode::SetError(const Status& status) {
   discard_result(ExecDebugAction(TExecNodePhase::SCANNER_ERROR, runtime_state_));
-  unique_lock<mutex> l(lock_);
+  unique_lock<timed_mutex> l(lock_);
   SetDoneInternal(status);
 }
diff --git a/be/src/exec/hdfs-scan-node.h b/be/src/exec/hdfs-scan-node.h
index c395b6c..3f2e56d 100644
--- a/be/src/exec/hdfs-scan-node.h
+++ b/be/src/exec/hdfs-scan-node.h
@@ -81,7 +81,7 @@ class HdfsScanNode : public HdfsScanNodeBase {
 
   virtual bool HasRowBatchQueue() const override { return true; }
 
-  bool done() const { return done_; }
+  bool done() const { return done_.Load(); }
 
   /// Adds ranges to the io mgr queue and starts up new scanner threads if possible.
   /// The enqueue_location parameter determines the location at which the scan ranges are
@@ -114,8 +114,10 @@ class HdfsScanNode : public HdfsScanNodeBase {
 
   /// Lock protects access between scanner thread and main query thread (the one calling
   /// GetNext()) for all fields below.  If this lock and any other locks needs to be taken
-  /// together, this lock must be taken first.
-  boost::mutex lock_;
+  /// together, this lock must be taken first. This is a "timed_mutex" to allow specifying
+  /// a timeout when acquiring the mutex. Almost all code locations acquire the mutex
+  /// without a timeout; see ThreadTokenAvailableCb for a location using a timeout.
+  boost::timed_mutex lock_;
 
   /// Protects file_type_counts_. Cannot be taken together with any other lock
   /// except lock_, and if so, lock_ must be taken first.
@@ -124,8 +126,9 @@ class HdfsScanNode : public HdfsScanNodeBase {
   /// Flag signaling that all scanner threads are done.  This could be because they
   /// are finished, an error/cancellation occurred, or the limit was reached.
   /// Setting this to true triggers the scanner threads to clean up.
-  /// This should not be explicitly set. Instead, call SetDone().
-  bool done_ = false;
+  /// This should not be explicitly set. Instead, call SetDone(). This is set while
+  /// holding lock_, but it is atomic to allow reads without holding the lock.
+  AtomicBool done_;
 
   /// Set to true if all ranges have started. Some of the ranges may still be in flight
   /// being processed by scanner threads, but no new ScannerThreads should be started.
@@ -177,8 +180,8 @@ class HdfsScanNode : public HdfsScanNodeBase {
   /// Called by scanner thread to return some or all of its reservation that is not
   /// needed. Always holds onto at least the minimum reservation to avoid violating the
   /// invariants of ExecNode::buffer_pool_client_. 'lock_' must be held via 'lock'.
-  void ReturnReservationFromScannerThread(const boost::unique_lock<boost::mutex>& lock,
-      int64_t bytes);
+  void ReturnReservationFromScannerThread(
+      const boost::unique_lock<boost::timed_mutex>& lock, int64_t bytes);
 
   /// Checks for eos conditions and returns batches from the row batch queue.
   Status GetNextInternal(RuntimeState* state, RowBatch* row_batch, bool* eos)
diff --git a/be/src/runtime/io/request-ranges.h b/be/src/runtime/io/request-ranges.h
index a459800..29d105d 100644
--- a/be/src/runtime/io/request-ranges.h
+++ b/be/src/runtime/io/request-ranges.h
@@ -343,13 +343,15 @@ class ScanRange : public RequestRange {
   /// hold 'lock_'.
   void CleanUpBuffers(std::vector<std::unique_ptr<BufferDescriptor>>&& buffers);
 
-  /// Same as Cancel() except doesn't remove the scan range from
-  /// reader_->active_scan_ranges_ or wait for in-flight reads to finish.
-  /// This is invoked by RequestContext::Cancel(), which removes the range itself
-  /// to avoid invalidating its active_scan_ranges_ iterator. If 'read_error' is
-  /// true, this is being called from a disk thread to propagate a read error, so
-  /// 'read_in_flight_' is set to false and threads in WaitForInFlightRead() are
-  /// woken up.
+  /// Same as Cancel() except it doesn't remove the scan range from
+  /// reader_->active_scan_ranges_ or call WaitForInFlightRead(). This allows for
+  /// custom handling of in flight reads or active scan ranges. For example, this is
+  /// invoked by RequestContext::Cancel(), which removes the range itself to avoid
+  /// invalidating its active_scan_ranges_ iterator. It is also invoked by disk IO
+  /// threads to propagate a read error for a range that is in flight (i.e. when
+  /// read_error is true), so 'read_in_flight_' is set to false and threads in
+  /// WaitForInFlightRead() are woken up. Note that this is tearing down the FileReader,
+  /// so it may block waiting for other threads that are performing IO.
   void CancelInternal(const Status& status, bool read_error);
 
   /// Marks the scan range as blocked waiting for a buffer. Caller must not hold 'lock_'.