Introduce DnsCache API + DnsResolver extensibility

Motivation:
Caching is currently nested in DnsResolver.
It should also be possible to extend DnsResolver to ba able to pass a different cache on each resolution attemp.

Modifications:

* Introduce DnsCache, NoopDnsCache and DefaultDnsCache. The latter contains all the current caching logic that was extracted.
* Introduce protected versions of doResolve and doResolveAll that can be used as extension points to build resolvers that bypass the main cache and use a different one on each resolution.

Result:

Isolated caching logic. Better extensibility.
This commit is contained in:
Stephane Landelle 2016-01-07 17:49:15 +01:00 committed by Norman Maurer
parent 07d0bcab8c
commit b4be040f30
8 changed files with 469 additions and 197 deletions

View File

@ -76,4 +76,24 @@ public final class ObjectUtil {
checkPositive(array.length, name + ".length");
return array;
}
/**
* Resolves a possibly null Integer to a primitive int, using a default value.
* @param wrapper the wrapper
* @param defaultValue the default value
* @return the primitive value
*/
public static int intValue(Integer wrapper, int defaultValue) {
return wrapper != null ? wrapper.intValue() : defaultValue;
}
/**
* Resolves a possibly null Long to a primitive long, using a default value.
* @param wrapper the wrapper
* @param defaultValue the default value
* @return the primitive value
*/
public static long longValue(Long wrapper, long defaultValue) {
return wrapper != null ? wrapper.longValue() : defaultValue;
}
}

View File

@ -0,0 +1,222 @@
/*
* Copyright 2016 The Netty Project
*
* The Netty Project licenses this file to you under the Apache License,
* version 2.0 (the "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at:
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations
* under the License.
*/
package io.netty.resolver.dns;
import io.netty.channel.EventLoop;
import io.netty.util.internal.OneTimeTask;
import io.netty.util.internal.PlatformDependent;
import java.net.InetAddress;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
import static io.netty.util.internal.ObjectUtil.checkNotNull;
import static io.netty.util.internal.ObjectUtil.checkPositiveOrZero;
/**
* Default implementation of {@link DnsCache}, backed by a {@link ConcurrentMap}.
*/
public class DefaultDnsCache implements DnsCache {
private final ConcurrentMap<String, List<DnsCacheEntry>> resolveCache = PlatformDependent.newConcurrentHashMap();
private final int minTtl;
private final int maxTtl;
private final int negativeTtl;
/**
* Create a cache that respects the TTL returned by the DNS server
* and doesn't cache negative responses.
*/
public DefaultDnsCache() {
this(0, Integer.MAX_VALUE, 0);
}
/**
* Create a cache.
* @param minTtl the minimum TTL
* @param maxTtl the maximum TTL
* @param negativeTtl the TTL for failed queries
*/
public DefaultDnsCache(int minTtl, int maxTtl, int negativeTtl) {
this.minTtl = checkPositiveOrZero(minTtl, "minTtl");
this.maxTtl = checkPositiveOrZero(maxTtl, "maxTtl");
if (minTtl > maxTtl) {
throw new IllegalArgumentException(
"minTtl: " + minTtl + ", maxTtl: " + maxTtl + " (expected: 0 <= minTtl <= maxTtl)");
}
this.negativeTtl = checkPositiveOrZero(negativeTtl, "negativeTtl");
}
/**
* Returns the minimum TTL of the cached DNS resource records (in seconds).
*
* @see #maxTtl()
*/
public int minTtl() {
return minTtl;
}
/**
* Returns the maximum TTL of the cached DNS resource records (in seconds).
*
* @see #minTtl()
*/
public int maxTtl() {
return maxTtl;
}
/**
* Returns the TTL of the cache for the failed DNS queries (in seconds). The default value is {@code 0}, which
* disables the cache for negative results.
*/
public int negativeTtl() {
return negativeTtl;
}
@Override
public void clear() {
for (Iterator<Map.Entry<String, List<DnsCacheEntry>>> i = resolveCache.entrySet().iterator(); i.hasNext();) {
final Map.Entry<String, List<DnsCacheEntry>> e = i.next();
i.remove();
cancelExpiration(e.getValue());
}
}
@Override
public boolean clear(String hostname) {
checkNotNull(hostname, "hostname");
boolean removed = false;
for (Iterator<Map.Entry<String, List<DnsCacheEntry>>> i = resolveCache.entrySet().iterator(); i.hasNext();) {
final Map.Entry<String, List<DnsCacheEntry>> e = i.next();
if (e.getKey().equals(hostname)) {
i.remove();
cancelExpiration(e.getValue());
removed = true;
}
}
return removed;
}
@Override
public List<DnsCacheEntry> get(String hostname) {
checkNotNull(hostname, "hostname");
return resolveCache.get(hostname);
}
private List<DnsCacheEntry> cachedEntries(String hostname) {
List<DnsCacheEntry> oldEntries = resolveCache.get(hostname);
final List<DnsCacheEntry> entries;
if (oldEntries == null) {
List<DnsCacheEntry> newEntries = new ArrayList<DnsCacheEntry>(8);
oldEntries = resolveCache.putIfAbsent(hostname, newEntries);
entries = oldEntries != null? oldEntries : newEntries;
} else {
entries = oldEntries;
}
return entries;
}
@Override
public void cache(String hostname, InetAddress address, long originalTtl, EventLoop loop) {
if (maxTtl == 0) {
return;
}
checkNotNull(hostname, "hostname");
checkNotNull(address, "address");
checkNotNull(loop, "loop");
final int ttl = Math.max(minTtl, (int) Math.min(maxTtl, originalTtl));
final List<DnsCacheEntry> entries = cachedEntries(hostname);
final DnsCacheEntry e = new DnsCacheEntry(hostname, address);
synchronized (entries) {
if (!entries.isEmpty()) {
final DnsCacheEntry firstEntry = entries.get(0);
if (firstEntry.cause() != null) {
assert entries.size() == 1;
firstEntry.cancelExpiration();
entries.clear();
}
}
entries.add(e);
}
scheduleCacheExpiration(entries, e, ttl, loop);
}
@Override
public void cache(String hostname, Throwable cause, EventLoop loop) {
if (negativeTtl == 0) {
return;
}
checkNotNull(hostname, "hostname");
checkNotNull(cause, "cause");
checkNotNull(loop, "loop");
final List<DnsCacheEntry> entries = cachedEntries(hostname);
final DnsCacheEntry e = new DnsCacheEntry(hostname, cause);
synchronized (entries) {
final int numEntries = entries.size();
for (int i = 0; i < numEntries; i ++) {
entries.get(i).cancelExpiration();
}
entries.clear();
entries.add(e);
}
scheduleCacheExpiration(entries, e, negativeTtl, loop);
}
private static void cancelExpiration(List<DnsCacheEntry> entries) {
final int numEntries = entries.size();
for (int i = 0; i < numEntries; i++) {
entries.get(i).cancelExpiration();
}
}
private void scheduleCacheExpiration(final List<DnsCacheEntry> entries,
final DnsCacheEntry e,
int ttl,
EventLoop loop) {
e.scheduleExpiration(loop, new OneTimeTask() {
@Override
public void run() {
synchronized (entries) {
entries.remove(e);
if (entries.isEmpty()) {
resolveCache.remove(e.hostname());
}
}
}
}, ttl, TimeUnit.SECONDS);
}
@Override
public String toString() {
return new StringBuilder()
.append("DefaultDnsCache(minTtl=")
.append(minTtl).append(", maxTtl=")
.append(maxTtl).append(", negativeTtl=")
.append(negativeTtl).append(", cached resolved hostname=")
.append(resolveCache.size()).append(")")
.toString();
}
}

View File

