From 27863612d4e61c561aa7a2c4eee0b9622c17510f Mon Sep 17 00:00:00 2001 From: scw00 Date: Mon, 23 Dec 2019 14:24:25 +0800 Subject: [PATCH] Introduce NetEvent to split UnixNetVConnection and NetHandler --- iocore/net/NetEvent.h | 92 +++++++++ iocore/net/P_UnixNet.h | 198 +++++++++---------- iocore/net/P_UnixNetState.h | 6 +- iocore/net/P_UnixNetVConnection.h | 75 +++---- iocore/net/QUICNetVConnection.cc | 2 +- iocore/net/UnixNet.cc | 319 +++++++++++++++--------------- iocore/net/UnixNetPages.cc | 12 +- iocore/net/UnixNetVConnection.cc | 19 +- 8 files changed, 417 insertions(+), 306 deletions(-) create mode 100644 iocore/net/NetEvent.h diff --git a/iocore/net/NetEvent.h b/iocore/net/NetEvent.h new file mode 100644 index 00000000000..a062e86dbc2 --- /dev/null +++ b/iocore/net/NetEvent.h @@ -0,0 +1,92 @@ +/** @file + + A brief file description + + @section license License + + Licensed to the Apache Software Foundation (ASF) under one + or more contributor license agreements. See the NOTICE file + distributed with this work for additional information + regarding copyright ownership. The ASF licenses this file + to you under the Apache License, Version 2.0 (the + "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + */ + +#pragma once + +#include "I_EventSystem.h" + +class NetHandler; + +// this class is used to NetHandler to hide some detail of NetEvent. +// To combine the `UDPConenction` and `NetEvent`. NetHandler should +// callback to net_read_io or net_write_io when net event happen. +class NetEvent +{ +public: + NetEvent() = default; + virtual ~NetEvent() {} + virtual void net_read_io(NetHandler *nh, EThread *lthread) = 0; + virtual void net_write_io(NetHandler *nh, EThread *lthread) = 0; + virtual void free(EThread *t) = 0; + + // since we want this class to be independent from VConnection, Continutaion. There should be + // a pure virtual function which connect sub class and NetHandler. + virtual int callback(int event = CONTINUATION_EVENT_NONE, void *data = nullptr) = 0; + + // Duplicate with `NetVConnection::set_inactivity_timeout` + // TODO: more abstraction. + virtual void set_inactivity_timeout(ink_hrtime timeout_in) = 0; + + // get this vc's thread + virtual EThread *get_thread() = 0; + + // Close when EventIO close; + virtual int close() = 0; + + // get fd + virtual int get_fd() = 0; + virtual Ptr &get_mutex() = 0; + virtual ContFlags &get_control_flags() = 0; + + EventIO ep{}; + NetState read{}; + NetState write{}; + + bool closed = false; + NetHandler *nh = nullptr; + + ink_hrtime inactivity_timeout_in = 0; + ink_hrtime active_timeout_in = 0; + ink_hrtime next_inactivity_timeout_at = 0; + ink_hrtime next_activity_timeout_at = 0; + ink_hrtime submit_time = 0; + + LINK(NetEvent, open_link); + LINK(NetEvent, cop_link); + LINKM(NetEvent, read, ready_link) + SLINKM(NetEvent, read, enable_link) + LINKM(NetEvent, write, ready_link) + SLINKM(NetEvent, write, enable_link) + LINK(NetEvent, keep_alive_queue_link); + LINK(NetEvent, active_queue_link); + + union { + unsigned int flags = 0; +#define NET_VC_SHUTDOWN_READ 1 +#define NET_VC_SHUTDOWN_WRITE 2 + struct { + unsigned int got_local_addr : 1; + unsigned int shutdown : 2; + } f; + }; +}; diff --git a/iocore/net/P_UnixNet.h b/iocore/net/P_UnixNet.h index 91ccb6fec0d..800be6b209c 100644 --- a/iocore/net/P_UnixNet.h +++ b/iocore/net/P_UnixNet.h @@ -75,7 +75,7 @@ struct PollDescriptor; typedef PollDescriptor *EventLoop; -class UnixNetVConnection; +class NetEvent; class UnixUDPConnection; struct DNSConnection; struct NetAccept; @@ -89,14 +89,14 @@ struct EventIO { int type = 0; union { Continuation *c; - UnixNetVConnection *vc; + NetEvent *ne; DNSConnection *dnscon; NetAccept *na; UnixUDPConnection *uc; } data; int start(EventLoop l, DNSConnection *vc, int events); int start(EventLoop l, NetAccept *vc, int events); - int start(EventLoop l, UnixNetVConnection *vc, int events); + int start(EventLoop l, NetEvent *ne, int events); int start(EventLoop l, UnixUDPConnection *vc, int events); int start(EventLoop l, int fd, Continuation *c, int events); // Change the existing events by adding modify(EVENTIO_READ) @@ -118,7 +118,7 @@ struct EventIO { #include "P_UnixPollDescriptor.h" #include -class UnixNetVConnection; +class NetEvent; class NetHandler; typedef int (NetHandler::*NetContHandler)(int, void *); typedef unsigned int uint32; @@ -171,9 +171,9 @@ struct PollCont : public Continuation { }; /** - NetHandler is the processor of NetVC for the Net sub-system. The NetHandler + NetHandler is the processor of NetEvent for the Net sub-system. The NetHandler is the core component of the Net sub-system. Once started, it is responsible - for polling socket fds and perform the I/O tasks in NetVC. + for polling socket fds and perform the I/O tasks in NetEvent. The NetHandler is executed periodically to perform read/write tasks for NetVConnection. The NetHandler::mainNetEvent() should be viewed as a part of @@ -181,7 +181,7 @@ struct PollCont : public Continuation { By get_NetHandler(this_ethread()), you can get the NetHandler object that runs inside the current EThread and then @c startIO / @c stopIO which - assign/release a NetVC to/from NetHandler. Before you call these functions, + assign/release a NetEvent to/from NetHandler. Before you call these functions, holding the mutex of this NetHandler is required. The NetVConnection provides a set of do_io functions through which you can @@ -192,10 +192,11 @@ struct PollCont : public Continuation { Multi-thread scheduler: The NetHandler should be viewed as multi-threaded schedulers which process - NetVCs from their queues. The NetVC can be made of NetProcessor (allocate_vc) - either by directly adding a NetVC to the queue (NetHandler::startIO), or more + NetEvents from their queues. If vc wants to be managed by NetHandler, the vc + should be derived from NetEvent. The vc can be made of NetProcessor (allocate_vc) + either by directly adding a NetEvent to the queue (NetHandler::startIO), or more conveniently, calling a method service call (NetProcessor::connect_re) which - synthesizes the NetVC and places it in the queue. + synthesizes the NetEvent and places it in the queue. Callback event codes: @@ -211,14 +212,14 @@ struct PollCont : public Continuation { NetVConnection allocation policy: - NetVCs are allocated by the NetProcessor and deallocated by NetHandler. - A state machine may access the returned, non-recurring NetVC / VIO until - it is closed by do_io_close. For recurring NetVC, the NetVC may be - accessed until it is closed. Once the NetVC is closed, it's the + VCs are allocated by the NetProcessor and deallocated by NetHandler. + A state machine may access the returned, non-recurring NetEvent / VIO until + it is closed by do_io_close. For recurring NetEvent, the NetEvent may be + accessed until it is closed. Once the NetEvent is closed, it's the NetHandler's responsibility to deallocate it. Before assign to NetHandler or after release from NetHandler, it's the - NetVC's responsibility to deallocate itself. + NetEvent's responsibility to deallocate itself. */ @@ -236,15 +237,15 @@ class NetHandler : public Continuation, public EThread::LoopTailHandler // If we don't get rid of @a trigger_event we should remove @a thread. EThread *thread = nullptr; Event *trigger_event = nullptr; - QueM(UnixNetVConnection, NetState, read, ready_link) read_ready_list; - QueM(UnixNetVConnection, NetState, write, ready_link) write_ready_list; - Que(UnixNetVConnection, link) open_list; - DList(UnixNetVConnection, cop_link) cop_list; - ASLLM(UnixNetVConnection, NetState, read, enable_link) read_enable_list; - ASLLM(UnixNetVConnection, NetState, write, enable_link) write_enable_list; - Que(UnixNetVConnection, keep_alive_queue_link) keep_alive_queue; + QueM(NetEvent, NetState, read, ready_link) read_ready_list; + QueM(NetEvent, NetState, write, ready_link) write_ready_list; + Que(NetEvent, open_link) open_list; + DList(NetEvent, cop_link) cop_list; + ASLLM(NetEvent, NetState, read, enable_link) read_enable_list; + ASLLM(NetEvent, NetState, write, enable_link) write_enable_list; + Que(NetEvent, keep_alive_queue_link) keep_alive_queue; uint32_t keep_alive_queue_size = 0; - Que(UnixNetVConnection, active_queue_link) active_queue; + Que(NetEvent, active_queue_link) active_queue; uint32_t active_queue_size = 0; /// configuration settings for managing the active and keep-alive queues @@ -293,10 +294,10 @@ class NetHandler : public Continuation, public EThread::LoopTailHandler void process_ready_list(); void manage_keep_alive_queue(); bool manage_active_queue(bool ignore_queue_size); - void add_to_keep_alive_queue(UnixNetVConnection *vc); - void remove_from_keep_alive_queue(UnixNetVConnection *vc); - bool add_to_active_queue(UnixNetVConnection *vc); - void remove_from_active_queue(UnixNetVConnection *vc); + void add_to_keep_alive_queue(NetEvent *ne); + void remove_from_keep_alive_queue(NetEvent *ne); + bool add_to_active_queue(NetEvent *ne); + void remove_from_active_queue(NetEvent *ne); /// Per process initialization logic. static void init_for_process(); @@ -304,58 +305,57 @@ class NetHandler : public Continuation, public EThread::LoopTailHandler void configure_per_thread_values(); /** - Start to handle read & write event on a UnixNetVConnection. - Initial the socket fd of netvc for polling system. + Start to handle read & write event on a NetEvent. + Initial the socket fd of ne for polling system. Only be called when holding the mutex of this NetHandler. - @param netvc UnixNetVConnection to be managed by this NetHandler. - @return 0 on success, netvc->nh set to this NetHandler. + @param ne NetEvent to be managed by this NetHandler. + @return 0 on success, ne->nh set to this NetHandler. -ERRNO on failure. */ - int startIO(UnixNetVConnection *netvc); + int startIO(NetEvent *ne); /** - Stop to handle read & write event on a UnixNetVConnection. - Remove the socket fd of netvc from polling system. - Only be called when holding the mutex of this NetHandler and must call stopCop(netvc) first. + Stop to handle read & write event on a NetEvent. + Remove the socket fd of ne from polling system. + Only be called when holding the mutex of this NetHandler and must call stopCop(ne) first. - @param netvc UnixNetVConnection to be released. - @return netvc->nh set to nullptr. + @param ne NetEvent to be released. + @return ne->nh set to nullptr. */ - void stopIO(UnixNetVConnection *netvc); + void stopIO(NetEvent *ne); /** - Start to handle active timeout and inactivity timeout on a UnixNetVConnection. - Put the netvc into open_list. All NetVCs in the open_list is checked for timeout by InactivityCop. - Only be called when holding the mutex of this NetHandler and must call startIO(netvc) first. + Start to handle active timeout and inactivity timeout on a NetEvent. + Put the ne into open_list. All NetEvents in the open_list is checked for timeout by InactivityCop. + Only be called when holding the mutex of this NetHandler and must call startIO(ne) first. - @param netvc UnixNetVConnection to be managed by InactivityCop + @param ne NetEvent to be managed by InactivityCop */ - void startCop(UnixNetVConnection *netvc); + void startCop(NetEvent *ne); /** - Stop to handle active timeout and inactivity on a UnixNetVConnection. - Remove the netvc from open_list and cop_list. - Also remove the netvc from keep_alive_queue and active_queue if its context is IN. + Stop to handle active timeout and inactivity on a NetEvent. + Remove the ne from open_list and cop_list. + Also remove the ne from keep_alive_queue and active_queue if its context is IN. Only be called when holding the mutex of this NetHandler. - @param netvc UnixNetVConnection to be released. + @param ne NetEvent to be released. */ - void stopCop(UnixNetVConnection *netvc); + void stopCop(NetEvent *ne); // Signal the epoll_wait to terminate. void signalActivity() override; /** - Release a netvc and free it. + Release a ne and free it. - @param netvc UnixNetVConnection to be detached. + @param ne NetEvent to be detached. */ - void free_netvc(UnixNetVConnection *netvc); + void free_netevent(NetEvent *ne); NetHandler(); private: - void _close_vc(UnixNetVConnection *vc, ink_hrtime now, int &handle_event, int &closed, int &total_idle_time, - int &total_idle_count); + void _close_ne(NetEvent *ne, ink_hrtime now, int &handle_event, int &closed, int &total_idle_time, int &total_idle_count); /// Static method used as the callback for runtime configuration updates. static int update_nethandler_config(const char *name, RecDataT, RecData data, void *); @@ -507,30 +507,30 @@ check_transient_accept_error(int res) } // -// Disable a UnixNetVConnection +// Disable a NetEvent // static inline void -read_disable(NetHandler *nh, UnixNetVConnection *vc) +read_disable(NetHandler *nh, NetEvent *ne) { - if (!vc->write.enabled) { - vc->set_inactivity_timeout(0); - Debug("socket", "read_disable updating inactivity_at %" PRId64 ", NetVC=%p", vc->next_inactivity_timeout_at, vc); + if (!ne->write.enabled) { + ne->set_inactivity_timeout(0); + Debug("socket", "read_disable updating inactivity_at %" PRId64 ", NetEvent=%p", ne->next_inactivity_timeout_at, ne); } - vc->read.enabled = 0; - nh->read_ready_list.remove(vc); - vc->ep.modify(-EVENTIO_READ); + ne->read.enabled = 0; + nh->read_ready_list.remove(ne); + ne->ep.modify(-EVENTIO_READ); } static inline void -write_disable(NetHandler *nh, UnixNetVConnection *vc) +write_disable(NetHandler *nh, NetEvent *ne) { - if (!vc->read.enabled) { - vc->set_inactivity_timeout(0); - Debug("socket", "write_disable updating inactivity_at %" PRId64 ", NetVC=%p", vc->next_inactivity_timeout_at, vc); + if (!ne->read.enabled) { + ne->set_inactivity_timeout(0); + Debug("socket", "write_disable updating inactivity_at %" PRId64 ", NetEvent=%p", ne->next_inactivity_timeout_at, ne); } - vc->write.enabled = 0; - nh->write_ready_list.remove(vc); - vc->ep.modify(-EVENTIO_WRITE); + ne->write.enabled = 0; + nh->write_ready_list.remove(ne); + ne->ep.modify(-EVENTIO_WRITE); } TS_INLINE int @@ -546,10 +546,10 @@ EventIO::start(EventLoop l, NetAccept *vc, int events) return start(l, vc->server.fd, (Continuation *)vc, events); } TS_INLINE int -EventIO::start(EventLoop l, UnixNetVConnection *vc, int events) +EventIO::start(EventLoop l, NetEvent *ne, int events) { type = EVENTIO_READWRITE_VC; - return start(l, vc->con.fd, (Continuation *)vc, events); + return start(l, ne->get_fd(), (Continuation *)ne, events); } TS_INLINE int EventIO::start(EventLoop l, UnixUDPConnection *vc, int events) @@ -576,7 +576,7 @@ EventIO::close() return data.na->server.close(); break; case EVENTIO_READWRITE_VC: - return data.vc->con.close(); + return data.ne->close(); break; } return -1; @@ -775,14 +775,14 @@ EventIO::stop() } TS_INLINE int -NetHandler::startIO(UnixNetVConnection *netvc) +NetHandler::startIO(NetEvent *ne) { ink_assert(this->mutex->thread_holding == this_ethread()); - ink_assert(netvc->thread == this_ethread()); + ink_assert(ne->get_thread() == this_ethread()); int res = 0; PollDescriptor *pd = get_PollDescriptor(this->thread); - if (netvc->ep.start(pd, netvc, EVENTIO_READ | EVENTIO_WRITE) < 0) { + if (ne->ep.start(pd, ne, EVENTIO_READ | EVENTIO_WRITE) < 0) { res = errno; // EEXIST should be ok, though it should have been cleared before we got back here if (errno != EEXIST) { @@ -791,51 +791,51 @@ NetHandler::startIO(UnixNetVConnection *netvc) } } - if (netvc->read.triggered == 1) { - read_ready_list.enqueue(netvc); + if (ne->read.triggered == 1) { + read_ready_list.enqueue(ne); } - netvc->nh = this; + ne->nh = this; return res; } TS_INLINE void -NetHandler::stopIO(UnixNetVConnection *netvc) +NetHandler::stopIO(NetEvent *ne) { - ink_release_assert(netvc->nh == this); + ink_release_assert(ne->nh == this); - netvc->ep.stop(); + ne->ep.stop(); - read_ready_list.remove(netvc); - write_ready_list.remove(netvc); - if (netvc->read.in_enabled_list) { - read_enable_list.remove(netvc); - netvc->read.in_enabled_list = 0; + read_ready_list.remove(ne); + write_ready_list.remove(ne); + if (ne->read.in_enabled_list) { + read_enable_list.remove(ne); + ne->read.in_enabled_list = 0; } - if (netvc->write.in_enabled_list) { - write_enable_list.remove(netvc); - netvc->write.in_enabled_list = 0; + if (ne->write.in_enabled_list) { + write_enable_list.remove(ne); + ne->write.in_enabled_list = 0; } - netvc->nh = nullptr; + ne->nh = nullptr; } TS_INLINE void -NetHandler::startCop(UnixNetVConnection *netvc) +NetHandler::startCop(NetEvent *ne) { ink_assert(this->mutex->thread_holding == this_ethread()); - ink_release_assert(netvc->nh == this); - ink_assert(!open_list.in(netvc)); + ink_release_assert(ne->nh == this); + ink_assert(!open_list.in(ne)); - open_list.enqueue(netvc); + open_list.enqueue(ne); } TS_INLINE void -NetHandler::stopCop(UnixNetVConnection *netvc) +NetHandler::stopCop(NetEvent *ne) { - ink_release_assert(netvc->nh == this); + ink_release_assert(ne->nh == this); - open_list.remove(netvc); - cop_list.remove(netvc); - remove_from_keep_alive_queue(netvc); - remove_from_active_queue(netvc); + open_list.remove(ne); + cop_list.remove(ne); + remove_from_keep_alive_queue(ne); + remove_from_active_queue(ne); } diff --git a/iocore/net/P_UnixNetState.h b/iocore/net/P_UnixNetState.h index 3a2265c92c9..46fddbc7164 100644 --- a/iocore/net/P_UnixNetState.h +++ b/iocore/net/P_UnixNetState.h @@ -40,13 +40,13 @@ #include "I_VIO.h" class Event; -class UnixNetVConnection; +class NetEvent; struct NetState { int enabled = 0; VIO vio; - Link ready_link; - SLink enable_link; + Link ready_link; + SLink enable_link; int in_enabled_list = 0; int triggered = 0; diff --git a/iocore/net/P_UnixNetVConnection.h b/iocore/net/P_UnixNetVConnection.h index 26d573de515..6e69ff99b8e 100644 --- a/iocore/net/P_UnixNetVConnection.h +++ b/iocore/net/P_UnixNetVConnection.h @@ -36,6 +36,7 @@ #include "P_UnixNetState.h" #include "P_Connection.h" #include "P_NetAccept.h" +#include "NetEvent.h" class UnixNetVConnection; class NetHandler; @@ -104,7 +105,7 @@ struct OOB_callback : public Continuation { enum tcp_congestion_control_t { CLIENT_SIDE, SERVER_SIDE }; -class UnixNetVConnection : public NetVConnection +class UnixNetVConnection : public NetVConnection, public NetEvent { public: int64_t outstanding() override; @@ -216,7 +217,45 @@ class UnixNetVConnection : public NetVConnection return false; } - virtual void net_read_io(NetHandler *nh, EThread *lthread); + // NetEvent + virtual void net_read_io(NetHandler *nh, EThread *lthread) override; + virtual void net_write_io(NetHandler *nh, EThread *lthread) override; + virtual void free(EThread *t) override; + virtual int + close() override + { + return this->con.close(); + } + virtual int + get_fd() override + { + return this->con.fd; + } + + virtual EThread * + get_thread() override + { + return this->thread; + } + + virtual int + callback(int event = CONTINUATION_EVENT_NONE, void *data = nullptr) override + { + return this->handleEvent(event, data); + } + + virtual Ptr & + get_mutex() override + { + return this->mutex; + } + + virtual ContFlags & + get_control_flags() override + { + return this->control_flags; + } + virtual int64_t load_buffer_and_write(int64_t towrite, MIOBufferAccessor &buf, int64_t &total_written, int &needs); void readDisable(NetHandler *nh); void readSignalError(NetHandler *nh, int err); @@ -233,40 +272,11 @@ class UnixNetVConnection : public NetVConnection UnixNetVConnection *migrateToCurrentThread(Continuation *c, EThread *t); Action action_; - int closed = 0; - NetState read; - NetState write; - - LINK(UnixNetVConnection, cop_link); - LINKM(UnixNetVConnection, read, ready_link) - SLINKM(UnixNetVConnection, read, enable_link) - LINKM(UnixNetVConnection, write, ready_link) - SLINKM(UnixNetVConnection, write, enable_link) - LINK(UnixNetVConnection, keep_alive_queue_link); - LINK(UnixNetVConnection, active_queue_link); - - ink_hrtime inactivity_timeout_in = 0; - ink_hrtime active_timeout_in = 0; - ink_hrtime next_inactivity_timeout_at = 0; - ink_hrtime next_activity_timeout_at = 0; - - EventIO ep; - NetHandler *nh = nullptr; - unsigned int id = 0; - union { - unsigned int flags; -#define NET_VC_SHUTDOWN_READ 1 -#define NET_VC_SHUTDOWN_WRITE 2 - struct { - unsigned int got_local_addr : 1; - unsigned int shutdown : 2; - } f; - }; + unsigned int id = 0; Connection con; int recursion = 0; - ink_hrtime submit_time = 0; OOB_callback *oob_ptr = nullptr; bool from_accept_thread = false; NetAccept *accept_object = nullptr; @@ -287,7 +297,6 @@ class UnixNetVConnection : public NetVConnection */ virtual int populate(Connection &con, Continuation *c, void *arg); virtual void clear(); - virtual void free(EThread *t); ink_hrtime get_inactivity_timeout() override; ink_hrtime get_active_timeout() override; diff --git a/iocore/net/QUICNetVConnection.cc b/iocore/net/QUICNetVConnection.cc index 22d2e61cec4..9c4aa9d3163 100644 --- a/iocore/net/QUICNetVConnection.cc +++ b/iocore/net/QUICNetVConnection.cc @@ -905,7 +905,7 @@ QUICNetVConnection::state_connection_closed(int event, Event *data) // FIXME I'm not sure whether we can block here, but it's needed to not crash. SCOPED_MUTEX_LOCK(lock, this->nh->mutex, this_ethread()); if (this->nh) { - this->nh->free_netvc(this); + this->nh->free_netevent(this); } else { this->free(this->mutex->thread_holding); } diff --git a/iocore/net/UnixNet.cc b/iocore/net/UnixNet.cc index 76900778a46..788c2ad6f07 100644 --- a/iocore/net/UnixNet.cc +++ b/iocore/net/UnixNet.cc @@ -41,7 +41,7 @@ extern "C" void fd_reify(struct ev_loop *); // INKqa10496 // One Inactivity cop runs on each thread once every second and -// loops through the list of NetVCs and calls the timeouts +// loops through the list of NetEvents and calls the timeouts class InactivityCop : public Continuation { public: @@ -54,49 +54,49 @@ class InactivityCop : public Continuation NetHandler &nh = *get_NetHandler(this_ethread()); Debug("inactivity_cop_check", "Checking inactivity on Thread-ID #%d", this_ethread()->id); - // The rest NetVCs in cop_list which are not triggered between InactivityCop runs. + // The rest NetEvents in cop_list which are not triggered between InactivityCop runs. // Use pop() to catch any closes caused by callbacks. - while (UnixNetVConnection *vc = nh.cop_list.pop()) { + while (NetEvent *ne = nh.cop_list.pop()) { // If we cannot get the lock don't stop just keep cleaning - MUTEX_TRY_LOCK(lock, vc->mutex, this_ethread()); + MUTEX_TRY_LOCK(lock, ne->get_mutex(), this_ethread()); if (!lock.is_locked()) { NET_INCREMENT_DYN_STAT(inactivity_cop_lock_acquire_failure_stat); continue; } - if (vc->closed) { - nh.free_netvc(vc); + if (ne->closed) { + nh.free_netevent(ne); continue; } - if (vc->next_inactivity_timeout_at && vc->next_inactivity_timeout_at < now) { - if (nh.keep_alive_queue.in(vc)) { + if (ne->next_inactivity_timeout_at && ne->next_inactivity_timeout_at < now) { + if (nh.keep_alive_queue.in(ne)) { // only stat if the connection is in keep-alive, there can be other inactivity timeouts - ink_hrtime diff = (now - (vc->next_inactivity_timeout_at - vc->inactivity_timeout_in)) / HRTIME_SECOND; + ink_hrtime diff = (now - (ne->next_inactivity_timeout_at - ne->inactivity_timeout_in)) / HRTIME_SECOND; NET_SUM_DYN_STAT(keep_alive_queue_timeout_total_stat, diff); NET_INCREMENT_DYN_STAT(keep_alive_queue_timeout_count_stat); } - Debug("inactivity_cop_verbose", "vc: %p now: %" PRId64 " timeout at: %" PRId64 " timeout in: %" PRId64, vc, - ink_hrtime_to_sec(now), vc->next_inactivity_timeout_at, vc->inactivity_timeout_in); - vc->handleEvent(VC_EVENT_INACTIVITY_TIMEOUT, e); - } else if (vc->next_activity_timeout_at && vc->next_activity_timeout_at < now) { - Debug("inactivity_cop_verbose", "active vc: %p now: %" PRId64 " timeout at: %" PRId64 " timeout in: %" PRId64, vc, - ink_hrtime_to_sec(now), vc->next_activity_timeout_at, vc->active_timeout_in); - vc->handleEvent(VC_EVENT_ACTIVE_TIMEOUT, e); + Debug("inactivity_cop_verbose", "ne: %p now: %" PRId64 " timeout at: %" PRId64 " timeout in: %" PRId64, ne, + ink_hrtime_to_sec(now), ne->next_inactivity_timeout_at, ne->inactivity_timeout_in); + ne->callback(VC_EVENT_INACTIVITY_TIMEOUT, e); + } else if (ne->next_activity_timeout_at && ne->next_activity_timeout_at < now) { + Debug("inactivity_cop_verbose", "active ne: %p now: %" PRId64 " timeout at: %" PRId64 " timeout in: %" PRId64, ne, + ink_hrtime_to_sec(now), ne->next_activity_timeout_at, ne->active_timeout_in); + ne->callback(VC_EVENT_ACTIVE_TIMEOUT, e); } } // The cop_list is empty now. // Let's reload the cop_list from open_list again. - forl_LL(UnixNetVConnection, vc, nh.open_list) + forl_LL(NetEvent, ne, nh.open_list) { - if (vc->thread == this_ethread()) { - nh.cop_list.push(vc); + if (ne->get_thread() == this_ethread()) { + nh.cop_list.push(ne); } } - // NetHandler will remove NetVC from cop_list if it is triggered. - // As the NetHandler runs, the number of NetVCs in the cop_list is decreasing. + // NetHandler will remove NetEvent from cop_list if it is triggered. + // As the NetHandler runs, the number of NetEvents in the cop_list is decreasing. // NetHandler runs 100 times maximum between InactivityCop runs. - // Therefore we don't have to check all the NetVCs as much as open_list. + // Therefore we don't have to check all the NetEvents as much as open_list. // Cleanup the active and keep-alive queues periodically nh.manage_active_queue(true); // close any connections over the active timeout @@ -333,23 +333,23 @@ NetHandler::init_for_process() } // -// Function used to release a UnixNetVConnection and free it. +// Function used to release a NetEvent and free it. // void -NetHandler::free_netvc(UnixNetVConnection *netvc) +NetHandler::free_netevent(NetEvent *ne) { EThread *t = this->thread; ink_assert(t == this_ethread()); - ink_release_assert(netvc->thread == t); - ink_release_assert(netvc->nh == this); - - // Release netvc from InactivityCop - stopCop(netvc); - // Release netvc from NetHandler - stopIO(netvc); - // Clear and deallocate netvc - netvc->free(t); + ink_release_assert(ne->get_thread() == t); + ink_release_assert(ne->nh == this); + + // Release ne from InactivityCop + stopCop(ne); + // Release ne from NetHandler + stopIO(ne); + // Clear and deallocate ne + ne->free(t); } // @@ -358,25 +358,25 @@ NetHandler::free_netvc(UnixNetVConnection *netvc) void NetHandler::process_enabled_list() { - UnixNetVConnection *vc = nullptr; - - SListM(UnixNetVConnection, NetState, read, enable_link) rq(read_enable_list.popall()); - while ((vc = rq.pop())) { - vc->ep.modify(EVENTIO_READ); - vc->ep.refresh(EVENTIO_READ); - vc->read.in_enabled_list = 0; - if ((vc->read.enabled && vc->read.triggered) || vc->closed) { - read_ready_list.in_or_enqueue(vc); + NetEvent *ne = nullptr; + + SListM(NetEvent, NetState, read, enable_link) rq(read_enable_list.popall()); + while ((ne = rq.pop())) { + ne->ep.modify(EVENTIO_READ); + ne->ep.refresh(EVENTIO_READ); + ne->read.in_enabled_list = 0; + if ((ne->read.enabled && ne->read.triggered) || ne->closed) { + read_ready_list.in_or_enqueue(ne); } } - SListM(UnixNetVConnection, NetState, write, enable_link) wq(write_enable_list.popall()); - while ((vc = wq.pop())) { - vc->ep.modify(EVENTIO_WRITE); - vc->ep.refresh(EVENTIO_WRITE); - vc->write.in_enabled_list = 0; - if ((vc->write.enabled && vc->write.triggered) || vc->closed) { - write_ready_list.in_or_enqueue(vc); + SListM(NetEvent, NetState, write, enable_link) wq(write_enable_list.popall()); + while ((ne = wq.pop())) { + ne->ep.modify(EVENTIO_WRITE); + ne->ep.refresh(EVENTIO_WRITE); + ne->write.in_enabled_list = 0; + if ((ne->write.enabled && ne->write.triggered) || ne->closed) { + write_ready_list.in_or_enqueue(ne); } } } @@ -387,63 +387,63 @@ NetHandler::process_enabled_list() void NetHandler::process_ready_list() { - UnixNetVConnection *vc = nullptr; + NetEvent *ne = nullptr; #if defined(USE_EDGE_TRIGGER) - // UnixNetVConnection * - while ((vc = read_ready_list.dequeue())) { + // NetEvent * + while ((ne = read_ready_list.dequeue())) { // Initialize the thread-local continuation flags - set_cont_flags(vc->control_flags); - if (vc->closed) { - free_netvc(vc); - } else if (vc->read.enabled && vc->read.triggered) { - vc->net_read_io(this, this->thread); - } else if (!vc->read.enabled) { - read_ready_list.remove(vc); + set_cont_flags(ne->get_control_flags()); + if (ne->closed) { + free_netevent(ne); + } else if (ne->read.enabled && ne->read.triggered) { + ne->net_read_io(this, this->thread); + } else if (!ne->read.enabled) { + read_ready_list.remove(ne); #if defined(solaris) - if (vc->read.triggered && vc->write.enabled) { - vc->ep.modify(-EVENTIO_READ); - vc->ep.refresh(EVENTIO_WRITE); - vc->writeReschedule(this); + if (ne->read.triggered && ne->write.enabled) { + ne->ep.modify(-EVENTIO_READ); + ne->ep.refresh(EVENTIO_WRITE); + ne->writeReschedule(this); } #endif } } - while ((vc = write_ready_list.dequeue())) { - set_cont_flags(vc->control_flags); - if (vc->closed) { - free_netvc(vc); - } else if (vc->write.enabled && vc->write.triggered) { - write_to_net(this, vc, this->thread); - } else if (!vc->write.enabled) { - write_ready_list.remove(vc); + while ((ne = write_ready_list.dequeue())) { + set_cont_flags(ne->get_control_flags()); + if (ne->closed) { + free_netevent(ne); + } else if (ne->write.enabled && ne->write.triggered) { + ne->net_write_io(this, this->thread); + } else if (!ne->write.enabled) { + write_ready_list.remove(ne); #if defined(solaris) - if (vc->write.triggered && vc->read.enabled) { - vc->ep.modify(-EVENTIO_WRITE); - vc->ep.refresh(EVENTIO_READ); - vc->readReschedule(this); + if (ne->write.triggered && ne->read.enabled) { + ne->ep.modify(-EVENTIO_WRITE); + ne->ep.refresh(EVENTIO_READ); + ne->readReschedule(this); } #endif } } #else /* !USE_EDGE_TRIGGER */ - while ((vc = read_ready_list.dequeue())) { - set_cont_flags(vc->control_flags); - if (vc->closed) - free_netvc(vc); - else if (vc->read.enabled && vc->read.triggered) - vc->net_read_io(this, this->thread); - else if (!vc->read.enabled) - vc->ep.modify(-EVENTIO_READ); + while ((ne = read_ready_list.dequeue())) { + set_cont_flags(ne->get_control_flags()); + if (ne->closed) + free_netevent(ne); + else if (ne->read.enabled && ne->read.triggered) + ne->net_read_io(this, this->thread); + else if (!ne->read.enabled) + ne->ep.modify(-EVENTIO_READ); } - while ((vc = write_ready_list.dequeue())) { - set_cont_flags(vc->control_flags); - if (vc->closed) - free_netvc(vc); - else if (vc->write.enabled && vc->write.triggered) - write_to_net(this, vc, this->thread); - else if (!vc->write.enabled) - vc->ep.modify(-EVENTIO_WRITE); + while ((ne = write_ready_list.dequeue())) { + set_cont_flags(ne->get_control_flags()); + if (ne->closed) + free_netevent(ne); + else if (ne->write.enabled && ne->write.triggered) + write_to_net(this, ne, this->thread); + else if (!ne->write.enabled) + ne->ep.modify(-EVENTIO_WRITE); } #endif /* !USE_EDGE_TRIGGER */ } @@ -482,35 +482,35 @@ NetHandler::waitForActivity(ink_hrtime timeout) p->do_poll(timeout); // Get & Process polling result - PollDescriptor *pd = get_PollDescriptor(this->thread); - UnixNetVConnection *vc = nullptr; + PollDescriptor *pd = get_PollDescriptor(this->thread); + NetEvent *ne = nullptr; for (int x = 0; x < pd->result; x++) { epd = static_cast get_ev_data(pd, x); if (epd->type == EVENTIO_READWRITE_VC) { - vc = epd->data.vc; - // Remove triggered NetVC from cop_list because it won't be timeout before next InactivityCop runs. - if (cop_list.in(vc)) { - cop_list.remove(vc); + ne = epd->data.ne; + // Remove triggered NetEvent from cop_list because it won't be timeout before next InactivityCop runs. + if (cop_list.in(ne)) { + cop_list.remove(ne); } if (get_ev_events(pd, x) & (EVENTIO_READ | EVENTIO_ERROR)) { - vc->read.triggered = 1; - if (!read_ready_list.in(vc)) { - read_ready_list.enqueue(vc); + ne->read.triggered = 1; + if (!read_ready_list.in(ne)) { + read_ready_list.enqueue(ne); } else if (get_ev_events(pd, x) & EVENTIO_ERROR) { // check for unhandled epoll events that should be handled Debug("iocore_net_main", "Unhandled epoll event on read: 0x%04x read.enabled=%d closed=%d read.netready_queue=%d", - get_ev_events(pd, x), vc->read.enabled, vc->closed, read_ready_list.in(vc)); + get_ev_events(pd, x), ne->read.enabled, ne->closed, read_ready_list.in(ne)); } } - vc = epd->data.vc; + ne = epd->data.ne; if (get_ev_events(pd, x) & (EVENTIO_WRITE | EVENTIO_ERROR)) { - vc->write.triggered = 1; - if (!write_ready_list.in(vc)) { - write_ready_list.enqueue(vc); + ne->write.triggered = 1; + if (!write_ready_list.in(ne)) { + write_ready_list.enqueue(ne); } else if (get_ev_events(pd, x) & EVENTIO_ERROR) { // check for unhandled epoll events that should be handled Debug("iocore_net_main", "Unhandled epoll event on write: 0x%04x write.enabled=%d closed=%d write.netready_queue=%d", - get_ev_events(pd, x), vc->write.enabled, vc->closed, write_ready_list.in(vc)); + get_ev_events(pd, x), ne->write.enabled, ne->closed, write_ready_list.in(ne)); } } else if (!(get_ev_events(pd, x) & EVENTIO_READ)) { Debug("iocore_net_main", "Unhandled epoll event: 0x%04x", get_ev_events(pd, x)); @@ -569,17 +569,17 @@ NetHandler::manage_active_queue(bool ignore_queue_size = false) ink_hrtime now = Thread::get_hrtime(); // loop over the non-active connections and try to close them - UnixNetVConnection *vc = active_queue.head; - UnixNetVConnection *vc_next = nullptr; - int closed = 0; - int handle_event = 0; - int total_idle_time = 0; - int total_idle_count = 0; - for (; vc != nullptr; vc = vc_next) { - vc_next = vc->active_queue_link.next; - if ((vc->inactivity_timeout_in && vc->next_inactivity_timeout_at <= now) || - (vc->active_timeout_in && vc->next_activity_timeout_at <= now)) { - _close_vc(vc, now, handle_event, closed, total_idle_time, total_idle_count); + NetEvent *ne = active_queue.head; + NetEvent *ne_next = nullptr; + int closed = 0; + int handle_event = 0; + int total_idle_time = 0; + int total_idle_count = 0; + for (; ne != nullptr; ne = ne_next) { + ne_next = ne->active_queue_link.next; + if ((ne->inactivity_timeout_in && ne->next_inactivity_timeout_at <= now) || + (ne->active_timeout_in && ne->next_activity_timeout_at <= now)) { + _close_ne(ne, now, handle_event, closed, total_idle_time, total_idle_count); } if (ignore_queue_size == false && max_connections_active_per_thread_in > active_queue_size) { return true; @@ -619,14 +619,14 @@ NetHandler::manage_keep_alive_queue() } // loop over the non-active connections and try to close them - UnixNetVConnection *vc_next = nullptr; - int closed = 0; - int handle_event = 0; - int total_idle_time = 0; - int total_idle_count = 0; - for (UnixNetVConnection *vc = keep_alive_queue.head; vc != nullptr; vc = vc_next) { - vc_next = vc->keep_alive_queue_link.next; - _close_vc(vc, now, handle_event, closed, total_idle_time, total_idle_count); + NetEvent *ne_next = nullptr; + int closed = 0; + int handle_event = 0; + int total_idle_time = 0; + int total_idle_count = 0; + for (NetEvent *ne = keep_alive_queue.head; ne != nullptr; ne = ne_next) { + ne_next = ne->keep_alive_queue_link.next; + _close_ne(ne, now, handle_event, closed, total_idle_time, total_idle_count); total_connections_in = active_queue_size + keep_alive_queue_size; if (total_connections_in <= max_connections_per_thread_in) { @@ -642,40 +642,39 @@ NetHandler::manage_keep_alive_queue() } void -NetHandler::_close_vc(UnixNetVConnection *vc, ink_hrtime now, int &handle_event, int &closed, int &total_idle_time, - int &total_idle_count) +NetHandler::_close_ne(NetEvent *ne, ink_hrtime now, int &handle_event, int &closed, int &total_idle_time, int &total_idle_count) { - if (vc->thread != this_ethread()) { + if (ne->get_thread() != this_ethread()) { return; } - MUTEX_TRY_LOCK(lock, vc->mutex, this_ethread()); + MUTEX_TRY_LOCK(lock, ne->get_mutex(), this_ethread()); if (!lock.is_locked()) { return; } - ink_hrtime diff = (now - (vc->next_inactivity_timeout_at - vc->inactivity_timeout_in)) / HRTIME_SECOND; + ink_hrtime diff = (now - (ne->next_inactivity_timeout_at - ne->inactivity_timeout_in)) / HRTIME_SECOND; if (diff > 0) { total_idle_time += diff; ++total_idle_count; NET_SUM_DYN_STAT(keep_alive_queue_timeout_total_stat, diff); NET_INCREMENT_DYN_STAT(keep_alive_queue_timeout_count_stat); } - Debug("net_queue", "closing connection NetVC=%p idle: %u now: %" PRId64 " at: %" PRId64 " in: %" PRId64 " diff: %" PRId64, vc, - keep_alive_queue_size, ink_hrtime_to_sec(now), ink_hrtime_to_sec(vc->next_inactivity_timeout_at), - ink_hrtime_to_sec(vc->inactivity_timeout_in), diff); - if (vc->closed) { - free_netvc(vc); + Debug("net_queue", "closing connection NetEvent=%p idle: %u now: %" PRId64 " at: %" PRId64 " in: %" PRId64 " diff: %" PRId64, ne, + keep_alive_queue_size, ink_hrtime_to_sec(now), ink_hrtime_to_sec(ne->next_inactivity_timeout_at), + ink_hrtime_to_sec(ne->inactivity_timeout_in), diff); + if (ne->closed) { + free_netevent(ne); ++closed; } else { - vc->next_inactivity_timeout_at = now; + ne->next_inactivity_timeout_at = now; // create a dummy event Event event; event.ethread = this_ethread(); - if (vc->inactivity_timeout_in && vc->next_inactivity_timeout_at <= now) { - if (vc->handleEvent(VC_EVENT_INACTIVITY_TIMEOUT, &event) == EVENT_DONE) { + if (ne->inactivity_timeout_in && ne->next_inactivity_timeout_at <= now) { + if (ne->callback(VC_EVENT_INACTIVITY_TIMEOUT, &event) == EVENT_DONE) { ++handle_event; } - } else if (vc->active_timeout_in && vc->next_activity_timeout_at <= now) { - if (vc->handleEvent(VC_EVENT_ACTIVE_TIMEOUT, &event) == EVENT_DONE) { + } else if (ne->active_timeout_in && ne->next_activity_timeout_at <= now) { + if (ne->callback(VC_EVENT_ACTIVE_TIMEOUT, &event) == EVENT_DONE) { ++handle_event; } } @@ -683,41 +682,41 @@ NetHandler::_close_vc(UnixNetVConnection *vc, ink_hrtime now, int &handle_event, } void -NetHandler::add_to_keep_alive_queue(UnixNetVConnection *vc) +NetHandler::add_to_keep_alive_queue(NetEvent *ne) { - Debug("net_queue", "NetVC: %p", vc); + Debug("net_queue", "NetEvent: %p", ne); ink_assert(mutex->thread_holding == this_ethread()); - if (keep_alive_queue.in(vc)) { + if (keep_alive_queue.in(ne)) { // already in the keep-alive queue, move the head - keep_alive_queue.remove(vc); + keep_alive_queue.remove(ne); } else { // in the active queue or no queue, new to this queue - remove_from_active_queue(vc); + remove_from_active_queue(ne); ++keep_alive_queue_size; } - keep_alive_queue.enqueue(vc); + keep_alive_queue.enqueue(ne); // if keep-alive queue is over size then close connections manage_keep_alive_queue(); } void -NetHandler::remove_from_keep_alive_queue(UnixNetVConnection *vc) +NetHandler::remove_from_keep_alive_queue(NetEvent *ne) { - Debug("net_queue", "NetVC: %p", vc); + Debug("net_queue", "NetEvent: %p", ne); ink_assert(mutex->thread_holding == this_ethread()); - if (keep_alive_queue.in(vc)) { - keep_alive_queue.remove(vc); + if (keep_alive_queue.in(ne)) { + keep_alive_queue.remove(ne); --keep_alive_queue_size; } } bool -NetHandler::add_to_active_queue(UnixNetVConnection *vc) +NetHandler::add_to_active_queue(NetEvent *ne) { - Debug("net_queue", "NetVC: %p", vc); + Debug("net_queue", "NetEvent: %p", ne); Debug("net_queue", "max_connections_per_thread_in: %d active_queue_size: %d keep_alive_queue_size: %d", max_connections_per_thread_in, active_queue_size, keep_alive_queue_size); ink_assert(mutex->thread_holding == this_ethread()); @@ -728,27 +727,27 @@ NetHandler::add_to_active_queue(UnixNetVConnection *vc) return false; } - if (active_queue.in(vc)) { + if (active_queue.in(ne)) { // already in the active queue, move the head - active_queue.remove(vc); + active_queue.remove(ne); } else { // in the keep-alive queue or no queue, new to this queue - remove_from_keep_alive_queue(vc); + remove_from_keep_alive_queue(ne); ++active_queue_size; } - active_queue.enqueue(vc); + active_queue.enqueue(ne); return true; } void -NetHandler::remove_from_active_queue(UnixNetVConnection *vc) +NetHandler::remove_from_active_queue(NetEvent *ne) { - Debug("net_queue", "NetVC: %p", vc); + Debug("net_queue", "NetEvent: %p", ne); ink_assert(mutex->thread_holding == this_ethread()); - if (active_queue.in(vc)) { - active_queue.remove(vc); + if (active_queue.in(ne)) { + active_queue.remove(ne); --active_queue_size; } } diff --git a/iocore/net/UnixNetPages.cc b/iocore/net/UnixNetPages.cc index bd8d64002bd..702e2ebb440 100644 --- a/iocore/net/UnixNetPages.cc +++ b/iocore/net/UnixNetPages.cc @@ -61,10 +61,11 @@ struct ShowNet : public ShowCont { } ink_hrtime now = Thread::get_hrtime(); - forl_LL(UnixNetVConnection, vc, nh->open_list) + forl_LL(NetEvent, ne, nh->open_list) { + auto vc = dynamic_cast(ne); // uint16_t port = ats_ip_port_host_order(&addr.sa); - if (ats_is_ip(&addr) && !ats_ip_addr_port_eq(&addr.sa, vc->get_remote_addr())) { + if (vc == nullptr || (ats_is_ip(&addr) && !ats_ip_addr_port_eq(&addr.sa, vc->get_remote_addr()))) { continue; } // if (port && port != ats_ip_port_host_order(&vc->server_addr.sa) && port != vc->accept_port) @@ -158,7 +159,12 @@ struct ShowNet : public ShowCont { CHECK_SHOW(show("

