mirror of
https://github.com/lightpanda-io/browser.git
synced 2025-10-29 15:13:28 +00:00
Remove the loop
Previously, the IO loop was doing three things: 1 - Managing timeouts (either from scripts or for our own needs) 2 - Handling browser IO events (page/script/xhr) 3 - Handling CDP events (accept, read, write, timeout) With the libcurl merge, 1 was moved to an in-process scheduler and 2 was moved to libcurl's own event loop. That means the entire loop code, including the dependency on tigerbeetle-io existed for handling a single TCP client. Not only is that a lot of code, there was also friction between the two loops (the libcurl one and our IO loop), which would result in latency - while one loop is waiting for the events, any events on the other loop go un-processed. This PR removes our IO loop. To accomplish this: 1 - The main accept loop is blocking. This is simpler and works perfectly well, given we only allow 1 active connection. 2 - The client socket is passed to libcurl - yes, libcurl's loop can take arbitrary FDs and poll them along with its own. In addition to having one less dependency, the CDP code is quite a bit simpler, especially around shutdowns and writes. This also removes _some_ of the latency caused by the friction between page process and CDP processing. Specifically, when CDP now blocks for input, http page events (script loading, xhr, ...) will still be processed. There's still friction. For one, the reverse isn't true: when the page is waiting for events, CDP events aren't going to be processed. But the page.wait already have some sensitivity to this (e.g. the page.request_intercepted flag). Also, when CDP waits, while we will process network events, page timeouts are still not processed. Because of both these remaining issues, we still need to jump between the two loops - but being able to block on CDP (even for a short time) WITHOUT stopping the page's network I/O, should reduce some latency.
This commit is contained in:
633
src/server.zig
633
src/server.zig
@@ -26,14 +26,6 @@ const Allocator = std.mem.Allocator;
|
||||
const ArenaAllocator = std.heap.ArenaAllocator;
|
||||
|
||||
const log = @import("log.zig");
|
||||
const IO = @import("runtime/loop.zig").IO;
|
||||
const Completion = IO.Completion;
|
||||
const AcceptError = IO.AcceptError;
|
||||
const RecvError = IO.RecvError;
|
||||
const SendError = IO.SendError;
|
||||
const TimeoutError = IO.TimeoutError;
|
||||
const Loop = @import("runtime/loop.zig").Loop;
|
||||
|
||||
const App = @import("app.zig").App;
|
||||
const CDP = @import("cdp/cdp.zig").CDP;
|
||||
|
||||
@@ -46,114 +38,143 @@ const MAX_HTTP_REQUEST_SIZE = 4096;
|
||||
// +140 for the max control packet that might be interleaved in a message
|
||||
const MAX_MESSAGE_SIZE = 512 * 1024 + 14;
|
||||
|
||||
const Server = struct {
|
||||
pub const Server = struct {
|
||||
app: *App,
|
||||
loop: *Loop,
|
||||
allocator: Allocator,
|
||||
client: ?*Client = null,
|
||||
|
||||
// internal fields
|
||||
listener: posix.socket_t,
|
||||
timeout: u64,
|
||||
|
||||
// I/O fields
|
||||
accept_completion: Completion,
|
||||
|
||||
// The response to send on a GET /json/version request
|
||||
client: ?posix.socket_t,
|
||||
listener: ?posix.socket_t,
|
||||
json_version_response: []const u8,
|
||||
|
||||
fn deinit(self: *Server) void {
|
||||
_ = self;
|
||||
}
|
||||
pub fn init(app: *App, address: net.Address) !Server {
|
||||
const allocator = app.allocator;
|
||||
const json_version_response = try buildJSONVersionResponse(allocator, address);
|
||||
errdefer allocator.free(json_version_response);
|
||||
|
||||
fn queueAccept(self: *Server) void {
|
||||
log.debug(.app, "accepting connection", .{});
|
||||
self.loop.io.accept(
|
||||
*Server,
|
||||
self,
|
||||
callbackAccept,
|
||||
&self.accept_completion,
|
||||
self.listener,
|
||||
);
|
||||
}
|
||||
|
||||
fn callbackAccept(
|
||||
self: *Server,
|
||||
completion: *Completion,
|
||||
result: AcceptError!posix.socket_t,
|
||||
) void {
|
||||
std.debug.assert(completion == &self.accept_completion);
|
||||
self.doCallbackAccept(result) catch |err| {
|
||||
log.err(.app, "server accept error", .{ .err = err });
|
||||
self.queueAccept();
|
||||
return .{
|
||||
.app = app,
|
||||
.client = null,
|
||||
.listener = null,
|
||||
.allocator = allocator,
|
||||
.json_version_response = json_version_response,
|
||||
};
|
||||
}
|
||||
|
||||
fn doCallbackAccept(
|
||||
self: *Server,
|
||||
result: AcceptError!posix.socket_t,
|
||||
) !void {
|
||||
const socket = try result;
|
||||
const client = try self.allocator.create(Client);
|
||||
client.* = Client.init(socket, self);
|
||||
client.start();
|
||||
self.client = client;
|
||||
|
||||
if (log.enabled(.app, .info)) {
|
||||
var address: std.net.Address = undefined;
|
||||
var socklen: posix.socklen_t = @sizeOf(net.Address);
|
||||
try std.posix.getsockname(socket, &address.any, &socklen);
|
||||
log.info(.app, "client connected", .{ .ip = address });
|
||||
pub fn deinit(self: *Server) void {
|
||||
self.allocator.free(self.json_version_response);
|
||||
if (self.listener) |listener| {
|
||||
posix.close(listener);
|
||||
}
|
||||
}
|
||||
|
||||
fn releaseClient(self: *Server, client: *Client) void {
|
||||
self.allocator.destroy(client);
|
||||
self.client = null;
|
||||
pub fn run(self: *Server, address: net.Address, timeout_ms: i32) !void {
|
||||
const flags = posix.SOCK.STREAM | posix.SOCK.CLOEXEC;
|
||||
const listener = try posix.socket(address.any.family, flags, posix.IPPROTO.TCP);
|
||||
self.listener = listener;
|
||||
|
||||
try posix.setsockopt(listener, posix.SOL.SOCKET, posix.SO.REUSEADDR, &std.mem.toBytes(@as(c_int, 1)));
|
||||
// TODO: Broken on darwin
|
||||
// https://github.com/ziglang/zig/issues/17260 (fixed in Zig 0.14)
|
||||
// if (@hasDecl(os.TCP, "NODELAY")) {
|
||||
// try os.setsockopt(socket.sockfd.?, os.IPPROTO.TCP, os.TCP.NODELAY, &std.mem.toBytes(@as(c_int, 1)));
|
||||
// }
|
||||
try posix.setsockopt(listener, posix.IPPROTO.TCP, 1, &std.mem.toBytes(@as(c_int, 1)));
|
||||
|
||||
try posix.bind(listener, &address.any, address.getOsSockLen());
|
||||
try posix.listen(listener, 1);
|
||||
|
||||
log.info(.app, "server running", .{ .address = address });
|
||||
while (true) {
|
||||
const socket = posix.accept(listener, null, null, posix.SOCK.NONBLOCK) catch |err| {
|
||||
log.err(.app, "CDP accept", .{ .err = err });
|
||||
std.time.sleep(std.time.ns_per_s);
|
||||
continue;
|
||||
};
|
||||
|
||||
self.client = socket;
|
||||
defer if (self.client) |s| {
|
||||
posix.close(s);
|
||||
self.client = null;
|
||||
};
|
||||
|
||||
if (log.enabled(.app, .info)) {
|
||||
var client_address: std.net.Address = undefined;
|
||||
var socklen: posix.socklen_t = @sizeOf(net.Address);
|
||||
try std.posix.getsockname(socket, &client_address.any, &socklen);
|
||||
log.info(.app, "client connected", .{ .ip = client_address });
|
||||
}
|
||||
|
||||
self.readLoop(socket, timeout_ms) catch |err| {
|
||||
log.err(.app, "CDP client loop", .{ .err = err });
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
fn readLoop(self: *Server, socket: posix.socket_t, timeout_ms: i32) !void {
|
||||
// This shouldn't be necessary, but the Client is HUGE (> 512KB) because
|
||||
// it has a large read buffer. I don't know why, but v8 crashes if this
|
||||
// is on the stack (and I assume it's related to its size).
|
||||
const client = try self.allocator.create(Client);
|
||||
defer self.allocator.destroy(client);
|
||||
|
||||
client.* = try Client.init(socket, self);
|
||||
defer client.deinit();
|
||||
|
||||
var last_message = timestamp();
|
||||
var http = &self.app.http;
|
||||
while (true) {
|
||||
if (http.poll(20, socket)) {
|
||||
const n = posix.read(socket, client.readBuf()) catch |err| {
|
||||
log.warn(.app, "CDP read", .{ .err = err });
|
||||
return;
|
||||
};
|
||||
if (n == 0) {
|
||||
log.info(.app, "CDP disconnect", .{});
|
||||
return;
|
||||
}
|
||||
const more = client.processData(n) catch false;
|
||||
if (!more) {
|
||||
return;
|
||||
}
|
||||
last_message = timestamp();
|
||||
} else if (timestamp() - last_message > timeout_ms) {
|
||||
log.info(.app, "CDP timeout", .{});
|
||||
return;
|
||||
}
|
||||
// We have 3 types of "events":
|
||||
// - Incoming CDP messages
|
||||
// - Network events from the browser
|
||||
// - Timeouts from the browser
|
||||
|
||||
// The call to http.poll above handles the first two (which is why
|
||||
// we pass the client socket to it). But browser timeouts aren't
|
||||
// hooked into that. So we need to go to the browser page (if there
|
||||
// is one), and ask it to process any pending events. That action
|
||||
// doesn't starve #2 (Network events from the browser), because
|
||||
// page.wait() handles that too. But it does starve #1 (Incoming CDP
|
||||
// messages). The good news is that, while the Page is mostly
|
||||
// unaware of CDP, it will only block if it actually has something to
|
||||
// do AND it knows if we're waiting on an intercept request, and will
|
||||
// eagerly return control here in those cases.
|
||||
if (client.mode == .cdp) {
|
||||
client.mode.cdp.pageWait();
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
// Client
|
||||
// --------
|
||||
|
||||
pub const Client = struct {
|
||||
// The client is initially serving HTTP requests but, under normal circumstances
|
||||
// should eventually be upgraded to a websocket connections
|
||||
mode: Mode,
|
||||
mode: union(enum) {
|
||||
http: void,
|
||||
cdp: CDP,
|
||||
},
|
||||
|
||||
// The CDP instance that processes messages from this client
|
||||
// (a generic so we can test with a mock
|
||||
// null until mode == .websocket
|
||||
cdp: ?CDP,
|
||||
|
||||
// Our Server (a generic so we can test with a mock)
|
||||
server: *Server,
|
||||
reader: Reader(true),
|
||||
socket: posix.socket_t,
|
||||
last_active: std.time.Instant,
|
||||
|
||||
// queue of messages to send
|
||||
send_queue: SendQueue,
|
||||
send_queue_node_pool: std.heap.MemoryPool(SendQueue.Node),
|
||||
|
||||
read_pending: bool,
|
||||
read_completion: Completion,
|
||||
|
||||
write_pending: bool,
|
||||
write_completion: Completion,
|
||||
|
||||
timeout_pending: bool,
|
||||
timeout_completion: Completion,
|
||||
|
||||
// Used along with xyx_pending to figure out the lifetime of
|
||||
// the client. When connected == false and we have no more pending
|
||||
// completions, we can kill the client
|
||||
connected: bool,
|
||||
|
||||
const Mode = enum {
|
||||
http,
|
||||
websocket,
|
||||
};
|
||||
socket_flags: usize,
|
||||
send_arena: ArenaAllocator,
|
||||
|
||||
const EMPTY_PONG = [_]u8{ 138, 0 };
|
||||
|
||||
@@ -164,118 +185,29 @@ pub const Client = struct {
|
||||
// "private-use" close codes must be from 4000-49999
|
||||
const CLOSE_TIMEOUT = [_]u8{ 136, 2, 15, 160 }; // code: 4000
|
||||
|
||||
const SendQueue = std.DoublyLinkedList(Outgoing);
|
||||
fn init(socket: posix.socket_t, server: *Server) !Client {
|
||||
const socket_flags = try posix.fcntl(socket, posix.F.GETFL, 0);
|
||||
const nonblocking = @as(u32, @bitCast(posix.O{ .NONBLOCK = true }));
|
||||
// we expect the socket to come to us as nonblocking
|
||||
std.debug.assert(socket_flags & nonblocking == nonblocking);
|
||||
|
||||
fn init(socket: posix.socket_t, server: *Server) Client {
|
||||
return .{
|
||||
.cdp = null,
|
||||
.mode = .http,
|
||||
.socket = socket,
|
||||
.server = server,
|
||||
.last_active = now(),
|
||||
.send_queue = .{},
|
||||
.read_pending = false,
|
||||
.read_completion = undefined,
|
||||
.write_pending = false,
|
||||
.write_completion = undefined,
|
||||
.timeout_pending = false,
|
||||
.timeout_completion = undefined,
|
||||
.connected = true,
|
||||
.mode = .{ .http = {} },
|
||||
.socket_flags = socket_flags,
|
||||
.reader = .{ .allocator = server.allocator },
|
||||
.send_queue_node_pool = std.heap.MemoryPool(SendQueue.Node).init(server.allocator),
|
||||
.send_arena = ArenaAllocator.init(server.allocator),
|
||||
};
|
||||
}
|
||||
|
||||
fn maybeDeinit(self: *Client) void {
|
||||
if (self.read_pending or self.write_pending) {
|
||||
// We cannot do anything as long as we still have these pending
|
||||
// They should not be pending for long as we're only here after
|
||||
// having shutdown the socket
|
||||
return;
|
||||
fn deinit(self: *Client) void {
|
||||
switch (self.mode) {
|
||||
.cdp => |*cdp| cdp.deinit(),
|
||||
.http => {},
|
||||
}
|
||||
|
||||
// We don't have a read nor a write completion pending, we can start
|
||||
// to shutdown.
|
||||
|
||||
self.reader.deinit();
|
||||
var node = self.send_queue.first;
|
||||
while (node) |n| {
|
||||
if (n.data.arena) |*arena| {
|
||||
arena.deinit();
|
||||
}
|
||||
node = n.next;
|
||||
}
|
||||
if (self.cdp) |*cdp| {
|
||||
cdp.deinit();
|
||||
}
|
||||
self.send_queue_node_pool.deinit();
|
||||
posix.close(self.socket);
|
||||
|
||||
// let the client accept a new connection
|
||||
self.server.queueAccept();
|
||||
|
||||
if (self.timeout_pending == false) {
|
||||
// We also don't have a pending timeout, we can release the client.
|
||||
// See callbackTimeout for more explanation about this. But, TL;DR
|
||||
// we want to call `queueAccept` as soon as we have no more read/write
|
||||
// but we don't want to wait for the timeout callback.
|
||||
self.server.releaseClient(self);
|
||||
}
|
||||
}
|
||||
|
||||
fn close(self: *Client) void {
|
||||
log.info(.app, "client disconnected", .{});
|
||||
self.connected = false;
|
||||
// recv only, because we might have pending writes we'd like to get
|
||||
// out (like the HTTP error response)
|
||||
posix.shutdown(self.socket, .recv) catch {};
|
||||
self.maybeDeinit();
|
||||
}
|
||||
|
||||
fn start(self: *Client) void {
|
||||
self.queueRead();
|
||||
self.queueTimeout();
|
||||
}
|
||||
|
||||
fn queueRead(self: *Client) void {
|
||||
self.server.loop.io.recv(
|
||||
*Client,
|
||||
self,
|
||||
callbackRead,
|
||||
&self.read_completion,
|
||||
self.socket,
|
||||
self.readBuf(),
|
||||
);
|
||||
self.read_pending = true;
|
||||
}
|
||||
|
||||
fn callbackRead(self: *Client, _: *Completion, result: RecvError!usize) void {
|
||||
self.read_pending = false;
|
||||
if (self.connected == false) {
|
||||
self.maybeDeinit();
|
||||
return;
|
||||
}
|
||||
|
||||
const size = result catch |err| {
|
||||
log.err(.app, "server read error", .{ .err = err });
|
||||
self.close();
|
||||
return;
|
||||
};
|
||||
|
||||
if (size == 0) {
|
||||
self.close();
|
||||
return;
|
||||
}
|
||||
|
||||
const more = self.processData(size) catch {
|
||||
self.close();
|
||||
return;
|
||||
};
|
||||
|
||||
// if more == false, the client is disconnecting
|
||||
if (more) {
|
||||
self.queueRead();
|
||||
}
|
||||
self.send_arena.deinit();
|
||||
}
|
||||
|
||||
fn readBuf(self: *Client) []u8 {
|
||||
@@ -283,19 +215,15 @@ pub const Client = struct {
|
||||
}
|
||||
|
||||
fn processData(self: *Client, len: usize) !bool {
|
||||
self.last_active = now();
|
||||
self.reader.len += len;
|
||||
|
||||
switch (self.mode) {
|
||||
.http => {
|
||||
try self.processHTTPRequest();
|
||||
return true;
|
||||
},
|
||||
.websocket => return self.processWebsocketMessage(),
|
||||
.cdp => |*cdp| return self.processWebsocketMessage(cdp),
|
||||
.http => return self.processHTTPRequest(),
|
||||
}
|
||||
}
|
||||
|
||||
fn processHTTPRequest(self: *Client) !void {
|
||||
fn processHTTPRequest(self: *Client) !bool {
|
||||
std.debug.assert(self.reader.pos == 0);
|
||||
const request = self.reader.buf[0..self.reader.len];
|
||||
|
||||
@@ -307,10 +235,12 @@ pub const Client = struct {
|
||||
// we're only expecting [body-less] GET requests.
|
||||
if (std.mem.endsWith(u8, request, "\r\n\r\n") == false) {
|
||||
// we need more data, put any more data here
|
||||
return;
|
||||
return true;
|
||||
}
|
||||
|
||||
self.handleHTTPRequest(request) catch |err| {
|
||||
// the next incoming data can go to the front of our buffer
|
||||
defer self.reader.len = 0;
|
||||
return self.handleHTTPRequest(request) catch |err| {
|
||||
switch (err) {
|
||||
error.NotFound => self.writeHTTPErrorResponse(404, "Not found"),
|
||||
error.InvalidRequest => self.writeHTTPErrorResponse(400, "Invalid request"),
|
||||
@@ -326,12 +256,9 @@ pub const Client = struct {
|
||||
}
|
||||
return err;
|
||||
};
|
||||
|
||||
// the next incoming data can go to the front of our buffer
|
||||
self.reader.len = 0;
|
||||
}
|
||||
|
||||
fn handleHTTPRequest(self: *Client, request: []u8) !void {
|
||||
fn handleHTTPRequest(self: *Client, request: []u8) !bool {
|
||||
if (request.len < 18) {
|
||||
// 18 is [generously] the smallest acceptable HTTP request
|
||||
return error.InvalidRequest;
|
||||
@@ -348,11 +275,12 @@ pub const Client = struct {
|
||||
const url = request[4..url_end];
|
||||
|
||||
if (std.mem.eql(u8, url, "/")) {
|
||||
return self.upgradeConnection(request);
|
||||
try self.upgradeConnection(request);
|
||||
return true;
|
||||
}
|
||||
|
||||
if (std.mem.eql(u8, url, "/json/version")) {
|
||||
try self.send(null, self.server.json_version_response);
|
||||
try self.send(self.server.json_version_response);
|
||||
// Chromedp (a Go driver) does an http request to /json/version
|
||||
// then to / (websocket upgrade) using a different connection.
|
||||
// Since we only allow 1 connection at a time, the 2nd one (the
|
||||
@@ -360,7 +288,7 @@ pub const Client = struct {
|
||||
// We can avoid that by closing the connection. json_version_response
|
||||
// has a Connection: Close header too.
|
||||
try posix.shutdown(self.socket, .recv);
|
||||
return;
|
||||
return false;
|
||||
}
|
||||
|
||||
return error.NotFound;
|
||||
@@ -426,8 +354,7 @@ pub const Client = struct {
|
||||
// our caller has already made sure this request ended in \r\n\r\n
|
||||
// so it isn't something we need to check again
|
||||
|
||||
var arena = ArenaAllocator.init(self.server.allocator);
|
||||
errdefer arena.deinit();
|
||||
const allocator = self.send_arena.allocator();
|
||||
|
||||
const response = blk: {
|
||||
// Response to an ugprade request is always this, with
|
||||
@@ -442,7 +369,7 @@ pub const Client = struct {
|
||||
|
||||
// The response will be sent via the IO Loop and thus has to have its
|
||||
// own lifetime.
|
||||
const res = try arena.allocator().dupe(u8, template);
|
||||
const res = try allocator.dupe(u8, template);
|
||||
|
||||
// magic response
|
||||
const key_pos = res.len - 32;
|
||||
@@ -458,9 +385,8 @@ pub const Client = struct {
|
||||
break :blk res;
|
||||
};
|
||||
|
||||
self.mode = .websocket;
|
||||
self.cdp = try CDP.init(self.server.app, self);
|
||||
return self.send(arena, response);
|
||||
self.mode = .{ .cdp = try CDP.init(self.server.app, self) };
|
||||
return self.send(response);
|
||||
}
|
||||
|
||||
fn writeHTTPErrorResponse(self: *Client, comptime status: u16, comptime body: []const u8) void {
|
||||
@@ -471,24 +397,21 @@ pub const Client = struct {
|
||||
|
||||
// we're going to close this connection anyways, swallowing any
|
||||
// error seems safe
|
||||
self.send(null, response) catch {};
|
||||
self.send(response) catch {};
|
||||
}
|
||||
|
||||
fn processWebsocketMessage(self: *Client) !bool {
|
||||
errdefer self.close();
|
||||
|
||||
fn processWebsocketMessage(self: *Client, cdp: *CDP) !bool {
|
||||
var reader = &self.reader;
|
||||
|
||||
while (true) {
|
||||
const msg = reader.next() catch |err| {
|
||||
switch (err) {
|
||||
error.TooLarge => self.send(null, &CLOSE_TOO_BIG) catch {},
|
||||
error.NotMasked => self.send(null, &CLOSE_PROTOCOL_ERROR) catch {},
|
||||
error.ReservedFlags => self.send(null, &CLOSE_PROTOCOL_ERROR) catch {},
|
||||
error.InvalidMessageType => self.send(null, &CLOSE_PROTOCOL_ERROR) catch {},
|
||||
error.ControlTooLarge => self.send(null, &CLOSE_PROTOCOL_ERROR) catch {},
|
||||
error.InvalidContinuation => self.send(null, &CLOSE_PROTOCOL_ERROR) catch {},
|
||||
error.NestedFragementation => self.send(null, &CLOSE_PROTOCOL_ERROR) catch {},
|
||||
error.TooLarge => self.send(&CLOSE_TOO_BIG) catch {},
|
||||
error.NotMasked => self.send(&CLOSE_PROTOCOL_ERROR) catch {},
|
||||
error.ReservedFlags => self.send(&CLOSE_PROTOCOL_ERROR) catch {},
|
||||
error.InvalidMessageType => self.send(&CLOSE_PROTOCOL_ERROR) catch {},
|
||||
error.ControlTooLarge => self.send(&CLOSE_PROTOCOL_ERROR) catch {},
|
||||
error.InvalidContinuation => self.send(&CLOSE_PROTOCOL_ERROR) catch {},
|
||||
error.NestedFragementation => self.send(&CLOSE_PROTOCOL_ERROR) catch {},
|
||||
error.OutOfMemory => {}, // don't borther trying to send an error in this case
|
||||
}
|
||||
return err;
|
||||
@@ -498,12 +421,10 @@ pub const Client = struct {
|
||||
.pong => {},
|
||||
.ping => try self.sendPong(msg.data),
|
||||
.close => {
|
||||
self.send(null, &CLOSE_NORMAL) catch {};
|
||||
self.close();
|
||||
self.send(&CLOSE_NORMAL) catch {};
|
||||
return false;
|
||||
},
|
||||
.text, .binary => if (self.cdp.?.handleMessage(msg.data) == false) {
|
||||
self.close();
|
||||
.text, .binary => if (cdp.handleMessage(msg.data) == false) {
|
||||
return false;
|
||||
},
|
||||
}
|
||||
@@ -520,18 +441,16 @@ pub const Client = struct {
|
||||
|
||||
fn sendPong(self: *Client, data: []const u8) !void {
|
||||
if (data.len == 0) {
|
||||
return self.send(null, &EMPTY_PONG);
|
||||
return self.send(&EMPTY_PONG);
|
||||
}
|
||||
var header_buf: [10]u8 = undefined;
|
||||
const header = websocketHeader(&header_buf, .pong, data.len);
|
||||
|
||||
var arena = ArenaAllocator.init(self.server.allocator);
|
||||
errdefer arena.deinit();
|
||||
|
||||
var framed = try arena.allocator().alloc(u8, header.len + data.len);
|
||||
const allocator = self.send_arena.allocator();
|
||||
var framed = try allocator.alloc(u8, header.len + data.len);
|
||||
@memcpy(framed[0..header.len], header);
|
||||
@memcpy(framed[header.len..], data);
|
||||
return self.send(arena, framed);
|
||||
return self.send(framed);
|
||||
}
|
||||
|
||||
// called by CDP
|
||||
@@ -541,10 +460,7 @@ pub const Client = struct {
|
||||
// buffer, where the first 10 bytes are reserved. We can then backfill
|
||||
// the header and send the slice.
|
||||
pub fn sendJSON(self: *Client, message: anytype, opts: std.json.StringifyOptions) !void {
|
||||
var arena = ArenaAllocator.init(self.server.allocator);
|
||||
errdefer arena.deinit();
|
||||
|
||||
const allocator = arena.allocator();
|
||||
const allocator = self.send_arena.allocator();
|
||||
|
||||
var buf: std.ArrayListUnmanaged(u8) = .{};
|
||||
try buf.ensureTotalCapacity(allocator, 512);
|
||||
@@ -554,144 +470,60 @@ pub const Client = struct {
|
||||
|
||||
try std.json.stringify(message, opts, buf.writer(allocator));
|
||||
const framed = fillWebsocketHeader(buf);
|
||||
return self.send(arena, framed);
|
||||
return self.send(framed);
|
||||
}
|
||||
|
||||
pub fn sendJSONRaw(
|
||||
self: *Client,
|
||||
arena: ArenaAllocator,
|
||||
buf: std.ArrayListUnmanaged(u8),
|
||||
) !void {
|
||||
// Dangerous API!. We assume the caller has reserved the first 10
|
||||
// bytes in `buf`.
|
||||
const framed = fillWebsocketHeader(buf);
|
||||
return self.send(arena, framed);
|
||||
return self.send(framed);
|
||||
}
|
||||
|
||||
fn queueTimeout(self: *Client) void {
|
||||
self.server.loop.io.timeout(
|
||||
*Client,
|
||||
self,
|
||||
callbackTimeout,
|
||||
&self.timeout_completion,
|
||||
TimeoutCheck,
|
||||
);
|
||||
self.timeout_pending = true;
|
||||
}
|
||||
fn send(self: *Client, data: []const u8) !void {
|
||||
var pos: usize = 0;
|
||||
var changed_to_blocking: bool = false;
|
||||
defer _ = self.send_arena.reset(.{ .retain_with_limit = 1024 * 32 });
|
||||
|
||||
fn callbackTimeout(self: *Client, _: *Completion, result: TimeoutError!void) void {
|
||||
self.timeout_pending = false;
|
||||
if (self.connected == false) {
|
||||
if (self.read_pending == false and self.write_pending == false) {
|
||||
// Timeout is problematic. Ideally, we'd just call maybeDeinit
|
||||
// here and check for timeout_pending == true. But that would
|
||||
// mean not being able to accept a new connection until this
|
||||
// callback fires - introducing a noticeable delay.
|
||||
// So, when read_pending and write_pending are both false, we
|
||||
// clean up as much as we can, and let the server accept a new
|
||||
// connection but we keep the client around to handle this
|
||||
// completion (if only we could cancel a completion!).
|
||||
// If we're here, with connected == false, read_pending == false
|
||||
// and write_pending == false, then everything has already been
|
||||
// cleaned up, and we just need to release the client.
|
||||
self.server.releaseClient(self);
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
if (result) |_| {
|
||||
if (now().since(self.last_active) >= self.server.timeout) {
|
||||
log.info(.app, "client connection timeout", .{});
|
||||
if (self.mode == .websocket) {
|
||||
self.send(null, &CLOSE_TIMEOUT) catch {};
|
||||
}
|
||||
self.close();
|
||||
return;
|
||||
}
|
||||
} else |err| {
|
||||
log.err(.app, "server timeout error", .{ .err = err });
|
||||
}
|
||||
|
||||
self.queueTimeout();
|
||||
}
|
||||
|
||||
fn send(self: *Client, arena: ?ArenaAllocator, data: []const u8) !void {
|
||||
const node = try self.send_queue_node_pool.create();
|
||||
errdefer self.send_queue_node_pool.destroy(node);
|
||||
|
||||
node.data = Outgoing{
|
||||
.arena = arena,
|
||||
.to_send = data,
|
||||
};
|
||||
self.send_queue.append(node);
|
||||
|
||||
if (self.send_queue.len > 1) {
|
||||
// if we already had a message in the queue, then our send loop
|
||||
// is already setup.
|
||||
return;
|
||||
}
|
||||
self.queueSend();
|
||||
}
|
||||
|
||||
fn queueSend(self: *Client) void {
|
||||
if (self.connected == false) {
|
||||
return;
|
||||
}
|
||||
|
||||
const node = self.send_queue.first orelse {
|
||||
// no more messages to send;
|
||||
return;
|
||||
defer if (changed_to_blocking) {
|
||||
// We had to change our socket to blocking me to get our write out
|
||||
// We need to change it back to non-blocking.
|
||||
_ = posix.fcntl(self.socket, posix.F.SETFL, self.socket_flags) catch |err| {
|
||||
log.err(.app, "CDP restore nonblocking", .{ .err = err });
|
||||
};
|
||||
};
|
||||
|
||||
self.server.loop.io.send(
|
||||
*Client,
|
||||
self,
|
||||
sendCallback,
|
||||
&self.write_completion,
|
||||
self.socket,
|
||||
node.data.to_send,
|
||||
);
|
||||
self.write_pending = true;
|
||||
}
|
||||
LOOP: while (pos < data.len) {
|
||||
const written = posix.write(self.socket, data[pos..]) catch |err| switch (err) {
|
||||
error.WouldBlock => {
|
||||
// self.socket is nonblocking, because we don't want to block
|
||||
// reads. But our life is a lot easier if we block writes,
|
||||
// largely, because we don't have to maintain a queue of pending
|
||||
// writes (which would each need their own allocations). So
|
||||
// if we get a WouldBlock error, we'll switch the socket to
|
||||
// blocking and switch it back to non-blocking after the write
|
||||
// is complete. Doesn't seem particularly efficiently, but
|
||||
// this should virtually never happen.
|
||||
std.debug.assert(changed_to_blocking == false);
|
||||
log.debug(.app, "CDP write would block", .{});
|
||||
changed_to_blocking = true;
|
||||
_ = try posix.fcntl(self.socket, posix.F.SETFL, self.socket_flags & ~@as(u32, @bitCast(posix.O{ .NONBLOCK = true })));
|
||||
continue :LOOP;
|
||||
},
|
||||
else => return err,
|
||||
};
|
||||
|
||||
fn sendCallback(self: *Client, _: *Completion, result: SendError!usize) void {
|
||||
self.write_pending = false;
|
||||
if (self.connected == false) {
|
||||
self.maybeDeinit();
|
||||
return;
|
||||
}
|
||||
|
||||
const sent = result catch |err| {
|
||||
log.warn(.app, "server send error", .{ .err = err });
|
||||
self.close();
|
||||
return;
|
||||
};
|
||||
|
||||
const node = self.send_queue.popFirst().?;
|
||||
const outgoing = &node.data;
|
||||
if (sent == outgoing.to_send.len) {
|
||||
if (outgoing.arena) |*arena| {
|
||||
arena.deinit();
|
||||
if (written == 0) {
|
||||
return error.Closed;
|
||||
}
|
||||
self.send_queue_node_pool.destroy(node);
|
||||
} else {
|
||||
// oops, we shouldn't have popped this node off, we need
|
||||
// to add it back to the front in order to send the unsent data
|
||||
// (this is less likely to happen, which is why we eagerly
|
||||
// pop it off)
|
||||
std.debug.assert(sent < outgoing.to_send.len);
|
||||
node.data.to_send = outgoing.to_send[sent..];
|
||||
self.send_queue.prepend(node);
|
||||
pos += written;
|
||||
}
|
||||
self.queueSend();
|
||||
}
|
||||
};
|
||||
|
||||
const Outgoing = struct {
|
||||
to_send: []const u8,
|
||||
arena: ?ArenaAllocator,
|
||||
};
|
||||
|
||||
// WebSocket message reader. Given websocket message, acts as an iterator that
|
||||
// can return zero or more Messages. When next returns null, any incomplete
|
||||
// message will remain in reader.data
|
||||
@@ -1008,72 +840,6 @@ fn websocketHeader(buf: []u8, op_code: OpCode, payload_len: usize) []const u8 {
|
||||
return buf[0..10];
|
||||
}
|
||||
|
||||
pub fn run(
|
||||
app: *App,
|
||||
address: net.Address,
|
||||
timeout: u64,
|
||||
) !void {
|
||||
// create socket
|
||||
const flags = posix.SOCK.STREAM | posix.SOCK.CLOEXEC | posix.SOCK.NONBLOCK;
|
||||
const listener = try posix.socket(address.any.family, flags, posix.IPPROTO.TCP);
|
||||
defer posix.close(listener);
|
||||
|
||||
try posix.setsockopt(listener, posix.SOL.SOCKET, posix.SO.REUSEADDR, &std.mem.toBytes(@as(c_int, 1)));
|
||||
// TODO: Broken on darwin
|
||||
// https://github.com/ziglang/zig/issues/17260 (fixed in Zig 0.14)
|
||||
// if (@hasDecl(os.TCP, "NODELAY")) {
|
||||
// try os.setsockopt(socket.sockfd.?, os.IPPROTO.TCP, os.TCP.NODELAY, &std.mem.toBytes(@as(c_int, 1)));
|
||||
// }
|
||||
try posix.setsockopt(listener, posix.IPPROTO.TCP, 1, &std.mem.toBytes(@as(c_int, 1)));
|
||||
|
||||
// bind & listen
|
||||
try posix.bind(listener, &address.any, address.getOsSockLen());
|
||||
try posix.listen(listener, 1);
|
||||
|
||||
var loop = app.loop;
|
||||
const allocator = app.allocator;
|
||||
const json_version_response = try buildJSONVersionResponse(allocator, address);
|
||||
defer allocator.free(json_version_response);
|
||||
|
||||
var server = Server{
|
||||
.app = app,
|
||||
.loop = loop,
|
||||
.timeout = timeout,
|
||||
.listener = listener,
|
||||
.allocator = allocator,
|
||||
.accept_completion = undefined,
|
||||
.json_version_response = json_version_response,
|
||||
};
|
||||
defer server.deinit();
|
||||
|
||||
// accept an connection
|
||||
server.queueAccept();
|
||||
log.info(.app, "server running", .{ .address = address });
|
||||
|
||||
// infinite loop on I/O events, either:
|
||||
// - cmd from incoming connection on server socket
|
||||
// - JS callbacks events from scripts
|
||||
// var http_client = app.http_client;
|
||||
while (true) {
|
||||
// @newhttp. This is a hack. We used to just have 1 loop, so we could
|
||||
// sleep it it "forever" and any activity (message to this server,
|
||||
// JS callback, http data) would wake it up.
|
||||
// Now we have 2 loops. If we block on one, the other won't get woken
|
||||
// up. We don't block "forever" but even 10ms adds a bunch of latency
|
||||
// since this is called in a loop.
|
||||
// Hopefully this is temporary and we can remove the io loop and then
|
||||
// only have 1 loop. But, until then, we need to check both loops and
|
||||
// pay some blocking penalty.
|
||||
if (server.client) |client| {
|
||||
if (client.cdp) |*cdp| {
|
||||
cdp.pageWait();
|
||||
}
|
||||
}
|
||||
|
||||
try loop.io.run_for_ns(10 * std.time.ns_per_ms);
|
||||
}
|
||||
}
|
||||
|
||||
// Utils
|
||||
// --------
|
||||
|
||||
@@ -1099,9 +865,9 @@ fn buildJSONVersionResponse(
|
||||
return try std.fmt.allocPrint(allocator, response_format, .{ body_len, address });
|
||||
}
|
||||
|
||||
fn now() std.time.Instant {
|
||||
// can only fail on platforms we don't support
|
||||
return std.time.Instant.now() catch unreachable;
|
||||
fn timestamp() u32 {
|
||||
const ts = std.posix.clock_gettime(std.posix.CLOCK.MONOTONIC) catch unreachable;
|
||||
return @intCast(ts.sec);
|
||||
}
|
||||
|
||||
// In-place string lowercase
|
||||
@@ -1465,8 +1231,7 @@ const MockCDP = struct {
|
||||
|
||||
allocator: Allocator = testing.allocator,
|
||||
|
||||
fn init(_: Allocator, client: anytype, loop: *Loop) MockCDP {
|
||||
_ = loop;
|
||||
fn init(_: Allocator, client: anytype) MockCDP {
|
||||
_ = client;
|
||||
return .{};
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user