Uploaded image for project: 'Mesos'
  1. Mesos
  2. MESOS-800

CHECK failure in cgroups_isolator.

Agile BoardAttach filesAttach ScreenshotVotersWatch issueWatchersLinkCloneUpdate Comment AuthorReplace String in CommentUpdate Comment VisibilityDelete Comments
    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 0.14.0, 0.14.1, 0.14.2, 0.15.0
    • 0.16.0
    • None
    • None

    Description

      F1105 17:22:04.206166 35860 cgroups_isolator.cpp:1205] Check failed: !info->killed OOM detected for an already killed executor

          • Check failure stack trace: ***
            @ 0x7f3ad114262d google::LogMessage::Fail()
            @ 0x7f3ad1146617 google::LogMessage::SendToLog()
            @ 0x7f3ad1144f14 google::LogMessage::Flush()
            @ 0x7f3ad1145146 google::LogMessageFatal::~LogMessageFatal()
            @ 0x7f3ad0f0c142 mesos::internal::slave::CgroupsIsolator::oom()
            @ 0x7f3ad0f0c571 mesos::internal::slave::CgroupsIsolator::oomWaited()
            @ 0x7f3ad0f1de61 std::tr1::_Function_handler<>::_M_invoke()
            @ 0x7f3ad0f1fb54 std::tr1::_Function_handler<>::_M_invoke()
            @ 0x7f3ad1033f84 process::ProcessManager::resume()
            @ 0x7f3ad10349df process::schedule()
            @ 0x7f3ad079d83d start_thread
            @ 0x7f3acf17ff8d clone

      This is because we're not ignoring killed executors in the OOM handler, see my comments below:

      void CgroupsIsolator::killExecutor(
      const FrameworkID& frameworkId,
      const ExecutorID& executorId)
      {
      ...
      // Stop the OOM listener if needed.
      // XXX: The OOM listener can already be ready at this point! This means we need to ignore killed executors in the OOM handler.
      if (info->oomNotifier.isPending())

      { info->oomNotifier.discard(); }

      info->killed = true;
      }

      Given my comment above, we need to ignore killed executors in the OOM handler. Instead, we perform a CHECK which can fail when the race between kill and OOM occurs:

      void CgroupsIsolator::oomWaited(
      const FrameworkID& frameworkId,
      const ExecutorID& executorId,
      const UUID& uuid,
      const Future<uint64_t>& future)
      {
      LOG(INFO) << "OOM notifier is triggered for executor "
      << executorId << " of framework " << frameworkId
      << " with uuid " << uuid;

      if (future.isDiscarded())

      { LOG(INFO) << "Discarded OOM notifier for executor " << executorId << " of framework " << frameworkId << " with uuid " << uuid; }

      else if (future.isFailed())

      { LOG(ERROR) << "Listening on OOM events failed for executor " << executorId << " of framework " << frameworkId << " with uuid " << uuid << ": " << future.failure(); }

      else

      { // Out-of-memory event happened, call the handler. oom(frameworkId, executorId, uuid); }

      }

      void CgroupsIsolator::oom(
      const FrameworkID& frameworkId,
      const ExecutorID& executorId,
      const UUID& uuid)
      {
      CgroupInfo* info = findCgroupInfo(frameworkId, executorId);
      if (info == NULL)

      { // It is likely that processExited is executed before this function (e.g. // The kill and OOM events happen at the same time, and the process exit // event arrives first.) Therefore, we should not report a fatal error here. LOG(INFO) << "OOM detected for an already terminated executor"; return; }

      // We can also ignore an OOM event that we are late to process for a
      // previous instance of an executor.
      CHECK_SOME(info->uuid);
      if (uuid != info->uuid.get())

      { LOG(INFO) << "OOM detected for a previous executor instance"; return; }

      // If killed is set, the OOM notifier will be discarded in oomWaited.
      // Therefore, we should not be able to reach this point.
      // XXX: ^ This comment is false. oomWaited does not ignore killed executors.
      CHECK(!info->killed) << "OOM detected for an already killed executor";
      ...

      Attachments

        Activity

          This comment will be Viewable by All Users Viewable by All Users
          Cancel

          People

            bmahler Benjamin Mahler
            bmahler Benjamin Mahler
            Votes:
            0 Vote for this issue
            Watchers:
            1 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved:

              Slack

                Issue deployment