]> granicus.if.org Git - libevent/commitdiff
Add a bufferevent function to resolve a name then connect to it.
authorNick Mathewson <nickm@torproject.org>
Tue, 3 Nov 2009 20:40:48 +0000 (20:40 +0000)
committerNick Mathewson <nickm@torproject.org>
Tue, 3 Nov 2009 20:40:48 +0000 (20:40 +0000)
This function, bufferevent_socket_connect_hostname() can either use
evdns to do the resolve, or use a new function (evutil_resolve) that
uses getaddrinfo or gethostbyname, like http.c does now.

This function is meant to eventually replace the hostname resolution mess in
http.c.

svn:r1496

13 files changed:
ChangeLog
Makefile.am
bufferevent-internal.h
bufferevent_sock.c
configure.in
evdns.c
evutil.c
http.c
include/event2/bufferevent.h
include/event2/dns.h
test/regress_dns.c
test/regress_util.c
util-internal.h

index e62e195ac35497779372927519eecdf09a901b9e..f0a046e4b224a5c8b167388714ce512428b90398 100644 (file)
--- a/ChangeLog
+++ b/ChangeLog
@@ -37,6 +37,7 @@ Changes in 2.0.3-alpha:
  o Add a return value to event_assign so that it can fail rather than asserting when the user gives it bad input.  event_set still dies on bad input.
  o The event_base_new() and event_base_new_with_config() functions now never call exit() on failure.  For backward "compatibility", event_init() still does, but more consistently.
  o Remove compat/sys/_time.h.  It interfered with system headers on HPUX, and its functionality has been subsumed by event2/util.h and util-internal.h.
+ o Add a new bufferevent_socket_connect_hostname() to encapsulate the resolve-then-connect operation.
 
 
 Changes in 2.0.2-alpha:
index e9383edd20f0f08205b7f2bed3eee715a9fcac25..ca805efa3fa6f9d9d07119ed7ac1eb55d2d720cc 100644 (file)
@@ -112,7 +112,7 @@ CORE_SRC = event.c buffer.c \
        bufferevent.c bufferevent_sock.c bufferevent_filter.c \
        bufferevent_pair.c listener.c \
        evmap.c log.c evutil.c strlcpy.c $(SYS_SRC)
-EXTRA_SRC = event_tagging.c http.c evdns.c evrpc.c
+EXTRA_SRC = event_tagging.c http.c evdns.c evrpc.c bufferevent_evdns.c
 
 
 libevent_la_SOURCES = $(CORE_SRC) $(EXTRA_SRC)
index 4c881316d4faee32c952cd79249ed5d60dea806f..eec60e1a9a2759addf18edb2564b5c412fd1ff8e 100644 (file)
@@ -227,6 +227,17 @@ void _bufferevent_generic_adj_timeouts(struct bufferevent *bev);
                        EVLOCK_UNLOCK(locking->lock, EVTHREAD_WRITE);   \
        } while(0)
 
+struct evdns_base;
+int _bufferevent_socket_connect_hostname_evdns(
+       struct bufferevent *bufev,
+       struct evdns_base *evdns_base,
+       int family,
+       const char *hostname,
+       int port);
+void _bufferevent_set_socket_connect_hostname_evdns_fn(
+       int (*fn)(struct bufferevent *, struct evdns_base *, int,
+           const char *, int));
+
 #ifdef __cplusplus
 }
 #endif
index 2cfef3856ca03ea7c35bc0a9b451027208042a87..11dd10d3a0809b2ba9b7d6d2d865f1de2f33c329 100644 (file)
 #ifdef _EVENT_HAVE_SYS_SOCKET_H
 #include <sys/socket.h>
 #endif
+#ifdef _EVENT_HAVE_NETINET_IN_H
+#include <netinet/in.h>
+#endif
+#ifdef _EVENT_HAVE_NETINET_IN6_H
+#include <netinet/in6.h>
+#endif
 
 #include "event2/util.h"
 #include "event2/bufferevent.h"
@@ -352,6 +358,73 @@ done:
        return result;
 }
 
