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

Race in Log shutdown

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • M5
    • None
    • log
    • None

    Description

      There is currently a race during Log shutdown due to sketchy early return in ~Log() I think... basically it's possible to end up with stuff in the log append queue at destruction time.

      The fix probably involves calling Shutdown() on the blocking queue during Log::Close() and not doing the early return thing... just always call Close() in ~Log().

      Check trace:

      I0512 17:30:08.830049 21196 raft_consensus.cc:1032] T TestTablet P peer-0 [term 1 LEADER]: Raft consensus shutting down.
      I0512 17:30:08.830078 21196 raft_consensus.cc:1047] T TestTablet P peer-0 [term 1 LEADER]: Raft consensus Shutdown!
      F0512 17:30:08.838702 21196 blocking_queue.h:55] Check failed: list_.empty() || !std::tr1::is_pointer<T>::value BlockingQueue holds bare pointers at destruction time
      *** Check failure stack trace: ***
      *** Aborted at 1431477008 (unix time) try "date -d @1431477008" if you are using GNU date ***
      PC: @     0x7faa5c49ccc9 (unknown)
      *** SIGABRT (@0x3e8000052cc) received by PID 21196 (TID 0x7faa5a01e800) from PID 21196; stack trace: ***
          @     0x7faa5d4d9340 (unknown) at ??:0
          @     0x7faa5c49ccc9 (unknown) at ??:0
          @     0x7faa5c4a00d8 (unknown) at ??:0
          @     0x7faa5e274d39 google::logging_fail() at ??:0
          @     0x7faa5e27638d google::LogMessage::Fail() at ??:0
          @     0x7faa5e27822d google::LogMessage::SendToLog() at ??:0
          @     0x7faa5e275f7c google::LogMessage::Flush() at ??:0
          @     0x7faa5e278b4e google::LogMessageFatal::~LogMessageFatal() at ??:0
          @     0x7faa5e33dfd8 kudu::BlockingQueue<>::~BlockingQueue() at ??:0
          @     0x7faa5e335faa kudu::log::Log::~Log() at ??:0
          @           0x50a727 kudu::RefCountedThreadSafe<>::DeleteInternal() at /home/mpercy/src/kudu/src/kudu/gutil/ref_counted.h:152
          @           0x50542d kudu::DefaultRefCountedThreadSafeTraits<>::Destruct() at /home/mpercy/src/kudu/src/kudu/gutil/ref_counted.h:117
          @           0x4fcd66 kudu::RefCountedThreadSafe<>::Release() at /home/mpercy/src/kudu/src/kudu/gutil/ref_counted.h:145
          @           0x4f3e09 scoped_refptr<>::~scoped_refptr() at /home/mpercy/src/kudu/src/kudu/gutil/ref_counted.h:252
          @           0x4f14ed kudu::consensus::RaftConsensusTest::~RaftConsensusTest() at /home/mpercy/src/kudu/src/kudu/consensus/raft_consensus-test.cc:141
          @           0x514840 kudu::consensus::RaftConsensusTest_TestCommittedIndexWhenInSameTerm_Test::~RaftConsensusTest_TestCommittedIndexWhenInSameTerm_Test() at /home/mpercy/src/kudu/src/kudu/consensus/raft_consensus-test.cc:286
          @           0x514870 kudu::consensus::RaftConsensusTest_TestCommittedIndexWhenInSameTerm_Test::~RaftConsensusTest_TestCommittedIndexWhenInSameTerm_Test() at /home/mpercy/src/kudu/src/kudu/consensus/raft_consensus-test.cc:286
          @     0x7faa5d08c553 testing::internal::HandleExceptionsInMethodIfSupported<>() at ??:0
          @     0x7faa5d07e7b7 testing::TestInfo::Run() at ??:0
          @     0x7faa5d07e895 testing::TestCase::Run() at ??:0
          @     0x7faa5d07eb48 testing::internal::UnitTestImpl::RunAllTests() at ??:0
          @     0x7faa5d07edf4 testing::UnitTest::Run() at ??:0
          @     0x7faa5e2a72e3 RUN_ALL_TESTS() at ??:0
          @     0x7faa5e2a6c9f main at ??:0
          @     0x7faa5c487ec5 (unknown) at ??:0
          @           0x4e0e19 (unknown) at ??:?
          @                0x0 (unknown)
      

      Attachments

        Activity

          People

            mpercy Mike Percy
            mpercy Mike Percy
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: