Async DNS search domain support (#25987)

Fixes #25825

Provides search domain support for Async DNS.

There are two parts here, first is the search domain support itself,
which is relatively straight forward - on lookup, create a list of hosts
to attempt to resolve based on the search domains configuration
(including the host with no postfix), exclude all those hosts that have
less dots than the configured ndots, and then search until you find one
that returns any records. In this case the search domains are applied for
each nameserver attempt, so if a particular nameserver times out, it
won't attempt to search any more domains on that nameserver.

The second part is in actually detecting the system configuration. I've
only implemented partial support here, it reads /etc/resolv.conf, and
nothing else. There are ways to detect it on other platforms, there are
also environment variables that are supposed to be honored on Linux and
other *nix's, but to keep this change manageable, I've focussed on just
supporting the minimum to meet the Kubernetes service discovery use case,
and documented the limitations.
This commit is contained in:
James Roper 2018-12-18 03:42:20 +11:00 committed by Arnout Engelen
parent 81b499cc7b
commit 762c9cb019
11 changed files with 400 additions and 71 deletions

View file

@ -30,6 +30,8 @@ class AsyncDnsResolverIntegrationSpec extends AkkaSpec(
akka.loggers = ["akka.testkit.SilenceAllTestEventListener"]
akka.io.dns.resolver = async-dns
akka.io.dns.async-dns.nameservers = ["localhost:${AsyncDnsResolverIntegrationSpec.dockerDnsServerPort}"]
akka.io.dns.async-dns.search-domains = ["foo.test", "test"]
akka.io.dns.async-dns.ndots = 2
// akka.io.dns.async-dns.nameservers = default
""") with DockerBindDnsService with WithLogCapturing {
val duration = 10.seconds
@ -152,6 +154,43 @@ class AsyncDnsResolverIntegrationSpec extends AkkaSpec(
answer.records.length should be(48)
}
"resolve using search domains where some have not enough ndots" in {
val name = "a-single"
val expectedName = "a-single.foo.test"
val answer = resolve(name, DnsProtocol.Ip(ipv6 = false))
withClue(answer) {
answer.name shouldEqual expectedName
answer.records.size shouldEqual 1
answer.records.head.name shouldEqual expectedName
answer.records.head.asInstanceOf[ARecord].ip shouldEqual InetAddress.getByName("192.168.1.20")
}
}
"resolve using search domains" in {
val name = "a-single.foo"
val expectedName = "a-single.foo.test"
val answer = resolve(name, DnsProtocol.Ip(ipv6 = false))
withClue(answer) {
answer.name shouldEqual expectedName
answer.records.size shouldEqual 1
answer.records.head.name shouldEqual expectedName
answer.records.head.asInstanceOf[ARecord].ip shouldEqual InetAddress.getByName("192.168.1.20")
}
}
"resolve localhost even though ndots is greater than 0" in {
// This currently works because the nameserver resolves localhost, but in future should work because we've
// implemented proper support for /etc/hosts
val name = "localhost"
val answer = resolve(name, DnsProtocol.Ip(ipv6 = false))
withClue(answer) {
answer.name shouldEqual "localhost"
answer.records.size shouldEqual 1
answer.records.head.name shouldEqual "localhost"
answer.records.head.asInstanceOf[ARecord].ip shouldEqual InetAddress.getByName("127.0.0.1")
}
}
def resolve(name: String, requestType: RequestType = Ip()): DnsProtocol.Resolved = {
(IO(Dns) ? DnsProtocol.Resolve(name, requestType)).mapTo[DnsProtocol.Resolved].futureValue
}

View file

@ -21,33 +21,102 @@ class DnsSettingsSpec extends AkkaSpec {
"""
nameservers = "default"
resolve-timeout = 1s
search-domains = []
ndots = 1
"""))
// Will differ based on name OS DNS servers so just validating it does not throw
dnsSettings.NameServers
}
"parse single name server" in {
"parse a single name server" in {
val dnsSettings = new DnsSettings(eas, ConfigFactory.parseString(
"""
nameservers = "127.0.0.1"
resolve-timeout = 1s
search-domains = []
ndots = 1
"""))
dnsSettings.NameServers.map(_.getAddress) shouldEqual List(InetAddress.getByName("127.0.0.1"))
}
"parse list" in {
"parse a list of name servers" in {
val dnsSettings = new DnsSettings(eas, ConfigFactory.parseString(
"""
nameservers = ["127.0.0.1", "127.0.0.2"]
resolve-timeout = 1s
search-domains = []
ndots = 1
"""))
dnsSettings.NameServers.map(_.getAddress) shouldEqual List(
InetAddress.getByName("127.0.0.1"), InetAddress.getByName("127.0.0.2")
)
}
"use host search domains if set to default" in {
val dnsSettings = new DnsSettings(eas, ConfigFactory.parseString(
"""
nameservers = "127.0.0.1"
resolve-timeout = 1s
search-domains = "default"
ndots = 1
"""))
// Will differ based on name OS DNS servers so just validating it does not throw
dnsSettings.SearchDomains shouldNot equal(List("default"))
}
"parse a single search domain" in {
val dnsSettings = new DnsSettings(eas, ConfigFactory.parseString(
"""
nameservers = "127.0.0.1"
resolve-timeout = 1s
search-domains = "example.com"
ndots = 1
"""))
dnsSettings.SearchDomains shouldEqual List("example.com")
}
"parse a single list of search domains" in {
val dnsSettings = new DnsSettings(eas, ConfigFactory.parseString(
"""
nameservers = "127.0.0.1"
resolve-timeout = 1s
search-domains = [ "example.com", "example.net" ]
ndots = 1
"""))
dnsSettings.SearchDomains shouldEqual List("example.com", "example.net")
}
"use host ndots if set to default" in {
val dnsSettings = new DnsSettings(eas, ConfigFactory.parseString(
"""
nameservers = "127.0.0.1"
resolve-timeout = 1s
search-domains = "example.com"
ndots = "default"
"""))
// Will differ based on name OS DNS servers so just validating it does not throw
dnsSettings.NDots
}
"parse ndots" in {
val dnsSettings = new DnsSettings(eas, ConfigFactory.parseString(
"""
nameservers = "127.0.0.1"
resolve-timeout = 1s
search-domains = "example.com"
ndots = 5
"""))
dnsSettings.NDots shouldEqual 5
}
}
}

View file

@ -145,6 +145,8 @@ class AsyncDnsResolverSpec extends AkkaSpec(
"""
nameservers = ["one","two"]
resolve-timeout = 25ms
search-domains = []
ndots = 1
"""))
system.actorOf(Props(new AsyncDnsResolver(settings, new AsyncDnsCache(), (arf, l) {
clients

View file

@ -0,0 +1,67 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.io.dns.internal
import org.scalatest.{ Matchers, WordSpec }
class ResolvConfParserSpec extends WordSpec with Matchers {
private def parse(str: String): ResolvConf = {
ResolvConfParser.parseLines(str.lines)
}
"The ResolvConfParser" should {
"parse an actual Kubernetes resolv.conf file" in {
val resolvConf = parse(
"""nameserver 172.30.0.2
|search myproject.svc.cluster.local svc.cluster.local cluster.local
|options ndots:5""".stripMargin)
resolvConf.search should be(List("myproject.svc.cluster.local", "svc.cluster.local", "cluster.local"))
resolvConf.ndots should be(5)
}
"ignore # comments" in {
parse(
"""search example.com
|#search foobar.com""".stripMargin).search should be(List("example.com"))
}
"ignore ; comments" in {
parse(
"""search example.com
|;search foobar.com""".stripMargin).search should be(List("example.com"))
}
"use the last search element found" in {
parse(
"""search example.com
|search foobar.com""".stripMargin).search should be(List("foobar.com"))
}
"support domain elements" in {
parse("domain example.com").search should be(List("example.com"))
}
"use the last domain element found" in {
parse(
"""domain example.com
|domain foobar.com
""".stripMargin).search should be(List("foobar.com"))
}
"ignore non ndots options" in {
parse("options\trotate\tinet6\tndots:3\tattempts:4").ndots should be(3)
}
"ignore tabs and spaces" in {
parse(" \t \n \t domain \t \texample.com \t \t\n\t\t ").search should be(List("example.com"))
}
"default to ndots 1" in {
parse("").ndots should be(1)
}
}
}

View file

@ -0,0 +1,7 @@
# Changed caching of async DNS lookups to store them directly against the DnsProtocol.RequestType #25987
# Internal API
ProblemFilters.exclude[MissingClassProblem]("akka.io.dns.internal.AsyncDnsResolver$QueryType")
ProblemFilters.exclude[MissingClassProblem]("akka.io.dns.internal.AsyncDnsResolver$SrvType$")
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.io.dns.internal.AsyncDnsCache.put")
ProblemFilters.exclude[MissingClassProblem]("akka.io.dns.internal.AsyncDnsResolver$Ipv6Type$")
ProblemFilters.exclude[MissingClassProblem]("akka.io.dns.internal.AsyncDnsResolver$Ipv4Type$")

View file

@ -1061,6 +1061,19 @@ akka {
# How often to sweep out expired cache entries.
# Note that this interval has nothing to do with TTLs
cache-cleanup-interval = 120s
# Configures the list of search domains.
# Defaults to a system dependent lookup (on Unix like OSes, will attempt to parse /etc/resolv.conf, on
# other platforms, will not make any attempt to lookup the search domains). Set to a single domain, or
# a list of domains, eg, [ "example.com", "example.net" ].
search-domains = default
# Any hosts that have a number of dots less than this will not be looked up directly, instead, a search on
# the search domains will be tried first. This corresponds to the ndots option in /etc/resolv.conf, see
# https://linux.die.net/man/5/resolver for more info.
# Defaults to a system dependent lookup (on Unix like OSes, will attempt to parse /etc/resolv.conf, on
# other platforms, will default to 1).
ndots = default
}
}
}

View file

@ -4,18 +4,21 @@
package akka.io.dns
import java.io.File
import java.net.{ InetSocketAddress, URI }
import java.util
import akka.actor.ExtendedActorSystem
import akka.annotation.InternalApi
import akka.io.dns.internal.{ ResolvConf, ResolvConfParser }
import akka.util.Helpers
import akka.util.JavaDurationConverters._
import com.typesafe.config.{ Config, ConfigValueType }
import scala.collection.JavaConverters._
import scala.collection.breakOut
import scala.concurrent.duration.FiniteDuration
import scala.util.Try
import scala.util.{ Failure, Success, Try }
/** INTERNAL API */
@InternalApi
@ -44,6 +47,54 @@ private[dns] final class DnsSettings(system: ExtendedActorSystem, c: Config) {
val ResolveTimeout: FiniteDuration = c.getDuration("resolve-timeout").asScala
private lazy val resolvConf: Option[ResolvConf] = {
val etcResolvConf = new File("/etc/resolv.conf")
// Avoid doing the check on Windows, no point
if (Helpers.isWindows) {
None
} else if (etcResolvConf.exists()) {
val parsed = ResolvConfParser.parseFile(etcResolvConf)
parsed match {
case Success(value) Some(value)
case Failure(exception)
if (system.log.isWarningEnabled) {
system.log.error(exception, "Error parsing /etc/resolv.conf, ignoring.")
}
None
}
} else None
}
val SearchDomains: List[String] = {
c.getValue("search-domains").valueType() match {
case ConfigValueType.STRING
c.getString("search-domains") match {
case "default" resolvConf.map(_.search).getOrElse(Nil)
case single List(single)
}
case ConfigValueType.LIST
c.getStringList("search-domains").asScala.toList
case _ throw new IllegalArgumentException("Invalid type for search-domains. Must be a string or string list.")
}
}
val NDots: Int = {
c.getValue("ndots").valueType() match {
case ConfigValueType.STRING
c.getString("ndots") match {
case "default" resolvConf.map(_.ndots).getOrElse(1)
case _ throw new IllegalArgumentException("Invalid value for ndots. Must be the string 'default' or an integer.")
}
case ConfigValueType.NUMBER
val ndots = c.getInt("ndots")
if (ndots < 0) {
throw new IllegalArgumentException("Invalid value for ndots, ndots must not be negative.")
}
ndots
case _ throw new IllegalArgumentException("Invalid value for ndots. Must be the string 'default' or an integer.")
}
}
// -------------------------
def failUnableToDetermineDefaultNameservers =

View file

@ -10,8 +10,7 @@ import akka.annotation.InternalApi
import akka.io.{ Dns, PeriodicCacheCleanup }
import akka.io.dns.CachePolicy.CachePolicy
import akka.io.SimpleDnsCache._
import akka.io.dns.internal.AsyncDnsResolver.{ Ipv4Type, Ipv6Type, QueryType }
import akka.io.dns.internal.DnsClient.Answer
import akka.io.dns.DnsProtocol.{ Ip, RequestType, Resolved }
import akka.io.dns.{ AAAARecord, ARecord }
import scala.annotation.tailrec
@ -21,7 +20,7 @@ import scala.collection.immutable
* Internal API
*/
@InternalApi class AsyncDnsCache extends Dns with PeriodicCacheCleanup {
private val cacheRef = new AtomicReference(new Cache[(String, QueryType), Answer](
private val cacheRef = new AtomicReference(new Cache[(String, RequestType), Resolved](
immutable.SortedSet()(expiryEntryOrdering()),
immutable.Map(), clock))
@ -32,15 +31,16 @@ import scala.collection.immutable
* To get Srv or just one type use DnsProtocol
*/
override def cached(name: String): Option[Dns.Resolved] = {
for {
ipv4 cacheRef.get().get((name, Ipv4Type))
ipv6 cacheRef.get().get((name, Ipv6Type))
} yield {
Dns.Resolved(name, (ipv4.rrs ++ ipv6.rrs).collect {
case r: ARecord r.ip
case r: AAAARecord r.ip
})
val ipv4 = cacheRef.get().get((name, Ip(ipv6 = false))).toList.flatMap(_.records)
val ipv6 = cacheRef.get().get((name, Ip(ipv4 = false))).toList.flatMap(_.records)
val both = cacheRef.get().get((name, Ip())).toList.flatMap(_.records)
val all = (ipv4 ++ ipv6 ++ both).collect {
case r: ARecord r.ip
case r: AAAARecord r.ip
}
if (all.isEmpty) None
else Some(Dns.Resolved(name, all))
}
// Milliseconds since start
@ -50,14 +50,14 @@ import scala.collection.immutable
else (now - nanoBase) / 1000000
}
private[io] final def get(key: (String, QueryType)): Option[Answer] = {
private[io] final def get(key: (String, RequestType)): Option[Resolved] = {
cacheRef.get().get(key)
}
@tailrec
private[io] final def put(key: (String, QueryType), records: Answer, ttl: CachePolicy): Unit = {
val cache: Cache[(String, QueryType), Answer] = cacheRef.get()
if (!cacheRef.compareAndSet(cache, cache.put(key, records, ttl)))
private[io] final def put(key: (String, RequestType), records: Resolved, ttl: CachePolicy): Unit = {
val c = cacheRef.get()
if (!cacheRef.compareAndSet(c, c.put(key, records, ttl)))
put(key, records, ttl)
}

View file

@ -17,6 +17,7 @@ import akka.util.{ Helpers, Timeout }
import scala.collection.immutable
import scala.concurrent.Future
import scala.concurrent.duration.Duration
import scala.util.Try
import scala.util.control.NonFatal
@ -38,7 +39,7 @@ private[io] final class AsyncDnsResolver(
val nameServers = settings.NameServers
log.debug("Using name servers [{}]", nameServers)
log.debug("Using name servers [{}] and search domains [{}] with ndots={}", nameServers, settings.SearchDomains, settings.NDots)
private var requestId: Short = 0
private def nextId(): Short = {
@ -50,10 +51,22 @@ private[io] final class AsyncDnsResolver(
override def receive: Receive = {
case DnsProtocol.Resolve(name, mode)
resolve(name, mode, resolvers) pipeTo sender()
cache.get((name, mode)) match {
case Some(resolved)
log.debug("{} cached {}", mode, resolved)
sender() ! resolved
case None
resolveWithResolvers(name, mode, resolvers).map { resolved
if (resolved.records.nonEmpty) {
val minTtl = resolved.records.minBy[Duration](_.ttl.value).ttl
cache.put((name, mode), resolved, minTtl)
}
resolved
} pipeTo sender()
}
}
private def resolve(name: String, requestType: RequestType, resolvers: List[ActorRef]): Future[DnsProtocol.Resolved] =
private def resolveWithResolvers(name: String, requestType: RequestType, resolvers: List[ActorRef]): Future[DnsProtocol.Resolved] =
if (isInetAddress(name)) {
Future.fromTry {
Try {
@ -69,10 +82,10 @@ private[io] final class AsyncDnsResolver(
resolvers match {
case Nil
Future.failed(ResolveFailedException(s"Timed out resolving $name with nameservers: $nameServers"))
case head :: tail resolve(name, requestType, head).recoverWith {
case head :: tail resolveWithSearch(name, requestType, head).recoverWith {
case NonFatal(t)
log.error(t, "Resolve failed. Trying next name server")
resolve(name, requestType, tail)
resolveWithResolvers(name, requestType, tail)
}
}
}
@ -85,63 +98,66 @@ private[io] final class AsyncDnsResolver(
result
}
private def resolveWithSearch(name: String, requestType: RequestType, resolver: ActorRef): Future[DnsProtocol.Resolved] = {
if (settings.SearchDomains.nonEmpty) {
val nameWithSearch = settings.SearchDomains.map(sd name + "." + sd)
// ndots is a heuristic used to try and work out whether the name passed in is a fully qualified domain name,
// or a name relative to one of the search names. The idea is to prevent the cost of doing a lookup that is
// obviously not going to resolve. So, if a host has less than ndots dots in it, then we don't try and resolve it,
// instead, we go directly to the search domains, or at least that's what the man page for resolv.conf says. In
// practice, Linux appears to implement something slightly different, if the name being searched contains less
// than ndots dots, then it should be searched last, rather than first. This means if the heuristic wrongly
// identifies a domain as being relative to the search domains, it will still be looked up if it doesn't resolve
// at any of the search domains, albeit with the latency of having to have done all the searches first.
val toResolve = if (name.count(_ == '.') >= settings.NDots) {
name :: nameWithSearch
} else {
nameWithSearch :+ name
}
resolveFirst(toResolve, requestType, resolver)
} else {
resolve(name, requestType, resolver)
}
}
private def resolveFirst(searchNames: List[String], requestType: RequestType, resolver: ActorRef): Future[DnsProtocol.Resolved] = {
searchNames match {
case searchName :: Nil
resolve(searchName, requestType, resolver)
case searchName :: remaining
resolve(searchName, requestType, resolver).flatMap { resolved
if (resolved.records.isEmpty) resolveFirst(remaining, requestType, resolver)
else Future.successful(resolved)
}
}
}
private def resolve(name: String, requestType: RequestType, resolver: ActorRef): Future[DnsProtocol.Resolved] = {
log.debug("Attempting to resolve {} with {}", name, resolver)
val caseFoldedName = Helpers.toRootLowerCase(name)
requestType match {
case Ip(ipv4, ipv6)
val ipv4Recs: Future[Answer] = if (ipv4)
cache.get((name, Ipv4Type)) match {
case Some(r)
log.debug("Ipv4 cached {}", r)
Future.successful(r)
case None
sendQuestion(resolver, Question4(nextId(), caseFoldedName))
}
sendQuestion(resolver, Question4(nextId(), caseFoldedName))
else
Empty
val ipv6Recs = if (ipv6)
cache.get((name, Ipv6Type)) match {
case Some(r)
log.debug("Ipv6 cached {}", r)
Future.successful(r)
case None
sendQuestion(resolver, Question6(nextId(), caseFoldedName))
}
sendQuestion(resolver, Question6(nextId(), caseFoldedName))
else
Empty
ipv4Recs.flatMap(ipv4Records {
// TODO, do we want config to specify a max for this?
if (ipv4Records.rrs.nonEmpty) {
val minTtl4 = ipv4Records.rrs.map(_.ttl).min
cache.put((name, Ipv4Type), ipv4Records, minTtl4)
}
ipv6Recs.map(ipv6Records {
if (ipv6Records.rrs.nonEmpty) {
val minTtl6 = ipv6Records.rrs.map(_.ttl).min
cache.put((name, Ipv6Type), ipv6Records, minTtl6)
}
ipv4Records.rrs ++ ipv6Records.rrs
}).map(recs DnsProtocol.Resolved(name, recs))
})
for {
ipv4 ipv4Recs
ipv6 ipv6Recs
} yield DnsProtocol.Resolved(name, ipv4.rrs ++ ipv6.rrs, ipv4.additionalRecs ++ ipv6.additionalRecs)
case Srv
cache.get((name, SrvType)) match {
case Some(r)
Future.successful(DnsProtocol.Resolved(name, r.rrs, r.additionalRecs))
case None
sendQuestion(resolver, SrvQuestion(nextId(), caseFoldedName))
.map(answer {
if (answer.rrs.nonEmpty) {
val minTtl = answer.rrs.map(_.ttl).min
cache.put((name, SrvType), answer, minTtl)
}
DnsProtocol.Resolved(name, answer.rrs, answer.additionalRecs)
})
}
sendQuestion(resolver, SrvQuestion(nextId(), caseFoldedName))
.map(answer {
DnsProtocol.Resolved(name, answer.rrs, answer.additionalRecs)
})
}
}
@ -165,10 +181,5 @@ private[io] object AsyncDnsResolver {
private val Empty = Future.successful(Answer(-1, immutable.Seq.empty[ResourceRecord], immutable.Seq.empty[ResourceRecord]))
sealed trait QueryType
final case object Ipv4Type extends QueryType
final case object Ipv6Type extends QueryType
final case object SrvType extends QueryType
case class ResolveFailedException(msg: String) extends Exception(msg)
}

View file

@ -0,0 +1,66 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.io.dns.internal
import java.io.File
import java.nio.file.Files
import scala.collection.JavaConverters._
import scala.util.Try
private[dns] case class ResolvConf(search: List[String], ndots: Int)
private[dns] object ResolvConfParser {
private val DomainLabel = "domain"
private val SearchLabel = "search"
private val OptionsLabel = "options"
private val NdotsOption = "ndots:"
/**
* Does a partial parse according to https://linux.die.net/man/5/resolver.
*/
def parseFile(file: File): Try[ResolvConf] = {
Try {
parseLines(Files.lines(file.toPath).iterator().asScala)
}
}
def parseLines(lines: Iterator[String]): ResolvConf = {
// A few notes - according to the spec, search and domain are mutually exclusive, the domain is used as the
// sole search domain if specified. Also, if multiple of either are specified, then last one wins, so as we
// parse the file, as we encounter either a domain or search element, we replace this list with what we find.
var search = List.empty[String]
var ndots = 1
lines.map(_.trim)
.filter { line
// Ignore blank lines and comments
line.nonEmpty && line(0) != ';' && line(0) != '#'
}
.foreach { line
val (label, args) = line.span(!_.isWhitespace)
def trimmedArgs = args.trim
label match {
case `DomainLabel`
search = List(trimmedArgs)
case `SearchLabel`
search = trimmedArgs.split("\\s+").toList
case `OptionsLabel`
args.split("\\s+").foreach { option
// We're only interested in ndots
if (option.startsWith(NdotsOption)) {
// Allow exception to fall through to Try
ndots = option.drop(NdotsOption.length).toInt
}
}
case _ // Ignore everything else
}
}
ResolvConf(search, ndots)
}
}

View file

@ -5,9 +5,13 @@
`async-dns` does not support:
* [Local hosts file](https://github.com/akka/akka/issues/25846) e.g. `/etc/hosts` on Unix systems
* [Search domains](https://github.com/akka/akka/issues/25825) e.g. in `/etc/resolve.conf` on Unix systems
* The [nsswitch.conf](https://linux.die.net/man/5/nsswitch.conf) file (no plan to support)
Additionally, while search domains are supported through configuration, detection of the system configured
[Search domains](https://github.com/akka/akka/issues/25825) is only supported on systems that provide this
configuration through a `/etc/resolv.conf` file, i.e. it isn't supported on Windows or OSX, and none of the
environment variables that are usually supported on most \*nix OSes are supported.
@@@
@@@ note