]> git.ipfire.org Git - thirdparty/unbound.git/commitdiff
LRU touch of RRsets. Pretty complicated to do.
authorWouter Wijngaards <wouter@nlnetlabs.nl>
Wed, 16 May 2007 12:48:48 +0000 (12:48 +0000)
committerWouter Wijngaards <wouter@nlnetlabs.nl>
Wed, 16 May 2007 12:48:48 +0000 (12:48 +0000)
git-svn-id: file:///svn/unbound/trunk@324 be551aaa-1e26-0410-a405-d3ace91eadb9

22 files changed:
Makefile.in
daemon/daemon.c
daemon/daemon.h
daemon/unbound.c
daemon/worker.c
doc/Changelog
iterator/iterator.c
services/cache/infra.c
services/cache/rrset.c [new file with mode: 0644]
services/cache/rrset.h [new file with mode: 0644]
testcode/lock_verify.c
testcode/unitlruhash.c
testcode/unitslabhash.c
util/data/msgreply.c
util/data/msgreply.h
util/data/packed_rrset.c
util/data/packed_rrset.h
util/module.h
util/storage/lruhash.c
util/storage/lruhash.h
util/storage/slabhash.c
util/storage/slabhash.h

index 528ba63c8b49f33d5b48a6777d750f24664461e7..ba1b75369b1299918188c1119958d18eaca13190 100644 (file)
@@ -157,4 +157,8 @@ $(BUILD)%.d: $(srcdir)/%.c
                      | sed '\''s!\(.*\)\.o[ :]*!$(dir $@)\1.o $@ : !g'\'' > $@; \
                      [ -s $@ ] || rm -f $@'
 
+ifneq ($(MAKECMDGOALS),clean)
+ifneq ($(MAKECMDGOALS),realclean)
 -include $(addprefix $(BUILD),$(ALL_SRC:.c=.d))
+endif
+endif
index 611c04039df3a80518d59dc22012cd6df4c0d79c..793d2e099310b4ea28b1cd531eb06cf4f28f519e 100644 (file)
@@ -50,6 +50,7 @@
 #include "util/data/msgreply.h"
 #include "util/storage/slabhash.h"
 #include "services/listen_dnsport.h"
+#include "services/cache/rrset.h"
 #include "util/module.h"
 #include "iterator/iterator.h"
 #include <signal.h>
@@ -128,16 +129,13 @@ daemon_init()
                free(daemon);
                return NULL;
        }
-       daemon->rrset_cache = slabhash_create(HASH_DEFAULT_SLABS, 
-               HASH_DEFAULT_STARTARRAY, HASH_DEFAULT_MAXMEM, 
-               ub_rrset_sizefunc, ub_rrset_compare,
-               ub_rrset_key_delete, rrset_data_delete, &daemon->superalloc);
+       alloc_init(&daemon->superalloc, NULL, 0);
+       daemon->rrset_cache = rrset_cache_create(NULL, &daemon->superalloc);
        if(!daemon->rrset_cache) {
                slabhash_delete(daemon->msg_cache);
                free(daemon);
                return NULL;
        }
-       alloc_init(&daemon->superalloc, NULL, 0);
        if(!(daemon->env = (struct module_env*)calloc(1, 
                sizeof(*daemon->env)))) {
                daemon_delete(daemon);
@@ -383,7 +381,7 @@ daemon_delete(struct daemon* daemon)
                return;
        listening_ports_free(daemon->ports);
        slabhash_delete(daemon->msg_cache);
-       slabhash_delete(daemon->rrset_cache);
+       rrset_cache_delete(daemon->rrset_cache);
        alloc_clear(&daemon->superalloc);
        free(daemon->cwd);
        free(daemon->pidfile);
index ef094cc241ef89bf2e0a3d03e54f5d9f4b24ad04..af00f2c3c0ee01513353e2aca97ed8eaafd960be 100644 (file)
@@ -49,6 +49,7 @@ struct worker;
 struct listen_port;
 struct slabhash;
 struct module_env;
+struct rrset_cache;
 
 /**
  * Structure holding worker list.
@@ -76,7 +77,7 @@ struct daemon {
        /** the message cache, content is struct msgreply_entry* */
        struct slabhash* msg_cache;
        /** the rrset cache, content is struct ub_packed_rrset_key* */
-       struct slabhash* rrset_cache;
+       struct rrset_cache* rrset_cache;
        /** the module environment master value, copied and changed by threads*/
        struct module_env* env;
        /** number of modules active, ids from 0 to num-1. */
index d35db0bcaf3b73a8caeb72e3423fc1a01084dfbd..37928f8872576135bfa44c2290021457a29806dc 100644 (file)
@@ -46,7 +46,9 @@
 #include "util/config_file.h"
 #include "util/storage/slabhash.h"
 #include "services/listen_dnsport.h"
+#include "services/cache/rrset.h"
 #include "util/data/msgreply.h"
+#include "util/module.h"
 #include <signal.h>
 #include <fcntl.h>
 #include <pwd.h>
@@ -129,18 +131,11 @@ apply_dir(struct daemon* daemon, struct config_file* cfg, int cmdline_verbose)
                        fatal_exit("malloc failure updating config settings");
                }
        }