@ -0,0 +1,68 @@
/*
* Copyright 2016 The Netty Project
*
* The Netty Project licenses this file to you under the Apache License,
* version 2.0 (the "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at:
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations
* under the License.
*/
package io.netty.resolver.dns;
import io.netty.channel.EventLoop;
import java.net.InetAddress;
import java.util.List;
/**
* A cache for DNS resolution entries.
*/
public interface DnsCache {
/**
* Clears all the resolved addresses cached by this resolver.
*
* @return {@code this}
*
* @see #clear(String)
*/
void clear();
/**
* Clears the resolved addresses of the specified host name from the cache of this resolver.
*
* @return {@code true} if and only if there was an entry for the specified host name in the cache and
* it has been removed by this method
*/
boolean clear(String hostname);
/**
* Return the cached entries for the given hostname.
* @param hostname the hostname
* @return the cached entries
*/
List<DnsCacheEntry> get(String hostname);
/**
* Cache a resolved address for a given hostname.
* @param hostname the hostname
* @param address the resolved adresse
* @param originalTtl the TLL as returned by the DNS server
* @param loop the {@link EventLoop} used to register the TTL timeout
*/
void cache(String hostname, InetAddress address, long originalTtl, EventLoop loop);
/**
* Cache the resolution failure for a given hostname.
* @param hostname the hostname
* @param cause the resolution failure
* @param loop the {@link EventLoop} used to register the TTL timeout
*/
void cache(String hostname, Throwable cause, EventLoop loop);
}

View File

