Browse Source

[1332] Merge branch 'trac1330' into trac1332

JINMEI Tatuya 13 years ago
parent
commit
9753568c85

+ 50 - 0
src/lib/datasrc/database.h

@@ -274,6 +274,56 @@ public:
      */
     virtual IteratorContextPtr getAllRecords(int id) const = 0;
 
+    /**
+     * \brief Creates an iterator context for a set of differences.
+     *
+     * Returns an IteratorContextPtr that contains all difference records for
+     * the given zone between two versions of a zone.
+     *
+     * The difference records are the set of records that would appear in an
+     * IXFR serving a request for the difference between two versions of a zone.
+     * The records are returned in the same order as they would be in the IXFR.
+     * This means that if the the difference between versions of a zone with SOA
+     * serial numbers of "start" and "end" is required, and the zone contains
+     * the differences between serial number "start" to serial number
+     * "intermediate" and from serial number "intermediate" to serial number
+     * "end", the returned records will be (in order):
+     *
+     * \li SOA for serial "start"
+     * \li Records removed from the zone between versions "start" and
+     *     "intermediate" of the zone.  The order of these is not guaranteed.
+     * \li SOA for serial "intermediate"
+     * \li Records added to the zone between versions "start" and
+     *     "intermediate" of the zone.  The order of these is not guaranteed.
+     * \li SOA for serial "intermediate"
+     * \li Records removed from the zone between versions "intermediate" and
+     *     "end" of the zone.  The order of these is not guaranteed.
+     * \li SOA for serial "end"
+     * \li Records added to the zone between versions "intermediate" and "end"
+     *     of the zone. The order of these is not guaranteed.
+     *
+     * Note that there is no requirement that "start" be less than "end". Owing
+     * to serial number arithmetic, it is entirely possible that a later version
+     * of a zone will have a smaller SOA serial number than an earlier version.
+     *
+     * Each call to getNext() on the returned iterator should copy all
+     * column fields of the array that is passed, as defined in the
+     * RecordColumns enum.
+     *
+     * \exception any Since any implementation can be used, the caller should
+     *                expect any exception to be thrown.
+     *
+     * \param id The ID of the zone, returned from getZone().
+     * \param start The SOA serial number of the version of the zone from
+     *        which the difference sequence should start.
+     * \param end The SOA serial number of the version of the zone at which
+     *        the difference sequence should end.
+     *
+     * \return Newly created iterator context. Must not be NULL.
+     */
+    virtual IteratorContextPtr
+    getDiffs(int id, uint32_t start, uint32_t end) const = 0;
+
     /// Start a transaction for updating a zone.
     ///
     /// Each derived class version of this method starts a database

+ 301 - 25
src/lib/datasrc/sqlite3_accessor.cc

@@ -23,6 +23,7 @@
 #include <datasrc/logger.h>
 #include <datasrc/data_source.h>
 #include <datasrc/factory.h>
+#include <datasrc/database.h>
 #include <util/filename.h>
 
 using namespace std;