-       if(cfg->rrset_cache_size != slabhash_get_size(daemon->rrset_cache) ||
-               cfg->rrset_cache_slabs != daemon->rrset_cache->size) {
-               slabhash_delete(daemon->rrset_cache);
-               daemon->rrset_cache = slabhash_create(cfg->rrset_cache_slabs, 
-                       HASH_DEFAULT_STARTARRAY, cfg->rrset_cache_size, 
-                       ub_rrset_sizefunc, ub_rrset_compare,
-                       ub_rrset_key_delete, rrset_data_delete, 
-                       &daemon->superalloc);
-               if(!daemon->rrset_cache) {
+       if((daemon->rrset_cache = rrset_cache_adjust(daemon->rrset_cache,
+                       cfg, &daemon->superalloc)) == 0)
                        fatal_exit("malloc failure updating config settings");
-               }
-       }
+       daemon->env->rrset_cache = daemon->rrset_cache;
+       daemon->env->msg_cache = daemon->msg_cache;
        checkrlimits(cfg);
 }
 
index d5e3c46445d82a61a0d21863844bbc449c425fa7..008d32db7070b1852b24ea4622f5505090d47c37 100644 (file)
@@ -51,6 +51,7 @@
 #include "util/storage/slabhash.h"
 #include "services/listen_dnsport.h"
 #include "services/outside_network.h"
+#include "services/cache/rrset.h"
 #include "util/data/msgparse.h"
 
 #ifdef HAVE_SYS_TYPES_H