@ -16,40 +16,45 @@
package io.netty.resolver.dns;
import static io.netty.util.internal.ObjectUtil.checkNotNull;
import io.netty.channel.EventLoop;
import io.netty.util.concurrent.ScheduledFuture;
import java.net.InetAddress;
import java.util.concurrent.TimeUnit;
final class DnsCacheEntry {
/**
* Entry in {@link DnsCache}.
*/
public final class DnsCacheEntry {
private final String hostname;
private final InetAddress address;
private final Throwable cause;
private volatile ScheduledFuture<?> expirationFuture;
DnsCacheEntry(String hostname, InetAddress address) {
this.hostname = hostname;
this.address = address;
public DnsCacheEntry(String hostname, InetAddress address) {
this.hostname = checkNotNull(hostname, "hostname");
this.address = checkNotNull(address, "address");
cause = null;
}
DnsCacheEntry(String hostname, Throwable cause) {
this.hostname = hostname;
this.cause = cause;
public DnsCacheEntry(String hostname, Throwable cause) {
this.hostname = checkNotNull(hostname, "hostname");
this.cause = checkNotNull(cause, "cause");
address = null;
}
String hostname() {
public String hostname() {
return hostname;
}
InetAddress address() {
public InetAddress address() {
return address;
}
Throwable cause() {
public Throwable cause() {
return cause;
}

View File

@ -39,8 +39,6 @@ import io.netty.util.ReferenceCountUtil;
import io.netty.util.concurrent.FastThreadLocal;
import io.netty.util.concurrent.Future;
import io.netty.util.concurrent.Promise;
import io.netty.util.internal.OneTimeTask;
import io.netty.util.internal.PlatformDependent;
import io.netty.util.internal.logging.InternalLogger;
import io.netty.util.internal.logging.InternalLoggerFactory;
@ -50,11 +48,7 @@ import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map.Entry;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
import static io.netty.util.internal.ObjectUtil.*;
@ -97,7 +91,7 @@ public class DnsNameResolver extends InetNameResolver {
/**
* Cache for {@link #doResolve(String, Promise)} and {@link #doResolveAll(String, Promise)}.
*/
private final ConcurrentMap<String, List<DnsCacheEntry>> resolveCache = PlatformDependent.newConcurrentHashMap();
private final DnsCache resolveCache;
private final FastThreadLocal<DnsServerAddressStream> nameServerAddrStream =
new FastThreadLocal<DnsServerAddressStream>() {
@ -108,10 +102,6 @@ public class DnsNameResolver extends InetNameResolver {
};
private final long queryTimeoutMillis;
// The default TTL values here respect the TTL returned by the DNS server and do not cache the negative response.
private final int minTtl;
private final int maxTtl;
private final int negativeTtl;
private final int maxQueriesPerResolve;
private final boolean traceEnabled;
private final InternetProtocolFamily[] resolvedAddressTypes;
@ -129,9 +119,7 @@ public class DnsNameResolver extends InetNameResolver {
* @param nameServerAddresses the addresses of the DNS server. For each DNS query, a new stream is created from
* this to determine which DNS server should be contacted for the next retry in case
* of failure.
* @param minTtl the minimum TTL of cached DNS records
* @param maxTtl the maximum TTL of cached DNS records
* @param negativeTtl the TTL for failed cached queries
* @param resolveCache the DNS resolved entries cache
* @param queryTimeoutMillis timeout of each DNS query in millis
* @param resolvedAddressTypes list of the protocol families
* @param recursionDesired if recursion desired flag must be set
@ -146,9 +134,7 @@ public class DnsNameResolver extends InetNameResolver {
ChannelFactory<? extends DatagramChannel> channelFactory,
InetSocketAddress localAddress,
DnsServerAddresses nameServerAddresses,
int minTtl,
int maxTtl,
int negativeTtl,
DnsCache resolveCache,
long queryTimeoutMillis,
InternetProtocolFamily[] resolvedAddressTypes,
boolean recursionDesired,
@ -162,13 +148,6 @@ public class DnsNameResolver extends InetNameResolver {
checkNotNull(channelFactory, "channelFactory");
checkNotNull(localAddress, "localAddress");
this.nameServerAddresses = checkNotNull(nameServerAddresses, "nameServerAddresses");
this.minTtl = checkPositiveOrZero(minTtl, "minTtl");
this.maxTtl = checkPositiveOrZero(maxTtl, "maxTtl");
if (minTtl > maxTtl) {
throw new IllegalArgumentException(
"minTtl: " + minTtl + ", maxTtl: " + maxTtl + " (expected: 0 <= minTtl <= maxTtl)");
}
this.negativeTtl = checkPositiveOrZero(negativeTtl, "negativeTtl");
this.queryTimeoutMillis = checkPositive(queryTimeoutMillis, "queryTimeoutMillis");
this.resolvedAddressTypes = checkNonEmpty(resolvedAddressTypes, "resolvedAddressTypes");
this.recursionDesired = recursionDesired;
@ -177,6 +156,7 @@ public class DnsNameResolver extends InetNameResolver {
this.maxPayloadSize = checkPositive(maxPayloadSize, "maxPayloadSize");
this.optResourceEnabled = optResourceEnabled;
this.hostsFileEntriesResolver = checkNotNull(hostsFileEntriesResolver, "hostsFileEntriesResolver");
this.resolveCache = resolveCache;
bindFuture = newChannel(channelFactory, localAddress);
ch = (DatagramChannel) bindFuture.channel();
@ -201,7 +181,7 @@ public class DnsNameResolver extends InetNameResolver {
bindFuture.channel().closeFuture().addListener(new ChannelFutureListener() {
@Override
public void operationComplete(ChannelFuture future) throws Exception {
clearCache();
resolveCache.clear();
}
});
@ -209,29 +189,10 @@ public class DnsNameResolver extends InetNameResolver {
}
/**
* Returns the minimum TTL of the cached DNS resource records (in seconds).
*
* @see #maxTtl()
* Returns the resolution cache.
*/
public int minTtl() {
return minTtl;
}
/**
* Returns the maximum TTL of the cached DNS resource records (in seconds).
*
* @see #minTtl()
*/
public int maxTtl() {
return maxTtl;
}
/**
* Returns the TTL of the cache for the failed DNS queries (in seconds). The default value is {@code 0}, which
* disables the cache for negative results.
*/
public int negativeTtl() {
return negativeTtl;
public DnsCache resolveCache() {
return resolveCache;
}
/**
@ -302,49 +263,6 @@ public class DnsNameResolver extends InetNameResolver {
return hostsFileEntriesResolver;
}
/**
* Clears all the resolved addresses cached by this resolver.
*
* @return {@code this}
*
* @see #clearCache(String)
*/
public DnsNameResolver clearCache() {
for (Iterator<Entry<String, List<DnsCacheEntry>>> i = resolveCache.entrySet().iterator(); i.hasNext();) {
final Entry<String, List<DnsCacheEntry>> e = i.next();
i.remove();
cancelExpiration(e);
}
return this;
}
/**
* Clears the resolved addresses of the specified host name from the cache of this resolver.
*
* @return {@code true} if and only if there was an entry for the specified host name in the cache and
* it has been removed by this method
*/
public boolean clearCache(String hostname) {
boolean removed = false;
for (Iterator<Entry<String, List<DnsCacheEntry>>> i = resolveCache.entrySet().iterator(); i.hasNext();) {
final Entry<String, List<DnsCacheEntry>> e = i.next();
if (e.getKey().equals(hostname)) {
i.remove();
cancelExpiration(e);
removed = true;
}
}
return removed;
}
private static void cancelExpiration(Entry<String, List<DnsCacheEntry>> e) {
final List<DnsCacheEntry> entries = e.getValue();
final int numEntries = entries.size();
for (int i = 0; i < numEntries; i++) {
entries.get(i).cancelExpiration();
}
}
/**
* Closes the internal datagram channel used for sending and receiving DNS messages, and clears all DNS resource
* records from the cache. Attempting to send a DNS query or to resolve a domain name will fail once this method
@ -366,6 +284,16 @@ public class DnsNameResolver extends InetNameResolver {
@Override
protected void doResolve(String inetHost, Promise<InetAddress> promise) throws Exception {
doResolve(inetHost, promise, resolveCache);
}
/**
* Hook designed for extensibility so one can pass a different cache on each resolution attempt
* instead of using the global one.
*/
protected void doResolve(String inetHost,
Promise<InetAddress> promise,
DnsCache resolveCache) throws Exception {
final byte[] bytes = NetUtil.createByteArrayFromIpAddressString(inetHost);
if (bytes != null) {
// The inetHost is actually an ipaddress.
@ -381,12 +309,14 @@ public class DnsNameResolver extends InetNameResolver {
return;
}
if (!doResolveCached(hostname, promise)) {
doResolveUncached(hostname, promise);
if (!doResolveCached(hostname, promise, resolveCache)) {
doResolveUncached(hostname, promise, resolveCache);
}
}
private boolean doResolveCached(String hostname, Promise<InetAddress> promise) {
private boolean doResolveCached(String hostname,
Promise<InetAddress> promise,
DnsCache resolveCache) {
final List<DnsCacheEntry> cachedEntries = resolveCache.get(hostname);
if (cachedEntries == null) {
return false;
@ -433,9 +363,11 @@ public class DnsNameResolver extends InetNameResolver {
}
}
private void doResolveUncached(String hostname, Promise<InetAddress> promise) {
private void doResolveUncached(String hostname,
Promise<InetAddress> promise,
DnsCache resolveCache) {
final DnsNameResolverContext<InetAddress> ctx =
new DnsNameResolverContext<InetAddress>(this, hostname, promise) {
new DnsNameResolverContext<InetAddress>(this, hostname, promise, resolveCache) {
@Override
protected boolean finishResolve(
Class<? extends InetAddress> addressType, List<DnsCacheEntry> resolvedEntries) {
@ -457,6 +389,16 @@ public class DnsNameResolver extends InetNameResolver {
@Override
protected void doResolveAll(String inetHost, Promise<List<InetAddress>> promise) throws Exception {
doResolveAll(inetHost, promise, resolveCache);
}
/**
* Hook designed for extensibility so one can pass a different cache on each resolution attempt
* instead of using the global one.
*/
protected void doResolveAll(String inetHost,
Promise<List<InetAddress>> promise,
DnsCache resolveCache) throws Exception {
final byte[] bytes = NetUtil.createByteArrayFromIpAddressString(inetHost);
if (bytes != null) {
@ -473,12 +415,14 @@ public class DnsNameResolver extends InetNameResolver {
return;
}
if (!doResolveAllCached(hostname, promise)) {
doResolveAllUncached(hostname, promise);
if (!doResolveAllCached(hostname, promise, resolveCache)) {
doResolveAllUncached(hostname, promise, resolveCache);
}
}
private boolean doResolveAllCached(String hostname, Promise<List<InetAddress>> promise) {
private boolean doResolveAllCached(String hostname,
Promise<List<InetAddress>> promise,
DnsCache resolveCache) {
final List<DnsCacheEntry> cachedEntries = resolveCache.get(hostname);
if (cachedEntries == null) {
return false;
@ -518,9 +462,11 @@ public class DnsNameResolver extends InetNameResolver {
return true;
}
private void doResolveAllUncached(final String hostname, final Promise<List<InetAddress>> promise) {
private void doResolveAllUncached(final String hostname,
final Promise<List<InetAddress>> promise,
DnsCache resolveCache) {
final DnsNameResolverContext<List<InetAddress>> ctx =
new DnsNameResolverContext<List<InetAddress>>(this, hostname, promise) {
new DnsNameResolverContext<List<InetAddress>>(this, hostname, promise, resolveCache) {
@Override
protected boolean finishResolve(
Class<? extends InetAddress> addressType, List<DnsCacheEntry> resolvedEntries) {
@ -552,81 +498,6 @@ public class DnsNameResolver extends InetNameResolver {
return IDN.toASCII(inetHost);
}
void cache(String hostname, InetAddress address, long originalTtl) {
final int maxTtl = maxTtl();
if (maxTtl == 0) {
return;
}
final int ttl = Math.max(minTtl(), (int) Math.min(maxTtl, originalTtl));
final List<DnsCacheEntry> entries = cachedEntries(hostname);
final DnsCacheEntry e = new DnsCacheEntry(hostname, address);
synchronized (entries) {
if (!entries.isEmpty()) {
final DnsCacheEntry firstEntry = entries.get(0);
if (firstEntry.cause() != null) {
assert entries.size() == 1;
firstEntry.cancelExpiration();
entries.clear();
}
}
entries.add(e);
}
scheduleCacheExpiration(entries, e, ttl);
}
void cache(String hostname, Throwable cause) {
final int negativeTtl = negativeTtl();
if (negativeTtl == 0) {
return;
}
final List<DnsCacheEntry> entries = cachedEntries(hostname);
final DnsCacheEntry e = new DnsCacheEntry(hostname, cause);
synchronized (entries) {
final int numEntries = entries.size();
for (int i = 0; i < numEntries; i ++) {
entries.get(i).cancelExpiration();
}
entries.clear();
entries.add(e);
}
scheduleCacheExpiration(entries, e, negativeTtl);
}
private List<DnsCacheEntry> cachedEntries(String hostname) {
List<DnsCacheEntry> oldEntries = resolveCache.get(hostname);
final List<DnsCacheEntry> entries;
if (oldEntries == null) {
List<DnsCacheEntry> newEntries = new ArrayList<DnsCacheEntry>();
oldEntries = resolveCache.putIfAbsent(hostname, newEntries);
entries = oldEntries != null? oldEntries : newEntries;
} else {
entries = oldEntries;
}
return entries;
}
private void scheduleCacheExpiration(final List<DnsCacheEntry> entries, final DnsCacheEntry e, int ttl) {
e.scheduleExpiration(
ch.eventLoop(),
new OneTimeTask() {
@Override
public void run() {
synchronized (entries) {
entries.remove(e);
if (entries.isEmpty()) {
resolveCache.remove(e.hostname());
}
}
}
}, ttl, TimeUnit.SECONDS);
}
/**
* Sends a DNS query with the specified question.
*/
@ -700,9 +571,7 @@ public class DnsNameResolver extends InetNameResolver {
final DnsQueryContext qCtx = queryContextManager.get(res.sender(), queryId);
if (qCtx == null) {
if (logger.isWarnEnabled()) {
logger.warn("{} Received a DNS response with an unknown ID: {}", ch, queryId);
}
logger.warn("{} Received a DNS response with an unknown ID: {}", ch, queryId);
return;
}

View File

@ -15,6 +15,8 @@
*/
package io.netty.resolver.dns;
import static io.netty.util.internal.ObjectUtil.intValue;
import io.netty.channel.ChannelFactory;
import io.netty.channel.EventLoop;
import io.netty.channel.ReflectiveChannelFactory;
@ -37,9 +39,10 @@ public final class DnsNameResolverBuilder {
private ChannelFactory<? extends DatagramChannel> channelFactory;
private InetSocketAddress localAddress = DnsNameResolver.ANY_LOCAL_ADDR;
private DnsServerAddresses nameServerAddresses;
private int minTtl;
private int maxTtl = Integer.MAX_VALUE;
private int negativeTtl;
private DnsCache resolveCache;
private Integer minTtl;
private Integer maxTtl;
private Integer negativeTtl;
private long queryTimeoutMillis = 5000;
private InternetProtocolFamily[] resolvedAddressTypes = DnsNameResolver.DEFAULT_RESOLVE_ADDRESS_TYPES;
private boolean recursionDesired = true;
@ -103,6 +106,17 @@ public final class DnsNameResolverBuilder {
return this;
}
/**
* Sets the cache for resolution results.
*
* @param resolveCache the DNS resolution results cache
* @return {@code this}
*/
public DnsNameResolverBuilder resolveCache(DnsCache resolveCache) {
this.resolveCache = resolveCache;
return this;
}
/**
* Sets the minimum and maximum TTL of the cached DNS resource records (in seconds). If the TTL of the DNS
* resource record returned by the DNS server is less than the minimum TTL or greater than the maximum TTL,
@ -291,14 +305,20 @@ public final class DnsNameResolverBuilder {
* @return a {@link DnsNameResolver}
*/
public DnsNameResolver build() {
if (resolveCache != null && (minTtl != null || maxTtl != null || negativeTtl != null)) {
throw new IllegalStateException("resolveCache and TTLs are mutually exclusive");
}
DnsCache cache = resolveCache != null ? resolveCache :
new DefaultDnsCache(intValue(minTtl, 0), intValue(maxTtl, Integer.MAX_VALUE), intValue(negativeTtl, 0));
return new DnsNameResolver(
eventLoop,
channelFactory,
localAddress,
nameServerAddresses,
minTtl,
maxTtl,
negativeTtl,
cache,
queryTimeoutMillis,
resolvedAddressTypes,
recursionDesired,

View File

@ -70,6 +70,7 @@ abstract class DnsNameResolverContext<T> {
private final DnsServerAddressStream nameServerAddrs;
private final Promise<T> promise;
private final String hostname;
private final DnsCache resolveCache;
private final boolean traceEnabled;
private final int maxAllowedQueries;
private final InternetProtocolFamily[] resolveAddressTypes;
@ -83,10 +84,14 @@ abstract class DnsNameResolverContext<T> {
private int allowedQueries;
private boolean triedCNAME;
protected DnsNameResolverContext(DnsNameResolver parent, String hostname, Promise<T> promise) {
protected DnsNameResolverContext(DnsNameResolver parent,
String hostname,
Promise<T> promise,
DnsCache resolveCache) {
this.parent = parent;
this.promise = promise;
this.hostname = hostname;
this.resolveCache = resolveCache;
nameServerAddrs = parent.nameServerAddresses.stream();
maxAllowedQueries = parent.maxQueriesPerResolve();
@ -245,7 +250,7 @@ abstract class DnsNameResolverContext<T> {
}
final DnsCacheEntry e = new DnsCacheEntry(hostname, resolved);
parent.cache(hostname, resolved, r.timeToLive());
resolveCache.cache(hostname, resolved, r.timeToLive(), parent.ch.eventLoop());
resolvedEntries.add(e);
found = true;
@ -426,7 +431,7 @@ abstract class DnsNameResolverContext<T> {
final UnknownHostException cause = new UnknownHostException(buf.toString());
parent.cache(hostname, cause);
resolveCache.cache(hostname, cause, parent.ch.eventLoop());
promise.tryFailure(cause);
}

View File

@ -0,0 +1,63 @@
/*
* Copyright 2016 The Netty Project
*
* The Netty Project licenses this file to you under the Apache License,
* version 2.0 (the "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at:
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations
* under the License.
*/
package io.netty.resolver.dns;
import io.netty.channel.EventLoop;
import java.net.InetAddress;
import java.util.Collections;
import java.util.List;
/**
* A noop DNS cache that actually never caches anything.
*/
public final class NoopDnsCache implements DnsCache {
public static final NoopDnsCache INSTANCE = new NoopDnsCache();
/**
* Private singleton constructor.
*/
private NoopDnsCache() {
}
@Override
public void clear() {
}
@Override
public boolean clear(String hostname) {
return false;
}
@Override
public List<DnsCacheEntry> get(String hostname) {
return Collections.emptyList();
}
@Override
public void cache(String hostname, InetAddress address, long originalTtl, EventLoop loop) {
}
@Override
public void cache(String hostname, Throwable cause, EventLoop loop) {
}
@Override
public String toString() {
return NoopDnsCache.class.getSimpleName();
}
}