@@ -54,7 +55,10 @@ enum StatementID {
     FIND_PREVIOUS = 10,
     ADD_RECORD_DIFF = 11,
     GET_RECORD_DIFF = 12,       // This is temporary for testing "add diff"
-    NUM_STATEMENTS = 13
+    LOW_DIFF_ID = 13,
+    HIGH_DIFF_ID = 14,
+    DIFF_RECS = 15,
+    NUM_STATEMENTS = 16
 };
 
 const char* const text_statements[NUM_STATEMENTS] = {
@@ -62,33 +66,48 @@ const char* const text_statements[NUM_STATEMENTS] = {
     // specifically chosen to match the enum values in RecordColumns
     "SELECT id FROM zones WHERE name=?1 AND rdclass = ?2", // ZONE
     "SELECT rdtype, ttl, sigtype, rdata FROM records "     // ANY
-    "WHERE zone_id=?1 AND name=?2",
+        "WHERE zone_id=?1 AND name=?2",
     "SELECT rdtype, ttl, sigtype, rdata " // ANY_SUB
-    "FROM records WHERE zone_id=?1 AND name LIKE (\"%.\" || ?2)",
+        "FROM records WHERE zone_id=?1 AND name LIKE (\"%.\" || ?2)",
     "BEGIN",                    // BEGIN
     "COMMIT",                   // COMMIT
     "ROLLBACK",                 // ROLLBACK
     "DELETE FROM records WHERE zone_id=?1", // DEL_ZONE_RECORDS
     "INSERT INTO records "      // ADD_RECORD
-    "(zone_id, name, rname, ttl, rdtype, sigtype, rdata) "
-    "VALUES (?1, ?2, ?3, ?4, ?5, ?6, ?7)",
+        "(zone_id, name, rname, ttl, rdtype, sigtype, rdata) "
+        "VALUES (?1, ?2, ?3, ?4, ?5, ?6, ?7)",
     "DELETE FROM records WHERE zone_id=?1 AND name=?2 " // DEL_RECORD
-    "AND rdtype=?3 AND rdata=?4",
+        "AND rdtype=?3 AND rdata=?4",
     "SELECT rdtype, ttl, sigtype, rdata, name FROM records " // ITERATE
-    "WHERE zone_id = ?1 ORDER BY rname, rdtype",
+        "WHERE zone_id = ?1 ORDER BY rname, rdtype",
     /*
      * This one looks for previous name with NSEC record. It is done by
      * using the reversed name. The NSEC is checked because we need to
      * skip glue data, which don't have the NSEC.
      */
     "SELECT name FROM records " // FIND_PREVIOUS
-    "WHERE zone_id=?1 AND rdtype = 'NSEC' AND "
-    "rname < $2 ORDER BY rname DESC LIMIT 1",
+        "WHERE zone_id=?1 AND rdtype = 'NSEC' AND "
+        "rname < $2 ORDER BY rname DESC LIMIT 1",
     "INSERT INTO diffs "        // ADD_RECORD_DIFF
-    "(zone_id, version, operation, name, rrtype, ttl, rdata) "
-    "VALUES (?1, ?2, ?3, ?4, ?5, ?6, ?7)"
-    , "SELECT name, rrtype, ttl, rdata, version, operation " // GET_RECORD_DIFF
-    "FROM diffs WHERE zone_id = ?1 ORDER BY id, operation"
+        "(zone_id, version, operation, name, rrtype, ttl, rdata) "
+        "VALUES (?1, ?2, ?3, ?4, ?5, ?6, ?7)",
+    "SELECT name, rrtype, ttl, rdata, version, operation " // GET_RECORD_DIFF
+        "FROM diffs WHERE zone_id = ?1 ORDER BY id, operation",
+
+    // Two statements to select the lowest ID and highest ID in a set of
+    // differences.
+    "SELECT id FROM diffs "     // LOW_DIFF_ID
+        "WHERE zone_id=?1 AND version=?2 and OPERATION=?3 "
+        "ORDER BY id ASC LIMIT 1",
+    "SELECT id FROM diffs "     // HIGH_DIFF_ID
+        "WHERE zone_id=?1 AND version=?2 and OPERATION=?3 "
+        "ORDER BY id DESC LIMIT 1",
+
+    // In the next statement, note the redundant ID.  This is to ensure
+    // that the columns match the column IDs passed to the iterator
+    "SELECT rrtype, ttl, id, rdata, name FROM diffs "   // DIFF_RECS
+        "WHERE zone_id=?1 AND id>=?2 and id<=?3 "
+        "ORDER BY id ASC"
 };
 
 struct SQLite3Parameters {
@@ -231,23 +250,26 @@ const char* const SCHEMA_LIST[] = {
     "dnssec BOOLEAN NOT NULL DEFAULT 0)",
     "CREATE INDEX zones_byname ON zones (name)",
     "CREATE TABLE records (id INTEGER PRIMARY KEY, "
-    "zone_id INTEGER NOT NULL, name STRING NOT NULL COLLATE NOCASE, "
-    "rname STRING NOT NULL COLLATE NOCASE, ttl INTEGER NOT NULL, "
-    "rdtype STRING NOT NULL COLLATE NOCASE, sigtype STRING COLLATE NOCASE, "
-    "rdata STRING NOT NULL)",
+        "zone_id INTEGER NOT NULL, name STRING NOT NULL COLLATE NOCASE, "
+        "rname STRING NOT NULL COLLATE NOCASE, ttl INTEGER NOT NULL, "
+        "rdtype STRING NOT NULL COLLATE NOCASE, sigtype STRING COLLATE NOCASE, "
+        "rdata STRING NOT NULL)",
     "CREATE INDEX records_byname ON records (name)",
     "CREATE INDEX records_byrname ON records (rname)",
     "CREATE TABLE nsec3 (id INTEGER PRIMARY KEY, zone_id INTEGER NOT NULL, "
-    "hash STRING NOT NULL COLLATE NOCASE, "
-    "owner STRING NOT NULL COLLATE NOCASE, "
-    "ttl INTEGER NOT NULL, rdtype STRING NOT NULL COLLATE NOCASE, "
-    "rdata STRING NOT NULL)",
+        "hash STRING NOT NULL COLLATE NOCASE, "
+        "owner STRING NOT NULL COLLATE NOCASE, "
+        "ttl INTEGER NOT NULL, rdtype STRING NOT NULL COLLATE NOCASE, "
+        "rdata STRING NOT NULL)",
     "CREATE INDEX nsec3_byhash ON nsec3 (hash)",
     "CREATE TABLE diffs (id INTEGER PRIMARY KEY, "
-    "zone_id INTEGER NOT NULL, version INTEGER NOT NULL, "
-    "operation INTEGER NOT NULL, name STRING NOT NULL COLLATE NOCASE, "
-    "rrtype STRING NOT NULL COLLATE NOCASE, ttl INTEGER NOT NULL, "
-    "rdata STRING NOT NULL)",
+        "zone_id INTEGER NOT NULL, "
+        "version INTEGER NOT NULL, "
+        "operation INTEGER NOT NULL, "
+        "name STRING NOT NULL COLLATE NOCASE, "
+        "rrtype STRING NOT NULL COLLATE NOCASE, "
+        "ttl INTEGER NOT NULL, "
+        "rdata STRING NOT NULL)",
     NULL
 };
 
@@ -558,6 +580,9 @@ private:
     const std::string name_;
 };
 
+
+// Methods to retrieve the various iterators
+
 DatabaseAccessor::IteratorContextPtr
 SQLite3Accessor::getRecords(const std::string& name, int id,
                             bool subdomains) const
@@ -571,6 +596,257 @@ SQLite3Accessor::getAllRecords(int id) const {
     return (IteratorContextPtr(new Context(shared_from_this(), id)));
 }
 
+
+/// \brief Difference Iterator
+///
+/// This iterator is used to search through the differences table for the
+/// resouce records making up an IXFR between two versions of a zone.
+
+class SQLite3Accessor::DiffContext : public DatabaseAccessor::IteratorContext {
+public:
+
+    /// \brief Constructor
+    ///
+    /// Constructs the iterator for the difference sequence.  It is
+    /// passed two parameters, the first and last versions in the difference
+    /// sequence.  Note that because of serial number rollover, it may well
+    /// be that the start serial number is greater than the end one.
+    ///
+    /// \param zone_id ID of the zone (in the zone table)
+    /// \param start Serial number of first version in difference sequence
+    /// \param end Serial number of last version in difference sequence
+    ///
+    /// \exception any A number of exceptions can be expected
+    DiffContext(const boost::shared_ptr<const SQLite3Accessor>& accessor,
+                int zone_id, uint32_t start, uint32_t end) :
+        accessor_(accessor),
+        last_status_(SQLITE_ROW)
+    {
+        try {
+            int low_id = findIndex(LOW_DIFF_ID, zone_id, start, DIFF_DELETE);
+            int high_id = findIndex(HIGH_DIFF_ID, zone_id, end, DIFF_ADD);
+
+            // Prepare the statement that will return data values
+            reset(DIFF_RECS);
+            bindInt(DIFF_RECS, 1, zone_id);
+            bindInt(DIFF_RECS, 2, low_id);
+            bindInt(DIFF_RECS, 3, high_id);
+
+        } catch (...) {
+            // Something wrong, clear up everything.
+            accessor_->dbparameters_->finalizeStatements();
+            throw;
+        }
+    }
+
+    /// \brief Destructor
+    virtual ~DiffContext()
+    {}
+
+    /// \brief Get Next Diff Record
+    ///
+    /// Returns the next difference record in the difference sequence.
+    ///
+    /// \param data Array of std::strings COLUMN_COUNT long.  The results
+    ///        are returned in this.
+    ///
+    /// \return bool true if data is returned, false if not.
+    ///
+    /// \exceptions any Varied
+    bool getNext(std::string (&data)[COLUMN_COUNT]) {
+
+        if (last_status_ != SQLITE_DONE) {
+            // Last call (if any) didn't reach end of result set, so we
+            // can read another row from it.
+            //
+            // Get a pointer to the statement for brevity (this does not
+            // transfer ownership of the statement to this class, so there is
+            // no need to tidy up after we have finished using it).
+            sqlite3_stmt* stmt =
+                accessor_->dbparameters_->getStatement(DIFF_RECS);
+
+            const int rc(sqlite3_step(stmt));
+            if (rc == SQLITE_ROW) {
+                // Copy the data across to the output array
+                copyColumn(DIFF_RECS, data, TYPE_COLUMN);
+                copyColumn(DIFF_RECS, data, TTL_COLUMN);
+                copyColumn(DIFF_RECS, data, NAME_COLUMN);
+                copyColumn(DIFF_RECS, data, RDATA_COLUMN);
+
+            } else if (rc != SQLITE_DONE) {
+                isc_throw(DataSourceError,
+                          "Unexpected failure in sqlite3_step: " <<
+                          sqlite3_errmsg(accessor_->dbparameters_->db_));
+            }
+            last_status_ = rc;
+        }
+        return (last_status_ == SQLITE_ROW);
+    }
+
+private:
+
+    /// \brief Reset prepared statement
+    ///
+    /// Sets up the statement so that new parameters can be attached to it and
+    /// that it can be used to query for another difference sequence.
+    ///
+    /// \param stindex Index of prepared statement to which to bind
+    void reset(int stindex) {
+        sqlite3_stmt* stmt = accessor_->dbparameters_->getStatement(stindex);
+        if ((sqlite3_reset(stmt) != SQLITE_OK) ||
+            (sqlite3_clear_bindings(stmt) != SQLITE_OK)) {
+            isc_throw(SQLite3Error, "Could not clear statement bindings in '" <<
+                      text_statements[stindex] << "': " <<
+                      sqlite3_errmsg(accessor_->dbparameters_->db_));
+        }
+    }
+
+    /// \brief Bind Int
+    ///
+    /// Binds an integer to a specific variable in a prepared statement.
+    ///
+    /// \param stindex Index of prepared statement to which to bind
+    /// \param varindex Index of variable to which to bind
+    /// \param value Value of variable to bind
+    /// \exception SQLite3Error on an error
+    void bindInt(int stindex, int varindex, sqlite3_int64 value) {
+        if (sqlite3_bind_int64(accessor_->dbparameters_->getStatement(stindex),
+                             varindex, value) != SQLITE_OK) {
+            isc_throw(SQLite3Error, "Could not bind value to parameter " <<
+                      varindex << " in statement '" <<
+                      text_statements[stindex] << "': " <<
+                      sqlite3_errmsg(accessor_->dbparameters_->db_));
+        }
+    }
+
+    ///\brief Get Single Value
+    ///
+    /// Executes a prepared statement (which has parameters bound to it)
+    /// for which the result of a single value is expected.
+    ///
+    /// \param stindex Index of prepared statement in statement table.
+    ///
+    /// \return Value of SELECT.
+    ///
+    /// \exception TooMuchData Multiple rows returned when one expected
+    /// \exception TooLittleData Zero rows returned when one expected
+    /// \exception DataSourceError SQLite3-related error
+    int getSingleValue(StatementID stindex) {
+
+        // Get a pointer to the statement for brevity (does not transfer
+        // resources)
+        sqlite3_stmt* stmt = accessor_->dbparameters_->getStatement(stindex);
+
+        // Execute the data.  Should be just one result
+        int rc = sqlite3_step(stmt);
+        int result = -1;
+        if (rc == SQLITE_ROW) {
+
+            // Got some data, extract the value
+            result = sqlite3_column_int(stmt, 0);
+            rc = sqlite3_step(stmt);
+            if (rc == SQLITE_DONE) {
+
+                // All OK, exit with the value.
+                return (result);
+
+            } else if (rc == SQLITE_ROW) {
+                isc_throw(TooMuchData, "request to return one value from "
+                          "diffs table returned multiple values");
+            }
+        } else if (rc == SQLITE_DONE) {
+
+            // No data in the table.  A bare exception with no explanation is
+            // thrown, as it will be replaced by a more informative one by
+            // the caller.
+            isc_throw(TooLittleData, "");
+        }
+
+        // We get here on an error.
+        isc_throw(DataSourceError, "could not get data from diffs table: " <<
+                  sqlite3_errmsg(accessor_->dbparameters_->db_));
+
+        // Keep the compiler happy with a return value.
+        return (result);
+    }
+
+    /// \brief Find index
+    ///
+    /// Executes the prepared statement locating the high or low index in
+    /// the diffs table and returns that index.
+    ///
+    /// \param stmt_id Index of the prepared statement to execute
+    /// \param zone_id ID of the zone for which the index is being sought
+    /// \param serial Zone serial number for which an index is being sought.
+    /// \param diff Code to delete record additions or deletions
+    ///
+    /// \return int ID of the row in the difss table corresponding to the
+    ///         statement.
+    ///
+    /// \exception TooLittleData Internal error, no result returned when one
+    ///            was expected.
+    /// \exception NoSuchSerial Serial number not found.
+    /// \exception NoDiffsData No data for this zone found in diffs table
+    int findIndex(StatementID stindex, int zone_id, uint32_t serial, int diff) {
+
+        // Set up the statement
+        reset(stindex);
+        bindInt(stindex, 1, zone_id);
+        bindInt(stindex, 2, serial);
+        bindInt(stindex, 3, diff);
+
+        // Execute the statement
+        int result = -1;
+        try {
+            result = getSingleValue(stindex);
+
+        } catch (const TooLittleData&) {
+
+            // No data returned but the SQL query succeeded.  Only possibility
+            // is that there is no entry in the differences table for the given
+            // zone and version.
+            isc_throw(NoSuchSerial, "No entry in differences table for " <<
+                      " zone ID " << zone_id << ", serial number " << serial);
+        }
+
+        return (result);
+    }
+
+    /// \brief Copy Column to Output
+    ///
+    /// Copies the textual data in the result set to the specified column
+    /// in the output.
+    ///
+    /// \param stindex Index of prepared statement used to access data
+    /// \param data Array of columns passed to getNext
+    /// \param column Column of output to copy
+    void copyColumn(StatementID stindex, std::string (&data)[COLUMN_COUNT],
+                    int column) {
+
+        // Get a pointer to the statement for brevity (does not transfer
+        // resources)
+        sqlite3_stmt* stmt = accessor_->dbparameters_->getStatement(stindex);
+        data[column] = convertToPlainChar(sqlite3_column_text(stmt,
+                                                              column),
+                                          accessor_->dbparameters_->db_);
+    }
+
+    // Attributes
+
+    boost::shared_ptr<const SQLite3Accessor> accessor_; // Accessor object
+    int last_status_;           // Last status received from sqlite3_step
+};
+
+// ... and return the iterator
+
+DatabaseAccessor::IteratorContextPtr
+SQLite3Accessor::getDiffs(int id, uint32_t start, uint32_t end) const {
+    return (IteratorContextPtr(new DiffContext(shared_from_this(), id, start,
+                               end)));
+}
+
+
+
 pair<bool, int>
 SQLite3Accessor::startUpdateZone(const string& zone_name, const bool replace) {
     if (dbparameters_->updating_zone) {

+ 69 - 4
src/lib/datasrc/sqlite3_accessor.h

@@ -17,6 +17,7 @@
 #define __DATASRC_SQLITE3_ACCESSOR_H
 
 #include <datasrc/database.h>
+#include <datasrc/data_source.h>
 
 #include <exceptions/exceptions.h>
 
@@ -40,12 +41,49 @@ namespace datasrc {
  * It might mean corrupt database file, invalid request or that something is
  * rotten in the library.
  */
-class SQLite3Error : public Exception {
+class SQLite3Error : public DataSourceError {
 public:
     SQLite3Error(const char* file, size_t line, const char* what) :
-        isc::Exception(file, line, what) {}
+        DataSourceError(file, line, what) {}
 };
 
+/**
+ * \brief Too Much Data
+ *
+ * Thrown if a query expecting a certain number of rows back returned too
+ * many rows.
+ */
+class TooMuchData : public DataSourceError {
+public:
+    TooMuchData(const char* file, size_t line, const char* what) :
+        DataSourceError(file, line, what) {}
+};
+
+/**
+ * \brief Too Little Data
+ *
+ * Thrown if a query expecting a certain number of rows back returned too
+ * few rows (including none).
+ */
+class TooLittleData : public DataSourceError {
+public:
+    TooLittleData(const char* file, size_t line, const char* what) :
+        DataSourceError(file, line, what) {}
+};
+
+/**
+ * \brief No such serial number when obtaining difference iterator
+ *
+ * Thrown if either the zone/start version or zone/end version combination
+ * does not exist in the differences table.
+ */
+class NoSuchSerial : public DataSourceError {
+public:
+    NoSuchSerial(const char* file, size_t line, const char* what) :
+        DataSourceError(file, line, what) {}
+};
+
+
 struct SQLite3Parameters;
 
 /**
@@ -128,6 +166,27 @@ public:
      */
     virtual IteratorContextPtr getAllRecords(int id) const;
 
+    /** \brief Creates an iterator context for a set of differences.
+     *
+     * Implements the getDiffs() method from DatabaseAccessor
+     *
+     * \exception NoSuchSerial if either of the versions do not exist in
+     *            the difference table.
+     * \exception SQLite3Error if there is an sqlite3 error when performing
+     *            the query
+     *
+     * \param id The ID of the zone, returned from getZone().
+     * \param start The SOA serial number of the version of the zone from
+     *        which the difference sequence should start.
+     * \param end The SOA serial number of the version of the zone at which
+     *        the difference sequence should end.
+     *
+     * \return Iterator containing difference records.
+     */
+    virtual IteratorContextPtr
+    getDiffs(int id, uint32_t start, uint32_t end) const;
+
+
     virtual std::pair<bool, int> startUpdateZone(const std::string& zone_name,
                                                  bool replace);
 
@@ -192,14 +251,20 @@ private:
     const std::string filename_;
     /// \brief The class for which the queries are done
     const std::string class_;
+    /// \brief Database name
+    const std::string database_name_;
+
     /// \brief Opens the database
     void open(const std::string& filename);
     /// \brief Closes the database
     void close();
-    /// \brief SQLite3 implementation of IteratorContext
+
+    /// \brief SQLite3 implementation of IteratorContext for all records
     class Context;
     friend class Context;
-    const std::string database_name_;
+    /// \brief SQLite3 implementation of IteratorContext for differences
+    class DiffContext;
+    friend class DiffContext;
 };
 
 /// \brief Creates an instance of the SQlite3 datasource client

+ 1 - 0
src/lib/datasrc/tests/Makefile.am

@@ -78,3 +78,4 @@ EXTRA_DIST += testdata/test-root.sqlite3
 EXTRA_DIST += testdata/test.sqlite3
 EXTRA_DIST += testdata/test.sqlite3.nodiffs
 EXTRA_DIST += testdata/rwtest.sqlite3
+EXTRA_DIST += testdata/diffs.sqlite3

+ 5 - 0
src/lib/datasrc/tests/database_unittest.cc

@@ -255,6 +255,11 @@ public:
                   "This database datasource can't be iterated");
     }
 
+    virtual IteratorContextPtr getDiffs(int, uint32_t, uint32_t) const {
+        isc_throw(isc::NotImplemented,
+                  "This database datasource can't be iterated");
+    }
+
     virtual std::string findPreviousName(int, const std::string&) const {
         isc_throw(isc::NotImplemented,
                   "This data source doesn't support DNSSEC");

+ 141 - 62
src/lib/datasrc/tests/sqlite3_accessor_unittest.cc

@@ -46,6 +46,7 @@ std::string SQLITE_DBNAME_EXAMPLE_ROOT = "sqlite3_test-root.sqlite3";
 std::string SQLITE_DBFILE_BROKENDB = TEST_DATA_DIR "/brokendb.sqlite3";
 std::string SQLITE_DBFILE_MEMORY = ":memory:";
 std::string SQLITE_DBFILE_EXAMPLE_ORG = TEST_DATA_DIR "/example.org.sqlite3";
+std::string SQLITE_DBFILE_DIFFS = TEST_DATA_DIR "/diffs.sqlite3";
 
 // The following file must be non existent and must be non"creatable";
 // the sqlite3 library will try to create a new DB file if it doesn't exist,
@@ -116,6 +117,26 @@ TEST_F(SQLite3AccessorTest, noClass) {
     EXPECT_FALSE(accessor->getZone("example.com.").first);
 }
 
+// Simple check to test that the sequence is valid.  It gets the next record
+// from the iterator, checks that it is not null, then checks the data.
+void checkRR(DatabaseAccessor::IteratorContextPtr& context,
+     std::string name, std::string ttl, std::string type, std::string rdata) {
+
+    // Mark where we are in the text
+    SCOPED_TRACE(name + " " + ttl + " " + type + " " + rdata);
+
+    std::string data[DatabaseAccessor::COLUMN_COUNT];
+
+    // Get next record
+    EXPECT_TRUE(context->getNext(data));
+
+    // ... and check expected values
+    EXPECT_EQ(name, data[DatabaseAccessor::NAME_COLUMN]);
+    EXPECT_EQ(ttl, data[DatabaseAccessor::TTL_COLUMN]);
+    EXPECT_EQ(type, data[DatabaseAccessor::TYPE_COLUMN]);
+    EXPECT_EQ(rdata, data[DatabaseAccessor::RDATA_COLUMN]);
+}
+
 // This tests the iterator context
 TEST_F(SQLite3AccessorTest, iterator) {
     // Our test zone is conveniently small, but not empty
@@ -130,80 +151,138 @@ TEST_F(SQLite3AccessorTest, iterator) {
     ASSERT_NE(DatabaseAccessor::IteratorContextPtr(), context);
 
     std::string data[DatabaseAccessor::COLUMN_COUNT];
-    // Get and check the first and only record
-    EXPECT_TRUE(context->getNext(data));
-    EXPECT_EQ("MX", data[DatabaseAccessor::TYPE_COLUMN]);
-    EXPECT_EQ("3600", data[DatabaseAccessor::TTL_COLUMN]);
-    EXPECT_EQ("10 mail.example.org.", data[DatabaseAccessor::RDATA_COLUMN]);
-    EXPECT_EQ("example.org.", data[DatabaseAccessor::NAME_COLUMN]);
 
-    EXPECT_TRUE(context->getNext(data));
-    EXPECT_EQ("NS", data[DatabaseAccessor::TYPE_COLUMN]);
-    EXPECT_EQ("3600", data[DatabaseAccessor::TTL_COLUMN]);
-    EXPECT_EQ("ns1.example.org.", data[DatabaseAccessor::RDATA_COLUMN]);
-    EXPECT_EQ("example.org.", data[DatabaseAccessor::NAME_COLUMN]);
+    checkRR(context, "example.org.", "3600", "MX", "10 mail.example.org.");
+    checkRR(context, "example.org.", "3600", "NS", "ns1.example.org.");
+    checkRR(context, "example.org.", "3600", "NS", "ns2.example.org.");
+    checkRR(context, "example.org.", "3600", "NS", "ns3.example.org.");
+    checkRR(context, "example.org.", "3600", "SOA",
+            "ns1.example.org. admin.example.org. 1234 3600 1800 2419200 7200");
+    checkRR(context, "dname.example.org.", "3600", "DNAME",
+            "dname.example.info.");
+    checkRR(context, "dname2.foo.example.org.", "3600", "DNAME",
+            "dname2.example.info.");
+    checkRR(context, "mail.example.org.", "3600", "A", "192.0.2.10");
+    checkRR(context, "sub.example.org.", "3600", "NS", "ns.sub.example.org.");
+    checkRR(context, "ns.sub.example.org.", "3600", "A", "192.0.2.101");
+    checkRR(context, "www.example.org.", "3600", "A", "192.0.2.1");
 
-    EXPECT_TRUE(context->getNext(data));
-    EXPECT_EQ("NS", data[DatabaseAccessor::TYPE_COLUMN]);
-    EXPECT_EQ("3600", data[DatabaseAccessor::TTL_COLUMN]);
-    EXPECT_EQ("ns2.example.org.", data[DatabaseAccessor::RDATA_COLUMN]);
-    EXPECT_EQ("example.org.", data[DatabaseAccessor::NAME_COLUMN]);
+    // Check there's no other
+    EXPECT_FALSE(context->getNext(data));
 
-    EXPECT_TRUE(context->getNext(data));
-    EXPECT_EQ("NS", data[DatabaseAccessor::TYPE_COLUMN]);
-    EXPECT_EQ("3600", data[DatabaseAccessor::TTL_COLUMN]);
-    EXPECT_EQ("ns3.example.org.", data[DatabaseAccessor::RDATA_COLUMN]);
-    EXPECT_EQ("example.org.", data[DatabaseAccessor::NAME_COLUMN]);
+    // And make sure calling it again won't cause problems.
+    EXPECT_FALSE(context->getNext(data));
+}
 
-    EXPECT_TRUE(context->getNext(data));
-    EXPECT_EQ("SOA", data[DatabaseAccessor::TYPE_COLUMN]);
-    EXPECT_EQ("3600", data[DatabaseAccessor::TTL_COLUMN]);
-    EXPECT_EQ("ns1.example.org. admin.example.org. "
-              "1234 3600 1800 2419200 7200",
-              data[DatabaseAccessor::RDATA_COLUMN]);
-    EXPECT_EQ("example.org.", data[DatabaseAccessor::NAME_COLUMN]);
+// This tests the difference iterator context
 
-    EXPECT_TRUE(context->getNext(data));
-    EXPECT_EQ("DNAME", data[DatabaseAccessor::TYPE_COLUMN]);
-    EXPECT_EQ("3600", data[DatabaseAccessor::TTL_COLUMN]);
-    EXPECT_EQ("dname.example.info.", data[DatabaseAccessor::RDATA_COLUMN]);
-    EXPECT_EQ("dname.example.org.", data[DatabaseAccessor::NAME_COLUMN]);
+// Test that at attempt to create a difference iterator for a serial number
+// that does not exist throws an exception.
+TEST_F(SQLite3AccessorTest, diffIteratorNoRecords) {
 
-    EXPECT_TRUE(context->getNext(data));
-    EXPECT_EQ("DNAME", data[DatabaseAccessor::TYPE_COLUMN]);
-    EXPECT_EQ("3600", data[DatabaseAccessor::TTL_COLUMN]);
-    EXPECT_EQ("dname2.example.info.", data[DatabaseAccessor::RDATA_COLUMN]);
-    EXPECT_EQ("dname2.foo.example.org.", data[DatabaseAccessor::NAME_COLUMN]);
+    // Our test zone is conveniently small, but not empty
+    initAccessor(SQLITE_DBFILE_DIFFS, "IN");
 
-    EXPECT_TRUE(context->getNext(data));
-    EXPECT_EQ("A", data[DatabaseAccessor::TYPE_COLUMN]);
-    EXPECT_EQ("3600", data[DatabaseAccessor::TTL_COLUMN]);
-    EXPECT_EQ("192.0.2.10", data[DatabaseAccessor::RDATA_COLUMN]);
-    EXPECT_EQ("mail.example.org.", data[DatabaseAccessor::NAME_COLUMN]);
+    const std::pair<bool, int> zone_info(accessor->getZone("example.org."));
+    ASSERT_TRUE(zone_info.first);
 
-    EXPECT_TRUE(context->getNext(data));
-    EXPECT_EQ("NS", data[DatabaseAccessor::TYPE_COLUMN]);
-    EXPECT_EQ("3600", data[DatabaseAccessor::TTL_COLUMN]);
-    EXPECT_EQ("ns.sub.example.org.", data[DatabaseAccessor::RDATA_COLUMN]);
-    EXPECT_EQ("sub.example.org.", data[DatabaseAccessor::NAME_COLUMN]);
+    // Get the iterator context.  Difference of version 1 does not exist, so
+    // this should throw an exception.
+    EXPECT_THROW(accessor->getDiffs(zone_info.second, 1, 1234),
+                 isc::datasrc::NoSuchSerial);
 
-    EXPECT_TRUE(context->getNext(data));
-    EXPECT_EQ("A", data[DatabaseAccessor::TYPE_COLUMN]);
-    EXPECT_EQ("3600", data[DatabaseAccessor::TTL_COLUMN]);
-    EXPECT_EQ("192.0.2.101", data[DatabaseAccessor::RDATA_COLUMN]);
-    EXPECT_EQ("ns.sub.example.org.", data[DatabaseAccessor::NAME_COLUMN]);
+    // Check that an invalid high version number also throws an exception.
+    EXPECT_THROW(accessor->getDiffs(zone_info.second, 1231, 2234),
+                 NoSuchSerial);
 
-    EXPECT_TRUE(context->getNext(data));
-    EXPECT_EQ("A", data[DatabaseAccessor::TYPE_COLUMN]);
-    EXPECT_EQ("3600", data[DatabaseAccessor::TTL_COLUMN]);
-    EXPECT_EQ("192.0.2.1", data[DatabaseAccessor::RDATA_COLUMN]);
-    EXPECT_EQ("www.example.org.", data[DatabaseAccessor::NAME_COLUMN]);
+    // Check that valid versions - but for the wrong zone which does not hold
+    // any records - also throws this exception.
+    EXPECT_THROW(accessor->getDiffs(zone_info.second + 42, 1231, 1234),
+                 NoSuchSerial);
 
-    // Check there's no other
-    EXPECT_FALSE(context->getNext(data));
+}
 
-    // And make sure calling it again won't cause problems.
-    EXPECT_FALSE(context->getNext(data));
+// Try to iterate through a valid sets of differences
+TEST_F(SQLite3AccessorTest, diffIteratorSequences) {
+    std::string data[DatabaseAccessor::COLUMN_COUNT];
+
+    // Our test zone is conveniently small, but not empty
+    initAccessor(SQLITE_DBFILE_DIFFS, "IN");
+    const std::pair<bool, int> zone_info(accessor->getZone("example.org."));
+    ASSERT_TRUE(zone_info.first);
+
+
+    // Check the difference sequence 1230-1231 (two adjacent differences)
+    // Get the iterator context
+    DatabaseAccessor::IteratorContextPtr
+        context1(accessor->getDiffs(zone_info.second, 1230, 1231));
+    ASSERT_NE(DatabaseAccessor::IteratorContextPtr(), context1);
+
+    // Change: 1230-1231
+    checkRR(context1, "example.org.", "1800", "SOA",
+            "ns1.example.org. admin.example.org. 1230 3600 1800 2419200 7200");
+    checkRR(context1, "example.org.", "3600", "SOA",
+            "ns1.example.org. admin.example.org. 1231 3600 1800 2419200 7200");
+
+    // Check there's no other and that calling it again after no records doesn't
+    // cause problems.
+    EXPECT_FALSE(context1->getNext(data));
+    EXPECT_FALSE(context1->getNext(data));
+
+
+    // Check that the difference sequence 1231-1233 (two separate difference
+    // sequences) is OK.
+    DatabaseAccessor::IteratorContextPtr
+        context2(accessor->getDiffs(zone_info.second, 1231, 1233));
+    ASSERT_NE(DatabaseAccessor::IteratorContextPtr(), context2);
+
+    // Change 1231-1232
+    checkRR(context2, "example.org.", "3600", "SOA",
+            "ns1.example.org. admin.example.org. 1231 3600 1800 2419200 7200");
+    checkRR(context2, "unused.example.org.", "3600", "A", "192.0.2.102");
+    checkRR(context2, "example.org.", "3600", "SOA",
+            "ns1.example.org. admin.example.org. 1232 3600 1800 2419200 7200");
+
+    // Change: 1232-1233
+    checkRR(context2, "example.org.", "3600", "SOA",
+            "ns1.example.org. admin.example.org. 1232 3600 1800 2419200 7200");
+    checkRR(context2, "example.org.", "3600", "SOA",
+            "ns1.example.org. admin.example.org. 1233 3600 1800 2419200 7200");
+    checkRR(context2, "sub.example.org.", "3600", "NS", "ns.sub.example.org.");
+    checkRR(context2, "ns.sub.example.org.", "3600", "A", "192.0.2.101");
+
+    // Check there's no other and that calling it again after no records doesn't
+    // cause problems.
+    EXPECT_FALSE(context2->getNext(data));
+    EXPECT_FALSE(context2->getNext(data));
+
+
+    // Check that the difference sequence 4294967280 to 1230 (serial number
+    // rollover) is OK
+    DatabaseAccessor::IteratorContextPtr
+        context3(accessor->getDiffs(zone_info.second, 4294967280U, 1230));
+    ASSERT_NE(DatabaseAccessor::IteratorContextPtr(), context3);
+
+    // Change 4294967280 to 1230.
+    checkRR(context3, "example.org.", "3600", "SOA",
+            "ns1.example.org. admin.example.org. 4294967280 3600 1800 2419200 7200");
+    checkRR(context3, "www.example.org.", "3600", "A", "192.0.2.31");
+    checkRR(context3, "example.org.", "1800", "SOA",
+            "ns1.example.org. admin.example.org. 1230 3600 1800 2419200 7200");
+    checkRR(context3, "www.example.org.", "3600", "A", "192.0.2.21");
+
+    EXPECT_FALSE(context3->getNext(data));
+    EXPECT_FALSE(context3->getNext(data));
+
+
+    // Check the difference sequence 1233-1231 (versions in wrong order).  This
+    // should give an empty difference set.
+    DatabaseAccessor::IteratorContextPtr
+        context4(accessor->getDiffs(zone_info.second, 1233, 1231));
+    ASSERT_NE(DatabaseAccessor::IteratorContextPtr(), context2);
+
+    EXPECT_FALSE(context4->getNext(data));
+    EXPECT_FALSE(context4->getNext(data));
 }
 
 TEST(SQLite3Open, getDBNameExample2) {

BIN
src/lib/datasrc/tests/testdata/brokendb.sqlite3


BIN
src/lib/datasrc/tests/testdata/diffs.sqlite3


+ 123 - 0
src/lib/datasrc/tests/testdata/diffs_table.sql

@@ -0,0 +1,123 @@
+-- Copyright (C) 2011  Internet Systems Consortium, Inc. ("ISC")
+--
+-- Permission to use, copy, modify, and/or distribute this software for any
+-- purpose with or without fee is hereby granted, provided that the above
+-- copyright notice and this permission notice appear in all copies.
+--
+-- THE SOFTWARE IS PROVIDED "AS IS" AND ISC DISCLAIMS ALL WARRANTIES WITH
+-- REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY
+-- AND FITNESS.  IN NO EVENT SHALL ISC BE LIABLE FOR ANY SPECIAL, DIRECT,
+-- INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM
+-- LOSS OF USE, DATA OR PROFITS, WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE
+-- OR OTHER TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE USE OR
+-- PERFORMANCE OF THIS SOFTWARE.
+
+-- \brief Create Differences Table
+--
+-- This is a short-term solution to creating the differences table for testing
+-- purposes.
+--
+-- It is assumed that the database used is a copy of the "example.org.sqlite3"
+-- database in this test directory.  The diffs table is created and populated
+-- with a set of RRs that purport to represent differences that end in the
+-- zone as is.
+--
+-- The file can be executed by the command:
+-- % sqlite3 -init <this-file> <database-file> ".quit"
+--
+-- The file gets executed as the set of SQL statements on the database file,
+-- the ".quit" on the command line then  getting executed to exit SQLite3.
+
+-- Create the diffs table
+DROP TABLE diffs;
+CREATE TABLE diffs (id INTEGER PRIMARY KEY,
+                    zone_id INTEGER NOT NULL,
+                    version INTEGER NOT NULL,
+                    operation INTEGER NOT NULL,
+                    name STRING NOT NULL COLLATE NOCASE,
+                    rrtype STRING NOT NULL COLLATE NOCASE,
+                    ttl INTEGER NOT NULL,
+                    rdata STRING NOT NULL);
+
+-- Populate it.  A dummy zone_id is used for now - this will be updated last of
+-- all.
+
+-- Change from 4294967280 (0xfffffff0) to 1230 to show serial rollover
+-- Update one record in the zone.
+INSERT INTO diffs(zone_id, version, operation, name, rrtype, ttl, rdata)
+    VALUES(1, 4294967280,  1, "example.org.", "SOA", 3600,
+           "ns1.example.org. admin.example.org. 4294967280 3600 1800 2419200 7200");
+INSERT INTO diffs(zone_id, version, operation, name, rrtype, ttl, rdata)
+    VALUES(1, 4294967280, 1, "www.example.org.", "A", 3600, "192.0.2.31");
+
+-- Records added in version 1230 of the zone
+INSERT INTO diffs(zone_id, version, operation, name, rrtype, ttl, rdata)
+    VALUES(1, 1230, 0, "example.org.", "SOA", 1800,
+           "ns1.example.org. admin.example.org. 1230 3600 1800 2419200 7200");
+INSERT INTO diffs(zone_id, version, operation, name, rrtype, ttl, rdata)
+    VALUES(1, 1230, 0, "www.example.org.", "A", 3600, "192.0.2.21");
+
+-- Change 1230 to 1231: Change change a parameter of the SOA record
+-- Records removed from version 1230 of the zone
+INSERT INTO diffs(zone_id, version, operation, name, rrtype, ttl, rdata)
+    VALUES(1, 1230, 1, "example.org.", "SOA", 1800,
+           "ns1.example.org. admin.example.org. 1230 3600 1800 2419200 7200");
+
+-- Records added in version 1231 of the zone
+INSERT INTO diffs(zone_id, version, operation, name, rrtype, ttl, rdata)
+    VALUES(1, 1231, 0, "example.org.", "SOA", 3600,
+           "ns1.example.org. admin.example.org. 1231 3600 1800 2419200 7200");
+
+
+-- Change 1231 to 1232: Remove one record, don't add anything.
+-- Records removed from version 1231 of the zone
+INSERT INTO diffs(zone_id, version, operation, name, rrtype, ttl, rdata)
+    VALUES(1, 1231, 1, "example.org.", "SOA", 3600,
+           "ns1.example.org. admin.example.org. 1231 3600 1800 2419200 7200");
+INSERT INTO diffs(zone_id, version, operation, name, rrtype, ttl, rdata)
+    VALUES(1, 1231, 1, "unused.example.org.", "A", 3600, "192.0.2.102");
+
+-- Records added in version 1232 of the zone
+INSERT INTO diffs(zone_id, version, operation, name, rrtype, ttl, rdata)
+    VALUES(1, 1232, 0, "example.org.", "SOA", 3600,
+           "ns1.example.org. admin.example.org. 1232 3600 1800 2419200 7200");
+
+-- Change 1232 to 1233: Add two, don't remove anything.
+-- Records removed from version 1232 of the zone
+INSERT INTO diffs(zone_id, version, operation, name, rrtype, ttl, rdata)
+    VALUES(1, 1232, 1, "example.org.", "SOA", 3600,
+           "ns1.example.org. admin.example.org. 1232 3600 1800 2419200 7200");
+
+-- Records added in version 1233 of the zone
+INSERT INTO diffs(zone_id, version, operation, name, rrtype, ttl, rdata)
+    VALUES(1, 1233, 0, "example.org.", "SOA", 3600,
+           "ns1.example.org. admin.example.org. 1233 3600 1800 2419200 7200");
+INSERT INTO diffs(zone_id, version, operation, name, rrtype, ttl, rdata)
+    VALUES(1, 1233, 0, "sub.example.org.", "NS", 3600, "ns.sub.example.org.");
+INSERT INTO diffs(zone_id, version, operation, name, rrtype, ttl, rdata)
+    VALUES(1, 1233, 0, "ns.sub.example.org.", "A", 3600, "192.0.2.101");
+
+
+-- Change 1233 to 1234: change addresses of two A records
+-- Records removed from version 1233 of the zone
+INSERT INTO diffs(zone_id, version, operation, name, rrtype, ttl, rdata)
+    VALUES(1, 1233, 1, "example.org.", "SOA", 3600,
+           "ns1.example.org. admin.example.org. 1233 3600 1800 2419200 7200");
+INSERT INTO diffs(zone_id, version, operation, name, rrtype, ttl, rdata)
+    VALUES(1, 1233, 1, "www.example.org.", "A", 3600, "192.0.2.21");
+INSERT INTO diffs(zone_id, version, operation, name, rrtype, ttl, rdata)
+    VALUES(1, 1233, 1, "mail.example.org.", "A", 3600, "192.0.2.210");
+
+-- Records added in version 1234 of the zone
+INSERT INTO diffs(zone_id, version, operation, name, rrtype, ttl, rdata)
+    VALUES(1, 1234, 0, "example.org.", "SOA", 3600,
+           "ns1.example.org. admin.example.org. 1234 3600 1800 2419200 7200");
+INSERT INTO diffs(zone_id, version, operation, name, rrtype, ttl, rdata)
+    VALUES(1, 1234, 0, "www.example.org.", "A", 3600, "192.0.2.1");
+INSERT INTO diffs(zone_id, version, operation, name, rrtype, ttl, rdata)
+    VALUES(1, 1234, 0, "mail.example.org.", "A", 3600, "192.0.2.10");
+
+-- Finally, update the zone_id in the diffs table with what is actually
+-- in the zone table.
+UPDATE diffs SET zone_id =
+   (SELECT id FROM ZONES LIMIT 1);

BIN
src/lib/datasrc/tests/testdata/example.org.sqlite3


BIN
src/lib/datasrc/tests/testdata/example2.com.sqlite3


BIN
src/lib/datasrc/tests/testdata/rwtest.sqlite3


BIN
src/lib/datasrc/tests/testdata/test-root.sqlite3