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

Performance issue in libprocess SocketManager.

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Blocker
    • Resolution: Fixed
    • 0.21.0
    • 0.21.1
    • libprocess
    • None
    • Twitter Mesos Q4 Sprint 5
    • 3

    Description

      Noticed an issue in production under which the master is slow to respond after failover for ~15 minutes.

      After looking at some perf data, the top offender is:

          12.02%  mesos-master  libmesos-0.21.0-rc3.so  [.] std::_Rb_tree<process::ProcessBase*, process::ProcessBase*, std::_Identity<process::ProcessBase*>, std::less<process::ProcessBase*>, std::allocator<process::ProcessBase*> >::erase(process::ProcessBase* const&)
      ...
           3.29%  mesos-master  libmesos-0.21.0-rc3.so  [.] process::SocketManager::exited(process::ProcessBase*)
      

      It appears that in the SocketManager, whenever an internal Process exits, we loop over all the links unnecessarily:

      void SocketManager::exited(ProcessBase* process)
      {
        // An exited event is enough to cause the process to get deleted
        // (e.g., by the garbage collector), which means we can't
        // dereference process (or even use the address) after we enqueue at
        // least one exited event. Thus, we save the process pid.
        const UPID pid = process->pid;
      
        // Likewise, we need to save the current time of the process so we
        // can update the clocks of linked processes as appropriate.
        const Time time = Clock::now(process);
      
        synchronized (this) {
          // Iterate through the links, removing any links the process might
          // have had and creating exited events for any linked processes.
          foreachpair (const UPID& linkee, set<ProcessBase*>& processes, links) {
            processes.erase(process);
      
            if (linkee == pid) {
              foreach (ProcessBase* linker, processes) {
                CHECK(linker != process) << "Process linked with itself";
                synchronized (timeouts) {
                  if (Clock::paused()) {
                    Clock::update(linker, time);
                  }
                }
                linker->enqueue(new ExitedEvent(linkee));
              }
            }
          }
      
          links.erase(pid);
        }
      }
      

      On clusters with 10,000s of slaves, this means we hold the socket manager lock for a very expensive loop erasing nothing from a set! This is because, the master contains links from the Master Process to each slave. However, when a random ephemeral Process terminates, we don't need to loop over each slave link.

      While we hold this lock, the following calls will block:

      class SocketManager
      {
      public:
        Socket accepted(int s);
        void link(ProcessBase* process, const UPID& to);
        PID<HttpProxy> proxy(const Socket& socket);
        void send(Encoder* encoder, bool persist);
        void send(const Response& response,
                  const Request& request,
                  const Socket& socket);
        void send(Message* message);
        Encoder* next(int s);
        void close(int s);
        void exited(const Node& node);
        void exited(ProcessBase* process);
      ...
      

      As a result, the slave observers and the master can block calling send()!

      Short term, we will try to fix this issue by removing the unnecessary looping. Longer term, it would be nice to avoid all this locking when sending on independent sockets.

      Attachments

        Activity

          People

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

            Dates

              Created:
              Updated:
              Resolved: