You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by ok...@apache.org on 2018/08/14 03:17:18 UTC

[trafficserver] branch master updated: Optimize: Assign nh->mutex to new NetVC first and switch to new mutex within UnixNetVC::acceptEvent

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 9e32c19  Optimize: Assign nh->mutex to new NetVC first and switch to new mutex within UnixNetVC::acceptEvent
9e32c19 is described below

commit 9e32c19872870705df255f7dfea46f6eaec226a9
Author: Oknet Xu <xu...@skyguard.com.cn>
AuthorDate: Fri Aug 10 15:09:09 2018 +0800

    Optimize: Assign nh->mutex to new NetVC first and switch to new mutex within UnixNetVC::acceptEvent
---
 iocore/net/UnixNetAccept.cc      | 58 +++++++++++++++++++++++-----------------
 iocore/net/UnixNetVConnection.cc | 15 +++--------
 2 files changed, 38 insertions(+), 35 deletions(-)

diff --git a/iocore/net/UnixNetAccept.cc b/iocore/net/UnixNetAccept.cc
index f4ee913..21f39c4 100644
--- a/iocore/net/UnixNetAccept.cc
+++ b/iocore/net/UnixNetAccept.cc
@@ -113,7 +113,6 @@ net_accept(NetAccept *na, void *ep, bool blockable)
     vc->id = net_next_connection_number();
     vc->con.move(con);
     vc->submit_time = Thread::get_hrtime();
-    vc->mutex       = new_ProxyMutex();
     vc->action_     = *na->action_;
     vc->set_is_transparent(na->opt.f_inbound_transparent);
     vc->set_context(NET_VCONNECTION_IN);
@@ -125,10 +124,25 @@ net_accept(NetAccept *na, void *ep, bool blockable)
 #endif
     SET_CONTINUATION_HANDLER(vc, (NetVConnHandler)&UnixNetVConnection::acceptEvent);
 
+    EThread *t;
+    NetHandler *h;
     if (e->ethread->is_event_type(na->opt.etype)) {
-      vc->dispatchEvent(EVENT_NONE, e);
+      t = e->ethread;
+      h = get_NetHandler(t);
+      // Assign NetHandler->mutex to NetVC
+      vc->mutex = h->mutex;
+      MUTEX_TRY_LOCK(lock, h->mutex, t);
+      if (!lock.is_locked()) {
+        t->schedule_in(vc, HRTIME_MSECONDS(net_retry_delay));
+      } else {
+        vc->handleEvent(EVENT_NONE, e);
+      }
     } else {
-      eventProcessor.schedule_imm(vc, na->opt.etype);
+      t = eventProcessor.assign_thread(na->opt.etype);
+      h = get_NetHandler(t);
+      // Assign NetHandler->mutex to NetVC
+      vc->mutex = h->mutex;
+      t->schedule_imm(vc);
     }
   } while (loop);
 
@@ -282,12 +296,11 @@ NetAccept::do_blocking_accept(EThread *t)
   int loop               = accept_till_done;
   UnixNetVConnection *vc = nullptr;
   Connection con;
+  con.sock_type = SOCK_STREAM;
 
   // do-while for accepting all the connections
   // added by YTS Team, yamsat
   do {
-    ink_hrtime now = Thread::get_hrtime();
-
     // Throttle accepts
     while (!opt.backdoor && check_net_throttle(ACCEPT)) {
       check_throttle_warning(ACCEPT);
@@ -296,7 +309,6 @@ NetAccept::do_blocking_accept(EThread *t)
         goto Lerror;
       }
       NET_SUM_DYN_STAT(net_connections_throttled_in_stat, num_throttled);
-      now = Thread::get_hrtime();
     }
 
     if ((res = server.accept(&con)) < 0) {
@@ -321,6 +333,8 @@ NetAccept::do_blocking_accept(EThread *t)
       return -1;
     }
 
+    NET_SUM_GLOBAL_DYN_STAT(net_tcp_accept_stat, 1);
+
     // Use 'nullptr' to Bypass thread allocator
     vc = (UnixNetVConnection *)this->getNetProcessor()->allocate_vc(nullptr);
     if (unlikely(!vc)) {
@@ -328,11 +342,9 @@ NetAccept::do_blocking_accept(EThread *t)
     }
 
     NET_SUM_GLOBAL_DYN_STAT(net_connections_currently_open_stat, 1);
-    NET_SUM_GLOBAL_DYN_STAT(net_tcp_accept_stat, 1);
     vc->id = net_next_connection_number();
     vc->con.move(con);
-    vc->submit_time = now;
-    vc->mutex       = new_ProxyMutex();
+    vc->submit_time = Thread::get_hrtime();
     vc->action_     = *action_;
     vc->set_is_transparent(opt.f_inbound_transparent);
     vc->options.packet_mark = opt.packet_mark;