@@ -279,16 +280,22 @@ answer_from_cache(struct worker* worker, struct lruhash_entry* e, uint16_t id,
        uint32_t timenow = time(0);
        uint16_t udpsize = edns->udp_size;
        size_t i;
+       hashvalue_t* h;
        /* see if it is possible */
        if(rep->ttl <= timenow) {
-               /* the rrsets may have been updated in the meantime */
-               /* but this ignores it */
+               /* the rrsets may have been updated in the meantime.
+                * we will refetch the message format from the
+                * authoritative server 
+                */
                return 0;
        }
        edns->edns_version = EDNS_ADVERTISED_VERSION;
        edns->udp_size = EDNS_ADVERTISED_SIZE;
        edns->ext_rcode = 0;
        edns->bits &= EDNS_DO;
+       if(!(h = (hashvalue_t*)region_alloc(worker->scratchpad, 
+               sizeof(hashvalue_t)*rep->rrset_count)))
+               return 0;
        /* check rrsets */
        for(i=0; i<rep->rrset_count; i++) {
                if(i>0 && rep->ref[i].key == rep->ref[i-1].key)
@@ -314,8 +321,16 @@ answer_from_cache(struct worker* worker, struct lruhash_entry* e, uint16_t id,
        for(i=0; i<rep->rrset_count; i++) {
                if(i>0 && rep->ref[i].key == rep->ref[i-1].key)
                        continue; /* only unlock items once */
+               h[i] = rep->ref[i].key->entry.hash;
                lock_rw_unlock(&rep->ref[i].key->entry.lock);
        }
+       /* LRU touch, with no rrset locks held */
+       for(i=0; i<rep->rrset_count; i++) {
+               if(i>0 && rep->ref[i].key == rep->ref[i-1].key)
+                       continue; /* only touch items once */
+               rrset_cache_touch(worker->env.rrset_cache, rep->ref[i].key,
+                       h[i], rep->ref[i].id);
+       }
        region_free_all(worker->scratchpad);
        /* go and return this buffer to the client */
        return 1;
@@ -375,7 +390,7 @@ worker_handle_request(struct comm_point* c, void* arg, int error,
        }
        if(c->type != comm_udp)
                edns.udp_size = 65535; /* max size for TCP replies */
-       if((e=slabhash_lookup(worker->daemon->msg_cache, h, &qinfo, 0))) {
+       if((e=slabhash_lookup(worker->env.msg_cache, h, &qinfo, 0))) {
                /* answer from cache - we have acquired a readlock on it */
                log_info("answer from the cache");
                if(answer_from_cache(worker, e, 
index 5eefcff8c0cf3bbd03c9a75350b94936cea5a5ad..129ad86f62de6754217ecaf65277e6eced7833f1 100644 (file)
@@ -1,3 +1,10 @@
+16 May 2007: Wouter
+       - lruhash_touch() would cause locking order problems. Fixup in 
+         lock-verify in case locking cycle is found.
+       - services/cache/rrset.c for rrset cache code.
+       - special rrset_cache LRU updating function that uses the rrset id.
+       - no dependencies calculation when make clean is called.
+
 15 May 2007: Wouter
        - host cache code.
        - unit test for host cache.
index a5e8b05d45023a98111fc547679081e7820b4dac..fb62cea9df98aaa50cb4084ca506340f0d624c16 100644 (file)
@@ -47,6 +47,7 @@
 #include "util/config_file.h"
 #include "util/net_help.h"
 #include "util/storage/slabhash.h"
+#include "services/cache/rrset.h"
 
 /** 
  * Set forwarder address 
@@ -122,70 +123,18 @@ iter_deinit(struct module_env* env, int id)
                free(iter_env);
 }
 
-/** see if rrset needs to be updated in the cache */
-static int
-need_to_update_rrset(struct packed_rrset_data* newd,
-        struct packed_rrset_data* cached)
-{
-        /*      o if current RRset is more trustworthy - insert it */
-        if( newd->trust > cached->trust )
-                return 1;
-        /*      o same trust, but different in data - insert it */
-        if( newd->trust == cached->trust &&
-                !rrsetdata_equal(newd, cached))
-                return 1;
-        /*      o see if TTL is better than TTL in cache. */
-        /*        if so, see if rrset+rdata is the same */
-        /*        if so, update TTL in cache, even if trust is worse. */
-        if( newd->ttl > cached->ttl &&
-                rrsetdata_equal(newd, cached))
-                return 1;
-        return 0;
-}
-
 /** store rrsets in the rrset cache. */
 static void
-worker_store_rrsets(struct module_env* env, struct reply_info* rep)
+store_rrsets(struct module_env* env, struct reply_info* rep, uint32_t now)
 {
-        struct lruhash_entry* e;
         size_t i;
         /* see if rrset already exists in cache, if not insert it. */
-        /* if it does exist: check to insert it */
         for(i=0; i<rep->rrset_count; i++) {
                 rep->ref[i].key = rep->rrsets[i];
                 rep->ref[i].id = rep->rrsets[i]->id;
-                /* looks up item with a readlock - no editing! */
-                if((e=slabhash_lookup(env->rrset_cache,
-                        rep->rrsets[i]->entry.hash, rep->rrsets[i]->entry.key,
-                        0)) != 0) {
-                        struct packed_rrset_data* data =
-                                (struct packed_rrset_data*)e->data;
-                        struct packed_rrset_data* rd =
-                                (struct packed_rrset_data*)
-                                rep->rrsets[i]->entry.data;
-                        rep->ref[i].key = (struct ub_packed_rrset_key*)e->key;
-                        rep->ref[i].id = rep->rrsets[i]->id;
-                        /* found in cache, do checks above */
-                        if(!need_to_update_rrset(rd, data)) {
-                                lock_rw_unlock(&e->lock);
-                                ub_packed_rrset_parsedelete(rep->rrsets[i],
-                                        env->alloc);
-                                rep->rrsets[i] = rep->ref[i].key;
-                                continue; /* use cached item instead */
-                        }
-                        if(rd->trust < data->trust)
-                                rd->trust = data->trust;
-                        lock_rw_unlock(&e->lock);
-                        /* small gap here, where entry is not locked.
-                         * possibly entry is updated with something else.
-                         * this is just too bad, its cache anyway. */
-                        /* use insert to update entry to manage lruhash
-                         * cache size values nicely. */
-                }
-                slabhash_insert(env->rrset_cache,
-                        rep->rrsets[i]->entry.hash, &rep->rrsets[i]->entry,
-                        rep->rrsets[i]->entry.data, env->alloc);
-                if(e) rep->rrsets[i] = rep->ref[i].key;
+               if(rrset_cache_update(env->rrset_cache, &rep->ref[i], 
+                       env->alloc, now)) /* it was in the cache */
+                       rep->rrsets[i] = rep->ref[i].key;
         }
 }
 
@@ -196,8 +145,9 @@ store_msg(struct module_qstate* qstate, struct query_info* qinfo,
        struct reply_info* rep)
 {
        struct msgreply_entry* e;
-       reply_info_set_ttls(rep, time(0));
-       worker_store_rrsets(qstate->env, rep);
+       uint32_t now = time(NULL);
+       reply_info_set_ttls(rep, now);
+       store_rrsets(qstate->env, rep, now);
        if(rep->ttl == 0) {
                log_info("TTL 0: dropped msg from cache");
                return;
index 11ab4e0b2b364760ccfdefdb78af00f504bdc46d..d3ed429a5235e548bd7da7a1e5996afba67f1e1c 100644 (file)
@@ -70,11 +70,12 @@ infra_host_compfunc(void* key1, void* key2)
 
 /** delete key, and destroy the lock */
 static void 
-infra_host_delkeyfunc(void* k, void* ATTR_UNUSED(arg))
+infra_host_delkeyfunc(void* k, void* ATTR_UNUSED(arg), int il)
 {
        struct infra_host_key* key = (struct infra_host_key*)k;
        if(!key)
                return;
+       if(il) lock_rw_unlock(&key->entry.lock);
        lock_rw_destroy(&key->entry.lock);
        free(key);
 }
@@ -297,11 +298,12 @@ infra_lame_compfunc(void* key1, void* key2)
 
 /** free key, lock and zonename */
 static void 
-infra_lame_delkeyfunc(void* k, void* ATTR_UNUSED(arg))
+infra_lame_delkeyfunc(void* k, void* ATTR_UNUSED(arg), int il)
 {
        struct infra_lame_key* key = (struct infra_lame_key*)k;
        if(!key) 
                return;
+       if(il) lock_rw_unlock(&key->entry.lock);
        lock_rw_destroy(&key->entry.lock);
        free(key->zonename);
        free(key);
diff --git a/services/cache/rrset.c b/services/cache/rrset.c
new file mode 100644 (file)
index 0000000..a35e14d
--- /dev/null
@@ -0,0 +1,176 @@
+/*
+ * services/cache/rrset.c - Resource record set cache.
+ *
+ * Copyright (c) 2007, NLnet Labs. All rights reserved.
+ *
+ * This software is open source.
+ * 
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 
+ * Redistributions of source code must retain the above copyright notice,
+ * this list of conditions and the following disclaimer.
+ * 
+ * Redistributions in binary form must reproduce the above copyright notice,
+ * this list of conditions and the following disclaimer in the documentation
+ * and/or other materials provided with the distribution.
+ * 
+ * Neither the name of the NLNET LABS nor the names of its contributors may
+ * be used to endorse or promote products derived from this software without
+ * specific prior written permission.
+ * 
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED
+ * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
+ * PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE REGENTS OR CONTRIBUTORS BE
+ * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+ * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+ * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+ * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+ * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
+ * POSSIBILITY OF SUCH DAMAGE.
+ */
+
+/**
+ * \file
+ *
+ * This file contains the rrset cache.
+ */
+#include "config.h"
+#include "services/cache/rrset.h"
+#include "util/storage/slabhash.h"
+#include "util/config_file.h"
+#include "util/data/packed_rrset.h"
+#include "util/data/msgreply.h"
+
+struct rrset_cache* rrset_cache_create(struct config_file* cfg, 
+       struct alloc_cache* alloc)
+{
+       size_t slabs = (cfg?cfg->rrset_cache_slabs:HASH_DEFAULT_SLABS);
+       size_t startarray = HASH_DEFAULT_STARTARRAY;
+       size_t maxmem = (cfg?cfg->rrset_cache_size:HASH_DEFAULT_MAXMEM);
+
+       struct rrset_cache *r = (struct rrset_cache*)slabhash_create(slabs,
+               startarray, maxmem, ub_rrset_sizefunc, ub_rrset_compare,
+               ub_rrset_key_delete, rrset_data_delete, alloc);
+       return r;
+}
+
+void rrset_cache_delete(struct rrset_cache* r)
+{
+       if(!r) 
+               return;
+       slabhash_delete(&r->table);
+}
+
+struct rrset_cache* rrset_cache_adjust(struct rrset_cache *r, 
+       struct config_file* cfg, struct alloc_cache* alloc)
+{
+       if(!r || !cfg || cfg->rrset_cache_slabs != r->table.size ||
+               cfg->rrset_cache_size != slabhash_get_size(&r->table))
+       {
+               rrset_cache_delete(r);
+               r = rrset_cache_create(cfg, alloc);
+       }
+       return r;
+}
+
+void 
+rrset_cache_touch(struct rrset_cache* r, struct ub_packed_rrset_key* key,
+        hashvalue_t hash, rrset_id_t id)
+{
+       struct lruhash* table = slabhash_gettable(&r->table, hash);
+       /* 
+        * This leads to locking problems, deadlocks, if the caller is 
+        * holding any other rrset lock.
+        * Because a lookup through the hashtable does:
+        *      tablelock -> entrylock  (for that entry caller holds)
+        * And this would do
+        *      entrylock(already held) -> tablelock
+        * And if two threads do this, it results in deadlock.
+        * So, the caller must not hold entrylock.
+        */
+       lock_quick_lock(&table->lock);
+       /* we have locked the hash table, the item can still be deleted.
+        * because it could already have been reclaimed, but not yet set id=0.
+        * This is because some lruhash routines have lazy deletion.
+        * so, we must acquire a lock on the item to verify the id != 0.
+        * also, with hash not changed, we are using the right slab.
+        */
+       lock_rw_rdlock(&key->entry.lock);
+       if(key->id == id && key->entry.hash == hash) {
+               lru_touch(table, &key->entry);
+       }
+       lock_rw_unlock(&key->entry.lock);
+       lock_quick_unlock(&table->lock);
+}
+
+/** see if rrset needs to be updated in the cache */
+static int
+need_to_update_rrset(void* nd, void* cd, uint32_t timenow)
+{
+       struct packed_rrset_data* newd = (struct packed_rrset_data*)nd;
+       struct packed_rrset_data* cached = (struct packed_rrset_data*)cd;
+        /*      o if current RRset is more trustworthy - insert it */
+        if( newd->trust > cached->trust )
+                return 1;
+       /*      o item in cache has expired */
+       if( cached->ttl < timenow )
+               return 1;
+        /*      o same trust, but different in data - insert it */
+        if( newd->trust == cached->trust &&
+                !rrsetdata_equal(newd, cached))
+                return 1;
+        /*      o see if TTL is better than TTL in cache. */
+        /*        if so, see if rrset+rdata is the same */
+        /*        if so, update TTL in cache, even if trust is worse. */
+        if( newd->ttl > cached->ttl &&
+                rrsetdata_equal(newd, cached)) {
+               /* since all else is the same, use the best trust value */
+               if(newd->trust < cached->trust)
+                       newd->trust = cached->trust;
+                return 1;
+       }
+        return 0;
+}
+
+int 
+rrset_cache_update(struct rrset_cache* r, struct rrset_ref* ref,
+       struct alloc_cache* alloc, uint32_t timenow)
+{
+       struct lruhash_entry* e;
+       struct ub_packed_rrset_key* k = ref->key;
+       hashvalue_t h = k->entry.hash;
+       /* looks up item with a readlock - no editing! */
+       if((e=slabhash_lookup(&r->table, h, k, 0)) != 0) {
+               /* return id and key as they will be used in the cache
+                * since the lruhash_insert, if item already exists, deallocs
+                * the passed key in favor of the already stored key.
+                * because of the small gap (see below) this key ptr and id
+                * may prove later to be already deleted, which is no problem
+                * as it only makes a cache miss. 
+                */
+               ref->key = (struct ub_packed_rrset_key*)e->key;
+               ref->id = ref->key->id;
+               if(!need_to_update_rrset(k->entry.data, e->data, timenow)) {
+                       /* cache is superior, return that value */
+                       lock_rw_unlock(&e->lock);
+                       ub_packed_rrset_parsedelete(k, alloc);
+                       return 1;
+               }
+               lock_rw_unlock(&e->lock);
+               /* Go on and insert the passed item.
+                * small gap here, where entry is not locked.
+                * possibly entry is updated with something else.
+                * we then overwrite that with our data.
+                * this is just too bad, its cache anyway. */
+               /* use insert to update entry to manage lruhash
+                * cache size values nicely. */
+       }
+       slabhash_insert(&r->table, h, &k->entry, k->entry.data, alloc);
+       if(e)
+               return 1;
+       return 0;
+}
diff --git a/services/cache/rrset.h b/services/cache/rrset.h
new file mode 100644 (file)
index 0000000..313a694
--- /dev/null
@@ -0,0 +1,131 @@
+/*
+ * services/cache/rrset.h - Resource record set cache.
+ *
+ * Copyright (c) 2007, NLnet Labs. All rights reserved.
+ *
+ * This software is open source.
+ * 
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ * 
+ * Redistributions of source code must retain the above copyright notice,
+ * this list of conditions and the following disclaimer.
+ * 
+ * Redistributions in binary form must reproduce the above copyright notice,
+ * this list of conditions and the following disclaimer in the documentation
+ * and/or other materials provided with the distribution.
+ * 
+ * Neither the name of the NLNET LABS nor the names of its contributors may
+ * be used to endorse or promote products derived from this software without
+ * specific prior written permission.
+ * 
+ * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+ * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED
+ * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
+ * PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE REGENTS OR CONTRIBUTORS BE
+ * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+ * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+ * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+ * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+ * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
+ * POSSIBILITY OF SUCH DAMAGE.
+ */
+
+/**
+ * \file
+ *
+ * This file contains the rrset cache.
+ */
+
+#ifndef SERVICES_CACHE_RRSET_H
+#define SERVICES_CACHE_RRSET_H
+#include "util/storage/lruhash.h"
+#include "util/storage/slabhash.h"
+#include "util/data/packed_rrset.h"
+struct config_file;
+struct alloc_cache;
+struct rrset_ref;
+
+/**
+ * The rrset cache
+ * Thin wrapper around hashtable, like a typedef.
+ */
+struct rrset_cache {
+       /** uses partitioned hash table */
+       struct slabhash table;
+};
+
+/**
+ * Create rrset cache
+ * @param cfg: config settings or NULL for defaults.
+ * @param alloc: initial default rrset key allocation.
+ * @return: NULL on error.
+ */
+struct rrset_cache* rrset_cache_create(struct config_file* cfg, 
+       struct alloc_cache* alloc);
+
+/**
+ * Delete rrset cache
+ * @param r: rrset cache to delete.
+ */
+void rrset_cache_delete(struct rrset_cache* r);
+
+/**
+ * Adjust settings of the cache to settings from the config file.
+ * May purge the cache. May recreate the cache.
+ * There may be no threading or use by other threads.
+ * @param r: rrset cache to adjust (like realloc).
+ * @param cfg: config settings or NULL for defaults.
+ * @param alloc: initial default rrset key allocation.
+ * @return 0 on error, or new rrset cache pointer on success.
+ */
+struct rrset_cache* rrset_cache_adjust(struct rrset_cache* r, 
+       struct config_file* cfg, struct alloc_cache* alloc);
+
+/**
+ * Touch rrset, with given pointer and id.
+ * Caller may not hold a lock on ANY rrset, this could give deadlock.
+ *
+ * This routine is faster than a hashtable lookup:
+ *     o no bin_lock is acquired.
+ *     o no walk through the bin-overflow-list. 
+ *     o no comparison of the entry key to find it.
+ *
+ * @param r: rrset cache.
+ * @param key: rrset key. Marked recently used (if it was not deleted
+ *     before the lock is acquired, in that case nothing happens).
+ * @param hash: hash value of the item. Please read it from the key when
+ *     you have it locked. Used to find slab from slabhash.
+ * @param id: used to check that the item is unchanged and not deleted.
+ */
+void rrset_cache_touch(struct rrset_cache* r, struct ub_packed_rrset_key* key,
+       hashvalue_t hash, rrset_id_t id);
+
+/**
+ * Update an rrset in the rrset cache. Stores the information for later use.
+ * Will lookup if the rrset is in the cache and perform an update if necessary.
+ * If the item was present, and superior, references are returned to that.
+ * The passed item is then deallocated with rrset_parsedelete.
+ *
+ * A superior rrset is:
+ *     o rrset with better trust value.
+ *     o same trust value, different rdata, newly passed rrset is inserted.
+ * If rdata is the same, TTL in the cache is updated.
+ *
+ * @param r: the rrset cache.
+ * @param ref: reference (ptr and id) to the rrset. Pass reference setup for
+ *     the new rrset. The reference may be changed if the cached rrset is
+ *     superior.
+ *     Before calling the rrset is presumed newly allocated and changeable.
+ *     Afer calling you do not hold a lock, and the rrset is inserted in
+ *     the hashtable so you need a lock to change it.
+ * @param alloc: how to allocate (and deallocate) the special rrset key.
+ * @param timenow: current time (to see if ttl in cache is expired).
+ * @return: true if the passed reference is updated, false if it is unchanged.
+ */
+int rrset_cache_update(struct rrset_cache* r, struct rrset_ref* ref, 
+       struct alloc_cache* alloc, uint32_t timenow);
+
+#endif /* SERVICES_CACHE_RRSET_H */
index 3da4ed4c0caffe61bc4a5135102e265dc814e959..18e5a667b1ea641e287f01040bd1ca19ba1ecdc3 100644 (file)
@@ -139,6 +139,9 @@ read_header(FILE* in)
                the_time = t;
                the_pid = p;
                memset(threads, 0, 256*sizeof(int));
+               if(thrno >= 256) {
+                       fatal_exit("Thread number too big. %d", thrno);
+               }
                threads[thrno] = 1;
                have_values = 1;
                printf(" trace %d from pid %u on %s", thrno, 
@@ -332,7 +335,7 @@ static void search_cycle(struct lock_ref* visit, int level,
        /* check for cycle */
        if(detect_cycle(visit, from) && level != 0) {
                found_cycle(visit, level);
-               return;
+               fatal_exit("found lock order cycle");
        }
        /* recurse */
        if(!visit->lock->visited)
index 09275564f95a57aa354a360b350bdc7a1866a186..d03f5a9ced50e5490e2035d246f5fb56437e76e7 100644 (file)
@@ -62,7 +62,7 @@ static size_t test_sizefunc(void*, void*);
 /** comparefunc for lruhash */
 static int test_compfunc(void*, void*);
 /** delkey for lruhash */
-static void test_delkey(void*, void*);
+static void test_delkey(void*, void*, int);
 /** deldata for lruhash */
 static void test_deldata(void*, void*);
 /* --- end test representation --- */
@@ -524,8 +524,9 @@ static int test_compfunc(void* key1, void* key2)
        return -1;
 }
 
-static void test_delkey(void* key, void* ATTR_UNUSED(arg))
+static void test_delkey(void* key, void* ATTR_UNUSED(arg), int l)
 {
+       if(l) lock_rw_unlock(&((struct testkey*)key)->entry.lock);
        delkey((struct testkey*)key);
 }
 
index 1db0ed20c4c80d07aca80b54f8c1ad8fa7827109..c525e1df72f76522cc03b8f82ac53fd6f908bf95 100644 (file)
@@ -62,7 +62,7 @@ static size_t test_sizefunc(void*, void*);
 /** comparefunc for lruhash */
 static int test_compfunc(void*, void*);
 /** delkey for lruhash */
-static void test_delkey(void*, void*);
+static void test_delkey(void*, void*, int);
 /** deldata for lruhash */
 static void test_deldata(void*, void*);
 /* --- end test representation --- */
@@ -404,8 +404,9 @@ static int test_compfunc(void* key1, void* key2)
        return -1;
 }
 
-static void test_delkey(void* key, void* ATTR_UNUSED(arg))
+static void test_delkey(void* key, void* ATTR_UNUSED(arg), int l)
 {
+       if(l) lock_rw_unlock(&((struct slabtestkey*)key)->entry.lock);
        delkey((struct slabtestkey*)key);
 }
 
index a721f5af2e4f1f936ac4cf8a21d3be00eaa6441a..f75655ee99d02beee0404e0e3e30fb5a1d53d1c2 100644 (file)
@@ -496,9 +496,11 @@ msgreply_sizefunc(void* k, void* d)
 }
 
 void 
-query_entry_delete(void *k, void* ATTR_UNUSED(arg))
+query_entry_delete(void *k, void* ATTR_UNUSED(arg), int is_locked)
 {
        struct msgreply_entry* q = (struct msgreply_entry*)k;
+       if(is_locked)
+               lock_rw_unlock(&q->entry.lock);
        lock_rw_destroy(&q->entry.lock);
        query_info_clear(&q->key);
        free(q);
index 4e712d6615e58c40c332e94fa1187e029480e356..566a8707e05ccc482db15d7f552886496d777c28 100644 (file)
@@ -239,7 +239,7 @@ void query_info_clear(struct query_info* m);
 size_t msgreply_sizefunc(void* k, void* d);
 
 /** delete msgreply_entry key structure */
-void query_entry_delete(void *q, void* arg);
+void query_entry_delete(void *q, void* arg, int is_locked);
 
 /** delete reply_info data structure */
 void reply_info_delete(void* d, void* arg);
index 0308157c9473dca28b1d6eadbaf9d27e41cbcdee..1393c58c2658c686e40d39fa8bf3324dde139095 100644 (file)
@@ -101,11 +101,13 @@ ub_rrset_compare(void* k1, void* k2)
 }
 
 void 
-ub_rrset_key_delete(void* key, void* userdata)
+ub_rrset_key_delete(void* key, void* userdata, int is_locked)
 {
        struct ub_packed_rrset_key* k = (struct ub_packed_rrset_key*)key;
        struct alloc_cache* a = (struct alloc_cache*)userdata;
        k->id = 0;
+       if(is_locked)
+               lock_rw_unlock(&k->entry.lock);
        free(k->rk.dname);
        k->rk.dname = NULL;
        alloc_special_release(a, k);
index f5ced474c21710b2bd4444b09cf6f7e0ee549a96..caa00b9e8d5151ee3c3499d3c06ced500cd12206 100644 (file)
@@ -256,8 +256,11 @@ int rrsetdata_equal(struct packed_rrset_data* d1, struct packed_rrset_data* d2);
  * Old key to be deleted. RRset keys are recycled via alloc.
  * @param key: struct ub_packed_rrset_key*.
  * @param userdata: alloc structure to use for recycling.
+ * @param is_locked: if the key is locked, the id is set to 0 while it is
+ *     still locked. So that other threads, after acquiring a lock always
+ *     get the correct value, in this case the 0 deleted-special value.
  */
-void ub_rrset_key_delete(void* key, void* userdata);
+void ub_rrset_key_delete(void* key, void* userdata, int is_locked);
 
 /**
  * Old data to be deleted.
index e80557faa92a21e53c695f50a7541584d5a2c9c9..e82f34777245431751716ab2280a6095fe1c8d66 100644 (file)
@@ -45,6 +45,7 @@
 #include "util/data/msgreply.h"
 #include "util/data/msgparse.h"
 struct alloc_cache;
+struct rrset_cache;
 struct config_file;
 struct slabhash;
 struct query_info;
@@ -67,7 +68,7 @@ struct module_env {
        /** shared message cache */
        struct slabhash* msg_cache;
        /** shared rrset cache */
-       struct slabhash* rrset_cache;
+       struct rrset_cache* rrset_cache;
 
        /* --- services --- */
        /** 
index 43eec5e61d2afac8e6b4be39578bba86e538addf..8068c36b4c544f917c94a57af2c4c835813b2e42 100644 (file)
@@ -102,7 +102,7 @@ bin_delete(struct lruhash* table, struct lruhash_bin* bin)
        while(p) {
                np = p->overflow_next;
                d = p->data;
-               (*table->delkeyfunc)(p->key, table->cb_arg);
+               (*table->delkeyfunc)(p->key, table->cb_arg, 0);
                (*table->deldatafunc)(d, table->cb_arg);
                p = np;
        }
@@ -313,7 +313,7 @@ lruhash_insert(struct lruhash* table, hashvalue_t hash,
                /* if so: update data - needs a writelock */
                table->space_used += need_size -
                        (*table->sizefunc)(found->key, found->data);
-               (*table->delkeyfunc)(entry->key, cb_arg);
+               (*table->delkeyfunc)(entry->key, cb_arg, 0);
                lru_touch(table, found);
                lock_rw_wrlock(&found->lock);
                (*table->deldatafunc)(found->data, cb_arg);
@@ -331,8 +331,7 @@ lruhash_insert(struct lruhash* table, hashvalue_t hash,
        while(reclaimlist) {
                struct lruhash_entry* n = reclaimlist->overflow_next;
                void* d = reclaimlist->data;
-               lock_rw_unlock(&reclaimlist->lock);
-               (*table->delkeyfunc)(reclaimlist->key, cb_arg);
+               (*table->delkeyfunc)(reclaimlist->key, cb_arg, 1);
                (*table->deldatafunc)(d, cb_arg);
                reclaimlist = n;
        }
@@ -383,9 +382,8 @@ lruhash_remove(struct lruhash* table, hashvalue_t hash, void* key)
        lock_rw_wrlock(&entry->lock);
        lock_quick_unlock(&bin->lock);
        /* finish removal */
-       lock_rw_unlock(&entry->lock);
        d = entry->data;
-       (*table->delkeyfunc)(entry->key, table->cb_arg);
+       (*table->delkeyfunc)(entry->key, table->cb_arg, 1);
        (*table->deldatafunc)(d, table->cb_arg);
 }
 
index 6257559b72e3343338b5cef313a35f5047dd96e1..f2053c6568fa541fe3dd896e0fc13aea96338e99 100644 (file)
@@ -123,8 +123,13 @@ typedef size_t (*lruhash_sizefunc_t)(void*, void*);
 /** type of function that compares two keys. return 0 if equal. */
 typedef int (*lruhash_compfunc_t)(void*, void*);
 
-/** old keys is deleted. This function is called: func(key, userarg) */
-typedef void (*lruhash_delkeyfunc_t)(void*, void*);
+/** old keys are deleted. 
+ * If is_locked is set, then the routine must unlock the item before deletion.
+ * If is_locked is not set, then this item is not locked. This allows the 
+ * routine to perform operations within the critical region of the lock 
+ * of the key. The critical region has been locked before the delete happened.
+ * This function is called: func(key, userarg, is_locked) */
+typedef void (*lruhash_delkeyfunc_t)(void*, void*, int);
 
 /** old data is deleted. This function is called: func(data, userarg). */
 typedef void (*lruhash_deldatafunc_t)(void*, void*);
@@ -267,6 +272,13 @@ void lruhash_insert(struct lruhash* table, hashvalue_t hash,
 struct lruhash_entry* lruhash_lookup(struct lruhash* table, hashvalue_t hash, 
        void* key, int wr);
 
+/**
+ * Touch entry, so it becomes the most recently used in the LRU list.
+ * Caller must hold hash table lock. The entry must be inserted already.
+ * @param table: hash table.
+ * @param entry: entry to make first in LRU.
+ */
+void lru_touch(struct lruhash* table, struct lruhash_entry* entry);
 
 /************************* Internal functions ************************/
 /*** these are only exposed for unit tests. ***/
@@ -337,14 +349,6 @@ void reclaim_space(struct lruhash* table, struct lruhash_entry** list);
  */
 void table_grow(struct lruhash* table);
 
-/**
- * Touch entry, so it becomes the most recently used in the LRU list.
- * Caller must hold hash table lock. The entry must be inserted already.
- * @param table: hash table.
- * @param entry: entry to make first in LRU.
- */
-void lru_touch(struct lruhash* table, struct lruhash_entry* entry);
-
 /**
  * Put entry at front of lru. entry must be unlinked from lru.
  * Caller must hold hash table lock.
index 1a31df41d08d266c56135a7c9f171ff12df1182e..61393c1e191aef086bc712982ae63c2a3ab150a4 100644 (file)
@@ -143,3 +143,8 @@ size_t slabhash_get_size(struct slabhash* sl)
        }
        return total;
 }
+
+struct lruhash* slabhash_gettable(struct slabhash* sl, hashvalue_t hash)
+{
+       return sl->array[slab_idx(sl, hash)];
+}
index a644ffe90a451e5c1896cb49ef26221238964d0f..d530d7a1e50fa50001a9d90f9a2cc790dba25b2b 100644 (file)
@@ -145,4 +145,12 @@ void slabhash_status(struct slabhash* table, const char* id, int extended);
  */
 size_t slabhash_get_size(struct slabhash* table);
 
+/**
+ * Get lruhash table for a given hash value
+ * @param table: slabbed hash table.
+ * @param hash: hash value.
+ * @return the lru hash table.
+ */
+struct lruhash* slabhash_gettable(struct slabhash* table, hashvalue_t hash);
+
 #endif /* UTIL_STORAGE_SLABHASH_H */