* For now only root zone import is supported.
*
* Import process consists of two stages.
- * 1) Zone file parsing.
- * 2) Import of parsed entries into the cache.
+ * 1) Zone file parsing and (optionally) ZONEMD verification.
+ * 2) DNSSEC validation and storage in cache.
*
* These stages are implemented as two separate functions
- * (zi_zone_import and zi_zone_process) which runs sequentially with the
+ * (zi_zone_import and zi_zone_process) which run sequentially with a
* pause between them. This is done because resolver is a single-threaded
* application, so it can't process user's requests during the whole import
* process. Separation into two stages allows to reduce the
* continuous time interval when resolver can't serve user requests.
- * Since root zone isn't large it is imported as single
- * chunk. If it would be considered as necessary, import stage can be
- * split into shorter stages.
- *
- * zi_zone_import() uses libzscanner to parse zone file.
- * Parsed records are stored to internal storage from where they are imported to
- * cache during the second stage.
- *
- * zi_zone_process() imports parsed resource records to cache.
- * It imports rrset by creating request that will never be sent to upstream.
- * After request creation resolver creates pseudo-answer which must contain
- * all necessary data for validation. Then resolver process answer as if he had
- * been received from network.
+ * Since root zone isn't large, it is imported as single chunk.
*/
+#include "daemon/zimport.h"
+
#include <inttypes.h> /* PRIu64 */
#include <limits.h>
+#include <math.h>
#include <stdlib.h>
+#include <sys/stat.h>
#include <uv.h>
-#include <ucw/mempool.h>
+
+#include "contrib/ucw/mempool.h"
#include <libknot/rrset.h>
#include <libzscanner/scanner.h>
-#include "lib/utils.h"
-#include "lib/dnssec/ta.h"
+#include <libknot/version.h>
+#define ENABLE_ZONEMD (KNOT_VERSION_HEX >= 0x030100)
+#if ENABLE_ZONEMD
+ #include <libdnssec/digest.h>
+
+ #if KNOT_VERSION_HEX < 0x030200
+ #define KNOT_ZONEMD_ALGORITHM_SHA384 KNOT_ZONEMD_ALORITHM_SHA384
+ #define KNOT_ZONEMD_ALGORITHM_SHA512 KNOT_ZONEMD_ALORITHM_SHA512
+ #endif
+#endif
+
#include "daemon/worker.h"
-#include "daemon/zimport.h"
+#include "lib/dnssec/ta.h"
+#include "lib/dnssec.h"
#include "lib/generic/map.h"
#include "lib/generic/array.h"
+#include "lib/generic/trie.h"
+#include "lib/utils.h"
-#define VERBOSE_MSG(qry, ...) kr_log_q(qry, ZIMPORT, __VA_ARGS__)
-
-/* Pause between parse and import stages, milliseconds.
- * See comment in zi_zone_import() */
+/* Pause between parse and import stages, milliseconds. */
#define ZONE_IMPORT_PAUSE 100
-typedef array_t(knot_rrset_t *) qr_rrsetlist_t;
+// NAN normally comes from <math.h> but it's not guaranteed.
+#ifndef NAN
+ #define NAN nan("")
+#endif
struct zone_import_ctx {
- struct worker_ctx *worker;
- bool started;
+ knot_mm_t *pool; /// memory pool for all allocations (including struct itself)
knot_dname_t *origin;
- knot_rrset_t *ta;
- knot_rrset_t *key;
- uint64_t start_timestamp;
- size_t rrset_idx;
uv_timer_t timer;
- map_t rrset_indexed;
- qr_rrsetlist_t rrset_sorted;
- knot_mm_t pool;
+
+ // from zi_config_t
zi_callback cb;
void *cb_param;
+
+ trie_t *rrsets; /// map: key_get() -> knot_rrset_t*, in ZONEMD order
+ uint32_t timestamp_rr; /// stamp of when RR data arrived (seconds since epoch)
+
+ struct kr_svldr_ctx *svldr; /// DNSSEC validator; NULL iff we don't validate
+ const knot_dname_t *last_cut; /// internal to zi_rrset_import()
+
+#if ENABLE_ZONEMD
+ uint8_t *digest_buf; /// temporary buffer for digest computation (on pool)
+ #define DIGEST_BUF_SIZE (64*1024 - 1)
+ #define DIGEST_ALG_COUNT 2
+ struct {
+ bool active; /// whether we want it computed
+ dnssec_digest_ctx_t *ctx;
+ const uint8_t *expected; /// expected digest (inside zonemd on pool)
+ } digests[DIGEST_ALG_COUNT]; /// we use indices 0 and 1 for SHA 384 and 512
+#endif
};
typedef struct zone_import_ctx zone_import_ctx_t;
-static int RRSET_IS_ALREADY_IMPORTED = 1;
-
-/** @internal Allocate zone import context.
- * @return pointer to zone import context or NULL. */
-static zone_import_ctx_t *zi_ctx_alloc()
-{
- return calloc(1, sizeof(zone_import_ctx_t));
-}
-/** @internal Free zone import context. */
-static void zi_ctx_free(zone_import_ctx_t *z_import)
-{
- if (z_import != NULL) {
- free(z_import);
- }
-}
-
-/** @internal Reset all fields in the zone import context to their default values.
- * Flushes memory pool, but doesn't reallocate memory pool buffer.
- * Doesn't affect timer handle, pointers to callback and callback parameter.
- * @return 0 if success; -1 if failed. */
-static int zi_reset(struct zone_import_ctx *z_import, size_t rrset_sorted_list_size)
-{
- mp_flush(z_import->pool.ctx);
-
- z_import->started = false;
- z_import->start_timestamp = 0;
- z_import->rrset_idx = 0;
- z_import->pool.alloc = (knot_mm_alloc_t) mp_alloc;
- z_import->rrset_indexed = map_make(&z_import->pool);
-
- array_init(z_import->rrset_sorted);
-
- int ret = 0;
- if (rrset_sorted_list_size) {
- ret = array_reserve_mm(z_import->rrset_sorted, rrset_sorted_list_size,
- kr_memreserve, &z_import->pool);
- }
-
- return ret;
-}
-
-/** @internal Close callback for timer handle.
- * @note Actually frees zone import context. */
-static void on_timer_close(uv_handle_t *handle)
+#define KEY_LEN (KNOT_DNAME_MAXLEN + 1 + 2 + 2)
+/** Construct key for name, type and signed type (if type == RRSIG).
+ *
+ * Return negative error code in asserted cases.
+ */
+static int key_get(char buf[KEY_LEN], const knot_dname_t *name,
+ uint16_t type, uint16_t type_maysig, char **key_p)
{
- zone_import_ctx_t *z_import = (zone_import_ctx_t *)handle->data;
- if (z_import != NULL) {
- zi_ctx_free(z_import);
- }
+ char *lf = (char *)knot_dname_lf(name, (uint8_t *)buf);
+ if (kr_fails_assert(lf && key_p))
+ return kr_error(EINVAL);
+ int len = lf[0];
+ lf++; // point to start of data
+ *key_p = lf;
+ // Check that LF is right-aligned to KNOT_DNAME_MAXLEN in buf.
+ if (kr_fails_assert(lf + len == buf + KNOT_DNAME_MAXLEN))
+ return kr_error(EINVAL);
+ buf[KNOT_DNAME_MAXLEN] = 0; // this ensures correct ZONEMD order
+ memcpy(buf + KNOT_DNAME_MAXLEN + 1, &type, sizeof(type));
+ len += 1 + sizeof(type);
+ if (type == KNOT_RRTYPE_RRSIG) {
+ memcpy(buf + KNOT_DNAME_MAXLEN + 1 + sizeof(type),
+ &type_maysig, sizeof(type_maysig));
+ len += sizeof(type_maysig);
+ }
+ return len;
}
-zone_import_ctx_t *zi_allocate(struct worker_ctx *worker,
- zi_callback cb, void *param)
+/** Simple helper to retreive from zone_import_ctx_t::rrsets */
+static knot_rrset_t * rrset_get(trie_t *rrsets, const knot_dname_t *name,
+ uint16_t type, uint16_t type_maysig)
{
- if (worker->loop == NULL) {
+ char key_buf[KEY_LEN], *key;
+ const int len = key_get(key_buf, name, type, type_maysig, &key);
+ if (len < 0)
return NULL;
- }
- zone_import_ctx_t *z_import = zi_ctx_alloc();
- if (!z_import) {
+ const trie_val_t *rrsig_p = trie_get_try(rrsets, key, len);
+ if (!rrsig_p)
return NULL;
- }
- void *mp = mp_new (8192);
- if (!mp) {
- zi_ctx_free(z_import);
- return NULL;
- }
- z_import->pool.ctx = mp;
- z_import->worker = worker;
- int ret = zi_reset(z_import, 0);
- if (ret < 0) {
- mp_delete(mp);
- zi_ctx_free(z_import);
- return NULL;
- }
- uv_timer_init(z_import->worker->loop, &z_import->timer);
- z_import->timer.data = z_import;
- z_import->cb = cb;
- z_import->cb_param = param;
- return z_import;
-}
-
-void zi_free(zone_import_ctx_t *z_import)
-{
- z_import->started = false;
- z_import->start_timestamp = 0;
- z_import->rrset_idx = 0;
- mp_delete(z_import->pool.ctx);
- z_import->pool.ctx = NULL;
- z_import->pool.alloc = NULL;
- z_import->worker = NULL;
- z_import->cb = NULL;
- z_import->cb_param = NULL;
- uv_close((uv_handle_t *)&z_import->timer, on_timer_close);
-}
-
-/** @internal Mark rrset that has been already imported
- * to avoid repeated import. */
-static inline void zi_rrset_mark_as_imported(knot_rrset_t *rr)
-{
- rr->additional = (void *)&RRSET_IS_ALREADY_IMPORTED;
-}
-
-/** @internal Check if rrset is marked as "already imported".
- * @return true if marked, false if isn't */
-static inline bool zi_rrset_is_marked_as_imported(knot_rrset_t *rr)
-{
- return (rr->additional == &RRSET_IS_ALREADY_IMPORTED);
-}
-
-/** @internal Try to find rrset with given requisites amongst parsed rrsets
- * and put it to given packet. If there is RRSIG which covers that rrset, it
- * will be added as well. If rrset found and successfully put, it marked as
- * "already imported" to avoid repeated import. The same is true for RRSIG.
- * @return -1 if failed
- * 0 if required record been actually put into the packet
- * 1 if required record could not be found */
-static int zi_rrset_find_put(struct zone_import_ctx *z_import,
- knot_pkt_t *pkt, const knot_dname_t *owner,
- uint16_t class, uint16_t type, uint16_t additional)
-{
- if (type != KNOT_RRTYPE_RRSIG) {
- /* If required rrset isn't rrsig, these must be the same values */
- additional = type;
- }
-
- char key[KR_RRKEY_LEN];
- int err = kr_rrkey(key, class, owner, type, additional);
- if (err <= 0) {
- return -1;
- }
- knot_rrset_t *rr = map_get(&z_import->rrset_indexed, key);
- if (!rr) {
- return 1;
- }
- err = knot_pkt_put(pkt, 0, rr, 0);
- if (err != KNOT_EOK) {
- return -1;
- }
- zi_rrset_mark_as_imported(rr);
-
- if (type != KNOT_RRTYPE_RRSIG) {
- /* Try to find corresponding rrsig */
- err = zi_rrset_find_put(z_import, pkt, owner,
- class, KNOT_RRTYPE_RRSIG, type);
- if (err < 0) {
- return err;
- }
- }
-
- return 0;
+ kr_assert(*rrsig_p);
+ return *rrsig_p;
}
-/** @internal Try to put given rrset to the given packet.
- * If there is RRSIG which covers that rrset, it will be added as well.
- * If rrset successfully put in the packet, it marked as
- * "already imported" to avoid repeated import.
- * The same is true for RRSIG.
- * @return -1 if failed
- * 0 if required record been actually put into the packet */
-static int zi_rrset_put(struct zone_import_ctx *z_import, knot_pkt_t *pkt,
- knot_rrset_t *rr)
+#if ENABLE_ZONEMD
+static int digest_rrset(trie_val_t *rr_p, void *z_import_v)
{
- if (kr_fails_assert(rr && rr->type != KNOT_RRTYPE_RRSIG))
- return -1;
- int err = knot_pkt_put(pkt, 0, rr, 0);
- if (err != KNOT_EOK) {
- return -1;
+ zone_import_ctx_t *z_import = z_import_v;
+ const knot_rrset_t *rr = *rr_p;
+
+ // ignore apex ZONEMD or its RRSIG, and also out of bailiwick records
+ const int origin_bailiwick = knot_dname_in_bailiwick(rr->owner, z_import->origin);
+ const bool is_apex = origin_bailiwick == 0;
+ if (is_apex && kr_rrset_type_maysig(rr) == KNOT_RRTYPE_ZONEMD)
+ return KNOT_EOK;
+ if (unlikely(origin_bailiwick < 0))
+ return KNOT_EOK;
+
+ const int len = knot_rrset_to_wire_extra(rr, z_import->digest_buf, DIGEST_BUF_SIZE,
+ 0, NULL, KNOT_PF_ORIGTTL);
+ if (len < 0)
+ return kr_error(len);
+
+ // digest serialized RRSet
+ for (int i = 0; i < DIGEST_ALG_COUNT; ++i) {
+ if (!z_import->digests[i].active)
+ continue;
+ dnssec_binary_t bufbin = { len, z_import->digest_buf };
+ int ret = dnssec_digest(z_import->digests[i].ctx, &bufbin);
+ if (ret != KNOT_EOK)
+ return kr_error(ret);
}
- zi_rrset_mark_as_imported(rr);
- /* Try to find corresponding RRSIG */
- err = zi_rrset_find_put(z_import, pkt, rr->owner, rr->rclass,
- KNOT_RRTYPE_RRSIG, rr->type);
- return (err < 0) ? err : 0;
+ return KNOT_EOK;
}
-/** @internal Try to put DS & NSEC* for rset->owner to given packet.
- * @return -1 if failed;
- * 0 if no errors occurred (it doesn't mean
- * that records were actually added). */
-static int zi_put_delegation(zone_import_ctx_t *z_import, knot_pkt_t *pkt,
- knot_rrset_t *rr)
+/** Verify ZONEMD in the stored zone, and return error code.
+ *
+ * ZONEMD signature is verified iff z_import->svldr != NULL
+ https://www.rfc-editor.org/rfc/rfc8976.html#name-verifying-zone-digest
+ */
+static int zonemd_verify(zone_import_ctx_t *z_import)
{
- int err = zi_rrset_find_put(z_import, pkt, rr->owner,
- rr->rclass, KNOT_RRTYPE_DS, 0);
- if (err == 1) {
- /* DS not found, maybe there are NSEC* */
- err = zi_rrset_find_put(z_import, pkt, rr->owner,
- rr->rclass, KNOT_RRTYPE_NSEC, 0);
- if (err >= 0) {
- err = zi_rrset_find_put(z_import, pkt, rr->owner,
- rr->rclass, KNOT_RRTYPE_NSEC3, 0);
+ bool zonemd_is_valid = false;
+ // Find ZONEMD RR + RRSIG
+ knot_rrset_t * const rr_zonemd
+ = rrset_get(z_import->rrsets, z_import->origin, KNOT_RRTYPE_ZONEMD, 0);
+ if (!rr_zonemd) {
+ // no zonemd; let's compute the shorter digest and print info later
+ z_import->digests[KNOT_ZONEMD_ALGORITHM_SHA384 - 1].active = true;
+ goto do_digest;
+ }
+ // Validate ZONEMD RRSIG, if desired
+ if (z_import->svldr) {
+ const knot_rrset_t *rrsig_zonemd
+ = rrset_get(z_import->rrsets, z_import->origin,
+ KNOT_RRTYPE_RRSIG, KNOT_RRTYPE_ZONEMD);
+ int ret = rrsig_zonemd
+ ? kr_svldr_rrset(rr_zonemd, &rrsig_zonemd->rrs, z_import->svldr)
+ : kr_error(ENOENT);
+ zonemd_is_valid = (ret == kr_ok());
+
+ if (!rrsig_zonemd) {
+ kr_log_error(PREFILL, "ZONEMD signature missing\n");
+ } else if (!zonemd_is_valid) {
+ kr_log_error(PREFILL, "ZONEMD signature failed to validate\n");
}
}
- return err < 0 ? err : 0;
-}
-/** @internal Try to put A & AAAA records for rset->owner to given packet.
- * @return -1 if failed;
- * 0 if no errors occurred (it doesn't mean
- * that records were actually added). */
-static int zi_put_glue(zone_import_ctx_t *z_import, knot_pkt_t *pkt,
- knot_rrset_t *rr)
-{
- int err = 0;
- knot_rdata_t *rdata_i = rr->rrs.rdata;
- for (uint16_t i = 0; i < rr->rrs.count;
- ++i, rdata_i = knot_rdataset_next(rdata_i)) {
- const knot_dname_t *ns_name = knot_ns_name(rdata_i);
- err = zi_rrset_find_put(z_import, pkt, ns_name,
- rr->rclass, KNOT_RRTYPE_A, 0);
- if (err < 0) {
- break;
+ // Get SOA serial
+ const knot_rrset_t *soa = rrset_get(z_import->rrsets, z_import->origin,
+ KNOT_RRTYPE_SOA, 0);
+ if (!soa) {
+ kr_log_error(PREFILL, "SOA record not found\n");
+ return kr_error(ENOENT);
+ }
+ if (soa->rrs.count != 1) {
+ kr_log_error(PREFILL, "the SOA RR set is weird\n");
+ return kr_error(EINVAL);
+ } // length is checked by parser already
+ const uint32_t soa_serial = knot_soa_serial(soa->rrs.rdata);
+
+ // Figure out SOA+ZONEMD RR contents.
+ bool some_active = false;
+ knot_rdata_t *rd = rr_zonemd->rrs.rdata;
+ for (int i = 0; i < rr_zonemd->rrs.count; ++i, rd = knot_rdataset_next(rd)) {
+ if (rd->len < 6 || knot_zonemd_scheme(rd) != KNOT_ZONEMD_SCHEME_SIMPLE
+ || knot_zonemd_soa_serial(rd) != soa_serial)
+ continue;
+ const int algo = knot_zonemd_algorithm(rd);
+ if (algo != KNOT_ZONEMD_ALGORITHM_SHA384 && algo != KNOT_ZONEMD_ALGORITHM_SHA512)
+ continue;
+ if (rd->len != 6 + knot_zonemd_digest_size(rd)) {
+ kr_log_error(PREFILL, "ZONEMD record has incorrect digest length\n");
+ return kr_error(EINVAL);
}
-
- err = zi_rrset_find_put(z_import, pkt, ns_name,
- rr->rclass, KNOT_RRTYPE_AAAA, 0);
- if (err < 0) {
- break;
+ if (z_import->digests[algo - 1].active) {
+ kr_log_error(PREFILL, "multiple clashing ZONEMD records found\n");
+ return kr_error(EINVAL);
+ }
+ some_active = true;
+ z_import->digests[algo - 1].active = true;
+ z_import->digests[algo - 1].expected = knot_zonemd_digest(rd);
+ }
+ if (!some_active) {
+ kr_log_error(PREFILL, "ZONEMD record(s) found but none were usable\n");
+ return kr_error(ENOENT);
+ }
+do_digest:
+ // Init memory, etc.
+ if (!z_import->digest_buf) {
+ z_import->digest_buf = mm_alloc(z_import->pool, DIGEST_BUF_SIZE);
+ if (!z_import->digest_buf)
+ return kr_error(ENOMEM);
+ }
+ for (int i = 0; i < DIGEST_ALG_COUNT; ++i) {
+ const int algo = i + 1;
+ if (!z_import->digests[i].active)
+ continue;
+ int ret = dnssec_digest_init(algo, &z_import->digests[i].ctx);
+ if (ret != KNOT_EOK) {
+ // free previous successful _ctx, if applicable
+ dnssec_binary_t digest = { 0 };
+ while (--i >= 0) {
+ if (z_import->digests[i].active)
+ dnssec_digest_finish(z_import->digests[i].ctx,
+ &digest);
+ }
+ return kr_error(ENOMEM);
}
}
- return err < 0 ? err : 0;
-}
-
-/** @internal Create query. */
-static knot_pkt_t *zi_query_create(zone_import_ctx_t *z_import, knot_rrset_t *rr)
-{
- knot_mm_t *pool = &z_import->pool;
-
- uint32_t msgid = kr_rand_bytes(2);
-
- knot_pkt_t *query = knot_pkt_new(NULL, KNOT_WIRE_MAX_PKTSIZE, pool);
- if (!query) {
- return NULL;
- }
-
- knot_pkt_put_question(query, rr->owner, rr->rclass, rr->type);
- knot_pkt_begin(query, KNOT_ANSWER);
- knot_wire_set_rd(query->wire);
- knot_wire_set_id(query->wire, msgid);
- int err = knot_pkt_parse(query, 0);
- if (err != KNOT_EOK) {
- knot_pkt_free(query);
- return NULL;
+ // Actually compute the digest(s).
+ int ret = trie_apply(z_import->rrsets, digest_rrset, z_import);
+ dnssec_binary_t digs[DIGEST_ALG_COUNT] = { { 0 } };
+ for (int i = 0; i < DIGEST_ALG_COUNT; ++i) {
+ if (!z_import->digests[i].active)
+ continue;
+ int ret2 = dnssec_digest_finish(z_import->digests[i].ctx, &digs[i]);
+ if (ret == DNSSEC_EOK)
+ ret = ret2;
+ // we need to keep going to free all digests[*].ctx
+ }
+ if (ret != DNSSEC_EOK) {
+ for (int i = 0; i < DIGEST_ALG_COUNT; ++i)
+ free(digs[i].data);
+ kr_log_error(PREFILL, "error when computing digest: %s\n",
+ kr_strerror(ret));
+ return kr_error(ret);
+ }
+ // Now only check that one of the hashes match.
+ bool has_match = false;
+ for (int i = 0; i < DIGEST_ALG_COUNT; ++i) {
+ if (!z_import->digests[i].active)
+ continue;
+ // hexdump the hash for logging
+ char hash_str[digs[i].size * 2 + 1];
+ for (ssize_t j = 0; j < digs[i].size; ++j)
+ sprintf(hash_str + 2*j, "%02x", digs[i].data[j]);
+
+ if (!z_import->digests[i].expected) {
+ kr_log_error(PREFILL, "no ZONEMD found; computed hash: %s\n",
+ hash_str);
+ } else if (memcmp(z_import->digests[i].expected, digs[i].data,
+ digs[i].size) != 0) {
+ kr_log_error(PREFILL, "ZONEMD hash mismatch; computed hash: %s\n",
+ hash_str);
+ } else {
+ kr_log_debug(PREFILL, "ZONEMD hash matches\n");
+ has_match = true;
+ continue;
+ }
}
- return query;
+ for (int i = 0; i < DIGEST_ALG_COUNT; ++i)
+ free(digs[i].data);
+ bool ok = has_match && (zonemd_is_valid || !z_import->svldr);
+ return ok ? kr_ok() : kr_error(ENOENT);
}
+#endif
-/** @internal Import given rrset to cache.
- * @return -1 if failed; 0 if success */
-static int zi_rrset_import(zone_import_ctx_t *z_import, knot_rrset_t *rr)
-{
- /* Create "pseudo query" which asks for given rrset. */
- knot_pkt_t *query = zi_query_create(z_import, rr);
- if (!query) {
- return -1;
- }
-
- knot_mm_t *pool = &z_import->pool;
- uint8_t *dname = rr->owner;
- uint16_t rrtype = rr->type;
- uint16_t rrclass = rr->rclass;
-
- /* Create "pseudo answer". */
- knot_pkt_t *answer = knot_pkt_new(NULL, KNOT_WIRE_MAX_PKTSIZE, pool);
- if (!answer) {
- knot_pkt_free(query);
- return -1;
- }
- knot_pkt_put_question(answer, dname, rrclass, rrtype);
- knot_pkt_begin(answer, KNOT_ANSWER);
-
- struct kr_qflags options = { 0 };
- options.DNSSEC_WANT = true;
- options.NO_MINIMIZE = true;
-
- /* This call creates internal structures which necessary for
- * resolving - qr_task & request_ctx. */
- struct qr_task *task = worker_resolve_start(query, options);
- if (!task) {
- knot_pkt_free(query);
- knot_pkt_free(answer);
- return -1;
- }
-
- /* Push query to the request resolve plan.
- * Actually query will never been sent to upstream. */
- struct kr_request *request = worker_task_request(task);
- struct kr_rplan *rplan = &request->rplan;
- struct kr_query *qry = kr_rplan_push(rplan, NULL, dname, rrclass, rrtype);
- int state = KR_STATE_FAIL;
- bool origin_is_owner = knot_dname_is_equal(rr->owner, z_import->origin);
- bool is_referral = (rrtype == KNOT_RRTYPE_NS && !origin_is_owner);
- uint32_t msgid = knot_wire_get_id(query->wire);
-
- qry->id = msgid;
- /* Prepare zonecut. It must have all the necessary requisites for
- * successful validation - matched zone name & keys & trust-anchors. */
- kr_zonecut_init(&qry->zone_cut, z_import->origin, pool);
- qry->zone_cut.key = z_import->key;
- qry->zone_cut.trust_anchor = z_import->ta;
-
- if (knot_pkt_init_response(answer, query) != 0) {
- goto cleanup;
- }
-
- /* Since "pseudo" query asks for NS for subzone,
- * "pseudo" answer must simulate referral. */
- if (is_referral) {
- knot_pkt_begin(answer, KNOT_AUTHORITY);
- }
-
- /* Put target rrset to ANSWER\AUTHORITY as well as corresponding RRSIG */
- int err = zi_rrset_put(z_import, answer, rr);
- if (err != 0) {
- goto cleanup;
- }
-
- if (!is_referral) {
- knot_wire_set_aa(answer->wire);
+/**
+ * @internal Import given rrset to cache.
+ *
+ * @return error code; we could've chosen to keep importing even if some RRset fails,
+ * but it would be harder to ensure that we don't generate too many logs
+ * and that we pass an error to the finishing callback.
+ */
+static int zi_rrset_import(trie_val_t *rr_p, void *z_import_v)
+{
+ zone_import_ctx_t *z_import = z_import_v;
+ knot_rrset_t *rr = *rr_p;
+
+ if (rr->type == KNOT_RRTYPE_RRSIG)
+ return 0; // we do RRSIGs at once with their types
+
+ const int origin_bailiwick = knot_dname_in_bailiwick(rr->owner, z_import->origin);
+ if (unlikely(origin_bailiwick < 0)) {
+ KR_DNAME_GET_STR(owner_str, rr->owner);
+ kr_log_warning(PREFILL, "ignoring out of bailiwick record(s) on %s\n",
+ owner_str);
+ return 0; // well, let's continue without error
+ }
+
+ // Determine if this RRset is authoritative.
+ // We utilize that iteration happens in canonical order.
+ bool is_auth;
+ const int kdib = knot_dname_in_bailiwick(rr->owner, z_import->last_cut);
+ if (kdib == 0 && (rr->type == KNOT_RRTYPE_DS || rr->type == KNOT_RRTYPE_NSEC
+ || rr->type == KNOT_RRTYPE_NSEC3)) {
+ // parent side of the zone cut (well, presumably in case of NSEC*)
+ is_auth = true;
+ } else if (kdib >= 0) {
+ // inside non-auth subtree
+ is_auth = false;
+ } else if (rr->type == KNOT_RRTYPE_NS && origin_bailiwick > 0) {
+ // entering non-auth subtree
+ z_import->last_cut = rr->owner;
+ is_auth = false;
} else {
- /* Type is KNOT_RRTYPE_NS and owner is not equal to origin.
- * It will be "referral" answer and must contain delegation. */
- err = zi_put_delegation(z_import, answer, rr);
- if (err < 0) {
- goto cleanup;
+ // outside non-auth subtree
+ is_auth = true;
+ z_import->last_cut = NULL; // so that the next _in_bailiwick() is faster
+ }
+ // Rare case: `A` exactly on zone cut would be misdetected and fail validation;
+ // it's the only type ordered before NS.
+ if (unlikely(is_auth && rr->type < KNOT_RRTYPE_NS)) {
+ if (rrset_get(z_import->rrsets, rr->owner, KNOT_RRTYPE_NS, 0))
+ is_auth = false;
+ }
+
+ // Get and validate the corresponding RRSIGs, if authoritative.
+ const knot_rrset_t *rrsig = NULL;
+ if (is_auth) {
+ rrsig = rrset_get(z_import->rrsets, rr->owner, KNOT_RRTYPE_RRSIG, rr->type);
+ if (unlikely(!rrsig && z_import->svldr)) {
+ KR_DNAME_GET_STR(owner_str, rr->owner);
+ KR_RRTYPE_GET_STR(type_str, rr->type);
+ kr_log_error(PREFILL, "no records found for %s RRSIG %s\n",
+ owner_str, type_str);
+ return kr_error(ENOENT);
}
}
-
- knot_pkt_begin(answer, KNOT_ADDITIONAL);
-
- if (rrtype == KNOT_RRTYPE_NS) {
- /* Try to find glue addresses. */
- err = zi_put_glue(z_import, answer, rr);
- if (err < 0) {
- goto cleanup;
+ if (is_auth && z_import->svldr) {
+ int ret = kr_svldr_rrset(rr, &rrsig->rrs, z_import->svldr);
+ if (unlikely(ret)) {
+ KR_DNAME_GET_STR(owner_str, rr->owner);
+ KR_RRTYPE_GET_STR(type_str, rr->type);
+ kr_log_error(PREFILL, "validation failed for %s %s: %s\n",
+ owner_str, type_str, kr_strerror(ret));
+ return kr_error(ret);
}
}
- knot_wire_set_id(answer->wire, msgid);
- answer->parsed = answer->size;
- err = knot_pkt_parse(answer, 0);
- if (err != KNOT_EOK) {
- goto cleanup;
+ uint8_t rank;
+ if (!is_auth) {
+ rank = KR_RANK_OMIT;
+ } else if (z_import->svldr) {
+ rank = KR_RANK_AUTH|KR_RANK_SECURE;
+ } else {
+ rank = KR_RANK_AUTH|KR_RANK_INSECURE;
}
- /* Importing doesn't imply communication with upstream at all.
- * "answer" contains pseudo-answer from upstream and must be successfully
- * validated in CONSUME stage. If not, something gone wrong. */
- state = kr_resolve_consume(request, NULL, answer);
-
-cleanup:
-
- knot_pkt_free(query);
- knot_pkt_free(answer);
- worker_task_finalize(task, state);
- return state == (is_referral ? KR_STATE_PRODUCE : KR_STATE_DONE) ? 0 : -1;
+ int ret = kr_cache_insert_rr(&the_worker->engine->resolver.cache, rr, rrsig,
+ rank, z_import->timestamp_rr,
+ // Optim.: only stash NSEC* params at the apex.
+ origin_bailiwick == 0);
+ if (ret) {
+ kr_log_error(PREFILL, "caching an RRset failed: %s\n",
+ kr_strerror(ret));
+ return kr_error(ret);
+ }
+ return 0; // success
}
-/** @internal Create element in qr_rrsetlist_t rrset_list for
- * given node of map_t rrset_sorted. */
-static int zi_mapwalk_preprocess(const char *k, void *v, void *baton)
+static void ctx_delete(zone_import_ctx_t *z_import)
{
- zone_import_ctx_t *z_import = (zone_import_ctx_t *)baton;
-
- int ret = array_push_mm(z_import->rrset_sorted, v, kr_memreserve, &z_import->pool);
-
- return (ret < 0);
+ if (kr_fails_assert(z_import)) return;
+ kr_svldr_free_ctx(z_import->svldr);
+ mm_ctx_delete(z_import->pool);
+}
+static void timer_close(uv_handle_t *handle)
+{
+ ctx_delete(handle->data);
}
/** @internal Iterate over parsed rrsets and try to import each of them. */
-static void zi_zone_process(uv_timer_t* handle)
+static void zi_zone_process(uv_timer_t *timer)
{
- zone_import_ctx_t *z_import = (zone_import_ctx_t *)handle->data;
-
- size_t failed = 0;
- size_t ns_imported = 0;
- size_t other_imported = 0;
-
- if (kr_fails_assert(z_import->worker)) {
- failed = 1;
- goto finish;
- }
-
- /* At the moment import of root zone only is supported.
- * Check the name of the parsed zone.
- * TODO - implement importing of arbitrary zone. */
- KR_DNAME_GET_STR(zone_name_str, z_import->origin);
-
- if (strcmp(".", zone_name_str) != 0) {
- kr_log_error(ZIMPORT, "unexpected zone name `%s` (root zone expected), fail\n",
- zone_name_str);
- failed = 1;
- goto finish;
- }
+ zone_import_ctx_t *z_import = timer->data;
- if (z_import->rrset_sorted.len <= 0) {
- kr_log_error(ZIMPORT, "zone `%s` is empty\n", zone_name_str);
- goto finish;
- }
-
- /* TA have been found, zone is secured.
- * DNSKEY must be somewhere amongst the imported records. Find it.
- * TODO - For those zones that provenly do not have TA this step must be skipped. */
- char key[KR_RRKEY_LEN];
- int err = kr_rrkey(key, KNOT_CLASS_IN, z_import->origin,
- KNOT_RRTYPE_DNSKEY, KNOT_RRTYPE_DNSKEY);
- if (err <= 0) {
- failed = 1;
- goto finish;
- }
-
- knot_rrset_t *rr_key = map_get(&z_import->rrset_indexed, key);
- if (!rr_key) {
- /* DNSKEY MUST be here. If not found - fail. */
- kr_log_error(ZIMPORT, "DNSKEY not found for `%s`, fail\n", zone_name_str);
- failed = 1;
- goto finish;
- }
- z_import->key = rr_key;
-
- map_t *trust_anchors = &z_import->worker->engine->resolver.trust_anchors;
- knot_rrset_t *rr_ta = kr_ta_get(trust_anchors, z_import->origin);
- if (!rr_ta) {
- kr_log_error(ZIMPORT, "error: TA for zone `%s` vanished, fail", zone_name_str);
- failed = 1;
- goto finish;
- }
- z_import->ta = rr_ta;
-
- VERBOSE_MSG(NULL, "started: zone: '%s'\n", zone_name_str);
-
- z_import->start_timestamp = kr_now();
-
- /* Import DNSKEY at first step. If any validation problems will appear,
- * cancel import of whole zone. */
- KR_DNAME_GET_STR(kname_str, rr_key->owner);
- KR_RRTYPE_GET_STR(ktype_str, rr_key->type);
-
- VERBOSE_MSG(NULL, "importing: name: '%s' type: '%s'\n",
- kname_str, ktype_str);
-
- int res = zi_rrset_import(z_import, rr_key);
- if (res != 0) {
- kr_log_error(ZIMPORT, "import failed: qname: '%s' type: '%s'\n",
- kname_str, ktype_str);
- failed = 1;
- goto finish;
- }
-
- /* Import all NS records */
- for (size_t i = 0; i < z_import->rrset_sorted.len; ++i) {
- knot_rrset_t *rr = z_import->rrset_sorted.at[i];
-
- if (rr->type != KNOT_RRTYPE_NS) {
- continue;
- }
-
- KR_DNAME_GET_STR(name_str, rr->owner);
- KR_RRTYPE_GET_STR(type_str, rr->type);
- VERBOSE_MSG(NULL, "importing: name: '%s' type: '%s'\n",
- name_str, type_str);
- int ret = zi_rrset_import(z_import, rr);
- if (ret == 0) {
- ++ns_imported;
- } else {
- VERBOSE_MSG(NULL, "import failed: name: '%s' type: '%s'\n",
- name_str, type_str);
- ++failed;
- }
- z_import->rrset_sorted.at[i] = NULL;
- }
-
- /* NS records have been imported as well as relative DS, NSEC* and glue.
- * Now import what's left. */
- for (size_t i = 0; i < z_import->rrset_sorted.len; ++i) {
-
- knot_rrset_t *rr = z_import->rrset_sorted.at[i];
- if (rr == NULL) {
- continue;
- }
+ kr_timer_t stopwatch;
+ kr_timer_start(&stopwatch);
- if (zi_rrset_is_marked_as_imported(rr)) {
- continue;
- }
-
- if (rr->type == KNOT_RRTYPE_DNSKEY || rr->type == KNOT_RRTYPE_RRSIG) {
- continue;
- }
-
- KR_DNAME_GET_STR(name_str, rr->owner);
- KR_RRTYPE_GET_STR(type_str, rr->type);
- VERBOSE_MSG(NULL, "importing: name: '%s' type: '%s'\n",
- name_str, type_str);
- res = zi_rrset_import(z_import, rr);
- if (res == 0) {
- ++other_imported;
- } else {
- VERBOSE_MSG(NULL, "import failed: name: '%s' type: '%s'\n",
- name_str, type_str);
- ++failed;
- }
- }
-
- uint64_t elapsed = kr_now() - z_import->start_timestamp;
- elapsed = elapsed > UINT_MAX ? UINT_MAX : elapsed;
-
- VERBOSE_MSG(NULL, "finished in %"PRIu64" ms; zone: `%s`; ns: %zd"
- "; other: %zd; failed: %zd\n",
- elapsed, zone_name_str, ns_imported, other_imported, failed);
-
-finish:
-
- uv_timer_stop(&z_import->timer);
- z_import->started = false;
-
- int import_state = 0;
-
- if (failed != 0) {
- if (ns_imported == 0 && other_imported == 0) {
- import_state = -1;
- kr_log_error(ZIMPORT, "import failed; zone `%s` \n", zone_name_str);
- } else {
- import_state = 1;
- }
- } else {
- import_state = 0;
+ int ret = trie_apply(z_import->rrsets, zi_rrset_import, z_import);
+ if (ret == 0) {
+ kr_log_info(PREFILL, "performance: validating and caching took %.3lf s\n",
+ kr_timer_elapsed(&stopwatch));
}
- if (z_import->cb != NULL) {
- z_import->cb(import_state, z_import->cb_param);
- }
+ if (z_import->cb)
+ z_import->cb(kr_error(ret), z_import->cb_param);
+ uv_close((uv_handle_t *)timer, timer_close);
}
/** @internal Store rrset that has been imported to zone import context memory pool.
{
if (s->r_data_length > UINT16_MAX) {
/* Due to knot_rrset_add_rdata(..., const uint16_t size, ...); */
- kr_log_error(ZSCANNER, "line %"PRIu64": rdata is too long\n",
+ kr_log_error(PREFILL, "line %"PRIu64": rdata is too long\n",
s->line_counter);
return -1;
}
if (knot_dname_size(s->r_owner) != strlen((const char *)(s->r_owner)) + 1) {
- kr_log_error(ZSCANNER, "line %"PRIu64
+ kr_log_error(PREFILL, "line %"PRIu64
": owner name contains zero byte, skip\n",
s->line_counter);
return 0;
zone_import_ctx_t *z_import = (zone_import_ctx_t *)s->process.data;
knot_rrset_t *new_rr = knot_rrset_new(s->r_owner, s->r_type, s->r_class,
- s->r_ttl, &z_import->pool);
+ s->r_ttl, z_import->pool);
if (!new_rr) {
- kr_log_error(ZSCANNER, "line %"PRIu64": error creating rrset\n",
+ kr_log_error(PREFILL, "line %"PRIu64": error creating rrset\n",
s->line_counter);
return -1;
}
int res = knot_rrset_add_rdata(new_rr, s->r_data, s->r_data_length,
- &z_import->pool);
+ z_import->pool);
if (res != KNOT_EOK) {
- kr_log_error(ZSCANNER, "line %"PRIu64": error adding rdata to rrset\n",
+ kr_log_error(PREFILL, "line %"PRIu64": error adding rdata to rrset\n",
s->line_counter);
return -1;
}
+ /* zscanner itself does not canonize - neither owner nor insides */
+ res = knot_rrset_rr_to_canonical(new_rr);
+ if (res != KNOT_EOK) {
+ kr_log_error(PREFILL, "line %"PRIu64": error when canonizing: %s\n",
+ s->line_counter, knot_strerror(res));
+ return -1;
+ }
/* Records in zone file may not be grouped by name and RR type.
* Use map to create search key and
* avoid ineffective searches across all the imported records. */
- char key[KR_RRKEY_LEN];
- uint16_t additional_key_field = kr_rrset_type_maysig(new_rr);
-
- res = kr_rrkey(key, new_rr->rclass, new_rr->owner, new_rr->type,
- additional_key_field);
- if (res <= 0) {
- kr_log_error(ZSCANNER, "line %"PRIu64": error constructing rrkey\n",
+ char key_buf[KEY_LEN], *key;
+ const int len = key_get(key_buf, new_rr->owner, new_rr->type,
+ kr_rrset_type_maysig(new_rr), &key);
+ if (len < 0) {
+ kr_log_error(PREFILL, "line %"PRIu64": error constructing rrkey\n",
s->line_counter);
return -1;
}
-
- knot_rrset_t *saved_rr = map_get(&z_import->rrset_indexed, key);
- if (saved_rr) {
- res = knot_rdataset_merge(&saved_rr->rrs, &new_rr->rrs,
- &z_import->pool);
+ trie_val_t *rr_p = trie_get_ins(z_import->rrsets, key, len);
+ if (!rr_p)
+ return -1; // ENOMEM
+ if (*rr_p) {
+ knot_rrset_t *rr = *rr_p;
+ res = knot_rdataset_merge(&rr->rrs, &new_rr->rrs, z_import->pool);
} else {
- res = map_set(&z_import->rrset_indexed, key, new_rr);
+ *rr_p = new_rr;
}
if (res != 0) {
- kr_log_error(ZSCANNER, "line %"PRIu64": error saving parsed rrset\n",
+ kr_log_error(PREFILL, "line %"PRIu64": error saving parsed rrset\n",
s->line_counter);
return -1;
}
return 0;
}
-/** @internal zscanner callback. */
static int zi_state_parsing(zs_scanner_t *s)
{
bool empty = true;
}
zone_import_ctx_t *z_import = (zone_import_ctx_t *) s->process.data;
empty = false;
- if (s->r_type == 6) {
+ if (s->r_type == KNOT_RRTYPE_SOA) {
z_import->origin = knot_dname_copy(s->r_owner,
- &z_import->pool);
+ z_import->pool);
}
break;
case ZS_STATE_ERROR:
- kr_log_error(ZSCANNER, "line: %"PRIu64
+ kr_log_error(PREFILL, "line: %"PRIu64
": parse error; code: %i ('%s')\n",
s->line_counter, s->error.code,
zs_strerror(s->error.code));
return -1;
case ZS_STATE_INCLUDE:
- kr_log_error(ZSCANNER, "line: %"PRIu64
+ kr_log_error(PREFILL, "line: %"PRIu64
": INCLUDE is not supported\n",
s->line_counter);
return -1;
case ZS_STATE_EOF:
case ZS_STATE_STOP:
if (empty) {
- kr_log_error(ZIMPORT, "empty zone file\n");
+ kr_log_error(PREFILL, "empty zone file\n");
return -1;
}
if (!((zone_import_ctx_t *) s->process.data)->origin) {
- kr_log_error(ZIMPORT, "zone file doesn't contain SOA record\n");
+ kr_log_error(PREFILL, "zone file doesn't contain SOA record\n");
return -1;
}
return (s->error.counter == 0) ? 0 : -1;
default:
- kr_log_error(ZSCANNER, "line: %"PRIu64
+ kr_log_error(PREFILL, "line: %"PRIu64
": unexpected parse state: %i\n",
s->line_counter, s->state);
return -1;
return -1;
}
-int zi_zone_import(struct zone_import_ctx *z_import,
- const char *zone_file, const char *origin,
- uint16_t rclass, uint32_t ttl)
+int zi_zone_import(const zi_config_t config)
{
- if (kr_fails_assert(z_import && z_import->worker && zone_file))
- return -1;
+ const zi_config_t *c = &config;
+ if (kr_fails_assert(c && c->zone_file))
+ return kr_error(EINVAL);
- zs_scanner_t *s = malloc(sizeof(zs_scanner_t));
- if (s == NULL) {
- kr_log_error(ZSCANNER, "error creating instance of zone scanner (malloc() fails)\n");
- return -1;
- }
+ knot_mm_t *pool = mm_ctx_mempool2(1024 * 1024);
+ zone_import_ctx_t *z_import = mm_calloc(pool, 1, sizeof(*z_import));
+ if (!z_import) return kr_error(ENOMEM);
+ z_import->pool = pool;
+
+ z_import->cb = c->cb;
+ z_import->cb_param = c->cb_param;
+ z_import->rrsets = trie_create(z_import->pool);
+
+ kr_timer_t stopwatch;
+ kr_timer_start(&stopwatch);
+ //// Parse the whole zone file into z_import->rrsets.
+ zs_scanner_t s_storage, *s = &s_storage;
/* zs_init(), zs_set_input_file(), zs_set_processing() returns -1 in case of error,
* so don't print error code as it meaningless. */
- int res = zs_init(s, origin, rclass, ttl);
- if (res != 0) {
- kr_log_error(ZSCANNER, "error initializing zone scanner instance, error: %i (%s)\n",
+ int ret = zs_init(s, c->origin, KNOT_CLASS_IN, c->ttl);
+ if (ret != 0) {
+ kr_log_error(PREFILL, "error initializing zone scanner instance, error: %i (%s)\n",
s->error.code, zs_strerror(s->error.code));
- free(s);
- return -1;
+ goto fail;
}
- res = zs_set_input_file(s, zone_file);
- if (res != 0) {
- kr_log_error(ZSCANNER, "error opening zone file `%s`, error: %i (%s)\n",
- zone_file, s->error.code, zs_strerror(s->error.code));
+ ret = zs_set_input_file(s, c->zone_file);
+ if (ret != 0) {
+ kr_log_error(PREFILL, "error opening zone file `%s`, error: %i (%s)\n",
+ c->zone_file, s->error.code, zs_strerror(s->error.code));
zs_deinit(s);
- free(s);
- return -1;
+ goto fail;
}
/* Don't set processing and error callbacks as we don't use automatic parsing.
* Parsing as well error processing will be performed in zi_state_parsing().
* Store pointer to zone import context for further use. */
- if (zs_set_processing(s, NULL, NULL, (void *)z_import) != 0) {
- kr_log_error(ZSCANNER, "zs_set_processing() failed for zone file `%s`, "
+ ret = zs_set_processing(s, NULL, NULL, (void *)z_import);
+ if (ret != 0) {
+ kr_log_error(PREFILL, "zs_set_processing() failed for zone file `%s`, "
"error: %i (%s)\n",
- zone_file, s->error.code, zs_strerror(s->error.code));
+ c->zone_file, s->error.code, zs_strerror(s->error.code));
zs_deinit(s);
- free(s);
- return -1;
+ goto fail;
}
- uint64_t elapsed = 0;
- int ret = zi_reset(z_import, 4096);
- if (ret == 0) {
- z_import->started = true;
- z_import->start_timestamp = kr_now();
- VERBOSE_MSG(NULL, "[zscanner] started; zone file `%s`\n",
- zone_file);
- ret = zi_state_parsing(s);
- if (ret == 0) {
- /* Try to find TA for worker->z_import.origin. */
- map_t *trust_anchors = &z_import->worker->engine->resolver.trust_anchors;
- knot_rrset_t *rr = kr_ta_get(trust_anchors, z_import->origin);
- if (!rr) {
- /* For now - fail.
- * TODO - query DS and continue after answer had been obtained. */
- KR_DNAME_GET_STR(zone_name_str, z_import->origin);
- kr_log_error(ZIMPORT, "no TA found for `%s`, fail\n", zone_name_str);
- ret = 1;
- }
- elapsed = kr_now() - z_import->start_timestamp;
- elapsed = elapsed > UINT_MAX ? UINT_MAX : elapsed;
- }
- }
+ ret = zi_state_parsing(s);
zs_deinit(s);
- free(s);
-
+ const double time_parse = kr_timer_elapsed(&stopwatch);
if (ret != 0) {
- kr_log_error(ZSCANNER, "error parsing zone file `%s`\n", zone_file);
- z_import->started = false;
- return ret;
+ kr_log_error(PREFILL, "error parsing zone file `%s`\n", c->zone_file);
+ goto fail;
}
+ kr_log_debug(PREFILL, "import started for zone file `%s`\n", c->zone_file);
- VERBOSE_MSG(NULL, "[zscanner] finished in %"PRIu64" ms; zone file `%s`\n",
- elapsed, zone_file);
- map_walk(&z_import->rrset_indexed, zi_mapwalk_preprocess, z_import);
+ KR_DNAME_GET_STR(zone_name_str, z_import->origin);
- /* Zone have been parsed already, so start the import. */
- uv_timer_start(&z_import->timer, zi_zone_process,
- ZONE_IMPORT_PAUSE, ZONE_IMPORT_PAUSE);
+ //// Choose timestamp_rr, according to config.
+ struct timespec now;
+ if (clock_gettime(CLOCK_REALTIME, &now)) {
+ ret = kr_error(errno);
+ kr_log_error(PREFILL, "failed to get current time: %s\n", kr_strerror(ret));
+ goto fail;
+ }
+ if (config.time_src == ZI_STAMP_NOW) {
+ z_import->timestamp_rr = now.tv_sec;
+ } else if (config.time_src == ZI_STAMP_MTIM) {
+ struct stat st;
+ if (stat(c->zone_file, &st) != 0) {
+ kr_log_debug(PREFILL, "failed to stat file `%s`: %s\n",
+ c->zone_file, strerror(errno));
+ goto fail;
+ }
+ z_import->timestamp_rr = st.st_mtime;
+ } else {
+ ret = kr_error(EINVAL);
+ goto fail;
+ }
+ //// Some sanity checks
+ const knot_rrset_t *soa = rrset_get(z_import->rrsets, z_import->origin,
+ KNOT_RRTYPE_SOA, 0);
+ if (z_import->timestamp_rr > now.tv_sec) {
+ kr_log_warning(PREFILL, "zone file `%s` comes from future\n", c->zone_file);
+ } else if (!soa) {
+ kr_log_warning(PREFILL, "missing %s SOA\n", zone_name_str);
+ } else if ((int64_t)z_import->timestamp_rr + soa->ttl < now.tv_sec) {
+ kr_log_warning(PREFILL, "%s SOA already expired\n", zone_name_str);
+ }
+
+ //// Initialize validator context with the DNSKEY.
+ if (c->downgrade)
+ goto zonemd;
+ struct kr_context *resolver = &the_worker->engine->resolver;
+ const knot_rrset_t * const ds = c->ds ? c->ds :
+ kr_ta_get(&resolver->trust_anchors, z_import->origin);
+ if (!ds) {
+ if (!kr_ta_closest(resolver, z_import->origin, KNOT_RRTYPE_DNSKEY))
+ goto zonemd; // our TAs say we're insecure
+ kr_log_error(PREFILL, "no DS found for `%s`, fail\n", zone_name_str);
+ ret = kr_error(ENOENT);
+ goto fail;
+ }
+ if (!knot_dname_is_equal(ds->owner, z_import->origin)) {
+ kr_log_error(PREFILL, "mismatching DS owner, fail\n");
+ ret = kr_error(EINVAL);
+ goto fail;
+ }
+
+ knot_rrset_t * const dnskey = rrset_get(z_import->rrsets, z_import->origin,
+ KNOT_RRTYPE_DNSKEY, 0);
+ if (!dnskey) {
+ kr_log_error(PREFILL, "no DNSKEY found for `%s`, fail\n", zone_name_str);
+ ret = kr_error(ENOENT);
+ goto fail;
+ }
+ knot_rrset_t * const dnskey_sigs = rrset_get(z_import->rrsets, z_import->origin,
+ KNOT_RRTYPE_RRSIG, KNOT_RRTYPE_DNSKEY);
+ if (!dnskey_sigs) {
+ kr_log_error(PREFILL, "no RRSIGs for DNSKEY found for `%s`, fail\n",
+ zone_name_str);
+ ret = kr_error(ENOENT);
+ goto fail;
+ }
+
+ z_import->svldr = kr_svldr_new_ctx(ds, dnskey, &dnskey_sigs->rrs,
+ z_import->timestamp_rr);
+ if (!z_import->svldr) {
+ kr_log_error(PREFILL, "failed to validate DNSKEY for `%s`\n", zone_name_str);
+ ret = kr_error(ENOENT);
+ goto fail;
+ /* TODO: more details about why validation failed.
+ * Perhaps extend the SVLDR API to somehow return EDE code. */
+ }
+
+ //// Do all ZONEMD processing, if desired.
+zonemd: (void)0; // C can't have a variable definition following a label
+ double time_zonemd = NAN;
+ if (c->zonemd) {
+ #if ENABLE_ZONEMD
+ kr_timer_start(&stopwatch);
+ ret = zonemd_verify(z_import);
+ time_zonemd = kr_timer_elapsed(&stopwatch);
+ #else
+ kr_log_error(PREFILL,
+ "ZONEMD check requested but not supported, fail\n");
+ ret = kr_error(ENOSYS);
+ #endif
+ } else {
+ ret = kr_ok();
+ }
+ kr_log_info(PREFILL, "performance: parsing took %.3lf s, hashing took %.3lf s\n",
+ time_parse, time_zonemd);
+ if (ret) goto fail;
- return 0;
+ //// Phase two, after a pause. Validate and import all the remaining records.
+ ret = uv_timer_init(the_worker->loop, &z_import->timer);
+ if (ret) goto fail;
+ z_import->timer.data = z_import;
+ ret = uv_timer_start(&z_import->timer, zi_zone_process, ZONE_IMPORT_PAUSE, 0);
+ if (ret) goto fail;
+
+ return kr_ok();
+fail:
+ if (z_import->cb)
+ z_import->cb(kr_error(ret), z_import->cb_param);
+ if (kr_fails_assert(ret))
+ ret = ENOENT;
+ ctx_delete(z_import);
+ return kr_error(ret);
}
-bool zi_import_started(struct zone_import_ctx *z_import)
-{
- return z_import ? z_import->started : false;
-}