@@ -348,8 +360,12 @@ NetAccept::do_blocking_accept(EThread *t)
     }
 #endif
     SET_CONTINUATION_HANDLER(vc, (NetVConnHandler)&UnixNetVConnection::acceptEvent);
-    // eventProcessor.schedule_imm(vc, getEtype());
-    eventProcessor.schedule_imm_signal(vc, opt.etype);
+
+    EThread *t    = eventProcessor.assign_thread(opt.etype);
+    NetHandler *h = get_NetHandler(t);
+    // Assign NetHandler->mutex to NetVC
+    vc->mutex = h->mutex;
+    t->schedule_imm_signal(vc);
   } while (loop);
 
   return 1;
@@ -406,6 +422,7 @@ NetAccept::acceptFastEvent(int event, void *ep)
   (void)e;
   int bufsz, res = 0;
   Connection con;
+  con.sock_type = SOCK_STREAM;
 
   UnixNetVConnection *vc = nullptr;
   int loop               = accept_till_done;
@@ -445,16 +462,6 @@ NetAccept::acceptFastEvent(int event, void *ep)
           }
         }
       }
-
-      if (opt.sockopt_flags & NetVCOptions::SOCK_OPT_NO_DELAY) {
-        safe_setsockopt(fd, IPPROTO_TCP, TCP_NODELAY, SOCKOPT_ON, sizeof(int));
-        Debug("socket", "::acceptFastEvent: setsockopt() TCP_NODELAY on socket");
-      }
-
-      if (opt.sockopt_flags & NetVCOptions::SOCK_OPT_KEEP_ALIVE) {
-        safe_setsockopt(fd, SOL_SOCKET, SO_KEEPALIVE, SOCKOPT_ON, sizeof(int));
-        Debug("socket", "::acceptFastEvent: setsockopt() SO_KEEPALIVE on socket");
-      }
     } else {
       res = fd;
     }
@@ -486,15 +493,13 @@ NetAccept::acceptFastEvent(int event, void *ep)
     vc->id = net_next_connection_number();
     vc->con.move(con);
     vc->submit_time = Thread::get_hrtime();
-    vc->mutex       = new_ProxyMutex();
-    // no need to set vc->action_
+    vc->action_     = *action_;
     vc->set_is_transparent(opt.f_inbound_transparent);
     vc->options.packet_mark = opt.packet_mark;
     vc->options.packet_tos  = opt.packet_tos;
     vc->options.ip_family   = opt.ip_family;
     vc->apply_options();
     vc->set_context(NET_VCONNECTION_IN);
-    vc->action_ = *action_;
 #ifdef USE_EDGE_TRIGGER
     // Set the vc as triggered and place it in the read ready queue later in case there is already data on the socket.
     if (server.http_accept_filter) {
@@ -502,6 +507,11 @@ NetAccept::acceptFastEvent(int event, void *ep)
     }
 #endif
     SET_CONTINUATION_HANDLER(vc, (NetVConnHandler)&UnixNetVConnection::acceptEvent);
+
+    EThread *t    = e->ethread;
+    NetHandler *h = get_NetHandler(t);
+    // Assign NetHandler->mutex to NetVC
+    vc->mutex = h->mutex;
     // We must be holding the lock already to do later do_io_read's
     SCOPED_MUTEX_LOCK(lock, vc->mutex, e->ethread);
     vc->handleEvent(EVENT_NONE, nullptr);
diff --git a/iocore/net/UnixNetVConnection.cc b/iocore/net/UnixNetVConnection.cc
index afc7dfb..d2834a3 100644
--- a/iocore/net/UnixNetVConnection.cc
+++ b/iocore/net/UnixNetVConnection.cc
@@ -1099,17 +1099,6 @@ UnixNetVConnection::acceptEvent(int event, Event *e)
   EThread *t    = (e == nullptr) ? this_ethread() : e->ethread;
   NetHandler *h = get_NetHandler(t);
 
-  MUTEX_TRY_LOCK(lock, h->mutex, t);
-  if (!lock.is_locked()) {
-    if (event == EVENT_NONE) {
-      t->schedule_in(this, HRTIME_MSECONDS(net_retry_delay));
-      return EVENT_DONE;
-    } else {
-      e->schedule_in(HRTIME_MSECONDS(net_retry_delay));
-      return EVENT_CONT;
-    }
-  }
-
   thread = t;
 
   // Send this NetVC to NetHandler and start to polling read & write event.
@@ -1118,6 +1107,10 @@ UnixNetVConnection::acceptEvent(int event, Event *e)
     return EVENT_DONE;
   }
 
+  // Switch vc->mutex from NetHandler->mutex to new mutex
+  mutex = new_ProxyMutex();
+  SCOPED_MUTEX_LOCK(lock2, mutex, t);
+
   // Setup a timeout callback handler.
   SET_HANDLER((NetVConnHandler)&UnixNetVConnection::mainEvent);