+static int (*_bufferevent_socket_connect_hostname_evdns_fn)(
+       struct bufferevent *, struct evdns_base *, int,
+       const char *, int) = NULL;
+
+void _bufferevent_set_socket_connect_hostname_evdns_fn(
+       int (*fn)(struct bufferevent *, struct evdns_base *, int,
+           const char *, int))
+{
+       if (!_bufferevent_socket_connect_hostname_evdns_fn)
+           _bufferevent_socket_connect_hostname_evdns_fn = fn;
+}
+
+int
+bufferevent_socket_connect_hostname(struct bufferevent *bev,
+    struct evdns_base *evdns_base, int family, const char *hostname, int port)
+{
+       struct sockaddr_storage ss;
+       ev_socklen_t socklen = sizeof(ss);
+       int socklen_int = sizeof(ss);
+
+       if (family != AF_INET && family != AF_INET6 && family != AF_UNSPEC)
+               return -1;
+       if (port < 1 || port > 65535)
+               return -1;
+
+       memset(&ss, 0, sizeof(ss));
+       if (!evutil_parse_sockaddr_port(hostname, (struct sockaddr*)&ss,
+               &socklen_int)) {
+               socklen = socklen_int;
+               if (ss.ss_family == AF_INET) {
+                       struct sockaddr_in *sin = (struct sockaddr_in*)&ss;
+                       if (family == AF_INET6)
+                               return -1;
+                       if (sin->sin_port)
+                               return -1;
+                       sin->sin_port = htons(port);
+               } else if (ss.ss_family == AF_INET6) {
+                       struct sockaddr_in6 *sin6 = (struct sockaddr_in6*)&ss;
+                       if (family == AF_INET)
+                               return -1;
+                       if (sin6->sin6_port)
+                               return -1;
+                       sin6->sin6_port = htons(port);
+               }
+               return bufferevent_socket_connect(bev, (struct sockaddr*)&ss,
+                   socklen);
+       }
+
+       if (evdns_base) {
+               EVUTIL_ASSERT(_bufferevent_socket_connect_hostname_evdns_fn);
+               return _bufferevent_socket_connect_hostname_evdns_fn(
+                       bev, evdns_base, family, hostname, port);
+       }
+
+       memset(&ss, 0, sizeof(ss));
+
+       if (evutil_resolve(family, hostname, (struct sockaddr*)&ss,
+               &socklen, port)<0) {
+               _bufferevent_incref_and_lock(bev);
+               _bufferevent_run_eventcb(bev, BEV_EVENT_ERROR);
+               _bufferevent_decref_and_unlock(bev);
+               return -1;
+       }
+
+       return bufferevent_socket_connect(bev, (struct sockaddr*)&ss, socklen);
+}
+
 /*
  * Create a new buffered event object.
  *
index 977646dd5a014cab62028e7895ad43d9c1d39c58..2fad969b3459d0f149e19716cae842ab45127332 100644 (file)
@@ -83,7 +83,7 @@ AC_SUBST(OPENSSL_LIBS)
 
 dnl Checks for header files.
 AC_HEADER_STDC
-AC_CHECK_HEADERS(fcntl.h stdarg.h inttypes.h stdint.h stddef.h poll.h unistd.h sys/epoll.h sys/time.h sys/queue.h sys/event.h sys/param.h sys/ioctl.h sys/select.h sys/devpoll.h port.h netinet/in.h netinet/in6.h sys/socket.h sys/uio.h arpa/inet.h sys/eventfd.h sys/mman.h sys/sendfile.h)
+AC_CHECK_HEADERS(fcntl.h stdarg.h inttypes.h stdint.h stddef.h poll.h unistd.h sys/epoll.h sys/time.h sys/queue.h sys/event.h sys/param.h sys/ioctl.h sys/select.h sys/devpoll.h port.h netinet/in.h netinet/in6.h sys/socket.h sys/uio.h arpa/inet.h sys/eventfd.h sys/mman.h sys/sendfile.h netdb.h)
 if test "x$ac_cv_header_sys_queue_h" = "xyes"; then
        AC_MSG_CHECKING(for TAILQ_FOREACH in sys/queue.h)
        AC_EGREP_CPP(yes,
diff --git a/evdns.c b/evdns.c
index c2b1388a5690e4d8fe5c1acaf58615c506d439a0..41a5cf8a403f5ebf0a8b812b0911ab23cbcf3054 100644 (file)
--- a/evdns.c
+++ b/evdns.c
 #include <event2/event_struct.h>
 #include <event2/thread.h>
 
+#include <event2/bufferevent.h>
+#include <event2/bufferevent_struct.h>
+#include "bufferevent-internal.h"
+
 #include "defer-internal.h"
 #include "log-internal.h"
 #include "mm-internal.h"
@@ -3584,6 +3588,13 @@ struct evdns_base *
 evdns_base_new(struct event_base *event_base, int initialize_nameservers)
 {
        struct evdns_base *base;
+
+       /* Give the bufferevent library a hook into its evdns-enabled
+        * functionality.  We can't do this correctly or else libevent-core
+        * will depend on libevent-extras. */
+       _bufferevent_set_socket_connect_hostname_evdns_fn(
+               _bufferevent_socket_connect_hostname_evdns);
+
        base = mm_malloc(sizeof(struct evdns_base));
        if (base == NULL)
                return (NULL);
index b41e7819060d7d1649123dfa88dfab87c4e1f97c..39d40b10228774593d2be89eb539e9662976e7ab 100644 (file)
--- a/evutil.c
+++ b/evutil.c
@@ -60,6 +60,9 @@
 #ifdef _EVENT_HAVE_NETINET_IN6_H
 #include <netinet/in6.h>
 #endif
+#ifdef _EVENT_HAVE_NETDB_H
+#include <netdb.h>
+#endif
 
 #ifndef _EVENT_HAVE_GETTIMEOFDAY
 #include <sys/timeb.h>
@@ -320,6 +323,92 @@ evutil_socket_finished_connecting(evutil_socket_t fd)
        return 1;
 }
 
+/** Internal helper: use the host's (blocking) resolver to look up 'hostname',
+ * and set the sockaddr pointed to by 'sa' to the answer.  Assume we have
+ * *socklen bytes of storage; adjust *socklen to the number of bytes used.
+ * Try to return answers of type 'family', unless family is AF_UNSPEC.
+ * Return 0 on success and -1 on failure.  If 'port' is nonzero, it is
+ * a port number in host order: set the port in any resulting sockaddr to
+ * the specified port.
+ */
+int
+evutil_resolve(int family, const char *hostname, struct sockaddr *sa,
+    ev_socklen_t *socklen, int port)
+{
+#ifdef _EVENT_HAVE_GETADDRINFO_XXX
+       struct addrinfo hint, *hintp=NULL;
+       struct addrinfo *ai=NULL;
+       int r;
+       memset(&hint, 0, sizeof(hint));
+
+       if (family != AF_UNSPEC) {
+               hint.ai_family = family;
+               hintp = &hint;
+       }
+
+       r = getaddrinfo(hostname, NULL, hintp, &ai);
+       if (!ai)
+               return -1;
+       if (r || ai->ai_addrlen > *socklen) {
+               /* log/report error? */
+               freeaddrinfo(ai);
+               return -1;
+       }
+       /* XXX handle multiple return values better. */
+       memcpy(sa, ai->ai_addr, ai->ai_addrlen);
+       if (port) {
+               if (sa->sa_family == AF_INET)
+                       ((struct sockaddr_in*)sa)->sin_port = htons(port);
+               else if (sa->sa_family == AF_INET6)
+                       ((struct sockaddr_in6*)sa)->sin6_port = htons(port);
+       }
+       *socklen = ai->ai_addrlen;
+       freeaddrinfo(ai);
+       return 0;
+#else
+       /* XXXX use gethostbyname_r/gethostbyname2_r where available */
+       struct hostent *he;
+       struct sockaddr *sa_ptr;
+       struct sockaddr_in sin;
+       struct sockaddr_in6 sin6;
+       ev_socklen_t slen;
+       he = gethostbyname(hostname);
+       if (!he || !he->h_length) {
+               return -1;
+       }
+       /* XXX handle multiple return values better. */
+       if (he->h_addrtype == AF_INET) {
+               if (family != AF_INET && family != AF_UNSPEC)
+                       return -1;
+               memset(&sin, 0, sizeof(sin));
+               sin.sin_family = AF_INET;
+               sin.sin_port = htons(port);
+               memcpy(&sin.sin_addr, he->h_addr_list[0], 4);
+               sa_ptr = (struct sockaddr*)&sin;
+               slen = sizeof(struct sockaddr_in);
+       } else if (he->h_addrtype == AF_INET6) {
+               if (family != AF_INET6 && family != AF_UNSPEC)
+                       return -1;
+               sin6.sin6_family = AF_INET6;
+               sin6.sin6_port = htons(port);
+               memset(&sin6, 0, sizeof(sin6));
+               memcpy(sin6.sin6_addr.s6_addr, &he->h_addr_list[1], 16);
+               sa_ptr = (struct sockaddr*)&sin6;
+               slen = sizeof(struct sockaddr_in6);
+       } else {
+               event_warnx("gethostbyname returned unknown family %d",
+                   he->h_addrtype);
+               return -1;
+       }
+       if (slen > *socklen) {
+               return -1;
+       }
+       memcpy(sa, sa_ptr, slen);
+       *socklen = slen;
+       return 0;
+#endif
+}
+
 #ifdef WIN32
 #define E(code, s) { code, (s " [" #code " ]") }
 static struct { int code; const char *msg; } windows_socket_errors[] = {
diff --git a/http.c b/http.c
index 86b6869e4ffb6f422f9a8566837395fa9826c527..dda2c012ed3fdb47e02887c593f59650e5eb6948 100644 (file)
--- a/http.c
+++ b/http.c
 
 #include <sys/queue.h>
 
-#ifndef WIN32
+#ifdef _EVENT_HAVE_NETINET_IN_H
 #include <netinet/in.h>
+#endif
+#ifdef _EVENT_HAVE_NETDB_H
 #include <netdb.h>
 #endif
 
index 3123a40983d12480fbff9dcd98cdfe39a2bc6cb4..523a7fce06e239cece6cf9048496689051217c35 100644 (file)
@@ -162,6 +162,36 @@ struct bufferevent *bufferevent_socket_new(struct event_base *base, evutil_socke
  */
 int bufferevent_socket_connect(struct bufferevent *, struct sockaddr *, int);
 
+struct evdns_base;
+/**
+   Resolve the hostname 'hostname' and connect to it as with
+   bufferevent_socket_connect().
+
+   @param bufev An existing bufferevent allocated with bufferevent_socket_new()
+   @param evdns_base Optionally, an evdns_base to use for resolving hostnames
+      asynchronously. May be set to NULL for a blocking resolve.
+   @param family A preferred address family to resolve addresses to, or
+      AF_UNSPEC for no preference.  Only AF_INET, AF_INET6, and AF_UNSPEC are
+      supported.
+   @param hostname The hostname to resolve; see below for notes on recognized
+      formats
+   @param port The port to connect to on the resolved address.
+   @return 0 if successful, -1 on failure.
+
+   Recognized hostname formats are:
+
+       www.example.com      (hostname)
+       1.2.3.4              (ipv4address)
+       ::1                  (ipv6address)
+       [::1]                ([ipv6address])
+
+   Performance note: If you do not provide an evdns_base, this function
+   may block while it waits for a DNS response.  This is probably not
+   what you want.
+ */
+int bufferevent_socket_connect_hostname(struct bufferevent *b,
+    struct evdns_base *, int, const char *, int);
+
 /**
   Assign a bufferevent to a specific event_base.
 
index bbe4309e1a957c8096a030d9a5de77507309e3da..5abcc2cf9cb1c0b0e083de9e9913ed363394077c 100644 (file)
@@ -205,10 +205,13 @@ extern "C" {
 
 /**
  * The callback that contains the results from a lookup.
+ * - result is one of the DNS_ERR_* values (DNS_ERR_NONE for success)
  * - type is either DNS_IPv4_A or DNS_PTR or DNS_IPv6_AAAA
  * - count contains the number of addresses of form type
  * - ttl is the number of seconds the resolution may be cached for.
- * - addresses needs to be cast according to type
+ * - addresses needs to be cast according to type.  It will be an array of
+ *   4-byte sequences for ipv4, or an array of 16-byte sequences for ipv6,
+ *   or a nul-terminated string for PTR.
  */
 typedef void (*evdns_callback_type) (int result, char type, int count, int ttl, void *addresses, void *arg);
 
index e17820c29316b90b528f58334b5cb325b5878e04..177261679a6528038e32b06d5c89bd4ec1e844e4 100644 (file)
@@ -63,6 +63,8 @@
 #include "event2/event.h"
 #include "event2/event_compat.h"
 #include <event2/util.h>
+#include <event2/listener.h>
+#include <event2/bufferevent.h>
 #include "evdns.h"
 #include "log-internal.h"
 #include "regress.h"
@@ -379,7 +381,7 @@ dns_server(void)
        tt_int_op(evdns_base_count_nameservers(base), ==, 1);
        /* Now configure a nameserver port. */
        sock = socket(AF_INET, SOCK_DGRAM, 0);
-        if (sock<=0) {
+        if (sock<0) {
                 tt_abort_perror("socket");
         }
 
@@ -840,6 +842,213 @@ end:
                evdns_close_server_port(port);
 }
 
+/* === Test for bufferevent_socket_connect_hostname */
+
+static int total_connected_or_failed = 0;
+static struct event_base *be_connect_hostname_base = NULL;
+
+/* Implements a DNS server for the connect_hostname test. */
+static void
+be_connect_hostname_server_cb(struct evdns_server_request *req, void *data)
+{
+       int i;
+       int *n_got_p=data;
+       int added_any=0;
+       ++*n_got_p;
+
+       for (i=0;i<req->nquestions;++i) {
+               const int qtype = req->questions[i]->type;
+               const int qclass = req->questions[i]->dns_question_class;
+               const char *qname = req->questions[i]->name;
+               struct in_addr ans;
+
+               if (qtype == EVDNS_TYPE_A &&
+                   qclass == EVDNS_CLASS_INET &&
+                   !evutil_ascii_strcasecmp(qname, "nobodaddy.example.com")) {
+                       ans.s_addr = htonl(0x7f000001);
+                       evdns_server_request_add_a_reply(req, qname,
+                           1, &ans.s_addr, 2000);
+                       added_any = 1;
+               } else if (!evutil_ascii_strcasecmp(qname,
+                       "nosuchplace.example.com")) {
+                       /* ok, just say notfound. */
+               } else {
+                       TT_GRIPE(("Got weird request for %s",qname));
+               }
+       }
+       if (added_any)
+               evdns_server_request_respond(req, 0);
+       else
+               evdns_server_request_respond(req, 3);
+}
+
+/* Implements a listener for connect_hostname test. */
+static void
+nil_accept_cb(struct evconnlistener *l, evutil_socket_t fd, struct sockaddr *s,
+    int socklen, void *arg)
+{
+       int *p = arg;
+       (*p)++;
+       /* don't do anything with the socket; let it close when we exit() */
+}
+
+/* Helper: return the port that a socket is bound on, in host order. */
+static int
+get_socket_port(evutil_socket_t fd)
+{
+       struct sockaddr_storage ss;
+       ev_socklen_t socklen = sizeof(ss);
+       if (getsockname(fd, (struct sockaddr*)&ss, &socklen) != 0)
+               return -1;
+       if (ss.ss_family == AF_INET)
+               return ntohs( ((struct sockaddr_in*)&ss)->sin_port);
+       else if (ss.ss_family == AF_INET6)
+               return ntohs( ((struct sockaddr_in6*)&ss)->sin6_port);
+       else
+               return -1;
+}
+
+/* Bufferevent event callback for the connect_hostname test: remembers what
+ * event we got. */
+static void
+be_connect_hostname_event_cb(struct bufferevent *bev, short what, void *ctx)
+{
+       int *got = ctx;
+       if (!*got) {
+               TT_BLATHER(("Got a bufferevent event %d", what));
+               *got = what;
+
+               if ((what & BEV_EVENT_CONNECTED) || (what & BEV_EVENT_ERROR)) {
+                       ++total_connected_or_failed;
+                       if (total_connected_or_failed >= 5)
+                               event_base_loopexit(be_connect_hostname_base,
+                                   NULL);
+               }
+       } else {
+               TT_FAIL(("Two events on one bufferevent. %d,%d",
+                       (int)*got, (int)what));
+       }
+}
+
+static void
+test_bufferevent_connect_hostname(void *arg)
+{
+       struct basic_test_data *data = arg;
+       struct evconnlistener *listener = NULL;
+       struct bufferevent *be1=NULL, *be2=NULL, *be3=NULL, *be4=NULL, *be5=NULL;
+       int be1_outcome=0, be2_outcome=0, be3_outcome=0, be4_outcome=0,
+           be5_outcome=0;
+       struct evdns_base *dns=NULL;
+       struct evdns_server_port *port=NULL;
+       evutil_socket_t server_fd=-1;
+       struct sockaddr_in sin;
+       int listener_port=-1, dns_port=-1;
+       int n_accept=0, n_dns=0;
+       char buf[128];
+
+       be_connect_hostname_base = data->base;
+
+       /* Bind an address and figure out what port it's on. */
+       memset(&sin, 0, sizeof(sin));
+       sin.sin_family = AF_INET;
+       sin.sin_addr.s_addr = htonl(0x7f000001); /* 127.0.0.1 */
+       sin.sin_port = 0;
+       listener = evconnlistener_new_bind(data->base, nil_accept_cb,
+           &n_accept,
+           LEV_OPT_REUSEABLE|LEV_OPT_CLOSE_ON_EXEC,
+           -1, (struct sockaddr *)&sin, sizeof(sin));
+       listener_port = get_socket_port(evconnlistener_get_fd(listener));
+
+       /* Start an evdns server that resolves nobodaddy.example.com to
+        * 127.0.0.1 */
+       memset(&sin, 0, sizeof(sin));
+       sin.sin_family = AF_INET;
+       sin.sin_addr.s_addr = htonl(0x7f000001); /* 127.0.0.1 */
+       sin.sin_port = 0;
+       server_fd = socket(AF_INET, SOCK_DGRAM, 0);
+       tt_int_op(server_fd, >=, 0);
+       if (bind(server_fd, (struct sockaddr*)&sin, sizeof(sin))<0) {
+               tt_abort_perror("bind");
+       }
+        evutil_make_socket_nonblocking(server_fd);
+       dns_port = get_socket_port(server_fd);
+       port = evdns_add_server_port_with_base(data->base, server_fd, 0,
+           be_connect_hostname_server_cb, &n_dns);
+
+       /* Start an evdns_base that uses the server as its resolver. */
+       dns = evdns_base_new(data->base, 0);
+       evutil_snprintf(buf, sizeof(buf), "127.0.0.1:%d", dns_port);
+       evdns_base_nameserver_ip_add(dns, buf);
+
+       /* Now, finally, at long last, launch the bufferevents.  One should do
+        * a failing lookup IP, one should do a successful lookup by IP,
+        * and one should do a successful lookup by hostname. */
+       be1 = bufferevent_socket_new(data->base, -1, BEV_OPT_CLOSE_ON_FREE);
+       be2 = bufferevent_socket_new(data->base, -1, BEV_OPT_CLOSE_ON_FREE);
+       be3 = bufferevent_socket_new(data->base, -1, BEV_OPT_CLOSE_ON_FREE);
+       be4 = bufferevent_socket_new(data->base, -1, BEV_OPT_CLOSE_ON_FREE);
+       be5 = bufferevent_socket_new(data->base, -1, BEV_OPT_CLOSE_ON_FREE);
+
+       bufferevent_setcb(be1, NULL, NULL, be_connect_hostname_event_cb,
+           &be1_outcome);
+       bufferevent_setcb(be2, NULL, NULL, be_connect_hostname_event_cb,
+           &be2_outcome);
+       bufferevent_setcb(be3, NULL, NULL, be_connect_hostname_event_cb,
+           &be3_outcome);
+       bufferevent_setcb(be4, NULL, NULL, be_connect_hostname_event_cb,
+           &be4_outcome);
+       bufferevent_setcb(be5, NULL, NULL, be_connect_hostname_event_cb,
+           &be5_outcome);
+
+       /* Launch an async resolve that will fail. */
+       tt_assert(!bufferevent_socket_connect_hostname(be1, dns, AF_INET,
+               "nosuchplace.example.com", listener_port));
+       /* Connect to the IP without resolving. */
+       tt_assert(!bufferevent_socket_connect_hostname(be2, dns, AF_INET,
+               "127.0.0.1", listener_port));
+       /* Launch an async resolve that will succeed. */
+       tt_assert(!bufferevent_socket_connect_hostname(be3, dns, AF_INET,
+               "nobodaddy.example.com", listener_port));
+       /* Use the blocking resolver.  This one will fail if your resolver
+        * can't resolve localhost to 127.0.0.1 */
+       tt_assert(!bufferevent_socket_connect_hostname(be4, NULL, AF_INET,
+               "localhost", listener_port));
+       /* Use the blocking resolver with a nonexistent hostname. */
+       tt_assert(bufferevent_socket_connect_hostname(be5, NULL, AF_INET,
+               "nonesuch.nowhere.example.com", 80) < 0);
+
+       event_base_dispatch(data->base);
+
+       tt_int_op(be1_outcome, ==, BEV_EVENT_ERROR);
+       tt_int_op(be2_outcome, ==, BEV_EVENT_CONNECTED);
+       tt_int_op(be3_outcome, ==, BEV_EVENT_CONNECTED);
+       tt_int_op(be4_outcome, ==, BEV_EVENT_CONNECTED);
+       tt_int_op(be5_outcome, ==, BEV_EVENT_ERROR);
+
+       tt_int_op(n_accept, ==, 3);
+       tt_int_op(n_dns, ==, 2);
+
+end:
+       if (listener)
+               evconnlistener_free(listener);
+       if (server_fd>=0)
+               EVUTIL_CLOSESOCKET(server_fd);
+       if (port)
+                evdns_close_server_port(port);
+       if (dns)
+               evdns_base_free(dns, 0);
+       if (be1)
+               bufferevent_free(be1);
+       if (be2)
+               bufferevent_free(be2);
+       if (be3)
+               bufferevent_free(be3);
+       if (be4)
+               bufferevent_free(be4);
+       if (be5)
+               bufferevent_free(be5);
+}
+
 #define DNS_LEGACY(name, flags)                                        \
        { #name, run_legacy_test_fn, flags|TT_LEGACY, &legacy_setup,   \
                     dns_##name }
@@ -854,6 +1063,8 @@ struct testcase_t dns_testcases[] = {
        { "retry", dns_retry_test, TT_FORK|TT_NEED_BASE, &basic_setup, NULL },
        { "reissue", dns_reissue_test, TT_FORK|TT_NEED_BASE, &basic_setup, NULL },
        { "inflight", dns_inflight_test, TT_FORK|TT_NEED_BASE, &basic_setup, NULL },
+       { "bufferevent_connnect_hostname", test_bufferevent_connect_hostname,
+         TT_FORK|TT_NEED_BASE, &basic_setup, NULL },
 
         END_OF_TESTCASES
 };
index 0ca2b06789f4ad69839cd93d4581638a9c064613..55aba1fe93567053a76c95a0be70fd01bc8ceb41 100644 (file)
@@ -490,6 +490,53 @@ end:
 
 }
 
+static void
+test_evutil_resolve(void *arg)
+{
+       struct sockaddr_storage ss;
+       struct sockaddr_in6 *sin6;
+       struct sockaddr_in *sin;
+       ev_socklen_t socklen = sizeof(ss);
+       char buf[128];
+       const char *cp;
+       int r;
+
+       memset(&ss, 0xff, sizeof(ss)); /* Make sure it starts out confused.*/
+       r = evutil_resolve(AF_INET, "www.google.com", (struct sockaddr*)&ss,
+           &socklen, 80);
+       if (r<0) {
+               TT_BLATHER(("Couldn't resolve www.google.com"));
+               tt_skip();
+       }
+       tt_int_op(ss.ss_family, ==, AF_INET);
+       tt_int_op(socklen, ==, sizeof(struct sockaddr_in));
+       sin = (struct sockaddr_in*)&ss;
+       tt_int_op(sin->sin_port, ==, htons(80));
+       tt_int_op(sin->sin_addr.s_addr, !=, 0xffffffff);
+
+       cp = evutil_inet_ntop(AF_INET, &sin->sin_addr, buf, sizeof(buf));
+       TT_BLATHER(("www.google.com resolved to %s",cp?cp:"<unwriteable>"));
+
+       memset(&ss, 0xff, sizeof(ss)); /* Make sure it starts out confused.*/
+       socklen = sizeof(ss);
+       r = evutil_resolve(AF_INET6, "ipv6.google.com", (struct sockaddr*)&ss,
+           &socklen, 80);
+       if (r<0) {
+               TT_BLATHER(("Couldn't do an ipv6 lookup for ipv6.google.com"));
+               goto end;
+       }
+       tt_int_op(ss.ss_family, ==, AF_INET6);
+       tt_int_op(socklen, ==, sizeof(struct sockaddr_in6));
+       sin6 = (struct sockaddr_in6*)&ss;
+       tt_int_op(sin6->sin6_port, ==, htons(80));
+
+       cp = evutil_inet_ntop(AF_INET6, &sin6->sin6_addr, buf, sizeof(buf));
+       TT_BLATHER(("ipv6.google.com resolved to %s",cp?cp:"<unwriteable>"));
+
+end:
+       ;
+}
+
 struct testcase_t util_testcases[] = {
        { "ipv4_parse", regress_ipv4_parse, 0, NULL, NULL },
        { "ipv6_parse", regress_ipv6_parse, 0, NULL, NULL },
@@ -500,7 +547,7 @@ struct testcase_t util_testcases[] = {
        { "strlcpy", test_evutil_strlcpy, 0, NULL, NULL },
        { "log", test_evutil_log, TT_FORK, NULL, NULL },
        { "upcast", test_evutil_upcast, 0, NULL, NULL },
+       { "resolve", test_evutil_resolve, TT_FORK, NULL, NULL },
        END_OF_TESTCASES,
 };
 
-
index dc1121166a84aa09e62f135aa6df5362c6c704e7..ef82af995872049eb67f81eab1307afe4aef9df8 100644 (file)
@@ -33,6 +33,9 @@
 #include "log-internal.h"
 #include <stdio.h>
 #include <stdlib.h>
+#ifdef _EVENT_HAVE_SYS_SOCKET_H
+#include <sys/socket.h>
+#endif
 
 #ifdef __cplusplus
 extern "C" {
@@ -137,6 +140,9 @@ int evutil_socket_connect(evutil_socket_t *fd_ptr, struct sockaddr *sa, int sock
 
 int evutil_socket_finished_connecting(evutil_socket_t fd);
 
+int evutil_resolve(int family, const char *hostname, struct sockaddr *sa,
+    ev_socklen_t *socklen, int port);
+
 /* Evaluates to the same boolean value as 'p', and hints to the compiler that
  * we expect this value to be false. */
 #ifdef __GNUC__X