int32_t info_code;
const char *extra_text;
};
+struct kr_cache_top_context {
+ uint64_t bloom[4];
+ uint32_t cnt;
+};
struct kr_request {
struct kr_context *ctx;
knot_pkt_t *answer;
uint8_t rule_score_apply;
uint8_t rule_score_log;
struct kr_rplan rplan;
+ struct kr_cache_top_context cache_top_context;
trace_log_f trace_log;
trace_callback_f trace_finish;
int vars_ref;
struct kr_rule_fwd_flags
typedef kr_rule_fwd_flags_t
struct kr_extended_error
+ struct kr_cache_top_context
struct kr_request
enum kr_rank
typedef kr_cdb_pt
qry->flags.TCP = false;
}
qr_task_step(task, NULL, NULL);
+
defer_sample_restart();
} else {
kr_assert(task->ctx->source.session == session);
const struct kr_cdb_api *api; /**< Storage engine */
struct kr_cdb_stats stats;
uint32_t ttl_min, ttl_max; /**< TTL limits; enforced primarily in iterator actually. */
- union kr_cache_top top;
+ struct kr_cache_top top;
/* A pair of stamps for detection of real-time shifts during runtime. */
struct timeval checkpoint_walltime; /**< Wall time on the last check-point. */
} txn;
bool is_cache; /**< cache vs. rules; from struct kr_cdb_opts::is_cache */
- union kr_cache_top *top; // TODO remove
+ struct kr_cache_top *top; // TODO remove
/* Cached part of struct stat for data.mdb. */
dev_t st_dev;
#define TICK_MSEC 1000
#define BASE_PRICE (((kru_price_t)1) << (KRU_PRICE_BITS - 16)) // increment by ones (16-bit)
- // -> instant limit: ~2^16
+ // -> instant limit: ~2^16 // TODO -> 2^14 ?
#define MAX_DECAY (BASE_PRICE / 2) // per sec
// -> rate limit: 1/2 per sec (more frequent accesses are incomparable)
- // -> half-life: ~ 25h 14min
+ // -> half-life: ~ 25h 14min // TODO -> 5h ?
struct top_data {
uint32_t version;
_Alignas(64) uint8_t kru[];
};
+
static inline uint32_t ticks_now(void)
{
// TODO use clock_gettime directly or maintain time offset
return kr_now() / TICK_MSEC; // not working over reboots
}
-int kr_cache_top_init(union kr_cache_top *top, char *mmap_file, size_t cache_size) {
+static inline bool first_access_ro(struct kr_cache_top_context *ctx, kru_hash_t hash) {
+ // struct kr_cache_top_context { uint64_t bloom[4]; }
+ static_assert(sizeof(((struct kr_cache_top_context *)0)->bloom[0]) * 8 == 64);
+ static_assert(sizeof(((struct kr_cache_top_context *)0)->bloom) * 8 == 64 * 4);
+ // expected around 16 unique cache accesses per request context;
+ // prob. of collision of the 16th unique access with the preceeding ones: 1/510;
+ // 32nd access: 1/45; 64th access: 1/6
+
+ uint8_t *h = (uint8_t *)&hash;
+ static_assert(sizeof(kru_hash_t) >= 4);
+
+ bool accessed = 1ull &
+ (ctx->bloom[0] >> (h[0] % 64)) &
+ (ctx->bloom[1] >> (h[1] % 64)) &
+ (ctx->bloom[2] >> (h[2] % 64)) &
+ (ctx->bloom[3] >> (h[3] % 64));
+
+ return !accessed;
+}
+
+static inline bool first_access(struct kr_cache_top_context *ctx, kru_hash_t hash) {
+ if (!first_access_ro(ctx, hash)) return false;
+
+ uint8_t *h = (uint8_t *)&hash;
+ static_assert(sizeof(kru_hash_t) >= 4);
+
+ ctx->bloom[0] |= 1ull << (h[0] % 64);
+ ctx->bloom[1] |= 1ull << (h[1] % 64);
+ ctx->bloom[2] |= 1ull << (h[2] % 64);
+ ctx->bloom[3] |= 1ull << (h[3] % 64);
+
+ kr_assert(!first_access_ro(ctx, hash));
+
+ if (++ctx->cnt > 16) {
+ VERBOSE_LOG("BLOOM overfull (%d unique accesses)\n", ctx->cnt);
+ }
+
+ return true;
+}
+
+
+int kr_cache_top_init(struct kr_cache_top *top, char *mmap_file, size_t cache_size) {
size_t size = 0, capacity_log = 0;
VERBOSE_LOG("INIT, cache size %d\n", cache_size);
}
VERBOSE_LOG("INIT mmapped_init\n");
- int state = mmapped_init(&top->mmapped, mmap_file, size, &header, header_size, true); // allocates top->data
+ int state = mmapped_init(&top->mmapped, mmap_file, size, &header, header_size, true);
+ top->data = top->mmapped.mem;
bool using_existing = false;
// try using existing data
if (!KRU.check_size((struct kru *)top->data->kru, top->mmapped.size - offsetof(struct top_data, kru))) {
VERBOSE_LOG("INIT reset, wrong size\n");
state = mmapped_init_reset(&top->mmapped, mmap_file, size, &header, header_size);
+ top->data = top->mmapped.mem;
} else {
using_existing = true;
VERBOSE_LOG("INIT finish existing\n");
if (state < 0) goto fail;
kr_assert(state == 0);
+ top->ctx = NULL;
kr_log_info(CACHE, "Cache top initialized %s (%s).\n",
using_existing ? "using existing data" : "as empty",
(kru_using_avx2() ? "AVX2" : "generic"));
fail:
VERBOSE_LOG("INIT error, deinit\n");
- mmapped_deinit(&top->mmapped);
+ kr_cache_top_deinit(top);
kr_log_crit(SYSTEM, "Initialization of cache top failed.\n");
return state;
}
-void kr_cache_top_deinit(union kr_cache_top *top) {
- mmapped_deinit(&top->mmapped); // sets top->data to NULL
+void kr_cache_top_deinit(struct kr_cache_top *top) {
+ top->data = NULL;
+ mmapped_deinit(&top->mmapped);
}
/* text mode: '\0' -> '|'
return str;
}
-void kr_cache_top_access(union kr_cache_top *top, void *key, size_t len, char *debug_label)
+void kr_cache_top_access(struct kr_cache_top *top, void *key, size_t len, char *debug_label)
{
- VERBOSE_LOG("ACCESS %-19s %s\n", debug_label, str_key(key, len));
- KRU.load_bytes((struct kru *)&top->data->kru, ticks_now(), (uint8_t *)key, len, top->data->base_price);
+ kru_hash_t hash = KRU.hash_bytes((struct kru *)&top->data->kru, (uint8_t *)key, len);
+ const bool unique = top->ctx ? first_access(top->ctx, hash) : true;
+ if (unique) {
+ KRU.load_hash((struct kru *)&top->data->kru, ticks_now(), hash, top->data->base_price);
+ }
+ VERBOSE_LOG("ACCESS %-19s %s%s%s\n", debug_label, str_key(key, len), unique ? "" : " (SKIP)", top->ctx ? "" : "(NO_CONTEXT, PASS)");
}
// temporal logging one level under _access
-void kr_cache_top_access_cdb(union kr_cache_top *top, void *key, size_t len, char *debug_label)
+void kr_cache_top_access_cdb(struct kr_cache_top *top, void *key, size_t len, char *debug_label)
{
- VERBOSE_LOG("ACCESS %-17s %s\n", debug_label, str_key(key, len));
+ // VERBOSE_LOG("ACCESS %-17s %s\n", debug_label, str_key(key, len));
+}
+
+struct kr_cache_top_context *kr_cache_top_context_switch(struct kr_cache_top *top,
+ struct kr_cache_top_context *new_ctx, char *debug_label)
+{
+ struct kr_cache_top_context *old_ctx = top->ctx;
+ top->ctx = new_ctx;
+ return old_ctx;
}
-uint16_t kr_cache_top_load(union kr_cache_top *top, void *key, size_t len) {
- uint16_t load = KRU.load_bytes((struct kru *)&top->data->kru, ticks_now(), (uint8_t *)key, len, 0);
+uint16_t kr_cache_top_load(struct kr_cache_top *top, void *key, size_t len) {
+ kru_hash_t hash = KRU.hash_bytes((struct kru *)&top->data->kru, (uint8_t *)key, len);
+ uint16_t load = KRU.load_hash((struct kru *)&top->data->kru, ticks_now(), hash, 0);
VERBOSE_LOG("LOAD %s -> %d\n", str_key(key, len), load);
return load;
#pragma once
#include "lib/mmapped.h"
-union kr_cache_top {
+struct kr_cache_top {
struct mmapped mmapped;
struct top_data *data;
+ struct kr_cache_top_context *ctx;
};
-static_assert(&((union kr_cache_top *)0)->mmapped.mem == (void *)&((union kr_cache_top *)0)->data);
+struct kr_cache_top_context {
+ uint64_t bloom[4];
+ uint32_t cnt; // TODO remove this (and propagate to kres-gen)
+};
+
+
+KR_EXPORT
+int kr_cache_top_init(struct kr_cache_top *top, char *mmap_file, size_t cache_size);
KR_EXPORT
-int kr_cache_top_init(union kr_cache_top *top, char *mmap_file, size_t cache_size);
+void kr_cache_top_deinit(struct kr_cache_top *top);
KR_EXPORT
-void kr_cache_top_deinit(union kr_cache_top *top);
+void kr_cache_top_access_cdb(struct kr_cache_top *top, void *key, size_t len, char *debug_label); // temporal, TODO remove
KR_EXPORT
-void kr_cache_top_access_cdb(union kr_cache_top *top, void *key, size_t len, char *debug_label); // temporal, TODO remove
+void kr_cache_top_access(struct kr_cache_top *top, void *key, size_t len, char *debug_label);
KR_EXPORT
-void kr_cache_top_access(union kr_cache_top *top, void *key, size_t len, char *debug_label);
+uint16_t kr_cache_top_load(struct kr_cache_top *top, void *key, size_t len);
+// ctx has to be kept valid until next call
KR_EXPORT
-uint16_t kr_cache_top_load(union kr_cache_top *top, void *key, size_t len);
+struct kr_cache_top_context *kr_cache_top_context_switch(struct kr_cache_top *top, struct kr_cache_top_context *ctx, char *debug_label);
// Exponential decay always uses randomized rounding on 32 bits.
typedef uint32_t kru_price_t;
+typedef uint64_t kru_hash_t;
+
#define KRU_PRICE_BITS (8 * sizeof(kru_price_t))
// maximal allowed sum of prices without limiting
uint8_t namespace, uint8_t key[static 16], uint8_t *prefixes, kru_price_t *prices, size_t queries_cnt, uint16_t *loads_out);
// TODO
- uint16_t (*load_bytes)(struct kru *kru, uint32_t time_now, uint8_t *key, uint8_t key_size, kru_price_t price);
+ /// Compute 64-bit hash to be used in load_hash.
+ kru_hash_t (*hash_bytes)(struct kru *kru, uint8_t *key, size_t key_size);
+ uint16_t (*load_hash)(struct kru *kru, uint32_t time_now, kru_hash_t hash, kru_price_t price);
};
// The functions are stored this way to make it easier to switch
#include "contrib/ucw/lib.h"
#include "libdnssec/error.h"
#include "libdnssec/random.h"
-typedef uint64_t hash_t;
#if USE_AES
/// 4-8 rounds should be an OK choice, most likely.
#define AES_ROUNDS 4
static size_t kru_get_size(int capacity_log)
{
uint32_t loads_bits = capacity2loads(capacity_log);
- if (8 * sizeof(hash_t) < TABLE_COUNT * loads_bits
+ if (8 * sizeof(kru_hash_t) < TABLE_COUNT * loads_bits
+ 8 * sizeof(((struct kru *)0)->load_cls[0]->ids[0])) {
assert(false);
return 0;
}
uint32_t loads_bits = capacity2loads(capacity_log);
- if (8 * sizeof(hash_t) < TABLE_COUNT * loads_bits
+ if (8 * sizeof(kru_hash_t) < TABLE_COUNT * loads_bits
+ 8 * sizeof(((struct kru *)0)->load_cls[0]->ids[0])) {
assert(false);
return false;
static inline void kru_limited_prefetch(struct kru *kru, uint32_t time_now, uint8_t key[static 16], kru_price_t price, struct query_ctx *ctx)
{
// Obtain hash of *buf.
- hash_t hash;
+ kru_hash_t hash;
+ static_assert(sizeof(kru_hash_t) * 8 <= 64);
#if !USE_AES
hash = SipHash(&kru->hash_key, SIPHASH_RC, SIPHASH_RF, key, 16);
#else
static inline void kru_limited_prefetch_prefix(struct kru *kru, uint32_t time_now, uint8_t namespace, uint8_t key[static 16], uint8_t prefix, kru_price_t price, struct query_ctx *ctx)
{
// Obtain hash of *buf.
- hash_t hash;
+ kru_hash_t hash;
+ static_assert(sizeof(kru_hash_t) * 8 <= 64);
#if !USE_AES
{
ctx->id = hash;
}
-/// Phase 1/3 of a query -- hash, prefetch, ctx init. Based on one 16-byte key.
-static inline void kru_limited_prefetch_bytes(struct kru *kru, uint32_t time_now, uint8_t *key, size_t key_size, kru_price_t price, struct query_ctx *ctx)
-{
+static kru_hash_t kru_hash_bytes(struct kru *kru, uint8_t *key, size_t key_size) {
// Obtain hash of *buf.
- hash_t hash;
+ kru_hash_t hash;
+ static_assert(sizeof(kru_hash_t) * 8 <= 64);
+
#if !USE_AES
hash = SipHash(&kru->hash_key, SIPHASH_RC, SIPHASH_RF, key, key_size);
#else
}
#endif
+ return hash;
+}
+
+/// Phase 1/3 of a query -- hash, prefetch, ctx init. Based on arbitrary-length byte-stream.
+static inline void kru_limited_prefetch_hash(struct kru *kru, uint32_t time_now, kru_hash_t hash, kru_price_t price, struct query_ctx *ctx)
+{
// Choose the cache-lines to operate on
const uint32_t loads_mask = (1 << kru->loads_bits) - 1;
// Fetch the two cache-lines in parallel before we really touch them.
return max_load;
}
-static uint16_t kru_load_bytes(struct kru *kru, uint32_t time_now, uint8_t *key, uint8_t key_size, kru_price_t price)
+static uint16_t kru_load_hash(struct kru *kru, uint32_t time_now, kru_hash_t hash, kru_price_t price)
{
struct query_ctx ctx;
- kru_limited_prefetch_bytes(kru, time_now, key, key_size, price, &ctx);
+ kru_limited_prefetch_hash(kru, time_now, hash, price, &ctx);
kru_limited_fetch(kru, &ctx);
if (price) {
.limited_multi_prefix_or = kru_limited_multi_prefix_or, \
.load_multi_prefix = kru_load_multi_prefix, \
.load_multi_prefix_max = kru_load_multi_prefix_max, \
- .load_bytes = kru_load_bytes, \
+ .load_hash = kru_load_hash, \
+ .hash_bytes = kru_hash_bytes, \
}
{
kr_require(request && transport && packet);
struct kr_rplan *rplan = &request->rplan;
+ kr_cache_top_context_switch(&the_resolver->cache.top, &request->cache_top_context, "produce");
/* No query left for resolution */
if (kr_rplan_empty(rplan)) {
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "produce");
return KR_STATE_FAIL;
}
}
switch(state) {
- case KR_STATE_FAIL: return KR_STATE_FAIL;
- case KR_STATE_DONE: return KR_STATE_PRODUCE;
+ case KR_STATE_FAIL:
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "produce");
+ return KR_STATE_FAIL;
+ case KR_STATE_DONE:
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "produce");
+ return KR_STATE_PRODUCE;
default: break;
}
VERBOSE_MSG(qry, "=> resuming yielded answer\n");
* this is normally not required, and incurs another cache lookups for cached answer. */
if (qry->flags.ALWAYS_CUT) { // LATER: maybe the flag doesn't work well anymore
switch(zone_cut_check(request, qry, packet)) {
- case KR_STATE_FAIL: return KR_STATE_FAIL;
- case KR_STATE_DONE: return KR_STATE_PRODUCE;
+ case KR_STATE_FAIL:
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "produce");
+ return KR_STATE_FAIL;
+ case KR_STATE_DONE:
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "produce");
+ return KR_STATE_PRODUCE;
default: break;
}
}
}
}
switch(request->state) {
- case KR_STATE_FAIL: return request->state;
+ case KR_STATE_FAIL:
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "produce");
+ return request->state;
case KR_STATE_CONSUME: break;
case KR_STATE_DONE:
default: /* Current query is done */
kr_rplan_pop(rplan, qry);
}
ITERATE_LAYERS(request, qry, reset);
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "produce");
return kr_rplan_empty(rplan) ? KR_STATE_DONE : KR_STATE_PRODUCE;
}
/* At this point we need to send a query upstream to proceed towards success. */
if (qry->stype == KNOT_RRTYPE_ANY ||
!knot_wire_get_rd(request->qsource.packet->wire)) {
VERBOSE_MSG(qry, "=> qtype is ANY or RD=0, bail out\n");
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "produce");
return KR_STATE_FAIL;
}
/* Update zone cut, spawn new subrequests. */
int state = zone_cut_check(request, qry, packet);
switch(state) {
- case KR_STATE_FAIL: return KR_STATE_FAIL;
- case KR_STATE_DONE: return KR_STATE_PRODUCE;
+ case KR_STATE_FAIL:
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "produce");
+ return KR_STATE_FAIL;
+ case KR_STATE_DONE:
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "produce");
+ return KR_STATE_PRODUCE;
default: break;
}
}
kr_request_set_extended_error(request, KNOT_EDNS_EDE_NREACH_AUTH, msg);
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "produce");
return KR_STATE_FAIL;
} else {
/* FIXME: This is probably quite inefficient:
* we go through the whole qr_task_step loop just because of the serve_stale
* module which might not even be loaded. */
qry->flags.NO_NS_FOUND = true;
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "produce");
return KR_STATE_PRODUCE;
}
}
uint16_t type = (*transport)->protocol == KR_TRANSPORT_RESOLVE_A ? KNOT_RRTYPE_A : KNOT_RRTYPE_AAAA;
ns_resolve_addr(qry, qry->request, *transport, type);
ITERATE_LAYERS(request, qry, reset);
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "produce");
return KR_STATE_PRODUCE;
}
* kr_resolve_checkout().
*/
qry->timestamp_mono = kr_now();
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "produce");
return request->state;
}
int kr_resolve_consume(struct kr_request *request, struct kr_transport **transport, knot_pkt_t *packet)
{
struct kr_rplan *rplan = &request->rplan;
+ kr_cache_top_context_switch(&the_resolver->cache.top, &request->cache_top_context, "consume");
/* Empty resolution plan, push packet as the new query */
if (packet && kr_rplan_empty(rplan)) {
- return resolve_query(request, packet);
+ int ret = resolve_query(request, packet);
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "consume");
+ return ret;
}
/* Different processing for network error */
/* Check overall resolution time */
if (kr_now() - qry->creation_time_mono >= KR_RESOLVE_TIME_LIMIT) {
kr_query_inform_timeout(request, qry);
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "consume");
return KR_STATE_FAIL;
}
bool tried_tcp = (qry->flags.TCP);
- if (!packet || packet->size == 0)
+ if (!packet || packet->size == 0) {
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "consume");
return KR_STATE_PRODUCE;
+ }
/* Packet cleared, derandomize QNAME. */
knot_dname_t *qname_raw = kr_pkt_qname_raw(packet);
if (transport && !qry->flags.CACHED) {
if (!(request->state & KR_STATE_FAIL)) {
/* Do not complete NS address resolution on soft-fail. */
- if (kr_fails_assert(packet->wire))
+ if (kr_fails_assert(packet->wire)) {
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "consume");
return KR_STATE_FAIL;
+ }
const int rcode = knot_wire_get_rcode(packet->wire);
if (rcode != KNOT_RCODE_SERVFAIL && rcode != KNOT_RCODE_REFUSED) {
qry->flags.AWAIT_IPV6 = false;
}
if (!qry->flags.NO_NS_FOUND) {
qry->flags.NO_NS_FOUND = true;
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "consume");
return KR_STATE_PRODUCE;
}
"OLX2: delegation ", cut_buf);
}
kr_request_set_extended_error(request, KNOT_EDNS_EDE_NREACH_AUTH, msg);
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "consume");
return KR_STATE_FAIL;
}
} else {
/* Pop query if resolved. */
if (request->state == KR_STATE_YIELD) { // NOLINT(bugprone-branch-clone)
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "consume");
return KR_STATE_PRODUCE; /* Requery */
} else if (qry->flags.RESOLVED) {
kr_rplan_pop(rplan, qry);
} else if (!tried_tcp && (qry->flags.TCP)) {
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "consume");
return KR_STATE_PRODUCE; /* Requery over TCP */
} else { /* Clear query flags for next attempt */
qry->flags.CACHED = false;
if (qry->flags.FORWARD || qry->flags.STUB
/* Probably CPU exhaustion attempt, so do not retry. */
|| qry->vld_limit_crypto_remains <= 0) {
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "consume");
return KR_STATE_FAIL;
}
/* Other servers might not have broken DNSSEC. */
qry->flags.DNSSEC_BOGUS = false;
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "consume");
return KR_STATE_PRODUCE;
}
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "consume");
return kr_rplan_empty(&request->rplan) ? KR_STATE_DONE : KR_STATE_PRODUCE;
}
int kr_resolve_finish(struct kr_request *request, int state)
{
+ kr_cache_top_context_switch(&the_resolver->cache.top, &request->cache_top_context, "finish");
request->state = state;
/* Finalize answer and construct whole wire-format (unless dropping). */
knot_pkt_t *answer = kr_request_ensure_answer(request);
request->trace_finish = NULL;
request->trace_log = NULL;
+ kr_cache_top_context_switch(&the_resolver->cache.top, NULL, "finish");
return KR_STATE_DONE;
}
bool auth_validated; /**< see answ_validated ^^ ; TODO */
bool stale_accounted;
bool ratelimited; /**< this request shot over the rate limit */
-
/** Overall rank for the request.
*
* Values from kr_rank, currently just KR_RANK_SECURE and _INITIAL.
uint8_t rule_score_log;
struct kr_rplan rplan;
+ struct kr_cache_top_context cache_top_context; // divided into two cache lines, TODO change placement (+update kres-gen)
trace_log_f trace_log; /**< Logging tracepoint */
trace_callback_f trace_finish; /**< Request finish tracepoint */
int vars_ref; /**< Reference to per-request variable table. LUA_NOREF if not set. */
}
struct kr_cache *cache = &qry->request->ctx->cache;
+ struct kr_cache_top_context *old_cache_top_ctx =
+ kr_cache_top_context_switch(&the_resolver->cache.top, &qry->request->cache_top_context, "update_rtt");
uint8_t *address = ip_to_bytes(&transport->address, transport->address_len);
/* This construct is a bit racy since the global state may change
qry->id, ns_name, ns_str ? ns_str : "", zonecut_str,
rtt, new_rtt_state.srtt, new_rtt_state.variance);
}
+
+ kr_cache_top_context_switch(&the_resolver->cache.top, old_cache_top_ctx, "update_rtt");
}
/// Update rtt_state (including caching) after a server timed out.
if (transport->timeout_capped)
return;
+ struct kr_cache_top_context *old_cache_top_ctx =
+ kr_cache_top_context_switch(&the_resolver->cache.top, &qry->request->cache_top_context, "server_timeout");
+
const uint8_t *address = ip_to_bytes(&transport->address, transport->address_len);
if (transport->address_len == sizeof(struct in6_addr))
no6_timed_out(qry, address);
} else {
kr_cache_commit(cache); // Avoid any risk of long transaction.
}
+
+ kr_cache_top_context_switch(&the_resolver->cache.top, old_cache_top_ctx, "server_timeout");
}
// Not everything can be checked in nice ways like static_assert()
static __attribute__((constructor)) void test_RTT_consts(void)
}
// TODO this is just an example, make this more clever
-category_t kr_gc_categorize(union kr_cache_top *top, gc_record_info_t * info, void *key, size_t key_len)
+category_t kr_gc_categorize(struct kr_cache_top *top, gc_record_info_t * info, void *key, size_t key_len)
{
category_t res;
#define CATEGORIES 100 // number of categories
-category_t kr_gc_categorize(union kr_cache_top *top, gc_record_info_t * info, void *key, size_t key_len);
+category_t kr_gc_categorize(struct kr_cache_top *top, gc_record_info_t * info, void *key, size_t key_len);
typedef struct {
size_t categories_sizes[CATEGORIES];
size_t records;
- union kr_cache_top *top;
+ struct kr_cache_top *top;
} ctx_compute_categories_t;
int cb_compute_categories(const knot_db_val_t * key, gc_record_info_t * info,
size_t cfg_temp_keys_space;
size_t used_space;
size_t oversize_records;
- union kr_cache_top *top;
+ struct kr_cache_top *top;
} ctx_delete_categories_t;
int cb_delete_categories(const knot_db_val_t * key, gc_record_info_t * info,