]> granicus.if.org Git - pdns/commitdiff
dnsdist: Add a 'keepStaleData' option to the packet cache
authorRemi Gacogne <remi.gacogne@powerdns.com>
Thu, 3 Jan 2019 15:16:46 +0000 (16:16 +0100)
committerRemi Gacogne <remi.gacogne@powerdns.com>
Thu, 10 Jan 2019 15:19:28 +0000 (16:19 +0100)
pdns/dnsdist-cache.cc
pdns/dnsdist-cache.hh
pdns/dnsdist-console.cc
pdns/dnsdist-lua-bindings.cc
pdns/dnsdist.cc
pdns/dnsdist.hh
pdns/dnsdistdist/docs/guides/cache.rst
pdns/dnsdistdist/docs/reference/config.rst
regression-tests.dnsdist/test_Caching.py

index b4c11df37ae420f375e052856f035da2bb7d85df..8c6f86f7cd59740f79929b5813b16525f155efa0 100644 (file)
@@ -168,7 +168,7 @@ void DNSDistPacketCache::insert(uint32_t key, const boost::optional<Netmask>& su
     return;
   }
 
-  const time_t now = time(NULL);
+  const time_t now = time(nullptr);
   time_t newValidity = now + minTTL;
   CacheValue newValue;
   newValue.qname = qname;
@@ -214,7 +214,7 @@ bool DNSDistPacketCache::get(const DNSQuestion& dq, uint16_t consumed, uint16_t
   }
 
   uint32_t shardIndex = getShardIndex(key);
-  time_t now = time(NULL);
+  time_t now = time(nullptr);
   time_t age;
   bool stale = false;
   auto& shard = d_shards.at(shardIndex);
@@ -294,10 +294,9 @@ bool DNSDistPacketCache::get(const DNSQuestion& dq, uint16_t consumed, uint16_t
 */
 void DNSDistPacketCache::purgeExpired(size_t upTo)
 {
-  time_t now = time(NULL);
   uint64_t size = getSize();
 
-  if (upTo >= size) {
+  if (size == 0 || upTo >= size) {
     return;
   }
 
@@ -305,6 +304,7 @@ void DNSDistPacketCache::purgeExpired(size_t upTo)
 
   size_t scannedMaps = 0;
 
+  const time_t now = time(nullptr);
   do {
     uint32_t shardIndex = (d_expungeIndex++ % d_shardCount);
     WriteLock w(&d_shards.at(shardIndex).d_lock);
index 0991857fee9aaa9fab0e0e015e26b00de5e75e89..830a01609ae83d3ca3732d9a24f71643995c9cd4 100644 (file)
@@ -53,8 +53,18 @@ public:
   uint64_t getTTLTooShorts() const { return d_ttlTooShorts; }
   uint64_t getEntriesCount();
   uint64_t dump(int fd);
+
   bool isECSParsingEnabled() const { return d_parseECS; }
 
+  bool keepStaleData() const
+  {
+    return d_keepStaleData;
+  }
+  void setKeepStaleData(bool keep)
+  {
+    d_keepStaleData = keep;
+  }
+
   static uint32_t getMinTTL(const char* packet, uint16_t length, bool* seenNoDataSOA);
   static uint32_t getKey(const std::string& qname, uint16_t consumed, const unsigned char* packet, uint16_t packetLen, bool tcp);
   static bool getClientSubnet(const char* packet, unsigned int consumed, uint16_t len, boost::optional<Netmask>& subnet);
@@ -124,4 +134,5 @@ private:
   bool d_dontAge;
   bool d_deferrableInsertLock;
   bool d_parseECS;
+  bool d_keepStaleData{false};
 };
index 091fa4bd85d2bf9abc3329bce4db8a4422804590..01458399d6f946a76c0c0be8e09f496739aeb986 100644 (file)
@@ -409,7 +409,7 @@ const std::vector<ConsoleKeyword> g_consoleKeywords{
   { "newDNSName", true, "name", "make a DNSName based on this .-terminated name" },
   { "newFrameStreamTcpLogger", true, "addr", "create a FrameStream logger object writing to a TCP address (addr should be ip:port), to use with `DnstapLogAction()` and `DnstapLogResponseAction()`" },
   { "newFrameStreamUnixLogger", true, "socket", "create a FrameStream logger object writing to a local unix socket, to use with `DnstapLogAction()` and `DnstapLogResponseAction()`" },
-  { "newPacketCache", true, "maxEntries[, maxTTL=86400, minTTL=0, temporaryFailureTTL=60, staleTTL=60, dontAge=false, numberOfShards=1, deferrableInsertLock=true]", "return a new Packet Cache" },
+  { "newPacketCache", true, "maxEntries[, maxTTL=86400, minTTL=0, temporaryFailureTTL=60, staleTTL=60, dontAge=false, numberOfShards=1, deferrableInsertLock=true, options={}]", "return a new Packet Cache" },
   { "newQPSLimiter", true, "rate, burst", "configure a QPS limiter with that rate and that burst capacity" },
   { "newRemoteLogger", true, "address:port [, timeout=2, maxQueuedEntries=100, reconnectWaitTime=1]", "create a Remote Logger object, to use with `RemoteLogAction()` and `RemoteLogResponseAction()`" },
   { "newRuleAction", true, "DNS rule, DNS action [, {uuid=\"UUID\"}]", "return a pair of DNS Rule and DNS Action, to be used with `setRules()`" },
index bec7c24bc9131cbd632031736cd1e96a34c661e3..963d13d90606aad180d07388ea562da9e5c38e92 100644 (file)
@@ -220,8 +220,63 @@ void setupLuaBindings(bool client)
 #endif /* HAVE_EBPF */
 
   /* PacketCache */
-  g_lua.writeFunction("newPacketCache", [](size_t maxEntries, boost::optional<uint32_t> maxTTL, boost::optional<uint32_t> minTTL, boost::optional<uint32_t> tempFailTTL, boost::optional<uint32_t> staleTTL, boost::optional<bool> dontAge, boost::optional<size_t> numberOfShards, boost::optional<bool> deferrableInsertLock, boost::optional<uint32_t> maxNegativeTTL, boost::optional<bool> ecsParsing) {
-      return std::make_shared<DNSDistPacketCache>(maxEntries, maxTTL ? *maxTTL : 86400, minTTL ? *minTTL : 0, tempFailTTL ? *tempFailTTL : 60, maxNegativeTTL ? *maxNegativeTTL : 3600, staleTTL ? *staleTTL : 60, dontAge ? *dontAge : false, numberOfShards ? *numberOfShards : 1, deferrableInsertLock ? *deferrableInsertLock : true, ecsParsing ? *ecsParsing : false);
+  g_lua.writeFunction("newPacketCache", [](size_t maxEntries, boost::optional<uint32_t> maxTTL, boost::optional<uint32_t> minTTL, boost::optional<uint32_t> tempFailTTL, boost::optional<uint32_t> staleTTL, boost::optional<bool> dontAge, boost::optional<size_t> numberOfShards, boost::optional<bool> deferrableInsertLock, boost::optional<uint32_t> maxNegativeTTL, boost::optional<bool> ecsParsing, boost::optional<std::unordered_map<std::string, boost::variant<bool, size_t>>> vars) {
+
+      bool keepStaleData = false;
+
+      if (vars) {
+
+        if (vars->count("deferrableInsertLock")) {
+          deferrableInsertLock = boost::get<bool>((*vars)["deferrableInsertLock"]);
+        }
+
+        if (vars->count("dontAge")) {
+          dontAge = boost::get<bool>((*vars)["dontAge"]);
+        }
+
+        if (vars->count("keepStaleData")) {
+          keepStaleData = boost::get<bool>((*vars)["keepStaleData"]);
+        }
+
+        if (vars->count("maxEntries")) {
+          maxEntries = boost::get<size_t>((*vars)["maxEntries"]);
+        }
+
+        if (vars->count("maxNegativeTTL")) {
+          maxNegativeTTL = boost::get<size_t>((*vars)["maxNegativeTTL"]);
+        }
+
+        if (vars->count("maxTTL")) {
+          maxTTL = boost::get<size_t>((*vars)["maxTTL"]);
+        }
+
+        if (vars->count("minTTL")) {
+          minTTL = boost::get<size_t>((*vars)["minTTL"]);
+        }
+
+        if (vars->count("numberOfShards")) {
+          numberOfShards = boost::get<size_t>((*vars)["numberOfShards"]);
+        }
+
+        if (vars->count("parseECS")) {
+          ecsParsing = boost::get<bool>((*vars)["parseECS"]);
+        }
+
+        if (vars->count("staleTTL")) {
+          staleTTL = boost::get<size_t>((*vars)["staleTTL"]);
+        }
+
+        if (vars->count("temporaryFailureTTL")) {
+          tempFailTTL = boost::get<size_t>((*vars)["temporaryFailureTTL"]);
+        }
+
+      }
+
+      auto res = std::make_shared<DNSDistPacketCache>(maxEntries, maxTTL ? *maxTTL : 86400, minTTL ? *minTTL : 0, tempFailTTL ? *tempFailTTL : 60, maxNegativeTTL ? *maxNegativeTTL : 3600, staleTTL ? *staleTTL : 60, dontAge ? *dontAge : false, numberOfShards ? *numberOfShards : 1, deferrableInsertLock ? *deferrableInsertLock : true, ecsParsing ? *ecsParsing : false);
+
+      res->setKeepStaleData(keepStaleData);
+
+      return res;
     });
   g_lua.registerFunction("toString", &DNSDistPacketCache::toString);
   g_lua.registerFunction("isFull", &DNSDistPacketCache::isFull);
@@ -248,6 +303,21 @@ void setupLuaBindings(bool client)
         g_outputBuffer+="TTL Too Shorts: " + std::to_string(cache->getTTLTooShorts()) + "\n";
       }
     });
+  g_lua.registerFunction<std::unordered_map<std::string, uint64_t>(std::shared_ptr<DNSDistPacketCache>::*)()>("getStats", [](const std::shared_ptr<DNSDistPacketCache> cache) {
+      std::unordered_map<std::string, uint64_t> stats;
+      if (cache) {
+        stats["entries"] = cache->getEntriesCount();
+        stats["maxEntries"] = cache->getMaxEntries();
+        stats["hits"] = cache->getHits();
+        stats["misses"] = cache->getMisses();
+        stats["deferredInserts"] = cache->getDeferredInserts();
+        stats["deferredLookups"] = cache->getDeferredLookups();
+        stats["lookupCollisions"] = cache->getLookupCollisions();
+        stats["insertCollisions"] = cache->getInsertCollisions();
+        stats["ttlTooShorts"] = cache->getTTLTooShorts();
+      }
+      return stats;
+    });
   g_lua.registerFunction<void(std::shared_ptr<DNSDistPacketCache>::*)(const std::string& fname)>("dump", [](const std::shared_ptr<DNSDistPacketCache> cache, const std::string& fname) {
       if (cache) {
 
index a6c4e7accbc0751173e3be6df55b7a818bc8af0e..cf2c44d5b566310620d642a38768d88c03b3b4d9 100644 (file)
@@ -1883,14 +1883,42 @@ void maintThread()
 
     counter++;
     if (counter >= g_cacheCleaningDelay) {
+      /* keep track, for each cache, of whether we should keep
+       expired entries */
+      std::map<std::shared_ptr<DNSDistPacketCache>, bool> caches;
+
+      /* gather all caches actually used by at least one pool, and see
+         if something prevents us from cleaning the expired entries */
       auto localPools = g_pools.getLocal();
-      std::shared_ptr<DNSDistPacketCache> packetCache = nullptr;
       for (const auto& entry : *localPools) {
-        packetCache = entry.second->packetCache;
-        if (packetCache) {
-          size_t upTo = (packetCache->getMaxEntries()* (100 - g_cacheCleaningPercentage)) / 100;
-          packetCache->purgeExpired(upTo);
+        auto& pool = entry.second;
+
+        auto packetCache = pool->packetCache;
+        if (!packetCache) {
+          continue;
+        }
+
+        auto pair = caches.insert({packetCache, false});
+        auto& iter = pair.first;
+        /* if we need to keep stale data for this cache (ie, not clear
+           expired entries when at least one pool using this cache
+           has all its backends down) */
+        if (packetCache->keepStaleData() && iter->second == false) {
+          /* so far all pools had at least one backend up */
+          if (pool->countServers(true) == 0) {
+            iter->second = true;
+          }
+        }
+      }
+
+      for (auto pair : caches) {
+        /* shall we keep expired entries ? */
+        if (pair.second == true) {
+          continue;
         }
+        auto& packetCache = pair.first;
+        size_t upTo = (packetCache->getMaxEntries()* (100 - g_cacheCleaningPercentage)) / 100;
+        packetCache->purgeExpired(upTo);
       }
       counter = 0;
     }
index bbeeb0bcba60ad59bacac72e93cd91fe5686f49a..66f154bee57caaf6e82824a68334b0fd721535ab 100644 (file)
@@ -826,8 +826,8 @@ struct ServerPool
     for (const auto& server : d_servers) {
       if (!upOnly || std::get<1>(server)->isUp() ) {
         count++;
-      };
-    };
+      }
+    }
     return count;
   }
 
index 7d6d6273303cdf1c30c14bf9f6830a01991bae46..1a6bdd3e3a7d7a17a8c3954a7f5de482d919f6bf 100644 (file)
@@ -40,6 +40,8 @@ Cache usage stats (hits, misses, deferred inserts and lookups, collisions) can b
 
   getPool("poolname"):getCache():printStats()
 
+The same values can also be returned as a Lua table, which is easier to work with from a script, using the :meth:`PacketCache:getStats` method.
+
 Expired cached entries can be removed from a cache using the :meth:`PacketCache:purgeExpired` method, which will remove expired entries from the cache until at most n entries remain in the cache.
 For example, to remove all expired entries::
 
index 6015f7808a5ccc45f7da3dc9bdbd1c80c034b2ca..77e4d5717356cfbc12e482d7fe7c2a9244d13e75 100644 (file)
@@ -535,7 +535,7 @@ PacketCache
 A Pool can have a packet cache to answer queries directly in stead of going to the backend.
 See :doc:`../guides/cache` for a how to.
 
-.. function:: newPacketCache(maxEntries[, maxTTL=86400[, minTTL=0[, temporaryFailureTTL=60[, staleTTL=60[, dontAge=false[, numberOfShards=1[, deferrableInsertLock=true[, maxNegativeTTL=3600[, parseECS=false]]]]]]]) -> PacketCache
+.. function:: newPacketCache(maxEntries[, maxTTL=86400[, minTTL=0[, temporaryFailureTTL=60[, staleTTL=60[, dontAge=false[, numberOfShards=1[, deferrableInsertLock=true[, maxNegativeTTL=3600[, parseECS=false [,options]]]]]]]]) -> PacketCache
 
   .. versionchanged:: 1.3.0
     ``numberOfShards`` and ``deferrableInsertLock`` parameters added.
@@ -543,7 +543,11 @@ See :doc:`../guides/cache` for a how to.
   .. versionchanged:: 1.3.1
     ``maxNegativeTTL`` and ``parseECS`` parameters added.
 
+  .. versionchanged:: 1.3.4
+    ``options`` parameter added.
+
   Creates a new :class:`PacketCache` with the settings specified.
+  Starting with 1.3.4, all parameters can be specified in the ``options`` table, overriding the value from the existing parameters if any.
 
   :param int maxEntries: The maximum number of entries in this cache
   :param int maxTTL: Cap the TTL for records to his number
@@ -555,6 +559,21 @@ See :doc:`../guides/cache` for a how to.
   :param bool deferrableInsertLock: Whether the cache should give up insertion if the lock is held by another thread, or simply wait to get the lock
   :param int maxNegativeTTL: Cache a NXDomain or NoData answer from the backend for at most this amount of seconds, even if the TTL of the SOA record is higher
   :param bool parseECS: Whether any EDNS Client Subnet option present in the query should be extracted and stored to be able to detect hash collisions involving queries with the same qname, qtype and qclass but a different incoming ECS value. Enabling this option adds a parsing cost and only makes sense if at least one backend might send different responses based on the ECS value, so it's disabled by default
+  :param table options: A table with key: value pairs with the options listed below:
+
+  Options:
+
+  * ``deferrableInsertLock=true``: bool - Whether the cache should give up insertion if the lock is held by another thread, or simply wait to get the lock.
+  * ``dontAge=false``: bool - Don't reduce TTLs when serving from the cache. Use this when :program:`dnsdist` fronts a cluster of authoritative servers.
+  * ``keepStaleData=false``: bool - Whether to suspend the removal of expired entries from the cache when there is no backend available in at least one of the pools using this cache.
+  * ``maxEntries``: int - The maximum number of entries in this cache.
+  * ``maxNegativeTTL=3600``: int - Cache a NXDomain or NoData answer from the backend for at most this amount of seconds, even if the TTL of the SOA record is higher.
+  * ``maxTTL=86400``: int - Cap the TTL for records to his number.
+  * ``minTTL=0``: int - Don't cache entries with a TTL lower than this.
+  * ``numberOfShards=1``: int - Number of shards to divide the cache into, to reduce lock contention.
+  * ``parseECS=false``: bool - Whether any EDNS Client Subnet option present in the query should be extracted and stored to be able to detect hash collisions involving queries with the same qname, qtype and qclass but a different incoming ECS value. Enabling this option adds a parsing cost and only makes sense if at least one backend might send different responses based on the ECS value, so it's disabled by default.
+  * ``staleTTL=60``: int - When the backend servers are not reachable, and global configuration ``setStaleCacheEntriesTTL`` is set appropriately, TTL that will be used when a stale cache entry is returned.
+  * ``temporaryFailureTTL=60``: int - On a SERVFAIL or REFUSED from the backend, cache for this amount of seconds..
 
 .. class:: PacketCache
 
@@ -585,13 +604,19 @@ See :doc:`../guides/cache` for a how to.
     :param int qtype: The type to expunge
     :param bool suffixMatch: When set to true, remove al entries under ``name``
 
+  .. method:: PacketCache:getStats()
+
+    .. versionadded:: 1.3.4
+
+    Return the cache stats (number of entries, hits, misses, deferred lookups, deferred inserts, lookup collisions, insert collisons and TTL too shorts) as a Lua table.
+
   .. method:: PacketCache:isFull() -> bool
 
     Return true if the cache has reached the maximum number of entries.
 
   .. method:: PacketCache:printStats()
 
-    Print the cache stats (hits, misses, deferred lookups and deferred inserts).
+    Print the cache stats (number of entries, hits, misses, deferred lookups, deferred inserts, lookup collisions, insert collisons and TTL too shorts).
 
   .. method:: PacketCache:purgeExpired(n)
 
index 33ba48f0ced955ebb3e50f6f1708080f5a6da889..d50aa88694c2316745a5e69651e25974114d6404 100644 (file)
@@ -8,6 +8,7 @@ from dnsdisttests import DNSDistTest
 class TestCaching(DNSDistTest):
 
     _config_template = """
+    -- maxTTL=86400, minTTL=1
     pc = newPacketCache(100, 86400, 1)
     getPool(""):setCache(pc)
     addAction(makeRule("nocache.cache.tests.powerdns.com."), SkipCacheAction())
@@ -16,7 +17,7 @@ class TestCaching(DNSDistTest):
         return DNSAction.None, ""
     end
     addAction("nocachevialua.cache.tests.powerdns.com.", LuaAction(skipViaLua))
-    newServer{address="127.0.0.1:%s"}
+    newServer{address="127.0.0.1:%d"}
     """
 
     def testCached(self):
@@ -416,10 +417,11 @@ class TestCaching(DNSDistTest):
 class TestTempFailureCacheTTLAction(DNSDistTest):
 
     _config_template = """
+    -- maxTTL=86400, minTTL=1
     pc = newPacketCache(100, 86400, 1)
     getPool(""):setCache(pc)
     addAction("servfail.cache.tests.powerdns.com.", TempFailureCacheTTLAction(1))
-    newServer{address="127.0.0.1:%s"}
+    newServer{address="127.0.0.1:%d"}
     """
 
     def testTempFailureCacheTTLAction(self):
@@ -462,9 +464,10 @@ class TestTempFailureCacheTTLAction(DNSDistTest):
 class TestCachingWithExistingEDNS(DNSDistTest):
 
     _config_template = """
+    -- maxTTL=86400, minTTL=1
     pc = newPacketCache(5, 86400, 1)
     getPool(""):setCache(pc)
-    newServer{address="127.0.0.1:%s"}
+    newServer{address="127.0.0.1:%d"}
     """
     def testCacheWithEDNS(self):
         """
@@ -519,9 +522,10 @@ class TestCachingWithExistingEDNS(DNSDistTest):
 class TestCachingCacheFull(DNSDistTest):
 
     _config_template = """
+    -- maxTTL=86400, minTTL=1
     pc = newPacketCache(1, 86400, 1)
     getPool(""):setCache(pc)
-    newServer{address="127.0.0.1:%s"}
+    newServer{address="127.0.0.1:%d"}
     """
     def testCacheFull(self):
         """
@@ -593,11 +597,12 @@ class TestCachingNoStale(DNSDistTest):
     _consoleKeyB64 = base64.b64encode(_consoleKey).decode('ascii')
     _config_params = ['_consoleKeyB64', '_consolePort', '_testServerPort']
     _config_template = """
+    -- maxTTL=86400, minTTL=1
     pc = newPacketCache(100, 86400, 1)
     getPool(""):setCache(pc)
     setKey("%s")
-    controlSocket("127.0.0.1:%s")
-    newServer{address="127.0.0.1:%s"}
+    controlSocket("127.0.0.1:%d")
+    newServer{address="127.0.0.1:%d"}
     """
     def testCacheNoStale(self):
         """
@@ -644,12 +649,13 @@ class TestCachingStale(DNSDistTest):
     _staleCacheTTL = 60
     _config_params = ['_staleCacheTTL', '_consoleKeyB64', '_consolePort', '_testServerPort']
     _config_template = """
-    pc = newPacketCache(100, 86400, 1, %s)
+    -- maxTTL=86400, minTTL=1, temporaryFailureTTL=0, staleTTL=XX
+    pc = newPacketCache(100, 86400, 1, 0, %d)
     getPool(""):setCache(pc)
     setStaleCacheEntriesTTL(600)
     setKey("%s")
-    controlSocket("127.0.0.1:%s")
-    newServer{address="127.0.0.1:%s"}
+    controlSocket("127.0.0.1:%d")
+    newServer{address="127.0.0.1:%d"}
     """
     def testCacheStale(self):
         """
@@ -698,17 +704,168 @@ class TestCachingStale(DNSDistTest):
 
         self.assertEquals(total, misses)
 
+class TestCachingStaleExpunged(DNSDistTest):
+
+    _consoleKey = DNSDistTest.generateConsoleKey()
+    _consoleKeyB64 = base64.b64encode(_consoleKey).decode('ascii')
+    _staleCacheTTL = 60
+    _config_params = ['_staleCacheTTL', '_consoleKeyB64', '_consolePort', '_testServerPort']
+    _config_template = """
+    -- maxTTL=86400, minTTL=1, temporaryFailureTTL=0, staleTTL=XX
+    pc = newPacketCache(100, 86400, 1, 0, %d)
+    getPool(""):setCache(pc)
+    setStaleCacheEntriesTTL(600)
+    -- try to remove all expired entries
+    setCacheCleaningPercentage(100)
+    -- clean the cache every second
+    setCacheCleaningDelay(1)
+    setKey("%s")
+    controlSocket("127.0.0.1:%d")
+    newServer{address="127.0.0.1:%d"}
+    """
+    def testCacheStale(self):
+        """
+        Cache: Cache entry, set backend down, wait for the cache cleaning to run and remove the entry, get no entry
+        """
+        misses = 0
+        drops = 0
+        ttl = 1
+        name = 'stale-but-expunged.cache.tests.powerdns.com.'
+        query = dns.message.make_query(name, 'A', 'IN')
+        response = dns.message.make_response(query)
+        rrset = dns.rrset.from_text(name,
+                                    ttl,
+                                    dns.rdataclass.IN,
+                                    dns.rdatatype.A,
+                                    '127.0.0.1')
+        response.answer.append(rrset)
+
+        # Miss
+        (receivedQuery, receivedResponse) = self.sendUDPQuery(query, response)
+        self.assertTrue(receivedQuery)
+        self.assertTrue(receivedResponse)
+        receivedQuery.id = query.id
+        self.assertEquals(query, receivedQuery)
+        self.assertEquals(response, receivedResponse)
+        misses += 1
+        self.assertEquals(int(self.sendConsoleCommand("getPool(\"\"):getCache():getStats()[\"misses\"]").strip("\n")), misses + drops)
+
+        # next queries should hit the cache
+        (_, receivedResponse) = self.sendUDPQuery(query, response=None, useQueue=False)
+        self.assertEquals(receivedResponse, response)
+        # the cache should have one entry
+        self.assertEquals(int(self.sendConsoleCommand("getPool(\"\"):getCache():getStats()[\"entries\"]").strip("\n")), 1)
+        self.assertEquals(int(self.sendConsoleCommand("getPool(\"\"):getCache():getStats()[\"hits\"]").strip("\n")), 1)
+
+        # ok, we mark the backend as down
+        self.sendConsoleCommand("getServer(0):setDown()")
+        # and we wait for the entry to expire
+        time.sleep(ttl + 1)
+        # wait a bit more to be sure that the cache cleaning algo has been run
+        time.sleep(1)
+        # the cache should be empty now
+        self.assertEquals(int(self.sendConsoleCommand("getPool(\"\"):getCache():getStats()[\"entries\"]").strip("\n")), 0)
+
+        # we should get a DROP (backend is down, nothing in the cache anymore)
+        (_, receivedResponse) = self.sendUDPQuery(query, response=None, useQueue=False)
+        self.assertEquals(receivedResponse, None)
+        drops += 1
+
+        self.assertEquals(int(self.sendConsoleCommand("getPool(\"\"):getCache():getStats()[\"misses\"]").strip("\n")), misses + drops)
+        self.assertEquals(int(self.sendConsoleCommand("getPool(\"\"):getCache():getStats()[\"hits\"]").strip("\n")), 1)
+
+        total = 0
+        for key in self._responsesCounter:
+            total += self._responsesCounter[key]
+
+        self.assertEquals(total, misses)
+
+class TestCachingStaleExpungePrevented(DNSDistTest):
+
+    _consoleKey = DNSDistTest.generateConsoleKey()
+    _consoleKeyB64 = base64.b64encode(_consoleKey).decode('ascii')
+    _config_params = ['_consoleKeyB64', '_consolePort', '_testServerPort']
+    _config_template = """
+    -- maxTTL=86400, minTTL=1, temporaryFailureTTL=0, staleTTL=60, dontAge=false, numberOfShards=1, deferrableInsertLock=true, maxNegativeTTL=3600, ecsParsing=false, keepStaleData=true
+    pc = newPacketCache(100, 86400, 1, 0, 60, false, 1, true, 3600, false, { keepStaleData=true})
+    getPool(""):setCache(pc)
+    setStaleCacheEntriesTTL(600)
+    -- try to remove all expired entries
+    setCacheCleaningPercentage(100)
+    -- clean the cache every second
+    setCacheCleaningDelay(1)
+    setKey("%s")
+    controlSocket("127.0.0.1:%d")
+    newServer{address="127.0.0.1:%d"}
+    """
+    def testCacheStale(self):
+        """
+        Cache: Cache entry, set backend down, wait for the cache cleaning to run and remove the entry, still get a cache HIT because the stale entry was not removed
+        """
+        misses = 0
+        ttl = 1
+        name = 'stale-not-expunged.cache.tests.powerdns.com.'
+        query = dns.message.make_query(name, 'A', 'IN')
+        response = dns.message.make_response(query)
+        rrset = dns.rrset.from_text(name,
+                                    ttl,
+                                    dns.rdataclass.IN,
+                                    dns.rdatatype.A,
+                                    '127.0.0.1')
+        response.answer.append(rrset)
+
+        # Miss
+        (receivedQuery, receivedResponse) = self.sendUDPQuery(query, response)
+        self.assertTrue(receivedQuery)
+        self.assertTrue(receivedResponse)
+        receivedQuery.id = query.id
+        self.assertEquals(query, receivedQuery)
+        self.assertEquals(response, receivedResponse)
+        misses += 1
+        self.assertEquals(int(self.sendConsoleCommand("getPool(\"\"):getCache():getStats()[\"misses\"]").strip("\n")), 1)
+
+        # next queries should hit the cache
+        (_, receivedResponse) = self.sendUDPQuery(query, response=None, useQueue=False)
+        self.assertEquals(receivedResponse, response)
+        # the cache should have one entry
+        self.assertEquals(int(self.sendConsoleCommand("getPool(\"\"):getCache():getStats()[\"entries\"]").strip("\n")), 1)
+        self.assertEquals(int(self.sendConsoleCommand("getPool(\"\"):getCache():getStats()[\"hits\"]").strip("\n")), 1)
+
+        # ok, we mark the backend as down
+        self.sendConsoleCommand("getServer(0):setDown()")
+        # and we wait for the entry to expire
+        time.sleep(ttl + 1)
+        # wait a bit more to be sure that the cache cleaning algo has been run
+        time.sleep(1)
+        # the cache should NOT be empty because the removal of the expired entry should have been prevented
+        # since all backends for this pool are down
+        self.assertEquals(int(self.sendConsoleCommand("getPool(\"\"):getCache():getStats()[\"entries\"]").strip("\n")), 1)
+
+        # we should get a HIT
+        (_, receivedResponse) = self.sendUDPQuery(query, response=None, useQueue=False)
+        self.assertEquals(receivedResponse, response)
+
+        self.assertEquals(int(self.sendConsoleCommand("getPool(\"\"):getCache():getStats()[\"misses\"]").strip("\n")), 1)
+        self.assertEquals(int(self.sendConsoleCommand("getPool(\"\"):getCache():getStats()[\"hits\"]").strip("\n")), 2)
+
+        total = 0
+        for key in self._responsesCounter:
+            total += self._responsesCounter[key]
+
+        self.assertEquals(total, misses)
+
 class TestCacheManagement(DNSDistTest):
 
     _consoleKey = DNSDistTest.generateConsoleKey()
     _consoleKeyB64 = base64.b64encode(_consoleKey).decode('ascii')
     _config_params = ['_consoleKeyB64', '_consolePort', '_testServerPort']
     _config_template = """
+    -- maxTTL=86400, minTTL=1
     pc = newPacketCache(100, 86400, 1)
     getPool(""):setCache(pc)
     setKey("%s")
-    controlSocket("127.0.0.1:%s")
-    newServer{address="127.0.0.1:%s"}
+    controlSocket("127.0.0.1:%d")
+    newServer{address="127.0.0.1:%d"}
     """
     def testCacheExpunge(self):
         """
@@ -1092,9 +1249,10 @@ class TestCachingTTL(DNSDistTest):
     _minCacheTTL = 600
     _config_params = ['_maxCacheTTL', '_minCacheTTL', '_testServerPort']
     _config_template = """
-    pc = newPacketCache(1000, %s, %s)
+    -- maxTTL=XX, minTTL=XX
+    pc = newPacketCache(1000, %d, %d)
     getPool(""):setCache(pc)
-    newServer{address="127.0.0.1:%s"}
+    newServer{address="127.0.0.1:%d"}
     """
     def testCacheShortTTL(self):
         """
@@ -1181,9 +1339,10 @@ class TestCachingLongTTL(DNSDistTest):
     _maxCacheTTL = 2
     _config_params = ['_maxCacheTTL', '_testServerPort']
     _config_template = """
-    pc = newPacketCache(1000, %s)
+    -- maxTTL=XX
+    pc = newPacketCache(1000, %d)
     getPool(""):setCache(pc)
-    newServer{address="127.0.0.1:%s"}
+    newServer{address="127.0.0.1:%d"}
     """
     def testCacheLongTTL(self):
         """
@@ -1244,9 +1403,10 @@ class TestCachingFailureTTL(DNSDistTest):
     _failureCacheTTL = 2
     _config_params = ['_failureCacheTTL', '_testServerPort']
     _config_template = """
+    -- maxTTL=86400, minTTL=0, temporaryFailureTTL=XX, staleTTL=60
     pc = newPacketCache(1000, 86400, 0, %d, 60)
     getPool(""):setCache(pc)
-    newServer{address="127.0.0.1:%s"}
+    newServer{address="127.0.0.1:%d"}
     """
     def testCacheServFailTTL(self):
         """
@@ -1380,9 +1540,10 @@ class TestCachingNegativeTTL(DNSDistTest):
     _negCacheTTL = 1
     _config_params = ['_negCacheTTL', '_testServerPort']
     _config_template = """
+    -- maxTTL=86400, minTTL=0, temporaryFailureTTL=60, staleTTL=60, dontAge=false, numberOfShards=1, deferrableInsertLock=true, maxNegativeTTL=XX
     pc = newPacketCache(1000, 86400, 0, 60, 60, false, 1, true, %d)
     getPool(""):setCache(pc)
-    newServer{address="127.0.0.1:%s"}
+    newServer{address="127.0.0.1:%d"}
     """
 
     def testCacheNegativeTTLNXDomain(self):
@@ -1484,9 +1645,10 @@ class TestCachingNegativeTTL(DNSDistTest):
 class TestCachingDontAge(DNSDistTest):
 
     _config_template = """
+    -- maxTTL=86400, minTTL=0, temporaryFailureTTL=60, staleTTL=60, dontAge=true
     pc = newPacketCache(100, 86400, 0, 60, 60, true)
     getPool(""):setCache(pc)
-    newServer{address="127.0.0.1:%s"}
+    newServer{address="127.0.0.1:%d"}
     """
     def testCacheDoesntDecreaseTTL(self):
         """
@@ -1544,6 +1706,7 @@ class TestCachingECSWithoutPoolECS(DNSDistTest):
     _consoleKeyB64 = base64.b64encode(_consoleKey).decode('ascii')
     _config_params = ['_consoleKeyB64', '_consolePort', '_testServerPort']
     _config_template = """
+    -- maxTTL=86400, minTTL=1
     pc = newPacketCache(100, 86400, 1)
     getPool(""):setCache(pc)
     setKey("%s")
@@ -1606,6 +1769,7 @@ class TestCachingECSWithPoolECS(DNSDistTest):
     _consoleKeyB64 = base64.b64encode(_consoleKey).decode('ascii')
     _config_params = ['_consoleKeyB64', '_consolePort', '_testServerPort']
     _config_template = """
+    -- maxTTL=86400, minTTL=1
     pc = newPacketCache(100, 86400, 1)
     getPool(""):setCache(pc)
     getPool(""):setECS(true)
@@ -1666,9 +1830,10 @@ class TestCachingECSWithPoolECS(DNSDistTest):
 class TestCachingCollisionNoECSParsing(DNSDistTest):
 
     _config_template = """
+    -- maxTTL=86400, minTTL=1
     pc = newPacketCache(100, 86400, 1)
     getPool(""):setCache(pc)
-    newServer{address="127.0.0.1:%s"}
+    newServer{address="127.0.0.1:%d"}
     """
 
     def testCacheCollisionNoECSParsing(self):
@@ -1708,9 +1873,10 @@ class TestCachingCollisionNoECSParsing(DNSDistTest):
 class TestCachingCollisionWithECSParsing(DNSDistTest):
 
     _config_template = """
+    -- maxTTL=86400, minTTL=1, temporaryFailureTTL=60, staleTTL=60, dontAge=false, numberOfShards=1, deferrableInsertLock=true, maxNegativeTTL=3600, parseECS=true
     pc = newPacketCache(100, 86400, 1, 60, 60, false, 1, true, 3600, true)
     getPool(""):setCache(pc)
-    newServer{address="127.0.0.1:%s"}
+    newServer{address="127.0.0.1:%d"}
     """
 
     def testCacheCollisionWithECSParsing(self):