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:
parent
81b499cc7b
commit
762c9cb019
11 changed files with 400 additions and 71 deletions
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -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$")
|
||||
|
|
@ -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
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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 =
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue