This commit is contained in:
Konrad `ktoso` Malawski 2018-06-04 23:30:35 +09:00 committed by Christopher Batey
parent 975d2d8371
commit d623593bc2
14 changed files with 716 additions and 2 deletions

View file

@ -1010,8 +1010,13 @@ akka {
dispatcher = "akka.actor.default-dispatcher" dispatcher = "akka.actor.default-dispatcher"
# Name of the subconfig at path akka.io.dns, see inet-address below # Name of the subconfig at path akka.io.dns, see inet-address below
#
# Change to `async-dns` to use the new akka "native" DNS resolver,
# which is also capable of resolving SRV records.
resolver = "inet-address" resolver = "inet-address"
# To-be-deprecated DNS resolver implementation which uses the Java InetAddress to resolve DNS records.
# To be replaced by `akka.io.dns.async` which implements the DNS protocol natively and without blocking (which InetAddress does)
inet-address { inet-address {
# Must implement akka.io.DnsProvider # Must implement akka.io.DnsProvider
provider-object = "akka.io.InetAddressDnsProvider" provider-object = "akka.io.InetAddressDnsProvider"
@ -1029,6 +1034,20 @@ akka {
# Note that this interval has nothing to do with TTLs # Note that this interval has nothing to do with TTLs
cache-cleanup-interval = 120s cache-cleanup-interval = 120s
} }
async {
# Must implement akka.io.DnsProvider
provider-object = "akka.io.dns.AsyncDnsProvider"
# Configures nameservers to query during DNS resolution.
#
# Defaults to the nameservers that would be used by the JVM by default.
# Set to a list of IPs to override the servers, e.g. [ "8.8.8.8", "8.8.4.4" ] for Google's servers
nameservers = default
# use /etc/resolve.conf on systems where it is available during DNS lookups
resolv-conf = on
}
} }
} }

View file

