Uploaded image for project: 'Kudu'
  1. Kudu
  2. KUDU-2088

UpdateReplica accesses stack object after it is destroyed

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 1.4.0
    • 1.5.0
    • consensus
    • None

    Description

      RaftConsensus::UpdateReplica() has this bit of code in it:

          // 5 - We wait for the writes to be durable.
      
          // Note that this is safe because dist consensus now only supports a single outstanding
          // request at a time and this way we can allow commits to proceed while we wait.
          TRACE("Waiting on the replicates to finish logging");
          TRACE_EVENT0("consensus", "Wait for log");
          Status s;
          do {
            s = log_synchronizer.WaitFor(
                MonoDelta::FromMilliseconds(FLAGS_raft_heartbeat_interval_ms));
            // If just waiting for our log append to finish lets snooze the timer.
            // We don't want to fire leader election because we're waiting on our own log.
            if (s.IsTimedOut()) {
              RETURN_NOT_OK(SnoozeFailureDetector());
            }
          } while (s.IsTimedOut());
          RETURN_NOT_OK(s);
      

      log_synchronizer is a stack-allocated Synchronizer. A reference to it is passed into an asynchronous log append function. The purpose of this code is to wait for that asynchronous function to finish while periodically snoozing the failure detector.

      However, if SnoozeFailureDetector() were to return an error, we'll exit the function early and destroy log_synchronizer. This can lead to a crash if the reference to log_synchronizer is accessed later by the asynchronous log append function. Here's one such crash stack trace:

      F0801 02:58:43.488010 13715 mutex.cc:76] Check failed: rv == 0 || rv == 16 . Invalid argument. Owner tid: 0; Self tid: 128; To collect the owner stack trace, enable the flag --debug_mutex_collect_stacktrace
      *** Check failure stack trace: ***
          @     0x7f843b5d22fd  google::LogMessage::Fail() at ??:0
          @     0x7f843b5d41bd  google::LogMessage::SendToLog() at ??:0
          @     0x7f843b5d1e39  google::LogMessage::Flush() at ??:0
          @     0x7f843b5d4c5f  google::LogMessageFatal::~LogMessageFatal() at ??:0
          @     0x7f843c49dc46  kudu::Mutex::TryAcquire() at ??:0
          @     0x7f843c49dcd1  kudu::Mutex::Acquire() at ??:0
          @     0x7f8444243290  kudu::MutexLock::MutexLock() at ??:0
          @     0x7f8444274d02  kudu::CountDownLatch::CountDown() at ??:0
          @     0x7f8444274dd1  kudu::CountDownLatch::CountDown() at ??:0
          @     0x7f84428c4d4f  kudu::Synchronizer::StatusCB() at ??:0
          @     0x7f84428cf73e  kudu::internal::RunnableAdapter<>::Run() at ??:0
          @     0x7f84428ce716  kudu::internal::InvokeHelper<>::MakeItSo() at ??:0
          @     0x7f84428ccf37  kudu::internal::Invoker<>::Run() at ??:0
          @     0x7f8442879e6f  kudu::Callback<>::Run() at ??:0
          @     0x7f844286e28e  kudu::consensus::PeerMessageQueue::LocalPeerAppendFinished() at ??:0
          @     0x7f8442882275  kudu::internal::RunnableAdapter<>::Run() at ??:0
          @     0x7f8442880649  kudu::internal::InvokeHelper<>::MakeItSo() at ??:0
          @     0x7f844287e87f  kudu::internal::Invoker<>::Run() at ??:0
          @     0x7f8442879e6f  kudu::Callback<>::Run() at ??:0
          @     0x7f8442891eec  kudu::consensus::LogCache::LogCallback() at ??:0
          @     0x7f8442897e94  kudu::internal::RunnableAdapter<>::Run() at ??:0
          @     0x7f844289797d  kudu::internal::InvokeHelper<>::MakeItSo() at ??:0
          @     0x7f8442896fff  kudu::internal::Invoker<>::Run() at ??:0
          @     0x7f8442879e6f  kudu::Callback<>::Run() at ??:0
          @     0x7f844250fa1f  kudu::log::Log::AppendThread::HandleGroup() at ??:0
          @     0x7f844250ee5c  kudu::log::Log::AppendThread::DoWork() at ??:0
          @     0x7f8442527c81  kudu::internal::RunnableAdapter<>::Run() at ??:0
          @     0x7f8442526773  kudu::internal::InvokeHelper<>::MakeItSo() at ??:0
          @     0x7f844252475a  kudu::internal::Invoker<>::Run() at ??:0
          @     0x7f844288b654  kudu::Callback<>::Run() at ??:0
          @     0x7f843c4e9a10  kudu::ClosureRunnable::Run() at ??:0
          @     0x7f843c4e8a73  kudu::ThreadPool::DispatchThread() at ??:0
      

      A simple fix would be to treat failures in SnoozeFailureDetectors as non-fatal and stay in the do-while loop.

      Attachments

        Issue Links

          Activity

            People

              adar Adar Dembo
              adar Adar Dembo
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: