kcat icon indicating copy to clipboard operation
kcat copied to clipboard

Broker DNS resolution failures on macOS

Open benweint opened this issue 5 years ago • 9 comments

Overview

We're seeing a strange issue related to DNS resolution with kafkacat 1.5.0 running on macOS Mojave and Catalina (possibly other versions too, haven't tried farther back). kafkacat is reporting DNS name resolution failures when trying to resolve the advertised hostnames of Kafka brokers, despite those DNS names resolving fine from the same host using other tools.

Here's an example of what this looks like (broker hostnames and topic names have been scrubbed in this and future examples):

▶ kafkacat -C -b kafka-a.example.com -t my-topic -c 1 -e
% ERROR: Local: Host resolution failure: kafka-xyz.example.com:9092/42: Failed to resolve 'kafka-xyz.example.com:9092': nodename nor servname provided, or not known (after 0ms in state CONNECT)

In this example:

  • kafka-a.example.com is a 'seed' DNS record (CNAME) pointing at some arbitrary broker in the cluster (different from the one that hosts the topic we're trying to consume from)
  • kafka-xyz.example.com is an A record, pointing at the broker with ID 42, which is leading the sole partition for my-topic. It's the name that broker 42 is advertising in ZooKeeper.

From what I can tell, this only happens on macOS. Running the same exact command from the same host in a Docker container or Linux VM yields the expected results, and no DNS resolution failure. Resolving both hostnames works fine using dig / nslookup locally on macOS.

The fact that kafkacat is able to even find kafka-xyz.example.com suggests that it was able to resolve kafka-a.example.com without issue. Interestingly, directly giving kafka-xyz.example.com (the advertised name of the lead broker) to kafkacat like this also yields working results:

▶ kafkacat -C -b kafka-xyz.example.com -t my-topic -c 1 -e # where kafka-xyz.example.com is the advertised name of the lead broker for the sole partition of my-topic
< ... message content ... >

... but if I use a topic that has leaders on more than one broker, DNS lookups for all but the first leader (the one specified with the -b flag) will fail in the same way.

Diagnosis

It looks like librdkafka uses the native getaddrinfo to do DNS resolution, so I tried tracing the calls to getaddrinfo to confirm that the args looked correct. I used dtrace to do this, and here's what I found (output cleaned up a little bit):

▶ sudo dtrace \
   -n 'pid$target::getaddrinfo:entry { printf("  getaddrinfo(%s, %s, ...)", copyinstr(arg0), copyinstr(arg1)) }' \
   -n 'pid$target::getaddrinfo:return { printf("  getaddrinfo returned %d", arg1); }' \
   -n 'pid$target::rd_getaddrinfo:entry { printf("rd_getaddrinfo(nodesvc=%s, defsvc=%s, flags=0x%x, family=%d, socktype=%d, protocol=%d)\n", copyinstr(arg0), copyinstr(arg1), arg2, arg3, arg4, arg5); }' \
   -n 'pid$target::rd_getaddrinfo:return { printf("rd_getaddrinfo returned %d\n", arg1); }' \
   -c "kafkacat -C -b kafka-a.example.com -t my-topic -c 1 -e"

% ERROR: Local: Host resolution failure: kafka-xyz.example.com:9092/42: Failed to resolve 'kafka-xyz.example.com:9092': nodename nor servname provided, or not known (after 0ms in state CONNECT)

CPU     ID                    FUNCTION:NAME
  8  41983             rd_getaddrinfo:entry rd_getaddrinfo(nodesvc=kafka-a.example.com:9092, defsvc=9092, flags=0x400, family=0, socktype=1, protocol=6)

  8  41981                getaddrinfo:entry   getaddrinfo(kafka-a.example.com, 9092, ...)
  4  41982               getaddrinfo:return   getaddrinfo returned 0
  4  41984            rd_getaddrinfo:return rd_getaddrinfo returned 140546978879280

 10  41983             rd_getaddrinfo:entry rd_getaddrinfo(nodesvc=kafka-xyz.example.com:9092, defsvc=9092, flags=0x400, family=0, socktype=1, protocol=6)

 10  41981                getaddrinfo:entry   getaddrinfo(kafka-xyz.example.com, 9092, ...)
 10  41982               getaddrinfo:return   getaddrinfo returned 8
 10  41984            rd_getaddrinfo:return rd_getaddrinfo returned 0

(Note that I'm tracing both entry and exit from rd_getaddrinfo and getaddrinfo itself, just because it makes it easier to get at the flags, family, socktype, and protocol attributes, which are otherwise tucked away inside the struct addrinfo arg to getaddrinfo.)

Note that the first resolution attempt (for kafka-a.example.com) works fine. The second attempt (for kafka-xyz) fails with error code 8, which is EAI_NONAME, matching the error message.

Weirdly, using the exact same getaddrinfo args from a standalone test program on the same system also works just fine:

Test program (excuse my awful C):

#include <stdio.h>
#include <sys/types.h>
#include <sys/socket.h>
#include <netdb.h>

void resolve(char *name) {
  printf("resolving %s\n", name);
  char *defsvc = "9092";

  int family = AF_UNSPEC;
  int socktype = SOCK_STREAM;
  int protocol = 6;
  int flags = AI_ADDRCONFIG;

  struct addrinfo *ais;
  struct addrinfo hints = { .ai_family = family,
          .ai_socktype = socktype,
          .ai_protocol = protocol,
          .ai_flags = flags };

  int r = getaddrinfo(name, defsvc, &hints, &ais);
  printf("got return code = %d\n", r);
}

int main(int argc, char **argv) {
  resolve("kafka-a.example.com");
  resolve("kafka-xyz.example.com");
  return 0;
}

Example output:

resolving kafka-a.example.com
got return code = 0
resolving kafka-xyz.example.com
got return code = 0

More Diagnosis Than You Probably Want

I spent some time tracing through the getaddrinfo guts on macOS to try to figure out what's going on, and found the RES_DEBUG environment variable, which you can set to trigger lots of debugging messages from the DNS resolver implementation on macOS. Here's the log spew for the first resolution (the one that works):

▶ RES_DEBUG=1 kafkacat -C -b kafka-a.example.com -t my-topic -c 10 -e
;; mdns_addrinfo node kafka-a.example.com serv �#�xj
;; _mdns_search wait loop
;; mdns query kafka-a.example.com type 1 class 1 ifindex 0 [ctx 0x7000056a7420]
;; mdns query kafka-a.example.com type 28 class 1 ifindex 0 [ctx 0x7000056a7478]
;; set kevent timeout 35.0 [ctx 0x7000056a7420 0x7000056a7478]
;; _mdns_search calling DNSServiceProcessResult
;; _mdns_query_callback ctx 0x7000056a7420 flags=0x00000003 (kDNSServiceFlagsMoreComing is set)
;; _mdns_hostent_append_alias(0x7000056a7698, kafka-a.example.com.)
;; _mdns_hostent_append_addr(0x7000056a7698, 0x7fbd1aa000b5, 4)
;; [kafka-a.example.com. type 1 class 1] reply [ctx 0x7000056a7420]
;; _mdns_query_callback sending kevent wakeup
;; _mdns_query_callback ctx 0x7000056a7478 flags=0x00000002
;; cleared kDNSServiceFlagsMoreComing flag for ctx 0x7000056a7420
;; [kafka-a.example.com. type 28 class 1]: error -65554 [ctx 0x7000056a7478]
;; _mdns_query_callback sending kevent wakeup
;; DNSServiceProcessResult -> (null)
;; mdns is_complete type 1 ctx 0x7000056a7420 clear more coming - complete
;; mdns is_complete type 1 ctx 0x7000056a7420 host addr count 1 complete -> true
;; mdns is_complete type 1 ctx 0x7000056a7420  - complete
;; ctx 0 0x7000056a7420 error=0 complete=true
;; type ns_t_a got_a_response=GOT_DATA ctx 0x7000056a7420
;; [kafka-a.example.com type 0 class 1] finished processing ctx 0x7000056a7420
;; mdns is_complete type 28 ctx 0x7000056a7478 clear more coming - complete
;; mdns is_complete type 28 ctx 0x7000056a7478  - incomplete
;; ctx 1 0x7000056a7478 error=-65554 complete=false
;; [kafka-a.example.com type 0 class 1] finished processing ctx 0x7000056a7478
;; [kafka-a.example.com type 0 class 1] done [ctx 0x7000056a7420 0x7000056a7478]
;; finished _mdns_search loop [ctx 0x7000056a7420 0x7000056a7478]
;; mdns is_complete type 1 ctx 0x7000056a7420 clear more coming - complete
;; mdns is_complete type 1 ctx 0x7000056a7420 host addr count 1 complete -> true
;; mdns is_complete type 1 ctx 0x7000056a7420  - complete
;; _mdns_search ctx 0x7000056a7420 complete
;; mdns is_complete type 28 ctx 0x7000056a7478 clear more coming - complete
;; mdns is_complete type 28 ctx 0x7000056a7478  - incomplete
;; _mdns_search ctx 0x7000056a7478 incomplete
;; _mdns_search overall complete
;; _mdns_search exit res 0

... and here's what it shows for the second resoultion (the one that fails):

;; mdns_addrinfo node kafka-xyz.example.com serv �#�Ȼ
;; finished _mdns_search loop [ctx 0x0 0x0]
;; _mdns_search overall incomplete
;; _mdns_search exit res -1
% ERROR: Local: Host resolution failure: kafka-xyz.example.com:9092/43: Failed to resolve 'kafka-xyz.example.com:9092': nodename nor servname provided, or not known (after 0ms in state CONNECT)
;; mdns_addrinfo node kafka-xyz.example.com serv �#�Ȼ
;; finished _mdns_search loop [ctx 0x0 0x0]
;; _mdns_search overall incomplete
;; _mdns_search exit res -1

Note that it jumps straight from the mdns_addrinfo line to the 'finished' line. The code that generates these messages is here from what I can tell.

Based on some more poking around with dtrace, it looks like the proximate cause of the early return on the second call to getaddrinfo is that the kevent() call on line 1703 of the above-linked mdns_module.c returns a non-zero value (consistently 9 on my machine) on the second getaddrinfo call. As to why that happens or what it suggests about this bug, I'm pretty out of my depth.

Wild Speculation

To be honest, this smells more like a macOS bug in the implementation of getaddrinfo than a bug in kafkacat or librdkafka to me, but I figured I'd start here.

benweint avatar Nov 13 '19 22:11 benweint

One other note: we have multiple Kafka clusters, and this issue only seems to affect the largest of them (a few hundred brokers). I'm not sure what that means, but it suggests to me that it may be timing related, or otherwise related to other work happening in the kafkacat process between the initial successful name resolution and the failure.

benweint avatar Nov 13 '19 23:11 benweint

What a great issue report! 💯

Error 9 is EBADFD, suggesting that the kevent kq file descriptor might have been closed for whatever reason.

Is it reproducible on multiple osx machines?

Are you building librdkafka on the same machine or using brew? If the latter, try building from source to rule out any ABI compatibility issues.

edenhill avatar Nov 18 '19 13:11 edenhill

Can confirm this is happening on my OSX (Catalina) machine as well. With librdkafka and kafkacat both built from source.

Something interesting (and probably a red herring) is that setting the sparse connections option to false exhibits slightly different behavior. Probably because of the slightly different bootstrap/retry logic in src/rdkafka_broker.c when not using sparse connections.

kafkacat -X enable.sparse.connections=false -b ... will fail immediately to resolve all of the brokers in the cluster with % ERROR: Local: Host resolution failure: ... (similar behavior as above). But within a few seconds the number of active broker connections will start to increase and eventually you can see your Kafka messages.

Still digging into the retry logic and timing of things at bootstrap time...

dakotasanchez avatar Nov 18 '19 21:11 dakotasanchez

Regarding the kevent return value, the man page (which might be wrong, but I'd be surprised) for kevent has this to say about the return values (emphasis mine):

The kevent(), kevent64() and kevent_qos() system calls return the number of events placed in the eventlist, up to the value given by nevents. If an error occurs while processing an element of the changelist and there is enough room in the eventlist, then the event will be placed in the eventlist with EV_ERROR set in flags and the system error in data. Otherwise, -1 will be returned, and errno will be set to indicate the error condition. If the time limit expires, then kevent(), kevent64() and kevent_qos() return 0.

I read that as saying that a return value of 9 indicates that there were 9 events placed in the eventlist.

As dakotasanchez mentioned, this is definitely reproducible on multiple macOS machines (all talking to the same cluster, though).

I can also confirm that building from source still exhibits the same problem.

benweint avatar Nov 22 '19 04:11 benweint

Huh, upon further investigation with dtrace, I'm thinking your theory about EBADF might be right. Tracing the return values of kqueue and kevent (along with a few other calls) for both the first (successful) DNS query and the second (failed) one, I see this:

Successful query

        mdns_addrinfo(?, kafka-a.example.com, ?, family=0, socktype=1, proto=6, flags=0x00001400, iface=0)
          _mdns_search(name=kafka-a.example.com, class=1, type=0, iface=0, flags=0x00008000)
            _mdns_deadline
          kqueue(...)
          kqueue returned 7
          kevent(7, ...)
          kevent returned 0
            DNSServiceCreateConnection
            _mdns_query_start
            _mdns_query_start returns 0
            _mdns_query_start
            _mdns_query_start returns 0
          kevent(7, ...)
          kevent returned 0
          kevent(7, ...)
          kevent returned 1
          kevent(7, ...)
          kevent returned 1
          kevent(7, ...)
          kevent returned 0
          kevent(7, ...)
          kevent returned 0
            _mdns_query_clear
            _mdns_query_clear
          _mdns_search returned 0
        mdns_addrinfo returns 140284930295824

Most notably, this bit:

          kqueue(...)
          kqueue returned 7
          kevent(7, ...)

... which shows kqueue returning 7, and then that value being passed directly into kevent, which matches what I see in the code:

	/* set up the kqueue */
	kq = kqueue();
	EV_SET(&ev, 1, EVFILT_USER, EV_ADD | EV_CLEAR, 0, 0, 0);
	n = kevent(kq, &ev, 1, NULL, 0, NULL);
	if (n != 0) wait = false;

	while (wait) {
            // do stuff
        }

Failed query

Starting from the same point, here's the trace for a failed DNS query:

        mdns_addrinfo(?, kafka-xyz.example.com, ?, family=0, socktype=1, proto=6, flags=0x00001400, iface=0)
          _mdns_search(name=kafka-xyz.example.com, class=1, type=0, iface=0, flags=0x00008000)
            _mdns_deadline
          kqueue(...)
          kqueue returned 24
          kevent(4294967295, ...)
          kevent returned 9
          _mdns_search returned 4294967295
        mdns_addrinfo returns 0

Note that 4294967295 == 0xffffffff. This is super suspect: the code shows no mutations to the kq variable between its assignment and use in handing off to kevent, so short of some kind of semi-random memory corruption issue from another thread, I don't see how kevent would wind up with that bogus first arg.

benweint avatar Nov 22 '19 04:11 benweint

Hmm, I'm starting to wonder if dtrace is behaving weirdly WRT function call args and return values for kevent. Tracing the syscall::kevent:return probe instead of pid$target::kevent:return, I see a return value of -1 (as indicated by the man page) rather than 9.

benweint avatar Nov 22 '19 04:11 benweint

Oof, I think I've figured this out.

dtrace was leading me astray, after poking around in lldb a bit, it looks like kqueue was just returning -1 (== 0xffffffff), which is why kevent was receiving that as its first arg.

Reading the kqueue manpage section on errors:

     The kqueue() system call fails if:

     [ENOMEM]           The kernel failed to allocate enough memory for the
                        kernel queue.

     [EMFILE]           The per-process descriptor table is full.

     [ENFILE]           The system file table is full.

Pretty sure my system's not out of memory, and it seems doubtful that the system file table is full, but the per-process descriptor table... seems plausible.

Lo and behold:

▶ ulimit -n
256

Guess how many brokers we have in that one cluster that doesn't work with kafkacat? (Hint: it's more than 256.)

So, the workaround here is simple:

▶ ulimit -n 1000
▶ kafkacat -C -b kafka-a.example.com -t my-topic -c 1 -e
< ... message content ... >

Sigh, sorry for the trouble!

benweint avatar Nov 23 '19 05:11 benweint

On second thought, I’m going to re-open this. If kafkacat is able to estimate its FD needs after receiving the metadata response, and can detect that the current ulimit is too low, is it reasonable for it to attempt to raise the limit with setrlimit?

benweint avatar Nov 23 '19 15:11 benweint

Good find!

librdkafka will create a separate thread per known broker, even if it doesn't need to connect to the broker, and each such broker thread has two file descriptors for queue-wakeups, so I could see how you quickly run out of those 256 fds.

kafkacat itself does not really know about the number of brokers, but we could have librdkafka adjust the fd limit automatically as it sees fit, but is this a kosher thing to do for a library?

edenhill avatar Nov 26 '19 08:11 edenhill