@ -51,10 +51,12 @@ object Dns extends ExtensionId[DnsExt] with ExtensionIdProvider {
} }
} }
// TODO tempted to deprecate this one?
def cached(name: String)(system: ActorSystem): Option[Resolved] = { def cached(name: String)(system: ActorSystem): Option[Resolved] = {
Dns(system).cache.cached(name) Dns(system).cache.cached(name)
} }
// TODO tempted to deprecate this one?
def resolve(name: String)(system: ActorSystem, sender: ActorRef): Option[Resolved] = { def resolve(name: String)(system: ActorSystem, sender: ActorRef): Option[Resolved] = {
Dns(system).cache.resolve(name)(system, sender) Dns(system).cache.resolve(name)(system, sender)
} }
@ -73,7 +75,6 @@ class DnsExt(system: ExtendedActorSystem) extends IO.Extension {
val Settings = new Settings(system.settings.config.getConfig("akka.io.dns")) val Settings = new Settings(system.settings.config.getConfig("akka.io.dns"))
class Settings private[DnsExt] (_config: Config) { class Settings private[DnsExt] (_config: Config) {
import _config._ import _config._
val Dispatcher: String = getString("dispatcher") val Dispatcher: String = getString("dispatcher")

View file

@ -6,8 +6,9 @@ package akka.io
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import akka.actor.{ ActorLogging, Actor, Deploy, Props } import akka.actor.{ Actor, ActorLogging, Deploy, Props }
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics } import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
import akka.event.Logging
import akka.routing.FromConfig import akka.routing.FromConfig
import scala.concurrent.duration.Duration import scala.concurrent.duration.Duration
@ -34,6 +35,16 @@ class SimpleDnsManager(val ext: DnsExt) extends Actor with RequiresMessageQueue[
case SimpleDnsManager.CacheCleanup case SimpleDnsManager.CacheCleanup
for (c cacheCleanup) for (c cacheCleanup)
c.cleanup() c.cleanup()
case m: dns.DnsProtocol.Protocol
val legacyDnsResolver = classOf[InetAddressDnsResolver]
if (ext.provider.actorClass == legacyDnsResolver) {
// FIXME technically we COULD adopt the protocol here, but not sure we should...
log.warning(
"Message of [akka.io.dns.DnsProtocol.Protocol] received ({}), while the legacy {} was configured! Dropping DNS resolve request." +
"Please use [akka.io.dns.DnsProtocol.resolve] to create resolution requests for the Async DNS resolver.",
Logging.simpleName(m), Logging.simpleName(legacyDnsResolver))
} else resolver.forward(m)
} }
override def postStop(): Unit = { override def postStop(): Unit = {

View file

@ -0,0 +1,69 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.io.dns
import java.util.concurrent.TimeUnit
import akka.actor.{ Actor, ActorLogging, Deploy, Props }
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
import akka.io.{ Dns, DnsExt, PeriodicCacheCleanup }
import akka.routing.FromConfig
import scala.concurrent.duration.Duration
final class AsyncDnsManager(val ext: DnsExt) extends Actor with RequiresMessageQueue[UnboundedMessageQueueSemantics] with ActorLogging {
import AsyncDnsManager._
implicit val ec = context.dispatcher
private var oldProtocolWarningLoggedTimes = 0
private val resolver = {
val props: Props = FromConfig.props(Props(ext.provider.actorClass, ext.cache, ext.Settings.ResolverConfig).withDeploy(Deploy.local).withDispatcher(ext.Settings.Dispatcher))
context.actorOf(props, ext.Settings.Resolver)
}
private val cacheCleanup = ext.cache match {
case cleanup: PeriodicCacheCleanup Some(cleanup)
case _ None
}
private val cleanupTimer = cacheCleanup map { _
val interval = Duration(ext.Settings.ResolverConfig.getDuration("cache-cleanup-interval", TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS)
context.system.scheduler.schedule(interval, interval, self, AsyncDnsManager.CacheCleanup)
}
override def receive = {
case r: DnsProtocol.Resolve
log.debug("Resolution request for {} {}from {}", r.name, r.mode, sender())
resolver.forward(r)
case r @ Dns.Resolve(name)
// adapt legacy protocol to new protocol
log.debug("Resolution request for {} from {}", name, sender())
warnAboutOldProtocolUse(name)
val adapted = DnsProtocol.resolve(name)
resolver.forward(adapted)
case CacheCleanup
for (c cacheCleanup)
c.cleanup()
}
private def warnAboutOldProtocolUse(name: String): Unit = {
val warnAtMostTimes = 10
if (oldProtocolWarningLoggedTimes < warnAtMostTimes) {
oldProtocolWarningLoggedTimes += 1
log.warning("Received Dns.Resolve({}) message while Async DNS resolver active. Please use the new API [akka.io.dns.DnsProtocol] to issue resolve requests. " +
"(This warning will be logged at most {} times)", name, warnAtMostTimes)
}
}
override def postStop(): Unit = {
for (t cleanupTimer) t.cancel()
}
}
object AsyncDnsManager {
private case object CacheCleanup
}

View file

@ -0,0 +1,13 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.io.dns
import akka.io._
class AsyncDnsProvider extends DnsProvider {
override def cache: Dns = new SimpleDnsCache()
override def actorClass = classOf[InetAddressDnsResolver]
override def managerClass = classOf[SimpleDnsManager]
}

View file

@ -0,0 +1,37 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.io.dns
import akka.actor.{ Actor, ActorLogging }
import akka.io.SimpleDnsCache
import akka.io.dns.protocol.DnsRecordType
import com.typesafe.config.Config
import akka.pattern.pipe
import scala.concurrent.Future
final class AsyncDnsResolver(cache: SimpleDnsCache, config: Config) extends Actor with ActorLogging {
implicit val ec = context.dispatcher
override def receive: Receive = {
case m: DnsProtocol.Protocol onReceive(m)
}
def onReceive(m: DnsProtocol.Protocol): Unit = m match {
case DnsProtocol.Resolve(name, mode)
resolve(name, mode) pipeTo sender()
}
def resolve(str: String, types: Set[DnsRecordType]): Future[DnsProtocol.Resolved] = {
???
}
}
object AsyncDnsResolver {
/** SRV lookups start with `_` so we use this heurestic to issue an SRV lookup TODO DO WE REALLY */
private[dns] def isLikelySrvLookup(name: String): Boolean =
name.startsWith("_")
}

View file

@ -0,0 +1,40 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.io.dns
import akka.io.dns.protocol.{ DnsRecordType, DnsResourceRecord }
import scala.collection.immutable
/**
* Supersedes [[akka.io.Dns]] protocol.
*
* Note that one MUST configure `akka.io.dns.resolver = async` to make use of this protocol and resolver.
*
* Allows for more detailed lookups, by specifying which records should be checked,
* and responses can more information than plain IP addresses (e.g. ports for SRV records).
*/
object DnsProtocol {
def resolve(name: String): Resolve =
Resolve(name, NormalLookupRecordTypes)
def resolve(name: String, recordTypes: Set[DnsRecordType]): Resolve =
Resolve(name, recordTypes)
sealed trait Protocol
private[akka] final case class Resolve(name: String, mode: Set[DnsRecordType]) extends Protocol
final case class Resolved(name: String, results: immutable.Seq[DnsResourceRecord]) extends Protocol
import DnsRecordType._
/** The default set of record types most applications are interested in: A, AAAA and CNAME */
final val NormalLookupRecordTypes = Set(A, AAAA, CNAME)
/** Request lookups of `SRV` records */
final val ServiceRecordTypes = Set(SRV)
}

View file

@ -0,0 +1,162 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
* Adopted from Apache v2 licensed: https://github.com/ilya-epifanov/akka-dns
*/
package akka.io.dns
import java.io.File
import java.net.{ InetSocketAddress, URI }
import java.nio.file.Paths
import java.util.concurrent.TimeUnit
import com.typesafe.config.{ Config, ConfigValueType }
import scala.collection.JavaConverters._
import scala.collection.{ breakOut, immutable }
import scala.concurrent.duration.FiniteDuration
import scala.io.Source
import scala.util.{ Failure, Success, Try }
import java.lang.reflect.Method
import java.security.{ AccessController, PrivilegedExceptionAction }
import java.util
import scala.util.control.NonFatal
/** INTERNAL API */
private[dns] final class DnsSettings(c: Config) {
import DnsSettings._
val systemNameServers =
if (c.getBoolean("resolv-conf"))
parseSystemNameServers(Paths.get("/etc/resolv.conf").toFile)
else
Option.empty[immutable.Seq[InetSocketAddress]]
val NameServers: immutable.Seq[InetSocketAddress] = {
val addrs = Try(c.getString("nameservers")).toOption.toList
.flatMap {
case "default" getDefaultSearchDomains().getOrElse(failUnableToDetermineDefaultNameservers)
case address parseNameserverAddress(address) :: Nil
}
if (addrs.nonEmpty) addrs
else c.getStringList("nameservers").asScala.map(parseNameserverAddress)(breakOut)
}
val NegativeTtl: Long = c.getDuration("negative-ttl", TimeUnit.MILLISECONDS)
val MinPositiveTtl: Long = c.getDuration("min-positive-ttl", TimeUnit.MILLISECONDS)
val MaxPositiveTtl: Long = c.getDuration("max-positive-ttl", TimeUnit.MILLISECONDS)
val ResolveTimeout: FiniteDuration = FiniteDuration(c.getDuration("request-ttl", TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS)
// -------------------------
private val inetSocketAddress = """(.*?)(?::(\d+))?""".r
def parseNameserverAddress(str: String): InetSocketAddress = {
val inetSocketAddress(host, port) = str
new InetSocketAddress(host, Option(port).fold(53)(_.toInt))
}
// TODO replace with actual parser, regex is very likely not efficient...
private val ipv4Address = """^[0-9]{1,3}\.[0-9]{1,3}\.[0-9]{1,3}\.[0-9]{1,3}$""".r
private val ipv6Address = """^\s*((([0-9A-Fa-f]{1,4}:){7}([0-9A-Fa-f]{1,4}|:))|(([0-9A-Fa-f]{1,4}:){6}(:[0-9A-Fa-f]{1,4}|((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3})|:))|(([0-9A-Fa-f]{1,4}:){5}(((:[0-9A-Fa-f]{1,4}){1,2})|:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3})|:))|(([0-9A-Fa-f]{1,4}:){4}(((:[0-9A-Fa-f]{1,4}){1,3})|((:[0-9A-Fa-f]{1,4})?:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){3}(((:[0-9A-Fa-f]{1,4}){1,4})|((:[0-9A-Fa-f]{1,4}){0,2}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){2}(((:[0-9A-Fa-f]{1,4}){1,5})|((:[0-9A-Fa-f]{1,4}){0,3}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){1}(((:[0-9A-Fa-f]{1,4}){1,6})|((:[0-9A-Fa-f]{1,4}){0,4}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:))|(:(((:[0-9A-Fa-f]{1,4}){1,7})|((:[0-9A-Fa-f]{1,4}){0,5}:((25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)(\.(25[0-5]|2[0-4]\d|1\d\d|[1-9]?\d)){3}))|:)))(%.+)?\s*$""".r
private def isInetAddress(name: String): Boolean =
ipv4Address.findAllMatchIn(name).nonEmpty || ipv6Address.findAllMatchIn(name).nonEmpty
// Note that the corresponding man page doesn't actually dictate the format of this field,
// just the keywords and their meanings. See http://man7.org/linux/man-pages/man5/resolv.conf.5.html
private[io] val NameserverLine = """^\s*nameserver\s+(.*)$""".r
// OS specific. No encoding or charset is specified by the man page as I recall.
// See http://man7.org/linux/man-pages/man5/resolv.conf.5.html.
def parseSystemNameServers(resolvConf: File): Option[immutable.Seq[InetSocketAddress]] =
try {
val addresses =
for {
line Source.fromFile(resolvConf).getLines()
addr NameserverLine.findFirstMatchIn(line).map(_.group(1))
} yield parseNameserverAddress(addr)
Some(addresses.toList)
} catch {
case NonFatal(_) Option.empty
}
def failUnableToDetermineDefaultNameservers =
throw new IllegalStateException("Unable to obtain default nameservers from JNDI or via reflection. " +
"Please set `akka.io.dns.async.nameservers` explicitly in order to be able to resolve domain names.")
}
object DnsSettings {
private final val DnsFallbackPort = 53
/**
* INTERNAL API
* Find out the default search domains that Java would use normally, e.g. when using InetAddress to resolve domains.
*
* The default nameservers are attempted to be obtained from: jndi-dns and from `sun.net.dnsResolverConfiguration`
* as a fallback (which is expected to fail though when running on JDK9+ due to the module encapsulation of sun packages).
*
* Based on: https://github.com/netty/netty/blob/4.1/resolver-dns/src/main/java/io/netty/resolver/dns/DefaultDnsServerAddressStreamProvider.java#L58-L146
*/
private[akka] def getDefaultSearchDomains(): Try[List[InetSocketAddress]] = {
def asInetSocketAddress(server: String): Try[InetSocketAddress] = {
Try {
val uri = new URI(server)
val host = uri.getHost
val port = uri.getPort match {
case -1 DnsFallbackPort
case selected selected
}
new InetSocketAddress(host, port)
}
}
def getNameserversUsingJNDI: Try[List[InetSocketAddress]] = {
import javax.naming.Context
import javax.naming.NamingException
import javax.naming.directory.InitialDirContext
import java.net.URI
import java.util
// Using jndi-dns to obtain the default name servers.
//
// See:
// - http://docs.oracle.com/javase/8/docs/technotes/guides/jndi/jndi-dns.html
// - http://mail.openjdk.java.net/pipermail/net-dev/2017-March/010695.html
val env = new util.Hashtable[String, String]
env.put(Context.INITIAL_CONTEXT_FACTORY, "com.sun.jndi.dns.DnsContextFactory")
env.put("java.naming.provider.url", "dns://")
Try {
val ctx = new InitialDirContext(env)
val dnsUrls = ctx.getEnvironment.get("java.naming.provider.url").asInstanceOf[String]
// Only try if not empty as otherwise we will produce an exception
if (dnsUrls != null && !dnsUrls.isEmpty) {
val servers = dnsUrls.split(" ")
servers.flatMap { server asInetSocketAddress(server).toOption }.toList
} else Nil
}
}
// this method is used as a fallback in case JNDI results in an empty list
// this method will not work when running modularised of course since it needs access to internal sun classes
def getNameserversUsingReflection: Try[List[InetSocketAddress]] = {
Try {
val configClass = Class.forName("sun.net.dns.ResolverConfiguration")
val open = configClass.getMethod("open")
val nameservers = configClass.getMethod("nameservers")
val instance = open.invoke(null)
val ns = nameservers.invoke(instance).asInstanceOf[util.List[String]]
val res = if (ns.isEmpty) throw new IllegalStateException("Empty nameservers list discovered using reflection. Consider configuring default nameservers manually!")
else ns.asScala.toList
res.flatMap(s asInetSocketAddress(s).toOption)
}
}
getNameserversUsingJNDI orElse getNameserversUsingReflection
}
}

View file

@ -0,0 +1,53 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
* Adopted from Apache v2 licensed: https://github.com/ilya-epifanov/akka-dns
*/
package akka.io.dns.protocol
import akka.util.{ ByteIterator, ByteString, ByteStringBuilder }
object DnsDomainName {
def length(name: String): Short = {
(name.length + 2).toShort
}
def write(it: ByteStringBuilder, name: String) {
for (label name.split('.')) {
it.putByte(label.length.toByte)
for (c label) {
it.putByte(c.toByte)
}
}
it.putByte(0)
}
def parse(it: ByteIterator, msg: ByteString): String = {
val ret = StringBuilder.newBuilder
// ret.sizeHint(getNameLength(it.clone(), 0))
// println("Parsing name")
while (true) {
val length = it.getByte
// println(s"Label length: $length")
if (length == 0) {
val r = ret.result()
// println(s"Name: $r")
return r
}
if (ret.nonEmpty)
ret.append('.')
if ((length & 0xc0) == 0xc0) {
val offset = ((length.toShort & 0x3f) << 8) | (it.getByte.toShort & 0x00ff)
// println(s"Computed offset: $offset")
return ret.result() + parse(msg.iterator.drop(offset), msg)
}
ret.appendAll(it.clone().take(length).map(_.toChar))
it.drop(length)
}
??? // FIXME!!!!!!!!!!
}
}

View file

@ -0,0 +1,26 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
* Adopted from Apache v2 licensed: https://github.com/ilya-epifanov/akka-dns
*/
package akka.io.dns.protocol
import akka.util.{ ByteIterator, ByteString, ByteStringBuilder }
/** INTERNAL API */
final case class DnsQuestion(name: String, qType: DnsRecordType, qClass: DnsRecordClass) {
def write(out: ByteStringBuilder) {
DnsDomainName.write(out, name)
DnsRecordType.write(out, qType)
DnsRecordClass.write(out, qClass)
}
}
object DnsQuestion {
def parse(it: ByteIterator, msg: ByteString): DnsQuestion = {
val name = DnsDomainName.parse(it, msg)
val qType = DnsRecordType.parse(it)
val qClass = DnsRecordClass.parse(it)
DnsQuestion(name, qType, qClass)
}
}

View file

@ -0,0 +1,34 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
* Adopted from Apache v2 licensed: https://github.com/ilya-epifanov/akka-dns
*/
package akka.io.dns.protocol
import akka.util.{ ByteIterator, ByteStringBuilder }
case class DnsRecordClass(code: Short, name: String)
object DnsRecordClass {
val IN = DnsRecordClass(1, "IN")
val CS = DnsRecordClass(2, "CS")
val CH = DnsRecordClass(3, "CH")
val HS = DnsRecordClass(4, "HS")
val WILDCARD = DnsRecordClass(255, "WILDCARD")
def parse(it: ByteIterator): DnsRecordClass = {
it.getShort match {
case 1 IN
case 2 CS
case 3 CH
case 255 WILDCARD
case _ ??? // FIXME
}
}
def write(out: ByteStringBuilder, c: DnsRecordClass): Unit = {
out.putShort(c.code)
}
}

View file

@ -0,0 +1,96 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.io.dns.protocol
import akka.annotation.InternalApi
import akka.util.{ ByteIterator, ByteStringBuilder, OptionVal }
/**
* DNS Record Type
*/
final case class DnsRecordType(code: Short, name: String)
object DnsRecordType {
// array for fast lookups by id
// wasteful, but we get trivial indexing into it for lookup
private final val lookupTable = Array.ofDim[DnsRecordType](256)
/** INTERNAL API: TODO move it somewhere off the datatype */
// TODO other type than ByteStringBuilder? (was used in akka-dns)
def write(out: ByteStringBuilder, value: DnsRecordType): Unit = {
out.putShort(value.code)
}
/** INTERNAL API: TODO move it somewhere off the datatype */
def parse(it: ByteIterator): DnsRecordType = {
val id = it.getShort
apply(id) match {
case OptionVal.None throw new IllegalArgumentException(s"Illegal id [${id}] for DnsRecordType")
case OptionVal.Some(t) t
}
}
private def register(t: DnsRecordType) = {
lookupTable(t.code) = t
t
}
def apply(id: Short): OptionVal[DnsRecordType] = {
if (id < 1 || id > 255) OptionVal.None
else OptionVal(lookupTable(id))
}
/** A host address */
final val A = register(new DnsRecordType(1, "A"))
/** An authoritative name server */
final val NS = register(new DnsRecordType(2, "NS"))
/** A mail destination (Obsolete - use MX) */
final val MD = register(new DnsRecordType(3, "MD"))
/** A mail forwarder (Obsolete - use MX) */
final val MF = register(new DnsRecordType(4, "MF"))
/** the canonical name for an alias */
final val CNAME = register(new DnsRecordType(5, "CNAME"))
/** marks the start of a zone of authority */
final val SOA = register(new DnsRecordType(6, "SOA"))
/** A mailbox domain name (EXPERIMENTAL) */
final val MB = register(new DnsRecordType(7, "MB"))
/** A mail group member (EXPERIMENTAL) */
final val MG = register(new DnsRecordType(8, "MG"))
/** A mail rename domain name (EXPERIMENTAL) */
final val MR = register(new DnsRecordType(9, "MR"))
/** A null RR (EXPERIMENTAL) */
final val NULL = register(new DnsRecordType(10, "NULL"))
/** A well known service description */
final val WKS = register(new DnsRecordType(11, "WKS"))
/** A domain name pointer */
final val PTR = register(new DnsRecordType(12, "PTR"))
/** host information */
final val HINFO = register(new DnsRecordType(13, "HINFO"))
/** mailbox or mail list information */
final val MINFO = register(new DnsRecordType(14, "MINFO"))
/** mail exchange */
final val MX = register(new DnsRecordType(15, "MX"))
/** text strings */
final val TXT = register(new DnsRecordType(16, "TXT"))
/** The AAAA resource record type is a record specific to the Internet class that stores a single IPv6 address. */
// See: https://tools.ietf.org/html/rfc3596
final val AAAA = register(new DnsRecordType(28, "AAAA"))
/**
* The SRV RR allows administrators to use several servers for a single
* domain, to move services from host to host with little fuss, and to
* designate some hosts as primary servers for a service and others as
* backups.
*/
// See: https://tools.ietf.org/html/rfc2782
final val SRV = register(new DnsRecordType(33, "SRV"))
final val AXFR = register(new DnsRecordType(252, "AXFR"))
final val MAILB = register(new DnsRecordType(253, "MAILB"))
final val MAILA = register(new DnsRecordType(254, "MAILA"))
final val WILDCARD = register(new DnsRecordType(255, "WILDCARD"))
}

View file

@ -0,0 +1,139 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
* Adopted from Apache v2 licensed: https://github.com/ilya-epifanov/akka-dns
*/
package akka.io.dns.protocol
import java.net.{ Inet4Address, Inet6Address, InetAddress }
import akka.annotation.InternalApi
import akka.util.{ ByteIterator, ByteString, ByteStringBuilder }
import scala.annotation.switch
@InternalApi
sealed abstract class DnsResourceRecord(val name: String, val ttl: Int, val recType: Short, val recClass: Short) {
def write(it: ByteStringBuilder): Unit = {
DnsDomainName.write(it, name)
it.putShort(recType)
it.putShort(recClass)
}
}
@InternalApi
final case class ARecord(override val name: String, override val ttl: Int,
ip: Inet4Address) extends DnsResourceRecord(name, ttl, DnsRecordType.A.code, DnsRecordClass.IN.code) {
override def write(it: ByteStringBuilder): Unit = {
super.write(it)
val addr = ip.getAddress
it.putShort(addr.length)
it.putBytes(addr)
}
}
@InternalApi
object ARecord {
def parseBody(name: String, ttl: Int, length: Short, it: ByteIterator): ARecord = {
val addr = Array.ofDim[Byte](4)
it.getBytes(addr)
ARecord(name, ttl, InetAddress.getByAddress(addr).asInstanceOf[Inet4Address])
}
}
@InternalApi
final case class AAAARecord(override val name: String, override val ttl: Int,
ip: Inet6Address) extends DnsResourceRecord(name, ttl, DnsRecordType.AAAA.code, DnsRecordClass.IN.code) {
override def write(it: ByteStringBuilder): Unit = {
super.write(it)
val addr = ip.getAddress
it.putShort(addr.length)
it.putBytes(addr)
}
}
@InternalApi
object AAAARecord {
def parseBody(name: String, ttl: Int, length: Short, it: ByteIterator): AAAARecord = {
val addr = Array.ofDim[Byte](16)
it.getBytes(addr)
AAAARecord(name, ttl, InetAddress.getByAddress(addr).asInstanceOf[Inet6Address])
}
}
@InternalApi
final case class CNAMERecord(override val name: String, override val ttl: Int,
canonicalName: String) extends DnsResourceRecord(name, ttl, DnsRecordType.CNAME.code, DnsRecordClass.IN.code) {
override def write(it: ByteStringBuilder): Unit = {
super.write(it)
it.putShort(DnsDomainName.length(name))
DnsDomainName.write(it, name)
}
}
@InternalApi
object CNAMERecord {
def parseBody(name: String, ttl: Int, length: Short, it: ByteIterator, msg: ByteString): CNAMERecord = {
CNAMERecord(name, ttl, DnsDomainName.parse(it, msg))
}
}
@InternalApi
final case class SRVRecord(override val name: String, override val ttl: Int,
priority: Int, weight: Int, port: Int, target: String) extends DnsResourceRecord(name, ttl, DnsRecordType.SRV.code, DnsRecordClass.IN.code) {
override def write(it: ByteStringBuilder): Unit = {
super.write(it)
it.putShort(priority)
it.putShort(weight)
it.putShort(port)
DnsDomainName.write(it, target)
}
}
@InternalApi
object SRVRecord {
def parseBody(name: String, ttl: Int, length: Short, it: ByteIterator, msg: ByteString): SRVRecord = {
val priority = it.getShort
val weight = it.getShort
val port = it.getShort
SRVRecord(name, ttl, priority, weight, port, DnsDomainName.parse(it, msg))
}
}
@InternalApi
final case class UnknownRecord(override val name: String, override val ttl: Int,
override val recType: Short, override val recClass: Short,
data: ByteString) extends DnsResourceRecord(name, ttl, recType, recClass) {
override def write(it: ByteStringBuilder): Unit = {
super.write(it)
it.putShort(data.length)
it.append(data)
}
}
@InternalApi
object UnknownRecord {
def parseBody(name: String, ttl: Int, recType: Short, recClass: Short, length: Short, it: ByteIterator): UnknownRecord =
UnknownRecord(name, ttl, recType, recClass, it.toByteString)
}
@InternalApi
object DnsResourceRecord {
def parse(it: ByteIterator, msg: ByteString): DnsResourceRecord = {
val name = DnsDomainName.parse(it, msg)
val recType = it.getShort
val recClass = it.getShort
val ttl = it.getInt
val rdLength = it.getShort
val data = it.clone().take(rdLength)
it.drop(rdLength)
(recType: @switch) match {
case 1 ARecord.parseBody(name, ttl, rdLength, data)
case 5 CNAMERecord.parseBody(name, ttl, rdLength, data, msg)
case 28 AAAARecord.parseBody(name, ttl, rdLength, data)
case 33 SRVRecord.parseBody(name, ttl, rdLength, data, msg)
case _ UnknownRecord.parseBody(name, ttl, recType, recClass, rdLength, data)
}
}
}

View file

@ -0,0 +1,14 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.io.dns
import java.nio.ByteOrder
package object protocol {
// We know we always want to use network byte order when writing
implicit val networkByteOrder: ByteOrder = ByteOrder.BIG_ENDIAN
}