Thread: %d

\n", ithread)); CHECK_SHOW(show("\n")); int connections = 0; - forl_LL(UnixNetVConnection, vc, nh->open_list) connections++; + forl_LL(NetEvent, ne, nh->open_list) + { + if (dynamic_cast(ne) != nullptr) { + ++connections; + } + } CHECK_SHOW(show("\n", "Connections", connections)); // CHECK_SHOW(show("\n", "Last Poll Size", pollDescriptor->nfds)); CHECK_SHOW(show("\n", "Last Poll Ready", pollDescriptor->result)); diff --git a/iocore/net/UnixNetVConnection.cc b/iocore/net/UnixNetVConnection.cc index 448ae426f14..96d092df8df 100644 --- a/iocore/net/UnixNetVConnection.cc +++ b/iocore/net/UnixNetVConnection.cc @@ -99,7 +99,7 @@ read_signal_and_update(int event, UnixNetVConnection *vc) if (!--vc->recursion && vc->closed) { /* BZ 31932 */ ink_assert(vc->thread == this_ethread()); - vc->nh->free_netvc(vc); + vc->nh->free_netevent(vc); return EVENT_DONE; } else { return EVENT_CONT; @@ -130,7 +130,7 @@ write_signal_and_update(int event, UnixNetVConnection *vc) if (!--vc->recursion && vc->closed) { /* BZ 31932 */ ink_assert(vc->thread == this_ethread()); - vc->nh->free_netvc(vc); + vc->nh->free_netevent(vc); return EVENT_DONE; } else { return EVENT_CONT; @@ -197,7 +197,7 @@ read_from_net(NetHandler *nh, UnixNetVConnection *vc, EThread *thread) // global session pool case. If so, the closed flag should be stable once we get the // s->vio.mutex (the global session pool mutex). if (vc->closed) { - vc->nh->free_netvc(vc); + vc->nh->free_netevent(vc); return; } // if it is not enabled. @@ -657,7 +657,7 @@ UnixNetVConnection::do_io_close(int alerrno /* = -1 */) if (close_inline) { if (nh) { - nh->free_netvc(this); + nh->free_netevent(this); } else { free(t); } @@ -866,8 +866,7 @@ UnixNetVConnection::reenable_re(VIO *vio) } } -UnixNetVConnection::UnixNetVConnection() : flags(0) - +UnixNetVConnection::UnixNetVConnection() { SET_HANDLER((NetVConnHandler)&UnixNetVConnection::startEvent); } @@ -891,6 +890,12 @@ UnixNetVConnection::net_read_io(NetHandler *nh, EThread *lthread) read_from_net(nh, this, lthread); } +void +UnixNetVConnection::net_write_io(NetHandler *nh, EThread *lthread) +{ + write_to_net(nh, this, lthread); +} + // This code was pulled out of write_to_net so // I could overwrite it for the SSL implementation // (SSL read does not support overlapped i/o) @@ -1139,7 +1144,7 @@ UnixNetVConnection::mainEvent(int event, Event *e) writer_cont = write.vio.cont; if (closed) { - nh->free_netvc(this); + nh->free_netevent(this); return EVENT_DONE; }
%s%d
%s%d
%s%d