|
@@ -37,14 +37,14 @@
|
|
#include <uv.h>
|
|
#include <uv.h>
|
|
|
|
|
|
#define conn_log_warn(conn, fmt, ...) \
|
|
#define conn_log_warn(conn, fmt, ...) \
|
|
- fprintf(stderr, "WARN: conn %p (reactor %p): " fmt, \
|
|
|
|
- conn, conn->reactor, __VA_ARGS__)
|
|
|
|
|
|
+ fprintf(stderr, "WARN: conn %s: " fmt, \
|
|
|
|
+ conn->name, __VA_ARGS__)
|
|
#define conn_log_info(conn, fmt, ...) \
|
|
#define conn_log_info(conn, fmt, ...) \
|
|
- fprintf(stderr, "INFO: conn %p (reactor %p): " fmt, \
|
|
|
|
- conn, conn->reactor, __VA_ARGS__)
|
|
|
|
|
|
+ fprintf(stderr, "INFO: conn %s: " fmt, \
|
|
|
|
+ conn->name, __VA_ARGS__)
|
|
#define conn_log_debug(conn, fmt, ...) \
|
|
#define conn_log_debug(conn, fmt, ...) \
|
|
- fprintf(stderr, "DEBUG: conn %p (reactor %p): " fmt, \
|
|
|
|
- conn, conn->reactor, __VA_ARGS__)
|
|
|
|
|
|
+ fprintf(stderr, "DEBUG: conn %s: " fmt, \
|
|
|
|
+ conn->name, __VA_ARGS__)
|
|
|
|
|
|
/**
|
|
/**
|
|
* The maximum length that we'll allocate to hold a response from the server.
|
|
* The maximum length that we'll allocate to hold a response from the server.
|
|
@@ -200,7 +200,7 @@ static struct hadoop_err *hrpc_conn_setup_payload(struct hrpc_conn *conn)
|
|
return NULL;
|
|
return NULL;
|
|
}
|
|
}
|
|
|
|
|
|
-static void conn_write_cb(uv_write_t* req, int status)
|
|
|
|
|
|
+static void conn_write_cb(uv_write_t *req, int status)
|
|
{
|
|
{
|
|
struct hrpc_conn *conn = req->data;
|
|
struct hrpc_conn *conn = req->data;
|
|
struct hrpc_conn_writer *writer = &conn->writer;
|
|
struct hrpc_conn_writer *writer = &conn->writer;
|
|
@@ -210,11 +210,30 @@ static void conn_write_cb(uv_write_t* req, int status)
|
|
|
|
|
|
if (status) {
|
|
if (status) {
|
|
err = hadoop_uverr_alloc(status,
|
|
err = hadoop_uverr_alloc(status,
|
|
- "conn_write_cb got error");
|
|
|
|
|
|
+ "conn_write_cb got error %d", status);
|
|
hrpc_conn_destroy(conn, err);
|
|
hrpc_conn_destroy(conn, err);
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
|
|
+
|
|
switch (writer->state) {
|
|
switch (writer->state) {
|
|
|
|
+ case HRPC_CONN_WRITE_CONNECTING:
|
|
|
|
+ writer->state = HRPC_CONN_WRITE_IPC_HEADER;
|
|
|
|
+ err = conn_setup_ipc_header(conn);
|
|
|
|
+ if (err) {
|
|
|
|
+ hrpc_conn_destroy(conn, err);
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+ conn_log_debug(conn, "writing %zd-byte IPC header\n",
|
|
|
|
+ writer->cur_writes[0].len);
|
|
|
|
+ res = uv_write(&writer->write_req, (uv_stream_t*)&conn->stream,
|
|
|
|
+ writer->cur_writes, 1, conn_write_cb);
|
|
|
|
+ if (res) {
|
|
|
|
+ err = hadoop_uverr_alloc(res, "conn_start_outbound: "
|
|
|
|
+ "got uv_write error %d when writing ipc header", res);
|
|
|
|
+ hrpc_conn_destroy(conn, err);
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+ break;
|
|
case HRPC_CONN_WRITE_IPC_HEADER:
|
|
case HRPC_CONN_WRITE_IPC_HEADER:
|
|
free_write_bufs(conn);
|
|
free_write_bufs(conn);
|
|
writer->state = HRPC_CONN_WRITE_PAYLOAD;
|
|
writer->state = HRPC_CONN_WRITE_PAYLOAD;
|
|
@@ -223,19 +242,19 @@ static void conn_write_cb(uv_write_t* req, int status)
|
|
hrpc_conn_destroy(conn, err);
|
|
hrpc_conn_destroy(conn, err);
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
- writer->write_req.data = conn;
|
|
|
|
|
|
+ conn_log_debug(conn, "writing %zd-byte and %zd-byte payloads\n",
|
|
|
|
+ writer->cur_writes[0].len, writer->cur_writes[1].len);
|
|
res = uv_write(&writer->write_req, (uv_stream_t*)&conn->stream,
|
|
res = uv_write(&writer->write_req, (uv_stream_t*)&conn->stream,
|
|
writer->cur_writes, 2, conn_write_cb);
|
|
writer->cur_writes, 2, conn_write_cb);
|
|
if (res) {
|
|
if (res) {
|
|
- err = hadoop_uverr_alloc(res,
|
|
|
|
- "failed to call uv_write on payload");
|
|
|
|
|
|
+ err = hadoop_uverr_alloc(res, "conn_write_cb: uv_write "
|
|
|
|
+ "failed with error %d. Falied to start writing the "
|
|
|
|
+ "payload.", res);
|
|
hrpc_conn_destroy(conn, err);
|
|
hrpc_conn_destroy(conn, err);
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
break;
|
|
break;
|
|
case HRPC_CONN_WRITE_PAYLOAD:
|
|
case HRPC_CONN_WRITE_PAYLOAD:
|
|
- conn_log_debug(conn, "%s", "conn_write_cb: finished writing payload. "
|
|
|
|
- "Now waiting for response.\n");
|
|
|
|
free_write_bufs(conn);
|
|
free_write_bufs(conn);
|
|
writer->state = HRPC_CONN_WRITE_IDLE;
|
|
writer->state = HRPC_CONN_WRITE_IDLE;
|
|
reader->state = HRPC_CONN_READ_LEN;
|
|
reader->state = HRPC_CONN_READ_LEN;
|
|
@@ -243,10 +262,13 @@ static void conn_write_cb(uv_write_t* req, int status)
|
|
res = uv_read_start((uv_stream_t*)&conn->stream, hrpc_conn_read_alloc,
|
|
res = uv_read_start((uv_stream_t*)&conn->stream, hrpc_conn_read_alloc,
|
|
conn_read_cb);
|
|
conn_read_cb);
|
|
if (res) {
|
|
if (res) {
|
|
- err = hadoop_uverr_alloc(res, "uv_read_start failed");
|
|
|
|
|
|
+ err = hadoop_uverr_alloc(res, "conn_write_cb: uv_read_start "
|
|
|
|
+ "failed with error %d", res);
|
|
hrpc_conn_destroy(conn, err);
|
|
hrpc_conn_destroy(conn, err);
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
|
|
+ conn_log_debug(conn, "%s", "conn_write_cb: finished writing payload. "
|
|
|
|
+ "Now waiting for response.\n");
|
|
break;
|
|
break;
|
|
default:
|
|
default:
|
|
conn_log_warn(conn, "conn_write_cb: got an unexpected write "
|
|
conn_log_warn(conn, "conn_write_cb: got an unexpected write "
|
|
@@ -256,39 +278,18 @@ static void conn_write_cb(uv_write_t* req, int status)
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
-static void conn_start_outbound(struct hrpc_conn *conn)
|
|
|
|
|
|
+void hrpc_conn_restart_outbound(struct hrpc_conn *conn, struct hrpc_call *call)
|
|
{
|
|
{
|
|
- struct hadoop_err *err = NULL;
|
|
|
|
- struct hrpc_conn_writer *writer = &conn->writer;
|
|
|
|
- int res;
|
|
|
|
-
|
|
|
|
- // Get next call ID to use.
|
|
|
|
|
|
+ conn->call = call;
|
|
if (conn->call_id >= 0x7fffffff) {
|
|
if (conn->call_id >= 0x7fffffff) {
|
|
conn->call_id = 1;
|
|
conn->call_id = 1;
|
|
} else {
|
|
} else {
|
|
conn->call_id++;
|
|
conn->call_id++;
|
|
}
|
|
}
|
|
- writer->state = HRPC_CONN_WRITE_IPC_HEADER;
|
|
|
|
- err = conn_setup_ipc_header(conn);
|
|
|
|
- if (err) {
|
|
|
|
- hrpc_conn_destroy(conn, err);
|
|
|
|
- return;
|
|
|
|
- }
|
|
|
|
- writer->write_req.data = conn;
|
|
|
|
- res = uv_write(&writer->write_req, (uv_stream_t*)&conn->stream,
|
|
|
|
- writer->cur_writes, 1, conn_write_cb);
|
|
|
|
- if (res) {
|
|
|
|
- err = hadoop_uverr_alloc(res,
|
|
|
|
- "failed to call uv_write on ipc_header_buf");
|
|
|
|
- hrpc_conn_destroy(conn, err);
|
|
|
|
- return;
|
|
|
|
- }
|
|
|
|
-}
|
|
|
|
-
|
|
|
|
-void hrpc_conn_start_outbound(struct hrpc_conn *conn, struct hrpc_call *call)
|
|
|
|
-{
|
|
|
|
- conn->call = call;
|
|
|
|
- conn_start_outbound(conn);
|
|
|
|
|
|
+ conn->writer.state = HRPC_CONN_WRITE_IPC_HEADER;
|
|
|
|
+ conn->reader.state = HRPC_CONN_UNREADABLE;
|
|
|
|
+ conn->writer.write_req.data = conn;
|
|
|
|
+ conn_write_cb(&conn->writer.write_req, 0);
|
|
}
|
|
}
|
|
|
|
|
|
static void conn_connect_cb(uv_connect_t *req, int status)
|
|
static void conn_connect_cb(uv_connect_t *req, int status)
|
|
@@ -296,9 +297,11 @@ static void conn_connect_cb(uv_connect_t *req, int status)
|
|
struct hrpc_conn *conn = req->data;
|
|
struct hrpc_conn *conn = req->data;
|
|
struct hadoop_err *err = NULL;
|
|
struct hadoop_err *err = NULL;
|
|
struct hrpc_conn_writer *writer = &conn->writer;
|
|
struct hrpc_conn_writer *writer = &conn->writer;
|
|
|
|
+ char buf[128] = { 0 };
|
|
|
|
|
|
if (status) {
|
|
if (status) {
|
|
- err = hadoop_uverr_alloc(status, "uv_tcp_connect failed");
|
|
|
|
|
|
+ err = hadoop_uverr_alloc(status, "uv_tcp_connect failed "
|
|
|
|
+ "with status %d", status);
|
|
hrpc_conn_destroy(conn, err);
|
|
hrpc_conn_destroy(conn, err);
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
@@ -310,25 +313,29 @@ static void conn_connect_cb(uv_connect_t *req, int status)
|
|
hrpc_conn_destroy(conn, err);
|
|
hrpc_conn_destroy(conn, err);
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
- conn_start_outbound(conn);
|
|
|
|
|
|
+ conn_log_debug(conn, "conn_connect_cb: finished connecting to %s.\n",
|
|
|
|
+ net_ipv4_name(&conn->remote, buf, sizeof(buf)));
|
|
|
|
+ conn->writer.write_req.data = conn;
|
|
|
|
+ conn_write_cb(&conn->writer.write_req, 0);
|
|
}
|
|
}
|
|
|
|
|
|
struct hadoop_err *hrpc_conn_create_outbound(struct hrpc_reactor *reactor,
|
|
struct hadoop_err *hrpc_conn_create_outbound(struct hrpc_reactor *reactor,
|
|
- struct hrpc_call *call,
|
|
|
|
|
|
+ struct hrpc_call *call, const char *name,
|
|
struct hrpc_conn **out)
|
|
struct hrpc_conn **out)
|
|
{
|
|
{
|
|
struct hadoop_err *err = NULL;
|
|
struct hadoop_err *err = NULL;
|
|
struct hrpc_conn *conn = NULL;
|
|
struct hrpc_conn *conn = NULL;
|
|
int res, tcp_init = 0;
|
|
int res, tcp_init = 0;
|
|
|
|
|
|
- conn = calloc(1, sizeof(struct hrpc_conn));
|
|
|
|
|
|
+ conn = calloc(1, sizeof(struct hrpc_conn) + strlen(name) + 1);
|
|
if (!conn) {
|
|
if (!conn) {
|
|
err = hadoop_lerr_alloc(ENOMEM, "hrpc_conn_create_outbound: OOM");
|
|
err = hadoop_lerr_alloc(ENOMEM, "hrpc_conn_create_outbound: OOM");
|
|
goto done;
|
|
goto done;
|
|
}
|
|
}
|
|
|
|
+ strcpy(conn->name, name);
|
|
conn->reactor = reactor;
|
|
conn->reactor = reactor;
|
|
conn->call = call;
|
|
conn->call = call;
|
|
- conn->remote = call->remote;
|
|
|
|
|
|
+ conn->remote = *call->remote;
|
|
conn->protocol = strdup(call->protocol);
|
|
conn->protocol = strdup(call->protocol);
|
|
conn->username = strdup(call->username);
|
|
conn->username = strdup(call->username);
|
|
if ((!conn->protocol) || (!conn->username)) {
|
|
if ((!conn->protocol) || (!conn->username)) {
|
|
@@ -338,11 +345,12 @@ struct hadoop_err *hrpc_conn_create_outbound(struct hrpc_reactor *reactor,
|
|
hrpc_client_id_generate_rand(&conn->client_id);
|
|
hrpc_client_id_generate_rand(&conn->client_id);
|
|
res = uv_tcp_init(&reactor->loop, &conn->stream);
|
|
res = uv_tcp_init(&reactor->loop, &conn->stream);
|
|
if (res) {
|
|
if (res) {
|
|
- err = hadoop_uverr_alloc(res,
|
|
|
|
- "hrpc_conn_create_outbound: uv_tcp_init failed");
|
|
|
|
|
|
+ err = hadoop_uverr_alloc(res, "hrpc_conn_create_outbound: "
|
|
|
|
+ "uv_tcp_init failed with status %d", res);
|
|
goto done;
|
|
goto done;
|
|
}
|
|
}
|
|
tcp_init = 1;
|
|
tcp_init = 1;
|
|
|
|
+ conn->call_id = 1;
|
|
conn->writer.state = HRPC_CONN_WRITE_CONNECTING;
|
|
conn->writer.state = HRPC_CONN_WRITE_CONNECTING;
|
|
conn->reader.state = HRPC_CONN_UNREADABLE;
|
|
conn->reader.state = HRPC_CONN_UNREADABLE;
|
|
conn->conn_req.data = conn;
|
|
conn->conn_req.data = conn;
|
|
@@ -350,12 +358,13 @@ struct hadoop_err *hrpc_conn_create_outbound(struct hrpc_reactor *reactor,
|
|
(struct sockaddr*)&conn->remote, conn_connect_cb);
|
|
(struct sockaddr*)&conn->remote, conn_connect_cb);
|
|
if (res) {
|
|
if (res) {
|
|
char remote_str[64] = { 0 };
|
|
char remote_str[64] = { 0 };
|
|
- err = hadoop_uverr_alloc(res,
|
|
|
|
- "hrpc_conn_create_outbound: uv_tcp_connect(%s) failed",
|
|
|
|
|
|
+ err = hadoop_uverr_alloc(res, "hrpc_conn_create_outbound: "
|
|
|
|
+ "uv_tcp_connect(%s) failed with status %d",
|
|
net_ipv4_name_and_port(&conn->remote, remote_str,
|
|
net_ipv4_name_and_port(&conn->remote, remote_str,
|
|
- sizeof(remote_str)));
|
|
|
|
|
|
+ sizeof(remote_str)), res);
|
|
goto done;
|
|
goto done;
|
|
}
|
|
}
|
|
|
|
+ conn_log_debug(conn, "%s", "hrpc_conn_create: created new connection.\n");
|
|
|
|
|
|
done:
|
|
done:
|
|
if (err) {
|
|
if (err) {
|
|
@@ -375,39 +384,50 @@ done:
|
|
|
|
|
|
int hrpc_conn_compare(const struct hrpc_conn *a, const struct hrpc_conn *b)
|
|
int hrpc_conn_compare(const struct hrpc_conn *a, const struct hrpc_conn *b)
|
|
{
|
|
{
|
|
- int proto_cmp, username_cmp, a_active, b_active;
|
|
|
|
|
|
+ int proto_cmp, username_cmp, client_id_cmp, a_active, b_active;
|
|
|
|
|
|
// Big-endian versus little-endian doesn't matter here.
|
|
// Big-endian versus little-endian doesn't matter here.
|
|
// We just want a consistent ordering on the same machine.
|
|
// We just want a consistent ordering on the same machine.
|
|
- if (a->remote.sin_addr.s_addr < b->remote.sin_addr.s_addr)
|
|
|
|
|
|
+ // We don't compare ports.
|
|
|
|
+ if (a->remote.sin_addr.s_addr < b->remote.sin_addr.s_addr) {
|
|
return -1;
|
|
return -1;
|
|
- else if (a->remote.sin_addr.s_addr > b->remote.sin_addr.s_addr)
|
|
|
|
|
|
+ } else if (a->remote.sin_addr.s_addr > b->remote.sin_addr.s_addr) {
|
|
return 1;
|
|
return 1;
|
|
- else if (a->remote.sin_port < b->remote.sin_port)
|
|
|
|
|
|
+ } else if (a->remote.sin_port < b->remote.sin_port) {
|
|
return -1;
|
|
return -1;
|
|
- else if (a->remote.sin_port > b->remote.sin_port)
|
|
|
|
|
|
+ } else if (a->remote.sin_port > b->remote.sin_port) {
|
|
return 1;
|
|
return 1;
|
|
- // Compare protocol name.
|
|
|
|
|
|
+ }
|
|
proto_cmp = strcmp(a->protocol, b->protocol);
|
|
proto_cmp = strcmp(a->protocol, b->protocol);
|
|
- if (proto_cmp != 0)
|
|
|
|
|
|
+ if (proto_cmp != 0) {
|
|
return proto_cmp;
|
|
return proto_cmp;
|
|
- // Compare username.
|
|
|
|
|
|
+ }
|
|
username_cmp = strcmp(a->username, b->username);
|
|
username_cmp = strcmp(a->username, b->username);
|
|
- if (username_cmp != 0)
|
|
|
|
|
|
+ if (username_cmp != 0) {
|
|
return username_cmp;
|
|
return username_cmp;
|
|
|
|
+ }
|
|
// Make the inactive connections sort before the active ones.
|
|
// Make the inactive connections sort before the active ones.
|
|
a_active = !!a->call;
|
|
a_active = !!a->call;
|
|
b_active = !!b->call;
|
|
b_active = !!b->call;
|
|
- if (a_active < b_active)
|
|
|
|
|
|
+ if (a_active < b_active) {
|
|
return -1;
|
|
return -1;
|
|
- else if (a_active > b_active)
|
|
|
|
|
|
+ }
|
|
|
|
+ else if (a_active > b_active) {
|
|
return 1;
|
|
return 1;
|
|
|
|
+ }
|
|
|
|
+ client_id_cmp = hrpc_client_id_compare(&a->client_id, &b->client_id);
|
|
|
|
+ if (client_id_cmp) {
|
|
|
|
+ return client_id_cmp;
|
|
|
|
+ }
|
|
|
|
+
|
|
// Compare pointer identity, so that no two distinct connections are
|
|
// Compare pointer identity, so that no two distinct connections are
|
|
// ever identical.
|
|
// ever identical.
|
|
- else if (a < b)
|
|
|
|
|
|
+ else if (a < b) {
|
|
return -1;
|
|
return -1;
|
|
- else if (a > b)
|
|
|
|
|
|
+ }
|
|
|
|
+ else if (a > b) {
|
|
return 1;
|
|
return 1;
|
|
|
|
+ }
|
|
return 0;
|
|
return 0;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -415,15 +435,23 @@ int hrpc_conn_usable(const struct hrpc_conn *conn,
|
|
const struct sockaddr_in *addr,
|
|
const struct sockaddr_in *addr,
|
|
const char *protocol, const char *username)
|
|
const char *protocol, const char *username)
|
|
{
|
|
{
|
|
- if (conn->remote.sin_addr.s_addr != addr->sin_addr.s_addr)
|
|
|
|
|
|
+ if (conn->call) {
|
|
|
|
+ // The current implementation can't use the same TCP connection
|
|
|
|
+ // simulatenously for multiple RPCs.
|
|
|
|
+ return 0;
|
|
|
|
+ } else if (conn->remote.sin_addr.s_addr != addr->sin_addr.s_addr) {
|
|
|
|
+ return 0;
|
|
|
|
+ } else if (conn->remote.sin_port != addr->sin_port) {
|
|
return 0;
|
|
return 0;
|
|
- else if (conn->remote.sin_port != addr->sin_port)
|
|
|
|
|
|
+ } else if (conn->remote.sin_port != addr->sin_port) {
|
|
return 0;
|
|
return 0;
|
|
- else if (strcmp(conn->protocol, protocol))
|
|
|
|
|
|
+ } else if (strcmp(conn->protocol, protocol)) {
|
|
return 0;
|
|
return 0;
|
|
- else if (strcmp(conn->username, username))
|
|
|
|
|
|
+ } else if (strcmp(conn->username, username)) {
|
|
return 0;
|
|
return 0;
|
|
- return 1;
|
|
|
|
|
|
+ } else {
|
|
|
|
+ return 1;
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
static void free_read_bufs(struct hrpc_conn *conn)
|
|
static void free_read_bufs(struct hrpc_conn *conn)
|
|
@@ -462,10 +490,16 @@ static struct hadoop_err *conn_deliver_resp(struct hrpc_conn *conn,
|
|
resp.base = reader->body;
|
|
resp.base = reader->body;
|
|
reader->body = NULL;
|
|
reader->body = NULL;
|
|
free_read_bufs(conn);
|
|
free_read_bufs(conn);
|
|
|
|
+ // Remove the connection from the map. When we put it back in, it will be
|
|
|
|
+ // no longer active because we have zeroed conn->call, which will put it
|
|
|
|
+ // in a different position.
|
|
|
|
+ reactor_remove_conn(conn->reactor, conn);
|
|
conn->call = NULL;
|
|
conn->call = NULL;
|
|
|
|
+ reactor_insert_conn(conn->reactor, conn);
|
|
|
|
+ // Shut down the read state for now.
|
|
conn->reader.state = HRPC_CONN_UNREADABLE;
|
|
conn->reader.state = HRPC_CONN_UNREADABLE;
|
|
- // TODO: cache connections
|
|
|
|
- hrpc_conn_destroy(conn, NULL);
|
|
|
|
|
|
+ uv_read_stop((uv_stream_t*)&conn->stream);
|
|
|
|
+ // Perform the callback.
|
|
hrpc_call_deliver_resp(call, &resp);
|
|
hrpc_call_deliver_resp(call, &resp);
|
|
return NULL;
|
|
return NULL;
|
|
}
|
|
}
|
|
@@ -550,16 +584,20 @@ static const char *conn_read_state_str(enum hrpc_conn_read_state state)
|
|
};
|
|
};
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Return a read buffer to libuv.
|
|
|
|
|
|
+ * Tell libuv where to put the bytes that arrive from the remote end of the
|
|
|
|
+ * connection.
|
|
|
|
+ *
|
|
|
|
+ * If we haven't read the length (we're in state HRPC_CONN_READ_LEN), we give
|
|
|
|
+ * libuv back bytes from the four-byte "message length" buffer.
|
|
|
|
+ *
|
|
|
|
+ * If we have read the message length (and allocated a buffer for it), we
|
|
|
|
+ * return back that.
|
|
*
|
|
*
|
|
- * We don't do the actual allocation here, for two reasons. The first is that
|
|
|
|
- * we'd like to know how big a buffer to allocate first (by reading the first
|
|
|
|
- * 4 bytes). The second is that libuv doesn't really take kindly to
|
|
|
|
- * failures here... returning a zero-length buffer triggers a crash.
|
|
|
|
- * So we simply return previously allocated buffers here.
|
|
|
|
|
|
+ * If we return 0 here, libuv treats that as a connection error.
|
|
*
|
|
*
|
|
* @param handle The TCP stream.
|
|
* @param handle The TCP stream.
|
|
- * @param suggested_size The suggested size.
|
|
|
|
|
|
+ * @param suggested_size The suggested size. We ignore this because we know
|
|
|
|
+ * exactly how many bytes we need.
|
|
*
|
|
*
|
|
* @return The read buffer to use.
|
|
* @return The read buffer to use.
|
|
*/
|
|
*/
|
|
@@ -610,7 +648,7 @@ static void conn_read_cb(uv_stream_t *stream, ssize_t nread,
|
|
|
|
|
|
if (nread < 0) {
|
|
if (nread < 0) {
|
|
hrpc_conn_destroy(conn,
|
|
hrpc_conn_destroy(conn,
|
|
- hadoop_uverr_alloc(-nread, "conn_read_cb error"));
|
|
|
|
|
|
+ hadoop_uverr_alloc(-nread, "conn_read_cb error %zd", -nread));
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
if (nread == 0) {
|
|
if (nread == 0) {
|