Bug 1305563 - Add a bookmark buffer and two-way merger for synced bookmarks. r?markh,rnewman,tcsc draft
authorKit Cambridge <kit@yakshaving.ninja>
Thu, 07 Sep 2017 23:48:22 -0700
changeset 714134 634103d6aa193ea5cf4492af2ab7c050c230bf86
parent 713607 cd50ddedbd7c4b4d976b4c290dcc1b1e20cdf66c
child 714135 1e594caec99e2d323033145ef9fcb4e5f9029710
push id93863
push userbmo:kit@mozilla.com
push dateThu, 21 Dec 2017 19:42:35 +0000
reviewersmarkh, rnewman, tcsc
bugs1305563
milestone59.0a1
Bug 1305563 - Add a bookmark buffer and two-way merger for synced bookmarks. r?markh,rnewman,tcsc This patch adds a Sync bookmark buffer and two-way merger. The buffer stages incoming bookmarks, attaches to Places, builds a merged tree from the local and remote trees, mass-inserts changes from the buffer into Places, applies remote deletions, and fires observer notifications for all items changed in the merge. MozReview-Commit-ID: MbeFQUargt
services/sync/modules/engines/bookmarks.js
toolkit/components/places/PlacesSyncUtils.jsm
toolkit/components/places/SyncedBookmarksMirror.jsm
toolkit/components/places/moz.build
toolkit/components/places/tests/moz.build
toolkit/components/places/tests/sync/head_sync.js
toolkit/components/places/tests/sync/livemark.xml
toolkit/components/places/tests/sync/test_bookmark_corruption.js
toolkit/components/places/tests/sync/test_bookmark_deduping.js
toolkit/components/places/tests/sync/test_bookmark_deletion.js
toolkit/components/places/tests/sync/test_bookmark_kinds.js
toolkit/components/places/tests/sync/test_bookmark_structure_changes.js
toolkit/components/places/tests/sync/test_bookmark_value_changes.js
toolkit/components/places/tests/sync/xpcshell.ini
tools/lint/eslint/modules.json
--- a/services/sync/modules/engines/bookmarks.js
+++ b/services/sync/modules/engines/bookmarks.js
@@ -123,17 +123,17 @@ PlacesItem.prototype = {
   toSyncBookmark() {
     let result = {
       kind: this.type,
       recordId: this.id,
       parentRecordId: this.parentid,
     };
     let dateAdded = PlacesSyncUtils.bookmarks.ratchetTimestampBackwards(
       this.dateAdded, +this.modified * 1000);
-    if (dateAdded !== undefined) {
+    if (dateAdded > 0) {
       result.dateAdded = dateAdded;
     }
     return result;
   },
 
   // Populates the record from a Sync bookmark object returned from
   // `PlacesSyncUtils.bookmarks.fetch`.
   fromSyncBookmark(item) {
--- a/toolkit/components/places/PlacesSyncUtils.jsm
+++ b/toolkit/components/places/PlacesSyncUtils.jsm
@@ -559,29 +559,36 @@ const BookmarkSyncUtils = PlacesSyncUtil
    *        A changeset containing sync change records, as returned by
    *        `pullChanges`.
    * @return {Promise} resolved once all records have been updated.
    */
   pushChanges(changeRecords) {
     return PlacesUtils.withConnectionWrapper(
       "BookmarkSyncUtils: pushChanges", async function(db) {
         let skippedCount = 0;
+        let weakCount = 0;
         let updateParams = [];
 
         for (let recordId in changeRecords) {
           // Validate change records to catch coding errors.
           let changeRecord = validateChangeRecord(
             "BookmarkSyncUtils: pushChanges",
             changeRecords[recordId], {
               tombstone: { required: true },
               counter: { required: true },
               synced: { required: true },
             }
           );
 
+          // Skip weakly uploaded records.
+          if (!changeRecord.counter) {
+            weakCount++;
+            continue;
+          }
+
           // Sync sets the `synced` flag for reconciled or successfully
           // uploaded items. If upload failed, ignore the change; we'll
           // try again on the next sync.
           if (!changeRecord.synced) {
             skippedCount++;
             continue;
           }
 
@@ -612,17 +619,18 @@ const BookmarkSyncUtils = PlacesSyncUtil
             await db.executeCached(`
               DELETE FROM moz_bookmarks_deleted
               WHERE guid = :guid`,
               deleteParams);
           });
         }
 
         BookmarkSyncLog.debug(`pushChanges: Processed change records`,
-                              { skipped: skippedCount,
+                              { weak: weakCount,
+                                skipped: skippedCount,
                                 updated: updateParams.length });
       }
     );
   },
 
   /**
    * Removes items from the database. Sync buffers incoming tombstones, and
    * calls this method to apply them at the end of each sync. Deletion
@@ -1019,24 +1027,24 @@ const BookmarkSyncUtils = PlacesSyncUtil
       WHERE type = :type AND
             fk = (SELECT id FROM moz_places WHERE url_hash = hash(:url) AND
                   url = :url)`,
       { syncChangeDelta, type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
         url: url.href });
   },
 
   /**
-   * Returns `undefined` if no sensible timestamp could be found.
+   * Returns `0` if no sensible timestamp could be found.
    * Otherwise, returns the earliest sensible timestamp between `existingMillis`
    * and `serverMillis`.
    */
   ratchetTimestampBackwards(existingMillis, serverMillis, lowerBound = BookmarkSyncUtils.EARLIEST_BOOKMARK_TIMESTAMP) {
     const possible = [+existingMillis, +serverMillis].filter(n => !isNaN(n) && n > lowerBound);
     if (!possible.length) {
-      return undefined;
+      return 0;
     }
     return Math.min(...possible);
   },
 
   /**
    * Rebuilds the left pane query for the mobile root under "All Bookmarks" if
    * necessary. Record calls this method at the end of each bookmark record. This
    * code should eventually move to `PlacesUIUtils#maybeRebuildLeftPane`; see
new file mode 100644
--- /dev/null
+++ b/toolkit/components/places/SyncedBookmarksMirror.jsm
@@ -0,0 +1,3698 @@
+/* This Source Code Form is subject to the terms of the Mozilla Public
+ * License, v. 2.0. If a copy of the MPL was not distributed with this
+ * file, You can obtain one at http://mozilla.org/MPL/2.0/. */
+
+"use strict";
+
+/**
+ * This file implements a mirror and two-way merger for synced bookmarks. The
+ * mirror matches the complete tree stored on the Sync server, and stages new
+ * bookmarks changed on the server since the last sync. The merger walks the
+ * local tree in Places and the mirrored remote tree, produces a new merged
+ * tree, then updates the local tree to reflect the merged tree.
+ *
+ * Let's start with an overview of the different classes, and how they fit
+ * together.
+ *
+ * - `SyncedBookmarksMirror` sets up the database, validates and upserts new
+ *   incoming records, attaches to Places, and applies the changed records.
+ *   During application, we fetch the local and remote bookmark trees, merge
+ *   them, and update Places to match. Merging and application happen in a
+ *   single transaction, so applying the merged tree won't collide with local
+ *   changes.
+ *
+ * - A `BookmarkTree` is a fully rooted tree that also notes deletions. A
+ *   `BookmarkNode` represents a local item in Places, or a remote item in the
+ *   mirror.
+ *
+ * - A `MergedBookmarkNode` holds a local node, a remote node, and a
+ *   `MergeState` that indicates which node to prefer when updating Places and
+ *   the server to match the merged tree.
+ *
+ * - `BookmarkObserverRecorder` records all changes made to Places during the
+ *   merge, then dispatches `nsINavBookmarkObserver` notifications once the
+ *   transaction commits and the database is consistent again. Places uses these
+ *   notifications to update the UI and internal caches. We can't dispatch
+ *   during the merge because the database might not be consistent.
+ *
+ * - After application, we flag all applied incoming items as unchanged, create
+ *   Sync records for the locally new and updated items in Places, and upload
+ *   the records to the server.
+ *
+ * - Once upload succeeds, we update the mirror with the uploaded records, so
+ *   that the mirror matches the server again.
+ */
+
+this.EXPORTED_SYMBOLS = ["SyncedBookmarksMirror"];
+
+const { utils: Cu, interfaces: Ci } = Components;
+
+Cu.importGlobalProperties(["URL"]);
+
+Cu.import("resource://gre/modules/Services.jsm");
+Cu.import("resource://gre/modules/XPCOMUtils.jsm");
+
+XPCOMUtils.defineLazyModuleGetters(this, {
+  AsyncShutdown: "resource://gre/modules/AsyncShutdown.jsm",
+  Log: "resource://gre/modules/Log.jsm",
+  OS: "resource://gre/modules/osfile.jsm",
+  PlacesSyncUtils: "resource://gre/modules/PlacesSyncUtils.jsm",
+  PlacesUtils: "resource://gre/modules/PlacesUtils.jsm",
+  Sqlite: "resource://gre/modules/Sqlite.jsm",
+});
+
+XPCOMUtils.defineLazyGetter(this, "MirrorLog", () =>
+  Log.repository.getLogger("Sync.Engine.Bookmarks.Mirror")
+);
+
+// These can be removed once they're exposed in a central location (bug
+// 1375896).
+const DB_URL_LENGTH_MAX = 65536;
+const DB_TITLE_LENGTH_MAX = 4096;
+const DB_DESCRIPTION_LENGTH_MAX = 256;
+
+const SQLITE_MAX_VARIABLE_NUMBER = 999;
+
+// The current mirror database schema version. Bump for migrations, then add
+// migration code to `migrateMirrorSchema`.
+const MIRROR_SCHEMA_VERSION = 1;
+
+/**
+ * A mirror stores a local copy of the remote bookmark tree in a separate
+ * SQLite database.
+ *
+ * The mirror schema is a hybrid of how Sync and Places represent bookmarks.
+ * The `items` table contains item attributes (title, kind, description,
+ * URL, etc.), while the `structure` table stores parent-child relationships and
+ * position. This is similar to how iOS encodes "value" and "structure" state,
+ * though we handle these differently when merging. See `BookmarkMerger` for
+ * details.
+ *
+ * There's no guarantee that the remote state is consistent. We might be missing
+ * parents or children, or a bookmark and its parent might disagree about where
+ * it belongs. We do what we can to build a complete tree from the remote state,
+ * even if we diverge from what's in the mirror.
+ *
+ * This means we need a strategy to handle missing parents and children. We
+ * treat the `children` of the last parent we see as canonical, and ignore the
+ * child's `parentid` entirely. We also ignore missing children, and temporarily
+ * reparent bookmarks with missing parents to "unfiled". When we eventually see
+ * the missing items, either during a later sync or as part of repair, we'll
+ * fill in the mirror's gaps and fix up the local tree.
+ *
+ * During merging, we won't intentionally try to fix inconsistencies on the
+ * server, because we might clobber a client that was interrupted uploading its
+ * records. We also want to avoid infinite sync loops, where two clients never
+ * converge, and each tries to make the other consistent by reuploading the same
+ * records. However, because we opt to continue syncing even if the remote tree
+ * is incomplete, we can still clobber partial uploaders if an inconsistent
+ * remote item was also changed locally. Once the server supports atomic
+ * uploads, we can revisit this decision.
+ *
+ * If a sync is interrupted, we resume downloading from the server collection
+ * last modified time, or the server last modified time of the most recent
+ * record if newer. New incoming records always replace existing records in the
+ * mirror.
+ *
+ * We delete the mirror database when the user is node reassigned, disables the
+ * bookmarks engine, or signs out.
+ */
+class SyncedBookmarksMirror {
+  constructor(db, { recordTelemetryEvent, finalizeAt =
+                      AsyncShutdown.profileBeforeChange } = {}) {
+    this.db = db;
+    this.recordTelemetryEvent = recordTelemetryEvent;
+
+    // Automatically finalize the mirror on shutdown.
+    this.finalizeAt = finalizeAt;
+    this.finalizeBound = () => this.finalize();
+    this.finalizeAt.addBlocker("SyncedBookmarksMirror: finalize",
+                               this.finalizeBound);
+  }
+
+  /**
+   * Sets up the mirror database connection and upgrades the mirror to the
+   * newest schema version.
+   *
+   * @param  {String} options.path
+   *         The full path to the mirror database file.
+   * @param  {Function} options.recordTelemetryEvent
+   *         A function with the signature `(object: String, method: String,
+   *         value: String?, extra: Object?)`, used to emit telemetry events.
+   * @param  {AsyncShutdown.Barrier} [options.finalizeAt]
+   *         A shutdown phase, barrier, or barrier client that should
+   *         automatically finalize the mirror when triggered. Exposed for
+   *         testing.
+   * @return {SyncedBookmarksMirror}
+   *         A mirror ready for use.
+   */
+  static async open(options) {
+    let db = await Sqlite.cloneStorageConnection({
+      connection: PlacesUtils.history.DBConnection,
+      readOnly: false,
+    });
+    try {
+      try {
+        await db.execute(`ATTACH :mirrorPath AS mirror`,
+                         { mirrorPath: options.path });
+      } catch (ex) {
+        if (ex.errors && isDatabaseCorrupt(ex.errors[0])) {
+          MirrorLog.warn("Error attaching mirror to Places; removing and " +
+                         "recreating mirror", ex);
+          options.recordTelemetryEvent("mirror", "open", "error",
+                                       { why: "corrupt" });
+          await OS.File.remove(options.path);
+          await db.execute(`ATTACH :mirrorPath AS mirror`,
+                           { mirrorPath: options.path });
+        } else {
+          MirrorLog.warn("Unrecoverable error attaching mirror to Places", ex);
+          throw ex;
+        }
+      }
+      await db.execute(`PRAGMA foreign_keys = ON`);
+      await migrateMirrorSchema(db);
+      await initializeTempMirrorEntities(db);
+    } catch (ex) {
+      options.recordTelemetryEvent("mirror", "open", "error",
+                                   { why: "initialize" });
+      await db.close();
+      throw ex;
+    }
+    return new SyncedBookmarksMirror(db, options);
+  }
+
+  /**
+   * Returns the newer of the bookmarks collection last modified time, or the
+   * server modified time of the newest record. The bookmarks engine uses this
+   * timestamp as the "high water mark" for all downloaded records. Each sync
+   * fetches and stores all records newer than this time.
+   *
+   * @return {Number}
+   *         The high water mark time, in seconds.
+   */
+  async getCollectionHighWaterMark() {
+    let rows = await this.db.execute(`
+      SELECT MAX(
+        IFNULL((SELECT MAX(serverModified) FROM items), 0),
+        IFNULL((SELECT CAST(value AS INTEGER) FROM meta
+                WHERE key = :modifiedKey), 0)
+      ) AS highWaterMark`,
+      { modifiedKey: SyncedBookmarksMirror.META.MODIFIED });
+    let highWaterMark = rows[0].getResultByName("highWaterMark");
+    return highWaterMark / 1000;
+  }
+
+  /**
+   * Updates the bookmarks collection last modified time. Note that this may
+   * be newer than the modified time of the most recent record.
+   *
+   * @param {Number|String} lastModifiedSeconds
+   *        The collection last modified time, in seconds.
+   */
+  async setCollectionLastModified(lastModifiedSeconds) {
+    let lastModified = lastModifiedSeconds * 1000;
+    if (!Number.isFinite(lastModified)) {
+      throw new TypeError("Invalid collection last modified time");
+    }
+    await this.db.execute(`
+      REPLACE INTO meta(key, value)
+      VALUES(:modifiedKey, :lastModified)`,
+      { modifiedKey: SyncedBookmarksMirror.META.MODIFIED, lastModified });
+  }
+
+  /**
+   * Stores incoming or uploaded Sync records in the mirror. Rejects if any
+   * records are invalid.
+   *
+   * @param {PlacesItem[]} records
+   *        An array of Sync records to store in the mirror.
+   * @param {Boolean} [options.needsMerge]
+   *        Indicates if the records were changed remotely since the last sync,
+   *        and should be merged into the local tree. This option is set to
+   *       `true` for incoming records, and `false` for successfully uploaded
+   *        records. Tests can also pass `false` to set up an existing mirror.
+   */
+  async store(records, { needsMerge = true } = {}) {
+    let options = { needsMerge };
+    await this.db.executeBeforeShutdown(
+      "SyncedBookmarksMirror: store",
+      db => db.executeTransaction(async () => {
+        for (let record of records) {
+          switch (record.type) {
+            case "bookmark":
+              MirrorLog.trace("Storing bookmark in mirror", record.cleartext);
+              await this.storeRemoteBookmark(record, options);
+              continue;
+
+            case "query":
+              MirrorLog.trace("Storing query in mirror", record.cleartext);
+              await this.storeRemoteQuery(record, options);
+              continue;
+
+            case "folder":
+              MirrorLog.trace("Storing folder in mirror", record.cleartext);
+              await this.storeRemoteFolder(record, options);
+              continue;
+
+            case "livemark":
+              MirrorLog.trace("Storing livemark in mirror", record.cleartext);
+              await this.storeRemoteLivemark(record, options);
+              continue;
+
+            case "separator":
+              MirrorLog.trace("Storing separator in mirror", record.cleartext);
+              await this.storeRemoteSeparator(record, options);
+              continue;
+
+            default:
+              if (record.deleted) {
+                MirrorLog.trace("Storing tombstone in mirror",
+                                record.cleartext);
+                await this.storeRemoteTombstone(record, options);
+                continue;
+              }
+          }
+          MirrorLog.warn("Ignoring record with unknown type", record.type);
+          this.recordTelemetryEvent("mirror", "ignore", "unknown",
+                                    { why: "kind" });
+        }
+      })
+    );
+  }
+
+  /**
+   * Builds a complete merged tree from the local and remote trees, resolves
+   * value and structure conflicts, dedupes local items, applies the merged
+   * tree back to Places, and notifies observers about the changes.
+   *
+   * Merging and application happen in an exclusive transaction, meaning code
+   * that uses the main Places connection, including the UI, will fail to read
+   * from or write to the database until the transaction commits. Asynchronous
+   * consumers will retry on `SQLITE_BUSY`; synchronous consumers will fail
+   * after waiting for 100ms. See bug 1305563, comment 122 for details.
+   *
+   * @param  {Number} [options.localTimeSeconds]
+   *         The current local time, in seconds.
+   * @param  {Number} [options.remoteTimeSeconds]
+   *         The current server time, in seconds.
+   * @return {Object.<String, BookmarkChangeRecord>}
+   *         A changeset containing locally changed and reconciled records to
+   *         upload to the server, and to store in the mirror once upload
+   *         succeeds.
+   */
+  async apply({ localTimeSeconds = Date.now() / 1000,
+                remoteTimeSeconds = 0 } = {}) {
+    // We intentionally don't use `executeBeforeShutdown` in this function,
+    // since merging can take a while for large trees, and we don't want to
+    // block shutdown.
+    try {
+      let { missingParents, missingChildren } =
+        await this.fetchRemoteOrphans();
+      if (missingParents.length) {
+        MirrorLog.debug("Temporarily reparenting remote items with missing " +
+                        "parents to unfiled", missingParents);
+        this.recordTelemetryEvent("mirror", "orphans", "parents",
+                                  { count: String(missingParents.length) });
+      }
+      if (missingChildren.length) {
+        MirrorLog.debug("Remote tree missing items", missingChildren);
+        this.recordTelemetryEvent("mirror", "orphans", "children",
+                                  { count: String(missingChildren.length) });
+      }
+
+      // It's safe to build the remote tree outside the transaction because
+      // `RemoteBookmarkStore.fetchTree` doesn't join to Places, only Sync
+      // writes to the mirror, and we're holding the Sync lock at this point.
+      MirrorLog.debug("Building remote tree from mirror");
+      let remoteTree = await this.fetchRemoteTree(remoteTimeSeconds);
+      MirrorLog.trace("Built remote tree from mirror", remoteTree);
+
+      let observersToNotify = await this.db.executeTransaction(async () => {
+        MirrorLog.debug("Building local tree from Places");
+        let localTree = await this.fetchLocalTree(localTimeSeconds);
+        MirrorLog.trace("Built local tree from Places", localTree);
+
+        MirrorLog.debug("Fetching content info for new mirror items");
+        let newRemoteContents = await this.fetchNewRemoteContents();
+
+        MirrorLog.debug("Fetching content info for new Places items");
+        let newLocalContents = await this.fetchNewLocalContents();
+
+        MirrorLog.debug("Building complete merged tree");
+        let merger = new BookmarkMerger(localTree, newLocalContents,
+                                        remoteTree, newRemoteContents);
+        let mergedRoot = merger.merge();
+        for (let { value, extra } of merger.telemetryEvents) {
+          this.recordTelemetryEvent("mirror", "merge", value, extra);
+        }
+
+        if (MirrorLog.level <= Log.Level.Trace) {
+          let newTreeRoot = mergedRoot.toBookmarkNode();
+          MirrorLog.trace("Built new merged tree", newTreeRoot);
+        }
+
+        // The merged tree should know about all items mentioned in the local
+        // and remote trees. Otherwise, it's incomplete, and we'll corrupt
+        // Places or lose data on the server if we try to apply it.
+        if (!merger.subsumes(localTree)) {
+          throw new SyncedBookmarksMirror.ConsistencyError(
+            "Merged tree doesn't mention all items from local tree");
+        }
+        if (!merger.subsumes(remoteTree)) {
+          throw new SyncedBookmarksMirror.ConsistencyError(
+            "Merged tree doesn't mention all items from remote tree");
+        }
+
+        let observersToNotify = new BookmarkObserverRecorder(this.db,
+          localTree, remoteTree);
+
+        let guidsToDelete = Array.from(merger.deleteLocally);
+        await this.applyMergedTree(mergedRoot, guidsToDelete, observersToNotify);
+
+        MirrorLog.debug("Flagging applied mirror rows as unchanged");
+        let mergedGuids = Array.from(merger.mergedGuids);
+        for (let chunk of PlacesSyncUtils.chunkArray(mergedGuids,
+          SQLITE_MAX_VARIABLE_NUMBER)) {
+
+          await this.db.execute(`
+            UPDATE items SET
+              needsMerge = 0
+            WHERE guid IN (${new Array(chunk.length).fill("?").join(",")})`,
+            chunk);
+        }
+
+        MirrorLog.debug("Staging locally changed items for upload");
+        await this.db.execute(`DELETE FROM itemsToUpload`);
+        await this.stageItemsToUpload();
+
+        return observersToNotify;
+      }, this.db.TRANSACTION_EXCLUSIVE);
+
+      let changeRecords;
+      try {
+        MirrorLog.debug("Fetching records for local items to upload");
+        changeRecords = await this.fetchLocalChangeRecords();
+      } finally {
+        await this.db.execute(`DELETE FROM itemsToUpload`);
+      }
+
+      MirrorLog.debug("Replaying recorded observer notifications");
+      await observersToNotify.notifyAll();
+
+      return changeRecords;
+    } finally {
+      await this.db.execute(`DELETE FROM mergeStates`);
+    }
+  }
+
+  /**
+   * Updates Places to reflect the merged tree and records observer
+   * notifications for changed items.
+   *
+   * @param {MergedBookmarkNode} mergedRoot
+   *        The root of the merged bookmark tree.
+   * @param {String[]} guidsToDelete
+   *        The GUIDs to delete from Places.
+   * @param {BookmarkObserverRecorder} observersToNotify
+   *        Records Places observer notifications for added, changed, and moved
+   *        items.
+   */
+  async applyMergedTree(mergedRoot, guidsToDelete, observersToNotify) {
+    // Add triggers to record item changes in temp tables. These are invoked by
+    // the triggers in `initializeTempMirrorEntities` that update Places.
+    await this.db.execute(`CREATE TEMP TABLE newItems(
+      guid TEXT PRIMARY KEY
+    ) WITHOUT ROWID`);
+
+    await this.db.execute(`
+      CREATE TEMP TRIGGER noteItemAdded
+      AFTER INSERT ON moz_bookmarks
+      BEGIN
+        INSERT INTO newItems(guid) VALUES(NEW.guid);
+      END`);
+
+    await this.db.execute(`CREATE TEMP TABLE changedItems(
+      itemId INTEGER PRIMARY KEY,
+      guid TEXT NOT NULL,
+      title TEXT,
+      placeId INTEGER,
+      parentId INTEGER NOT NULL DEFAULT -1,
+      position INTEGER NOT NULL DEFAULT -1
+    ) WITHOUT ROWID`);
+
+    // Records value and structure changes for existing bookmarks. We
+    // `INSERT OR IGNORE` then `UPDATE` because this trigger might be
+    // fired twice for the same item, since we update value and structure
+    // separately. In that case, we *don't* want changes to the parent
+    // or position to clobber changes that we already recorded for the
+    // GUID, title, and URL. We also skip items with placeholder parents and
+    // positions of "-1", since they're already in `newItems`.
+    await this.db.execute(`
+      CREATE TEMP TRIGGER noteItemChanged
+      AFTER UPDATE OF guid, title, fk, parent, position ON moz_bookmarks
+      WHEN -1 NOT IN (OLD.parent, OLD.position)
+      BEGIN
+        INSERT OR IGNORE INTO changedItems(itemId, guid, title, placeId)
+        VALUES(NEW.id, OLD.guid, OLD.title, OLD.fk);
+        UPDATE changedItems SET
+          parentId = OLD.parent,
+          position = OLD.position
+        WHERE itemId = NEW.id;
+      END`);
+
+    await this.db.execute(`CREATE TEMP TABLE removedItems(
+      itemId INTEGER PRIMARY KEY,
+      parentId INTEGER NOT NULL,
+      position INTEGER NOT NULL,
+      type INTEGER NOT NULL,
+      placeId INTEGER,
+      guid TEXT NOT NULL
+    ) WITHOUT ROWID`);
+
+    await this.db.execute(`
+      CREATE TEMP TRIGGER noteItemRemoved
+      AFTER DELETE ON moz_bookmarks
+      BEGIN
+        INSERT INTO removedItems(itemId, parentId, position, type, placeId, guid)
+        VALUES(OLD.id, OLD.parent, OLD.position, OLD.type, OLD.fk, OLD.guid);
+
+        /* Remove any local tombstones for remotely deleted items. */
+        DELETE FROM moz_bookmarks_deleted WHERE guid = OLD.guid;
+
+        /* Remove annos for the deleted items. */
+        DELETE FROM moz_items_annos WHERE item_id = OLD.id;
+
+        /* Recalculate frecency. */
+        UPDATE moz_places SET
+          frecency = -1
+        WHERE id = OLD.fk;
+      END`);
+
+    await this.db.execute(`CREATE TEMP TABLE changedAnnos(
+      itemId INTEGER NOT NULL,
+      annoName TEXT NOT NULL,
+      wasRemoved BOOLEAN NOT NULL,
+      PRIMARY KEY(itemId, annoName, wasRemoved)
+    ) WITHOUT ROWID`);
+
+    await this.db.execute(`
+      CREATE TEMP TRIGGER noteAnnoAdded
+      AFTER INSERT ON moz_items_annos
+      BEGIN
+        REPLACE INTO changedAnnos(itemId, annoName, wasRemoved)
+        SELECT NEW.item_id, n.name, 0 FROM moz_anno_attributes n
+        WHERE n.id = NEW.anno_attribute_id;
+      END`);
+
+    await this.db.execute(`
+      CREATE TEMP TRIGGER noteAnnoRemoved
+      AFTER DELETE ON moz_items_annos
+      BEGIN
+        REPLACE INTO changedAnnos(itemId, annoName, wasRemoved)
+        SELECT OLD.item_id, n.name, 1 FROM moz_anno_attributes n
+        WHERE n.id = OLD.anno_attribute_id;
+      END`);
+
+    await this.db.execute(`CREATE TEMP TABLE changedKeywords(
+      itemId INTEGER NOT NULL,
+      placeId INTEGER NOT NULL,
+      keyword TEXT
+    )`);
+
+    await this.db.execute(`
+      CREATE TEMP TRIGGER noteKeywordAdded
+      AFTER INSERT ON moz_keywords
+      BEGIN
+        INSERT INTO changedKeywords(itemId, placeId, keyword)
+        SELECT b.id, b.fk, NEW.keyword FROM moz_bookmarks b
+        WHERE b.fk = NEW.place_id;
+      END`);
+
+    await this.db.execute(`
+      CREATE TEMP TRIGGER noteKeywordRemoved
+      AFTER DELETE ON moz_keywords
+      BEGIN
+        /* A NULL keyword means we're removing all keywords from this URL. */
+        INSERT INTO changedKeywords(itemId, placeId, keyword)
+        SELECT b.id, b.fk, NULL FROM moz_bookmarks b
+        WHERE b.fk = OLD.place_id;
+      END`);
+
+    MirrorLog.debug("Updating Places to match merged tree");
+    await this.updateLocalItemsInPlaces(mergedRoot, guidsToDelete);
+
+    // At this point, the database is consistent, and we can fetch info to
+    // pass to observers. Note that we can't fetch observer info in the
+    // triggers above, because the structure might not be complete yet. An
+    // incomplete structure might cause us to miss or record wrong parents and
+    // positions.
+
+    MirrorLog.debug("Recording notifications for new items");
+    let newRows = await this.db.execute(`SELECT guid FROM newItems`);
+    for (let row of newRows) {
+      let newGuid = row.getResultByName("guid");
+      observersToNotify.noteItemAdded(newGuid);
+    }
+
+    MirrorLog.debug("Recording notifications for updated items");
+    let changedRows = await this.db.execute(`
+      SELECT b.id, v.guid, v.parentId, p.guid AS parentGuid,
+             IFNULL(b.title, "") AS title, h.url, v.position
+      FROM changedItems v
+      JOIN moz_bookmarks b ON b.id = v.itemId
+      JOIN moz_bookmarks p ON p.id = v.parentId
+      JOIN moz_places h ON h.id = v.placeId
+      JOIN mergeStates r ON r.mergedGuid = b.guid
+      JOIN mergeStates s ON s.mergedGuid = p.guid
+      WHERE :mergeState IN (r.valueState, s.structureState) OR
+            v.guid <> b.guid`,
+      { mergeState: BookmarkMergeState.TYPE.REMOTE });
+    for (let row of changedRows) {
+      observersToNotify.noteItemChanged({
+        id: row.getResultByName("id"),
+        oldGuid: row.getResultByName("guid"),
+        oldParentId: row.getResultByName("parentId"),
+        oldParentGuid: row.getResultByName("parentGuid"),
+        oldTitle: row.getResultByName("title"),
+        oldURLHref: row.getResultByName("url"),
+        oldPosition: row.getResultByName("position"),
+      });
+    }
+
+    MirrorLog.debug("Recording notifications for changed annos");
+    let annoRows = await this.db.execute(`
+      SELECT itemId, annoName, wasRemoved FROM changedAnnos
+      ORDER BY itemId`);
+    for (let row of annoRows) {
+      let id = row.getResultByName("itemId");
+      let name = row.getResultByName("annoName");
+      if (row.getResultByName("wasRemoved")) {
+        observersToNotify.noteAnnoRemoved(id, name);
+      } else {
+        observersToNotify.noteAnnoSet(id, name);
+      }
+    }
+
+    MirrorLog.debug("Recording notifications for changed keywords");
+    let changedKeywordRows = await this.db.execute(`
+      SELECT b.id, IFNULL(k.keyword, "") AS keyword, b.lastModified, b.type,
+             p.id AS parentId, b.guid, p.guid AS parentGuid, h.url
+      FROM changedKeywords k
+      JOIN moz_bookmarks b ON b.id = k.itemId
+      JOIN moz_bookmarks p ON p.id = b.parent
+      JOIN moz_places h ON h.id = k.placeId
+      /* Ensure we replay additions and deletions for the same keyword or URL
+         in order. */
+      ORDER BY k.ROWID`);
+    for (let row of changedKeywordRows) {
+      let info = {
+        id: row.getResultByName("id"),
+        keyword: row.getResultByName("keyword"),
+        lastModified: row.getResultByName("lastModified"),
+        type: row.getResultByName("type"),
+        parentId: row.getResultByName("parentId"),
+        guid: row.getResultByName("guid"),
+        parentGuid: row.getResultByName("parentGuid"),
+        urlHref: row.getResultByName("url"),
+      };
+      observersToNotify.noteKeywordChanged(info);
+    }
+
+    MirrorLog.debug("Recording observer notifications for removed items");
+    let removedItemInfos = [];
+    let removedRows = await this.db.execute(`
+      SELECT v.itemId, v.parentId, v.position, v.type, h.url, v.guid,
+             p.guid AS parentGuid
+      FROM removedItems v
+      LEFT JOIN moz_places h ON h.id = v.placeId
+      LEFT JOIN (
+        SELECT id, guid FROM moz_bookmarks
+        UNION ALL
+        SELECT itemId AS id, guid FROM removedItems
+      ) p ON p.id = v.parentId`);
+    for (let row of removedRows) {
+      let urlHref = row.getResultByName("url");
+      removedItemInfos.push({
+        id: row.getResultByName("itemId"),
+        parentId: row.getResultByName("parentId"),
+        position: row.getResultByName("position"),
+        type: row.getResultByName("type"),
+        uri: urlHref ? Services.io.newURI(urlHref) : null,
+        guid: row.getResultByName("guid"),
+        parentGuid: row.getResultByName("parentGuid"),
+      });
+    }
+    observersToNotify.noteItemsRemoved(removedItemInfos);
+
+    await this.db.execute(`DROP TABLE newItems`);
+    await this.db.execute(`DROP TRIGGER noteItemAdded`);
+    await this.db.execute(`DROP TABLE changedItems`);
+    await this.db.execute(`DROP TRIGGER noteItemChanged`);
+    await this.db.execute(`DROP TABLE removedItems`);
+    await this.db.execute(`DROP TRIGGER noteItemRemoved`);
+    await this.db.execute(`DROP TABLE changedAnnos`);
+    await this.db.execute(`DROP TRIGGER noteAnnoAdded`);
+    await this.db.execute(`DROP TRIGGER noteAnnoRemoved`);
+    await this.db.execute(`DROP TABLE changedKeywords`);
+    await this.db.execute(`DROP TRIGGER noteKeywordAdded`);
+    await this.db.execute(`DROP TRIGGER noteKeywordRemoved`);
+  }
+
+  /**
+   * Discards the mirror contents. This is called when the user is node
+   * reassigned, disables the bookmarks engine, or signs out.
+   */
+  async reset() {
+    await this.db.executeBeforeShutdown(
+      "SyncedBookmarksMirror: reset",
+      async function(db) {
+        await db.executeTransaction(async function() {
+          await db.execute(`DELETE FROM meta`);
+          await db.execute(`DELETE FROM items`);
+          await db.execute(`DELETE FROM urls`);
+
+          // Since we need to reset the modified times for the syncable roots,
+          // we simply delete and recreate them.
+          await createMirrorRoots(db);
+        });
+      }
+    );
+  }
+
+  async storeRemoteBookmark(record, { needsMerge }) {
+    let guid = validateGuid(record.id);
+    if (!guid) {
+      MirrorLog.warn("Ignoring bookmark with invalid ID", record.id);
+      this.recordTelemetryEvent("mirror", "ignore", "bookmark",
+                                { why: "id" });
+      return;
+    }
+
+    let url = validateURL(record.bmkUri);
+    if (!url) {
+      MirrorLog.trace("Ignoring bookmark ${guid} with invalid URL ${url}",
+                      { guid, url: record.bmkUri });
+      this.recordTelemetryEvent("mirror", "ignore", "bookmark",
+                                { why: "url" });
+      return;
+    }
+
+    await this.maybeStoreRemoteURL(url);
+
+    let serverModified = determineServerModified(record);
+    let dateAdded = determineDateAdded(record);
+    let title = validateTitle(record.title);
+    let keyword = validateKeyword(record.keyword);
+    let description = validateDescription(record.description);
+    let loadInSidebar = record.loadInSidebar === true ? "1" : null;
+
+    await this.db.executeCached(`
+      REPLACE INTO items(guid, serverModified, needsMerge, kind,
+                         dateAdded, title, keyword,
+                         urlId, description, loadInSidebar)
+      VALUES(:guid, :serverModified, :needsMerge, :kind,
+             :dateAdded, NULLIF(:title, ""), :keyword,
+             (SELECT id FROM urls WHERE hash = hash(:url) AND
+                                        url = :url),
+             :description, :loadInSidebar)`,
+      { guid, serverModified, needsMerge,
+        kind: SyncedBookmarksMirror.KIND.BOOKMARK, dateAdded, title, keyword,
+        url: url.href, description, loadInSidebar });
+
+    let tags = record.tags;
+    if (tags && Array.isArray(tags)) {
+      for (let rawTag of tags) {
+        let tag = validateTag(rawTag);
+        if (!tag) {
+          continue;
+        }
+        await this.db.executeCached(`
+          INSERT INTO tags(itemId, tag)
+          SELECT id, :tag FROM items WHERE guid = :guid`,
+          { tag, guid });
+      }
+    }
+  }
+
+  async storeRemoteQuery(record, { needsMerge }) {
+    let guid = validateGuid(record.id);
+    if (!guid) {
+      MirrorLog.warn("Ignoring query with invalid ID", record.id);
+      this.recordTelemetryEvent("mirror", "ignore", "query",
+                                { why: "id" });
+      return;
+    }
+
+    let url = validateURL(record.bmkUri);
+    if (!url) {
+      MirrorLog.trace("Ignoring query ${guid} with invalid URL ${url}",
+                      { guid, url: record.bmkUri });
+      this.recordTelemetryEvent("mirror", "ignore", "query",
+                                { why: "url" });
+      return;
+    }
+
+    await this.maybeStoreRemoteURL(url);
+
+    let serverModified = determineServerModified(record);
+    let dateAdded = determineDateAdded(record);
+    let title = validateTitle(record.title);
+    let tagFolderName = validateTag(record.folderName);
+    let description = validateDescription(record.description);
+    let smartBookmarkName = typeof record.queryId == "string" ?
+                            record.queryId : null;
+
+    await this.db.executeCached(`
+      REPLACE INTO items(guid, serverModified, needsMerge, kind,
+                         dateAdded, title, tagFolderName,
+                         urlId, description, smartBookmarkName)
+      VALUES(:guid, :serverModified, :needsMerge, :kind,
+             :dateAdded, NULLIF(:title, ""), :tagFolderName,
+             (SELECT id FROM urls WHERE hash = hash(:url) AND
+                                        url = :url),
+             :description, :smartBookmarkName)`,
+      { guid, serverModified, needsMerge,
+        kind: SyncedBookmarksMirror.KIND.QUERY, dateAdded, title, tagFolderName,
+        url: url.href, description, smartBookmarkName });
+  }
+
+  async storeRemoteFolder(record, { needsMerge }) {
+    let guid = validateGuid(record.id);
+    if (!guid) {
+      MirrorLog.warn("Ignoring folder with invalid ID", record.id);
+      this.recordTelemetryEvent("mirror", "ignore", "folder",
+                                { why: "id" });
+      return;
+    }
+    if (guid == PlacesUtils.bookmarks.rootGuid) {
+      // The Places root shouldn't be synced at all.
+      MirrorLog.warn("Ignoring Places root record", record.cleartext);
+      this.recordTelemetryEvent("mirror", "ignore", "folder",
+                                { why: "root" });
+      return;
+    }
+
+    let serverModified = determineServerModified(record);
+    let dateAdded = determineDateAdded(record);
+    let title = validateTitle(record.title);
+    let description = validateDescription(record.description);
+
+    await this.db.executeCached(`
+      REPLACE INTO items(guid, serverModified, needsMerge, kind,
+                         dateAdded, title, description)
+      VALUES(:guid, :serverModified, :needsMerge, :kind,
+             :dateAdded, NULLIF(:title, ""),
+             :description)`,
+      { guid, serverModified, needsMerge, kind: SyncedBookmarksMirror.KIND.FOLDER,
+        dateAdded, title, description });
+
+    let children = record.children;
+    if (children && Array.isArray(children)) {
+      for (let position = 0; position < children.length; ++position) {
+        let childRecordId = children[position];
+        let childGuid = validateGuid(childRecordId);
+        if (!childGuid) {
+          MirrorLog.warn("Ignoring child of folder ${parentGuid} with " +
+                         "invalid ID ${childRecordId}", { parentGuid: guid,
+                                                          childRecordId });
+          this.recordTelemetryEvent("mirror", "ignore", "child",
+                                    { why: "id" });
+          continue;
+        }
+        await this.db.executeCached(`
+          REPLACE INTO structure(guid, parentGuid, position)
+          VALUES(:childGuid, :parentGuid, :position)`,
+          { childGuid, parentGuid: guid, position });
+      }
+    }
+  }
+
+  async storeRemoteLivemark(record, { needsMerge }) {
+    let guid = validateGuid(record.id);
+    if (!guid) {
+      MirrorLog.warn("Ignoring livemark with invalid ID", record.id);
+      this.recordTelemetryEvent("mirror", "ignore", "livemark",
+                                { why: "id" });
+      return;
+    }
+
+    let feedURL = validateURL(record.feedUri);
+    if (!feedURL) {
+      MirrorLog.trace("Ignoring livemark ${guid} with invalid feed URL ${url}",
+                      { guid, url: record.feedUri });
+      this.recordTelemetryEvent("mirror", "ignore", "livemark",
+                                { why: "feed" });
+      return;
+    }
+
+    let serverModified = determineServerModified(record);
+    let dateAdded = determineDateAdded(record);
+    let title = validateTitle(record.title);
+    let description = validateDescription(record.description);
+    let siteURL = validateURL(record.siteUri);
+
+    await this.db.executeCached(`
+      REPLACE INTO items(guid, serverModified, needsMerge, kind, dateAdded,
+                         title, description, feedURL, siteURL)
+      VALUES(:guid, :serverModified, :needsMerge, :kind, :dateAdded,
+             NULLIF(:title, ""), :description, :feedURL, :siteURL)`,
+      { guid, serverModified, needsMerge,
+        kind: SyncedBookmarksMirror.KIND.LIVEMARK,
+        dateAdded, title, description, feedURL: feedURL.href,
+        siteURL: siteURL ? siteURL.href : null });
+  }
+
+  async storeRemoteSeparator(record, { needsMerge }) {
+    let guid = validateGuid(record.id);
+    if (!guid) {
+      MirrorLog.warn("Ignoring separator with invalid ID", record.id);
+      this.recordTelemetryEvent("mirror", "ignore", "separator",
+                                { why: "id" });
+      return;
+    }
+
+    let serverModified = determineServerModified(record);
+    let dateAdded = determineDateAdded(record);
+
+    await this.db.executeCached(`
+      REPLACE INTO items(guid, serverModified, needsMerge, kind,
+                         dateAdded)
+      VALUES(:guid, :serverModified, :needsMerge, :kind,
+             :dateAdded)`,
+      { guid, serverModified, needsMerge, kind: SyncedBookmarksMirror.KIND.SEPARATOR,
+        dateAdded });
+  }
+
+  async storeRemoteTombstone(record, { needsMerge }) {
+    let guid = validateGuid(record.id);
+    if (!guid) {
+      MirrorLog.warn("Ignoring tombstone with invalid ID", record.id);
+      this.recordTelemetryEvent("mirror", "ignore", "tombstone",
+                                { why: "id" });
+      return;
+    }
+
+    if (PlacesUtils.bookmarks.userContentRoots.includes(guid)) {
+      MirrorLog.warn("Ignoring tombstone for syncable root", guid);
+      this.recordTelemetryEvent("mirror", "ignore", "tombstone",
+                                { why: "root" });
+      return;
+    }
+
+    await this.db.executeCached(`
+      REPLACE INTO items(guid, serverModified, needsMerge, isDeleted)
+      VALUES(:guid, :serverModified, :needsMerge, 1)`,
+      { guid, serverModified: determineServerModified(record), needsMerge });
+  }
+
+  async maybeStoreRemoteURL(url) {
+    await this.db.executeCached(`
+      INSERT OR IGNORE INTO urls(guid, url, hash, revHost)
+      VALUES(IFNULL((SELECT guid FROM urls
+                     WHERE hash = hash(:url) AND
+                                  url = :url),
+                    GENERATE_GUID()), :url, hash(:url), :revHost)`,
+      { url: url.href, revHost: PlacesUtils.getReversedHost(url) });
+  }
+
+  async fetchRemoteOrphans() {
+    let infos = {
+      missingParents: [],
+      missingChildren: [],
+    };
+
+    let orphanRows = await this.db.execute(`
+      SELECT v.guid AS guid, 1 AS missingParent, 0 AS missingChild
+      FROM items v
+      LEFT JOIN structure s ON s.guid = v.guid
+      WHERE NOT isDeleted AND
+            s.guid IS NULL
+      UNION ALL
+      SELECT s.guid AS guid, 0 AS missingParent, 1 AS missingChild
+      FROM structure s
+      LEFT JOIN items v ON v.guid = s.guid
+      WHERE v.guid IS NULL`);
+
+    for (let row of orphanRows) {
+      let guid = row.getResultByName("guid");
+      let missingParent = row.getResultByName("missingParent");
+      if (missingParent) {
+        infos.missingParents.push(guid);
+      }
+      let missingChild = row.getResultByName("missingChild");
+      if (missingChild) {
+        infos.missingChildren.push(guid);
+      }
+    }
+
+    return infos;
+  }
+
+  /**
+   * Builds a fully rooted, consistent tree from the items and tombstones in the
+   * mirror.
+   *
+   * @param  {Number} remoteTimeSeconds
+   *         The current server time, in seconds.
+   * @return {BookmarkTree}
+   *         The remote bookmark tree.
+   */
+  async fetchRemoteTree(remoteTimeSeconds) {
+    let remoteTree = new BookmarkTree(BookmarkNode.root());
+
+    // First, build a flat mapping of parents to children. The `LEFT JOIN`
+    // includes items orphaned by an interrupted upload on another device.
+    // We keep the orphans in "unfiled" until the other device returns and
+    // uploads the missing parent.
+    let itemRows = await this.db.execute(`
+      SELECT v.guid, IFNULL(s.parentGuid, :unfiledGuid) AS parentGuid,
+             IFNULL(s.position, -1) AS position, v.serverModified, v.kind,
+             v.needsMerge
+      FROM items v
+      LEFT JOIN structure s ON s.guid = v.guid
+      WHERE NOT v.isDeleted AND
+            v.guid <> :rootGuid
+      ORDER BY parentGuid, position = -1, position, v.guid`,
+      { rootGuid: PlacesUtils.bookmarks.rootGuid,
+        unfiledGuid: PlacesUtils.bookmarks.unfiledGuid });
+
+    let pseudoTree = new Map();
+    for (let row of itemRows) {
+      let parentGuid = row.getResultByName("parentGuid");
+      let node = BookmarkNode.fromRemoteRow(row, remoteTimeSeconds);
+      if (pseudoTree.has(parentGuid)) {
+        let nodes = pseudoTree.get(parentGuid);
+        nodes.push(node);
+      } else {
+        pseudoTree.set(parentGuid, [node]);
+      }
+    }
+
+    // Second, build a complete tree from the pseudo-tree. We could do these
+    // two steps in SQL, but it's extremely inefficient. An equivalent
+    // recursive query, with joins in the base and recursive clauses, takes
+    // 10 seconds for a mirror with 5k items. Building the pseudo-tree and
+    // the pseudo-tree and recursing in JS takes 30ms for 5k items.
+    inflateTree(remoteTree, pseudoTree, PlacesUtils.bookmarks.rootGuid);
+
+    // Note tombstones for remotely deleted items.
+    let tombstoneRows = await this.db.execute(`
+      SELECT guid FROM items WHERE isDeleted AND
+                                   needsMerge`);
+
+    for (let row of tombstoneRows) {
+      let guid = row.getResultByName("guid");
+      remoteTree.noteDeleted(guid);
+    }
+
+    return remoteTree;
+  }
+
+  /**
+   * Fetches content info for all items in the mirror that changed since the
+   * last sync and don't exist locally.
+   *
+   * @return {Map.<String, BookmarkContent>}
+   *         Changed items in the mirror that don't exist in Places, keyed by
+   *         their GUIDs.
+   */
+  async fetchNewRemoteContents() {
+    let newRemoteContents = new Map();
+
+    let rows = await this.db.execute(`
+      SELECT v.guid, IFNULL(v.title, "") AS title, u.url, v.smartBookmarkName,
+             IFNULL(s.position, -1) AS position
+      FROM items v
+      LEFT JOIN urls u ON u.id = v.urlId
+      LEFT JOIN structure s ON s.guid = v.guid
+      LEFT JOIN moz_bookmarks b ON b.guid = v.guid
+      WHERE NOT v.isDeleted AND
+            v.needsMerge AND
+            b.guid IS NULL AND
+            IFNULL(s.parentGuid, :unfiledGuid) <> :rootGuid`,
+      { unfiledGuid: PlacesUtils.bookmarks.unfiledGuid,
+        rootGuid: PlacesUtils.bookmarks.rootGuid });
+    for (let row of rows) {
+      let guid = row.getResultByName("guid");
+      let content = BookmarkContent.fromRow(row);
+      newRemoteContents.set(guid, content);
+    }
+
+    return newRemoteContents;
+  }
+
+  /**
+   * Builds a fully rooted, consistent tree from the items and tombstones in
+   * Places.
+   *
+   * @param  {Number} localTimeSeconds
+   *         The current local time, in seconds.
+   * @return {BookmarkTree}
+   *         The local bookmark tree.
+   */
+  async fetchLocalTree(localTimeSeconds) {
+    let localTree = new BookmarkTree(BookmarkNode.root());
+
+    // This unsightly query collects all descendants and maps their Places types
+    // to the Sync record kinds. We start with the roots, and work our way down.
+    let itemRows = await this.db.execute(`
+      WITH RECURSIVE
+      syncedItems(id, level) AS (
+        SELECT b.id, 0 AS level FROM moz_bookmarks b
+        WHERE b.guid IN (:menuGuid, :toolbarGuid, :unfiledGuid, :mobileGuid)
+        UNION ALL
+        SELECT b.id, s.level + 1 AS level FROM moz_bookmarks b
+        JOIN syncedItems s ON s.id = b.parent
+      )
+      SELECT b.id, b.guid, p.guid AS parentGuid,
+             /* Map Places item types to Sync record kinds. */
+             (CASE b.type
+                WHEN :bookmarkType THEN (
+                  CASE SUBSTR((SELECT h.url FROM moz_places h
+                               WHERE h.id = b.fk), 1, 6)
+                  /* Queries are bookmarks with a "place:" URL scheme. */
+                  WHEN 'place:' THEN :queryKind
+                  ELSE :bookmarkKind END)
+                WHEN :folderType THEN (
+                  CASE WHEN EXISTS(
+                    /* Livemarks are folders with a feed URL annotation. */
+                    SELECT 1 FROM moz_items_annos a
+                    JOIN moz_anno_attributes n ON n.id = a.anno_attribute_id
+                    WHERE a.item_id = b.id AND
+                          n.name = :feedURLAnno
+                  ) THEN :livemarkKind
+                  ELSE :folderKind END)
+                ELSE :separatorKind END) AS kind,
+             b.lastModified, b.syncChangeCounter
+      FROM moz_bookmarks b
+      JOIN moz_bookmarks p ON p.id = b.parent
+      JOIN syncedItems s ON s.id = b.id
+      ORDER BY s.level, b.parent, b.position`,
+      { menuGuid: PlacesUtils.bookmarks.menuGuid,
+        toolbarGuid: PlacesUtils.bookmarks.toolbarGuid,
+        unfiledGuid: PlacesUtils.bookmarks.unfiledGuid,
+        mobileGuid: PlacesUtils.bookmarks.mobileGuid,
+        bookmarkType: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+        queryKind: SyncedBookmarksMirror.KIND.QUERY,
+        bookmarkKind: SyncedBookmarksMirror.KIND.BOOKMARK,
+        folderType: PlacesUtils.bookmarks.TYPE_FOLDER,
+        feedURLAnno: PlacesUtils.LMANNO_FEEDURI,
+        livemarkKind: SyncedBookmarksMirror.KIND.LIVEMARK,
+        folderKind: SyncedBookmarksMirror.KIND.FOLDER,
+        separatorKind: SyncedBookmarksMirror.KIND.SEPARATOR });
+
+    for (let row of itemRows) {
+      let parentGuid = row.getResultByName("parentGuid");
+      let node = BookmarkNode.fromLocalRow(row, localTimeSeconds);
+      localTree.insert(parentGuid, node);
+    }
+
+    // Note tombstones for locally deleted items.
+    let tombstoneRows = await this.db.execute(`
+      SELECT guid FROM moz_bookmarks_deleted`);
+
+    for (let row of tombstoneRows) {
+      let guid = row.getResultByName("guid");
+      localTree.noteDeleted(guid);
+    }
+
+    return localTree;
+  }
+
+  /**
+   * Fetches content info for all NEW local items that don't exist in the
+   * mirror. We'll try to dedupe them to changed items with similar contents and
+   * different GUIDs in the mirror.
+   *
+   * @return {Map.<String, BookmarkContent>}
+   *         New items in Places that don't exist in the mirror, keyed by their
+   *         GUIDs.
+   */
+  async fetchNewLocalContents() {
+    let newLocalContents = new Map();
+
+    let rows = await this.db.execute(`
+      SELECT b.guid, IFNULL(b.title, "") AS title, h.url,
+             (SELECT a.content FROM moz_items_annos a
+              JOIN moz_anno_attributes n ON n.id = a.anno_attribute_id
+              WHERE a.item_id = b.id AND
+                    n.name = :smartBookmarkAnno) AS smartBookmarkName,
+             b.position
+      FROM moz_bookmarks b
+      JOIN moz_bookmarks p ON p.id = b.parent
+      LEFT JOIN moz_places h ON h.id = b.fk
+      LEFT JOIN items v ON v.guid = b.guid
+      WHERE v.guid IS NULL AND
+            p.guid <> :rootGuid AND
+            b.syncStatus <> :syncStatus`,
+      { smartBookmarkAnno: PlacesSyncUtils.bookmarks.SMART_BOOKMARKS_ANNO,
+        rootGuid: PlacesUtils.bookmarks.rootGuid,
+        syncStatus: PlacesUtils.bookmarks.SYNC_STATUS.NORMAL });
+    for (let row of rows) {
+      let guid = row.getResultByName("guid");
+      let content = BookmarkContent.fromRow(row);
+      newLocalContents.set(guid, content);
+    }
+
+    return newLocalContents;
+  }
+
+  /**
+   * Builds a temporary table with the merge states of all nodes in the merged
+   * tree, rewrites tag queries, and updates Places to match the merged tree.
+   *
+   * Conceptually, we examine the merge state of each item, and either keep the
+   * complete local state, takes the complete remote state, or apply a new
+   * structure state and flag the item for reupload.
+   *
+   * Note that we update Places and flag items *before* upload, while iOS
+   * updates the mirror *after* a successful upload. This simplifies our
+   * implementation, though we lose idempotent merges. If upload is interrupted,
+   * the next sync won't distinguish between new merge states from the previous
+   * sync, and local changes. Since this is how Desktop behaved before
+   * structured application, that's OK. In the future, we can make this more
+   * like iOS.
+   *
+   * @param {MergedBookmarkNode} mergedRoot
+   *        The root of the merged bookmark tree.
+   * @param {String[]} guidsToDelete
+   *        The GUIDs to delete from Places.
+   */
+  async updateLocalItemsInPlaces(mergedRoot, guidsToDelete) {
+    MirrorLog.debug("Setting up merge states table");
+    let mergeStatesParams = Array.from(mergedRoot.mergeStatesParams());
+    if (mergeStatesParams.length) {
+      await this.db.execute(`
+        INSERT INTO mergeStates(localGuid, mergedGuid, parentGuid,
+                                position, valueState, structureState)
+        VALUES(IFNULL(:localGuid, :mergedGuid), :mergedGuid, :parentGuid,
+               :position, :valueState, :structureState)`,
+        mergeStatesParams);
+    }
+
+    MirrorLog.debug("Rewriting tag queries in mirror");
+    await this.rewriteRemoteTagQueries();
+
+    MirrorLog.debug("Inserting new URLs into Places");
+    await this.db.execute(`
+      INSERT OR IGNORE INTO moz_places(url, url_hash, rev_host, hidden,
+                                       frecency, guid)
+      SELECT u.url, u.hash, u.revHost, 0,
+             (CASE SUBSTR(u.url, 1, 6) WHEN 'place:' THEN 0 ELSE -1 END),
+             IFNULL(h.guid, u.guid)
+      FROM items v
+      JOIN urls u ON u.id = v.urlId
+      LEFT JOIN moz_places h ON h.url_hash = u.hash AND
+                                h.url = u.url
+      JOIN mergeStates r ON r.mergedGuid = v.guid
+      WHERE r.valueState = :valueState`,
+      { valueState: BookmarkMergeState.TYPE.REMOTE });
+    await this.db.execute(`DELETE FROM moz_updatehostsinsert_temp`);
+
+    // Deleting from `newRemoteItems` fires the `insertNewLocalItems` and
+    // `updateExistingLocalItems` triggers.
+    MirrorLog.debug("Updating value states for local bookmarks");
+    await this.db.execute(`DELETE FROM newRemoteItems`);
+
+    // Update the structure. The mirror stores structure info in a separate
+    // table, like iOS, while Places stores structure info on children. We don't
+    // check the parent's merge state here because our merged tree might
+    // diverge from the server if we're missing children, or moved children
+    // without parents to "unfiled". In that case, we *don't* want to reupload
+    // the new local structure to the server.
+    MirrorLog.debug("Updating structure states for local bookmarks");
+    await this.db.execute(`DELETE FROM newRemoteStructure`);
+
+    MirrorLog.debug("Dropping remote deletions from Places");
+    for (let chunk of PlacesSyncUtils.chunkArray(guidsToDelete,
+      SQLITE_MAX_VARIABLE_NUMBER)) {
+
+      await this.db.execute(`
+        DELETE FROM moz_bookmarks
+        WHERE guid IN (${new Array(chunk.length).fill("?").join(",")})`,
+        chunk);
+    }
+  }
+
+  /**
+   * Creates local tag folders mentioned in remotely changed tag queries, then
+   * rewrites the query URLs in the mirror to point to the new local folders.
+   *
+   * This can be removed once bug 1293445 lands.
+   */
+  async rewriteRemoteTagQueries() {
+    // Create local tag folders that don't already exist. This fires the
+    // `tagLocalPlace` trigger.
+    await this.db.execute(`
+      INSERT INTO localTags(tag)
+      SELECT v.tagFolderName FROM items v
+      JOIN mergeStates r ON r.mergedGuid = v.guid
+      WHERE r.valueState = :valueState AND
+            v.tagFolderName NOT NULL`,
+      { valueState: BookmarkMergeState.TYPE.REMOTE });
+
+    let urlsParams = [];
+    let queryRows = await this.db.execute(`
+      SELECT u.id AS urlId, u.url, b.id AS newTagFolderId
+      FROM urls u
+      JOIN items v ON v.urlId = u.id
+      JOIN mergeStates r ON r.mergedGuid = v.guid
+      JOIN moz_bookmarks b ON b.title = v.tagFolderName
+      JOIN moz_bookmarks p ON p.id = b.parent
+      WHERE p.guid = :tagsGuid AND
+            r.valueState = :valueState AND
+            v.kind = :queryKind AND
+            v.tagFolderName NOT NULL`,
+      { tagsGuid: PlacesUtils.bookmarks.tagsGuid,
+        valueState: BookmarkMergeState.TYPE.REMOTE,
+        queryKind: SyncedBookmarksMirror.KIND.QUERY });
+    for (let row of queryRows) {
+      let url = new URL(row.getResultByName("url"));
+      let tagQueryParams = new URLSearchParams(url.pathname);
+      let type = Number(tagQueryParams.get("type"));
+      if (type != Ci.nsINavHistoryQueryOptions.RESULTS_AS_TAG_CONTENTS) {
+        continue;
+      }
+
+      // Rewrite the query URL to point to the new folder.
+      let newTagFolderId = row.getResultByName("newTagFolderId");
+      tagQueryParams.set("folder", newTagFolderId);
+
+      let newURLHref = url.protocol + tagQueryParams;
+      urlsParams.push({
+        urlId: row.getResultByName("urlId"),
+        url: newURLHref,
+      });
+    }
+
+    if (urlsParams.length) {
+      await this.db.execute(`
+        UPDATE urls SET
+          url = :url,
+          hash = hash(:url)
+        WHERE id = :urlId`,
+        urlsParams);
+    }
+  }
+
+  /**
+   * Stores a snapshot of all locally changed items in a temporary table for
+   * upload. This is called from within the merge transaction, to ensure that
+   * structure changes made during the sync don't cause us to upload an
+   * inconsistent tree.
+   *
+   * For an example of why we use a temporary table instead of reading directly
+   * from Places, consider a user adding a bookmark, then changing its parent
+   * folder. We first add the bookmark to the default folder, bump the change
+   * counter of the new bookmark and the default folder, then trigger a sync.
+   * Depending on how quickly the user picks the new parent, we might upload
+   * a record for the default folder, commit the move, then upload the bookmark.
+   * We'll still upload the new parent on the next sync, but, in the meantime,
+   * we've introduced a parent-child disagreement. This can also happen if the
+   * user moves many items between folders.
+   */
+  async stageItemsToUpload() {
+    // Stage all locally changed items for upload, along with any remotely
+    // changed records with older local creation dates. These are tracked
+    // "weakly", in the in-memory table only. If the upload is interrupted
+    // or fails, we won't reupload the record on the next sync.
+    await this.db.execute(`
+      WITH RECURSIVE
+      syncedItems(id, level) AS (
+        SELECT b.id, 0 AS level FROM moz_bookmarks b
+        WHERE b.guid IN (:menuGuid, :toolbarGuid, :unfiledGuid, :mobileGuid)
+        UNION ALL
+        SELECT b.id, s.level + 1 AS level FROM moz_bookmarks b
+        JOIN syncedItems s ON s.id = b.parent
+      ),
+      annos(itemId, name, content) AS (
+        SELECT a.id, n.name, a.content FROM moz_items_annos a
+        JOIN moz_anno_attributes n ON n.id = a.anno_attribute_id
+      )
+      INSERT INTO itemsToUpload(guid, syncChangeCounter, parentGuid,
+                                parentTitle, dateAdded, type, title, isQuery,
+                                url, tags, description, loadInSidebar,
+                                smartBookmarkName, keyword, feedURL, siteURL,
+                                position)
+      SELECT b.guid, b.syncChangeCounter, p.guid, p.title, b.dateAdded, b.type,
+             b.title, IFNULL(SUBSTR(h.url, 1, 6) = 'place:', 0), h.url,
+             (SELECT GROUP_CONCAT(t.title, ',') FROM moz_bookmarks e
+              JOIN moz_bookmarks t ON t.id = e.parent
+              JOIN moz_bookmarks r ON r.id = t.parent
+              WHERE r.guid = :tagsGuid AND
+                    e.fk = h.id),
+             (SELECT content FROM annos WHERE name = :descriptionAnno),
+             IFNULL((SELECT content FROM annos WHERE name = :sidebarAnno), 0),
+             (SELECT content FROM ANNOS WHERE name = :smartBookmarkAnno),
+             (SELECT keyword FROM moz_keywords WHERE place_id = h.id),
+             (SELECT content FROM annos WHERE name = :feedURLAnno),
+             (SELECT content FROM annos WHERE name = :siteURLAnno),
+             b.position
+      FROM moz_bookmarks b
+      JOIN moz_bookmarks p ON p.id = b.parent
+      JOIN syncedItems s ON s.id = b.id
+      LEFT JOIN moz_places h ON h.id = b.fk
+      JOIN mergeStates r ON r.mergedGuid = b.guid
+      LEFT JOIN items v ON v.guid = r.mergedGuid
+      WHERE b.syncChangeCounter >= 1 OR
+            (r.valueState = :valueState AND
+              b.dateAdded < v.dateAdded)`,
+      { menuGuid: PlacesUtils.bookmarks.menuGuid,
+        toolbarGuid: PlacesUtils.bookmarks.toolbarGuid,
+        unfiledGuid: PlacesUtils.bookmarks.unfiledGuid,
+        mobileGuid: PlacesUtils.bookmarks.mobileGuid,
+        tagsGuid: PlacesUtils.bookmarks.tagsGuid,
+        descriptionAnno: PlacesSyncUtils.bookmarks.DESCRIPTION_ANNO,
+        sidebarAnno: PlacesSyncUtils.bookmarks.SIDEBAR_ANNO,
+        smartBookmarkAnno: PlacesSyncUtils.bookmarks.SMART_BOOKMARKS_ANNO,
+        feedURLAnno: PlacesUtils.LMANNO_FEEDURI,
+        siteURLAnno: PlacesUtils.LMANNO_SITEURI,
+        valueState: BookmarkMergeState.TYPE.REMOTE });
+
+    // Record tag folder names for tag queries. Parsing query URLs one by one
+    // is inefficient, but queries aren't common today, and we can remove this
+    // logic entirely once bug 1293445 lands.
+    let tagFolderNameParams = [];
+    let queryRows = await this.db.execute(`
+      SELECT guid, url FROM itemsToUpload
+      WHERE isQuery`);
+    for (let row of queryRows) {
+      let url = new URL(row.getResultByName("url"));
+      let tagQueryParams = new URLSearchParams(url.pathname);
+      let type = Number(tagQueryParams.get("type"));
+      if (type == Ci.nsINavHistoryQueryOptions.RESULTS_AS_TAG_CONTENTS) {
+        continue;
+      }
+      let tagFolderId = Number(tagQueryParams.get("folder"));
+      tagFolderNameParams.push({
+        guid: row.getResultByName("guid"),
+        tagFolderId,
+        folderType: PlacesUtils.bookmarks.TYPE_FOLDER,
+      });
+    }
+    if (tagFolderNameParams.length) {
+      await this.db.execute(`
+        UPDATE itemsToUpload SET
+          tagFolderName = (SELECT b.title FROM moz_bookmarks b
+                           WHERE b.id = :tagFolderId AND
+                                 b.type = :folderType)
+        WHERE guid = :guid`);
+    }
+
+    // Record the child GUIDs of locally changed folders, which we use to
+    // populate the `children` array in the record.
+    await this.db.execute(`
+      INSERT INTO structureToUpload(guid, parentGuid, position)
+      SELECT b.guid, p.guid, b.position
+      FROM moz_bookmarks b
+      JOIN moz_bookmarks p ON p.id = b.parent
+      JOIN itemsToUpload o ON o.guid = p.guid`);
+
+    // Finally, stage tombstones for deleted items. Ignore conflicts if we have
+    // tombstones for undeleted items; Places Maintenance should clean these up.
+    await this.db.execute(`
+      INSERT OR IGNORE INTO itemsToUpload(guid, syncChangeCounter, isDeleted,
+                                          dateAdded)
+      SELECT guid, 1, 1, dateRemoved FROM moz_bookmarks_deleted`);
+  }
+
+  /**
+   * Inflates Sync records for all staged outgoing items.
+   *
+   * @return {Object.<String, BookmarkChangeRecord>}
+   *         A changeset containing Sync record cleartexts for outgoing items
+   *         and tombstones, keyed by their Sync record IDs.
+   */
+  async fetchLocalChangeRecords() {
+    let changeRecords = {};
+
+    // Create records for bookmarks to upload.
+    MirrorLog.debug("Fetching items for strong upload");
+    let itemRows = await this.db.execute(`
+      SELECT syncChangeCounter, guid, isDeleted, type, isQuery,
+             smartBookmarkName, IFNULL(tagFolderName, "") AS tagFolderName,
+             loadInSidebar, keyword, tags, url, IFNULL(title, "") AS title,
+             description, feedURL, siteURL, position, parentGuid,
+             IFNULL(parentTitle, "") AS parentTitle, dateAdded
+      FROM itemsToUpload`);
+    for (let row of itemRows) {
+      let syncChangeCounter = row.getResultByName("syncChangeCounter");
+
+      let guid = row.getResultByName("guid");
+      let recordId = PlacesSyncUtils.bookmarks.guidToRecordId(guid);
+
+      // Tombstones don't carry additional properties.
+      let isDeleted = row.getResultByName("isDeleted");
+      if (isDeleted) {
+        changeRecords[recordId] = new BookmarkChangeRecord(syncChangeCounter, {
+          id: recordId,
+          deleted: true,
+        });
+        continue;
+      }
+
+      let parentGuid = row.getResultByName("parentGuid");
+      let parentRecordId = PlacesSyncUtils.bookmarks.guidToRecordId(parentGuid);
+      let dateAdded = PlacesUtils.toDate(
+        row.getResultByName("dateAdded")).getTime();
+
+      let type = row.getResultByName("type");
+      switch (type) {
+        case PlacesUtils.bookmarks.TYPE_BOOKMARK: {
+          let isQuery = row.getResultByName("isQuery");
+          if (isQuery) {
+            let queryCleartext = {
+              id: recordId,
+              type: "query",
+              // We ignore `parentid` and use the parent's `children`, but older
+              // Desktops and Android use `parentid` as the canonical parent.
+              // iOS is stricter and requires both `children` and `parentid` to
+              // match.
+              parentid: parentRecordId,
+              // Older Desktops use `hasDupe` and `parentName` for deduping.
+              hasDupe: false,
+              parentName: row.getResultByName("parentTitle"),
+              dateAdded,
+              bmkUri: row.getResultByName("url"),
+              title: row.getResultByName("title"),
+              queryId: row.getResultByName("smartBookmarkName"),
+              folderName: row.getResultByName("tagFolderName"),
+            };
+            let description = row.getResultByName("description");
+            if (description) {
+              queryCleartext.description = description;
+            }
+            changeRecords[recordId] = new BookmarkChangeRecord(
+              syncChangeCounter, queryCleartext);
+            continue;
+          }
+
+          let bookmarkCleartext = {
+            id: recordId,
+            type: "bookmark",
+            parentid: parentRecordId,
+            hasDupe: false,
+            parentName: row.getResultByName("parentTitle"),
+            dateAdded,
+            bmkUri: row.getResultByName("url"),
+            title: row.getResultByName("title"),
+          };
+          let description = row.getResultByName("description");
+          if (description) {
+            bookmarkCleartext.description = description;
+          }
+          let loadInSidebar = row.getResultByName("loadInSidebar");
+          if (loadInSidebar) {
+            bookmarkCleartext.loadInSidebar = true;
+          }
+          let keyword = row.getResultByName("keyword");
+          if (keyword) {
+            bookmarkCleartext.keyword = keyword;
+          }
+          let tags = row.getResultByName("tags");
+          if (tags) {
+            bookmarkCleartext.tags = tags.split(",");
+          }
+          changeRecords[recordId] = new BookmarkChangeRecord(
+            syncChangeCounter, bookmarkCleartext);
+          continue;
+        }
+
+        case PlacesUtils.bookmarks.TYPE_FOLDER: {
+          let feedURLHref = row.getResultByName("feedURL");
+          if (feedURLHref) {
+            // Places stores livemarks as folders with feed and site URL annos.
+            // See bug 1072833 for discussion about changing them to queries.
+            let livemarkCleartext = {
+              id: recordId,
+              type: "livemark",
+              parentid: parentRecordId,
+              hasDupe: false,
+              parentName: row.getResultByName("parentTitle"),
+              dateAdded,
+              title: row.getResultByName("title"),
+              feedUri: feedURLHref,
+            };
+            let description = row.getResultByName("description");
+            if (description) {
+              livemarkCleartext.description = description;
+            }
+            let siteURLHref = row.getResultByName("siteURL");
+            if (siteURLHref) {
+              livemarkCleartext.siteUri = siteURLHref;
+            }
+            changeRecords[recordId] = new BookmarkChangeRecord(
+              syncChangeCounter, livemarkCleartext);
+            continue;
+          }
+
+          let folderCleartext = {
+            id: recordId,
+            type: "folder",
+            parentid: parentRecordId,
+            hasDupe: false,
+            parentName: row.getResultByName("parentTitle"),
+            dateAdded,
+            title: row.getResultByName("title"),
+          };
+          let description = row.getResultByName("description");
+          if (description) {
+            folderCleartext.description = description;
+          }
+          let childGuidRows = await this.db.executeCached(`
+            SELECT guid FROM structureToUpload
+            WHERE parentGuid = :guid
+            ORDER BY position`,
+            { guid });
+          folderCleartext.children = childGuidRows.map(row => {
+            let childGuid = row.getResultByName("guid");
+            return PlacesSyncUtils.bookmarks.guidToRecordId(childGuid);
+          });
+          changeRecords[recordId] = new BookmarkChangeRecord(
+            syncChangeCounter, folderCleartext);
+          continue;
+        }
+
+        case PlacesUtils.bookmarks.TYPE_SEPARATOR: {
+          let separatorCleartext = {
+            id: recordId,
+            type: "separator",
+            parentid: parentRecordId,
+            hasDupe: false,
+            parentName: row.getResultByName("parentTitle"),
+            dateAdded,
+            // Older Desktops use `pos` for deduping.
+            pos: row.getResultByName("position"),
+          };
+          changeRecords[recordId] = new BookmarkChangeRecord(
+            syncChangeCounter, separatorCleartext);
+          continue;
+        }
+
+        default:
+          throw new TypeError("Can't create record for unknown Places item");
+      }
+    }
+
+    return changeRecords;
+  }
+
+  /**
+   * Closes the mirror database connection. This is called automatically on
+   * shutdown, but may also be called explicitly when the mirror is no longer
+   * needed.
+   */
+  finalize() {
+    if (!this.finalizePromise) {
+      this.finalizePromise = (async () => {
+        await this.db.close();
+        this.finalizeAt.removeBlocker(this.finalizeBound);
+      })();
+    }
+    return this.finalizePromise;
+  }
+}
+
+this.SyncedBookmarksMirror = SyncedBookmarksMirror;
+
+SyncedBookmarksMirror.KIND = {
+  BOOKMARK: 1,
+  QUERY: 2,
+  FOLDER: 3,
+  LIVEMARK: 4,
+  SEPARATOR: 5,
+};
+
+SyncedBookmarksMirror.META = {
+  MODIFIED: 1,
+};
+
+SyncedBookmarksMirror.ConsistencyError =
+  class ConsistencyError extends Error {};
+
+function isDatabaseCorrupt(error) {
+  return error instanceof Ci.mozIStorageError &&
+         (error.result == Ci.mozIStorageError.CORRUPT ||
+          error.result == Ci.mozIStorageError.NOTADB);
+}
+
+function migrateMirrorSchema(db) {
+  return db.executeTransaction(async function() {
+    let currentSchemaVersion = (await db.execute(`
+      PRAGMA mirror.user_version`))[0].getInt32(0);
+    if (currentSchemaVersion < 1) {
+      await initializeMirrorDatabase(db);
+    }
+    // Downgrading from a newer profile to an older profile rolls back the
+    // schema version, but leaves all new rows in place. We'll run the
+    // migration logic again on the next upgrade.
+    await db.execute(`PRAGMA mirror.user_version = ${MIRROR_SCHEMA_VERSION}`);
+  });
+}
+
+async function initializeMirrorDatabase(db) {
+  // Key-value metadata table. Currently stores just the server collection
+  // last modified time.
+  await db.execute(`CREATE TABLE mirror.meta(
+    key INTEGER PRIMARY KEY,
+    value NOT NULL
+    CHECK(key = ${SyncedBookmarksMirror.META.MODIFIED})
+  )`);
+
+  await db.execute(`CREATE TABLE mirror.items(
+    id INTEGER PRIMARY KEY,
+    guid TEXT UNIQUE NOT NULL,
+    /* The server modified time, in milliseconds. */
+    serverModified INTEGER NOT NULL DEFAULT 0,
+    needsMerge BOOLEAN NOT NULL DEFAULT 0,
+    isDeleted BOOLEAN NOT NULL DEFAULT 0,
+    kind INTEGER NOT NULL DEFAULT -1,
+    /* The creation date, in microseconds. */
+    dateAdded INTEGER NOT NULL DEFAULT 0,
+    title TEXT,
+    urlId INTEGER REFERENCES urls(id)
+                  ON DELETE SET NULL,
+    keyword TEXT,
+    tagFolderName TEXT,
+    description TEXT,
+    loadInSidebar BOOLEAN,
+    smartBookmarkName TEXT,
+    feedURL TEXT,
+    siteURL TEXT,
+    /* Only bookmarks and queries must have URLs. */
+    CHECK(CASE WHEN kind IN (${[
+                      SyncedBookmarksMirror.KIND.BOOKMARK,
+                      SyncedBookmarksMirror.KIND.QUERY,
+                    ].join(",")}) THEN urlId NOT NULL
+               ELSE urlId IS NULL END)
+  )`);
+
+  await db.execute(`CREATE TABLE mirror.structure(
+    guid TEXT NOT NULL PRIMARY KEY,
+    parentGuid TEXT NOT NULL REFERENCES items(guid)
+                             ON DELETE CASCADE,
+    position INTEGER NOT NULL
+  ) WITHOUT ROWID`);
+
+  await db.execute(`CREATE TABLE mirror.urls(
+    id INTEGER PRIMARY KEY,
+    guid TEXT NOT NULL,
+    url TEXT NOT NULL,
+    hash INTEGER NOT NULL,
+    revHost TEXT NOT NULL
+  )`);
+
+  await db.execute(`CREATE TABLE mirror.tags(
+    itemId INTEGER NOT NULL REFERENCES items(id)
+                            ON DELETE CASCADE,
+    tag TEXT NOT NULL
+  )`);
+
+  await db.execute(`CREATE INDEX mirror.urlHashes ON urls(hash)`);
+
+  await db.execute(`CREATE INDEX mirror.locations ON structure(
+    parentGuid,
+    position
+  )`);
+
+  // Set up the syncable roots.
+  await createMirrorRoots(db);
+}
+
+async function createMirrorRoots(db) {
+  const syncableRoots = [{
+    guid: PlacesUtils.bookmarks.rootGuid,
+    // The Places root is its own parent, to satisfy the foreign key and
+    // `NOT NULL` constraints on `structure`.
+    parentGuid: PlacesUtils.bookmarks.rootGuid,
+    position: -1,
+  }, {
+    guid: PlacesUtils.bookmarks.menuGuid,
+    parentGuid: PlacesUtils.bookmarks.rootGuid,
+    position: 0,
+  }, {
+    guid: PlacesUtils.bookmarks.toolbarGuid,
+    parentGuid: PlacesUtils.bookmarks.rootGuid,
+    position: 1,
+  }, {
+    guid: PlacesUtils.bookmarks.unfiledGuid,
+    parentGuid: PlacesUtils.bookmarks.rootGuid,
+    position: 2,
+  }, {
+    guid: PlacesUtils.bookmarks.mobileGuid,
+    parentGuid: PlacesUtils.bookmarks.rootGuid,
+    position: 3,
+  }];
+  for (let info of syncableRoots) {
+    await db.executeCached(`
+      INSERT INTO items(guid, kind)
+      VALUES(:guid, :kind)`,
+      { guid: info.guid, kind: SyncedBookmarksMirror.KIND.FOLDER });
+
+    await db.executeCached(`
+      INSERT INTO structure(guid, parentGuid, position)
+      VALUES(:guid, :parentGuid, :position)`,
+      info);
+  }
+}
+
+async function initializeTempMirrorEntities(db) {
+  // A table of column names in `items` that correspond to annos stored in
+  // `moz_items_annos`. We use this to build SQL fragments for the insert
+  // and update triggers below.
+  const syncedAnnoTriggers = [{
+    annoName: PlacesSyncUtils.bookmarks.DESCRIPTION_ANNO,
+    columnName: "description",
+    type: PlacesUtils.annotations.TYPE_STRING,
+  }, {
+    annoName: PlacesSyncUtils.bookmarks.SIDEBAR_ANNO,
+    columnName: "loadInSidebar",
+    type: PlacesUtils.annotations.TYPE_INT32,
+  }, {
+    annoName: PlacesSyncUtils.bookmarks.SMART_BOOKMARKS_ANNO,
+    columnName: "smartBookmarkName",
+    type: PlacesUtils.annotations.TYPE_STRING,
+  }, {
+    annoName: PlacesUtils.LMANNO_FEEDURI,
+    columnName: "feedURL",
+    type: PlacesUtils.annotations.TYPE_STRING,
+  }, {
+    annoName: PlacesUtils.LMANNO_SITEURI,
+    columnName: "siteURL",
+    type: PlacesUtils.annotations.TYPE_STRING,
+  }];
+
+  // An in-memory table with the value and structure states of all nodes
+  // in the merged tree.
+  await db.execute(`CREATE TEMP TABLE mergeStates(
+    localGuid TEXT NOT NULL,
+    mergedGuid TEXT NOT NULL,
+    parentGuid TEXT NOT NULL,
+    position INTEGER NOT NULL,
+    valueState INTEGER NOT NULL,
+    structureState INTEGER NOT NULL,
+    PRIMARY KEY(localGuid, mergedGuid)
+  ) WITHOUT ROWID`);
+
+  // A view of the value states for all bookmarks in the mirror. We use `INSTEAD
+  // OF DELETE` triggers on this view to update Places. This is effectively the
+  // same as issuing `SELECT * FROM newRemoteItems`, followed by separate
+  // `INSERT` and `UPDATE` statements. Note that we can't just `REPLACE
+  // INTO moz_bookmarks`, because `REPLACE` doesn't fire the `AFTER DELETE`
+  // triggers that Places uses to maintain schema coherency.
+  await db.execute(`
+    CREATE TEMP VIEW newRemoteItems(localId, remoteId, localGuid, mergedGuid,
+                                    needsUpdate, type, dateAdded, title,
+                                    oldPlaceId, newPlaceId, newKeyword,
+                                    description, loadInSidebar,
+                                    smartBookmarkName, feedURL, siteURL,
+                                    syncChangeCounter) AS
+    SELECT b.id, v.id, r.localGuid, r.mergedGuid,
+           r.valueState = ${BookmarkMergeState.TYPE.REMOTE},
+           (CASE WHEN v.kind IN (${[
+                        SyncedBookmarksMirror.KIND.BOOKMARK,
+                        SyncedBookmarksMirror.KIND.QUERY,
+                      ].join(",")}) THEN ${PlacesUtils.bookmarks.TYPE_BOOKMARK}
+                 WHEN v.kind IN (${[
+                        SyncedBookmarksMirror.KIND.FOLDER,
+                        SyncedBookmarksMirror.KIND.LIVEMARK,
+                      ].join(",")}) THEN ${PlacesUtils.bookmarks.TYPE_FOLDER}
+                 ELSE ${PlacesUtils.bookmarks.TYPE_SEPARATOR} END),
+           (CASE WHEN b.dateAdded < v.dateAdded THEN b.dateAdded
+                 ELSE v.dateAdded END),
+           v.title, h.id, u.newPlaceId, v.keyword, v.description,
+           v.loadInSidebar, v.smartBookmarkName, v.feedURL, v.siteURL,
+           (CASE r.structureState WHEN ${BookmarkMergeState.TYPE.REMOTE} THEN 0
+            ELSE 1 END)
+    FROM items v
+    JOIN mergeStates r ON r.mergedGuid = v.guid
+    LEFT JOIN moz_bookmarks b ON b.guid = r.localGuid
+    LEFT JOIN moz_places h ON h.id = b.fk
+    LEFT JOIN (
+      SELECT h.id AS newPlaceId, u.id AS urlId
+      FROM urls u
+      JOIN moz_places h ON h.url_hash = u.hash AND
+                           h.url = u.url
+    ) u ON u.urlId = v.urlId
+    WHERE r.mergedGuid <> '${PlacesUtils.bookmarks.rootGuid}'`);
+
+  // Changes local GUIDs to remote GUIDs, then drop local tombstones for revived
+  // remote items. We do this here, instead of in the `updateExistingLocalItems`
+  // trigger, because deduped items where we're keeping the local value state
+  // won't have `needsUpdate` set.
+  await db.execute(`
+    CREATE TEMP TRIGGER updateLocalGuids
+    INSTEAD OF DELETE ON newRemoteItems
+    BEGIN
+      UPDATE moz_bookmarks SET
+        guid = OLD.mergedGuid
+      WHERE guid = OLD.localGuid;
+      DELETE FROM moz_bookmarks_deleted WHERE guid = OLD.mergedGuid;
+    END`);
+
+  // Inserts items from the mirror that don't exist locally.
+  await db.execute(`
+    CREATE TEMP TRIGGER insertNewLocalItems
+    INSTEAD OF DELETE ON newRemoteItems WHEN OLD.localId IS NULL
+    BEGIN
+      /* Sync associates keywords with bookmarks, and doesn't sync POST data;
+         Places associates keywords with (URL, POST data) pairs, and multiple
+         bookmarks may have the same URL. For simplicity, we bump the change
+         counter for all local bookmarks with the remote URL (bug 1328737),
+         then remove all local keywords from remote URLs, and the remote keyword
+         from local URLs, . */
+      UPDATE moz_bookmarks SET
+        syncChangeCounter = syncChangeCounter + 1
+      WHERE fk IN (
+        /* We intentionally use "place_id = OLD.newPlaceId" in the subquery,
+           instead of "fk = OLD.newPlaceId" above, because we only want to bump
+           the counter if the URL has keywords. */
+        SELECT place_id FROM moz_keywords
+        WHERE place_id = OLD.newPlaceId OR
+              keyword = OLD.newKeyword);
+      DELETE FROM moz_keywords WHERE place_id = OLD.newPlaceId OR
+                                     keyword = OLD.newKeyword;
+
+      /* Remove existing tags. */
+      DELETE FROM localTags WHERE placeId = OLD.newPlaceId;
+
+      /* Insert the new item, using "-1" as the placeholder parent and
+         position. We'll update hese later, in the "updateLocalStructure"
+         trigger. */
+      INSERT INTO moz_bookmarks(guid, parent, position, type, fk, title,
+                                dateAdded, lastModified, syncStatus,
+                                syncChangeCounter)
+      VALUES(OLD.mergedGuid, -1, -1, OLD.type, OLD.newPlaceId, OLD.title,
+             OLD.dateAdded, STRFTIME('%s', 'now', 'localtime', 'utc') * 1000000,
+             ${PlacesUtils.bookmarks.SYNC_STATUS.NORMAL},
+             OLD.syncChangeCounter);
+
+      /* Insert new keywords after the item, so that "noteKeywordAdded" can find
+         the new item by Place ID. */
+      INSERT INTO moz_keywords(keyword, place_id)
+      SELECT OLD.newKeyword, OLD.newPlaceId
+      WHERE OLD.newKeyword NOT NULL;
+
+      /* Insert new tags for the URL. */
+      INSERT INTO localTags(tag, placeId)
+      SELECT t.tag, OLD.newPlaceId FROM tags t
+      WHERE t.itemId = OLD.remoteId;
+
+      /* Insert new synced annos. These are almost identical to the statements
+         for updates, except we need an additional subquery to fetch the new
+         item's ID. We can also skip removing existing annos. */
+      INSERT OR IGNORE INTO moz_anno_attributes(name)
+      VALUES ${syncedAnnoTriggers.map(annoTrigger =>
+        `('${annoTrigger.annoName}')`
+      ).join(",")};
+      ${syncedAnnoTriggers.map(annoTrigger => `
+        INSERT INTO moz_items_annos(item_id, anno_attribute_id, content, flags,
+                                    expiration, type, lastModified, dateAdded)
+        SELECT (SELECT id FROM moz_bookmarks WHERE guid = OLD.mergedGuid),
+               (SELECT id FROM moz_anno_attributes
+                WHERE name = '${annoTrigger.annoName}'),
+               OLD.${annoTrigger.columnName}, 0,
+               ${PlacesUtils.annotations.EXPIRE_NEVER}, ${annoTrigger.type},
+               STRFTIME('%s', 'now', 'localtime', 'utc') * 1000000,
+               STRFTIME('%s', 'now', 'localtime', 'utc') * 1000000
+        WHERE OLD.${annoTrigger.columnName} NOT NULL;
+      `).join("")}
+    END`);
+
+  // Updates existing items with new values from the mirror.
+  await db.execute(`
+    CREATE TEMP TRIGGER updateExistingLocalItems
+    INSTEAD OF DELETE ON newRemoteItems WHEN OLD.needsUpdate AND
+                                             OLD.localId NOT NULL
+    BEGIN
+      UPDATE moz_bookmarks SET
+        title = OLD.title,
+        dateAdded = OLD.dateAdded,
+        lastModified = STRFTIME('%s', 'now', 'localtime', 'utc') * 1000000,
+        syncStatus = ${PlacesUtils.bookmarks.SYNC_STATUS.NORMAL},
+        syncChangeCounter = OLD.syncChangeCounter
+      WHERE id = OLD.localId;
+
+      /* Remove existing keywords. */
+      UPDATE moz_bookmarks SET
+        syncChangeCounter = syncChangeCounter + 1
+      WHERE fk IN (SELECT place_id FROM moz_keywords
+                   WHERE place_id IN (OLD.oldPlaceId, OLD.newPlaceId) OR
+                         keyword = OLD.newKeyword);
+      DELETE FROM moz_keywords WHERE place_id IN (OLD.oldPlaceId,
+                                                  OLD.newPlaceId) OR
+                                     keyword = OLD.newKeyword;
+
+      /* Remove existing tags. */
+      DELETE FROM localTags WHERE placeId IN (OLD.oldPlaceId, OLD.newPlaceId);
+
+      /* Update the URL and recalculate frecency. It's important we do this
+         *after* removing old keywords and *before* inserting new ones, so that
+         the "noteKeywordRemoved" and "noteKeywordAdded" triggers can find the
+         affected bookmarks by Place ID. */
+      UPDATE moz_bookmarks SET
+        fk = OLD.newPlaceId
+      WHERE OLD.oldPlaceId <> OLD.newPlaceId AND
+            id = OLD.localId;
+      UPDATE moz_places SET
+        frecency = -1
+      WHERE OLD.oldPlaceId <> OLD.newPlaceId AND
+            id IN (OLD.oldPlaceId, OLD.newPlaceId);
+
+      /* Insert new keywords for the new URL. */
+      INSERT INTO moz_keywords(keyword, place_id)
+      SELECT OLD.newKeyword, OLD.newPlaceId
+      WHERE OLD.newKeyword NOT NULL;
+
+      /* Insert new tags for the new URL. */
+      INSERT INTO localTags(tag, placeId)
+      SELECT t.tag, OLD.newPlaceId FROM tags t
+      WHERE t.itemId = OLD.remoteId;
+
+      /* Remove existing synced annos. */
+      DELETE FROM moz_items_annos
+      WHERE item_id = OLD.localId AND
+            anno_attribute_id IN (SELECT id FROM moz_anno_attributes
+                                  WHERE name IN (${syncedAnnoTriggers.map(
+                                    annoTrigger => `'${annoTrigger.annoName}'`
+                                  ).join(",")}));
+
+      /* Insert new synced annos. */
+      INSERT OR IGNORE INTO moz_anno_attributes(name)
+      VALUES ${syncedAnnoTriggers.map(annoTrigger =>
+        `('${annoTrigger.annoName}')`
+      ).join(",")};
+      ${syncedAnnoTriggers.map(annoTrigger => `
+        INSERT INTO moz_items_annos(item_id, anno_attribute_id, content, flags,
+                                    expiration, type, lastModified, dateAdded)
+        SELECT OLD.localId, (SELECT id FROM moz_anno_attributes
+                             WHERE name = '${annoTrigger.annoName}'),
+               OLD.${annoTrigger.columnName}, 0,
+               ${PlacesUtils.annotations.EXPIRE_NEVER}, ${annoTrigger.type},
+               STRFTIME('%s', 'now', 'localtime', 'utc') * 1000000,
+               STRFTIME('%s', 'now', 'localtime', 'utc') * 1000000
+         WHERE OLD.${annoTrigger.columnName} NOT NULL;
+      `).join("")}
+    END`);
+
+  // A view of the new structure state for all items in the merged tree. The
+  // mirror stores structure info in a separate table, like iOS, while Places
+  // stores structure info on children. Unlike iOS, we can't simply check the
+  // parent's merge state to know if its children changed. This is because our
+  // merged tree might diverge from the mirror if we're missing children, or if
+  // we temporarily reparented children without parents to "unfiled". In that
+  // case, we want to keep syncing, but *don't* want to reupload the new local
+  // structure to the server.
+  await db.execute(`
+    CREATE TEMP VIEW newRemoteStructure(localId, parentId, position) AS
+    SELECT b.id, p.id, r.position FROM moz_bookmarks b
+    JOIN mergeStates r ON r.mergedGuid = b.guid
+    JOIN moz_bookmarks p ON p.guid = r.parentGuid
+    WHERE r.parentGuid <> '${PlacesUtils.bookmarks.rootGuid}'`);
+
+  // Updates all parents and positions to reflect the merged tree.
+  await db.execute(`
+    CREATE TEMP TRIGGER updateLocalStructure
+    INSTEAD OF DELETE ON newRemoteStructure
+    BEGIN
+      UPDATE moz_bookmarks SET
+        parent = OLD.parentId
+      WHERE id = OLD.localId AND
+            parent <> OLD.parentId;
+      UPDATE moz_bookmarks SET
+        position = OLD.position
+      WHERE id = OLD.localId AND
+            position <> OLD.position;
+    END`);
+
+  // A view of local bookmark tags. Tags, like keywords, are associated with
+  // URLs, so two bookmarks with the same URL should have the same tags. Unlike
+  // keywords, one tag may be associated with many different URLs. Tags are also
+  // different because they're implemented as bookmarks under the hood. Each tag
+  // is stored as a folder under the tags root, and tagged URLs are stored as
+  // untitled bookmarks under these folders. This complexity, along with tag
+  // query rewriting, can be removed once bug 1293445 lands.
+  await db.execute(`
+    CREATE TEMP VIEW localTags(tagEntryId, tagEntryPosition, tagFolderId,
+                               tag, placeId) AS
+    SELECT b.id, b.position, p.id, p.title, b.fk
+    FROM moz_bookmarks b
+    JOIN moz_bookmarks p ON p.id = b.parent
+    JOIN moz_bookmarks r ON r.id = p.parent
+    WHERE b.type = ${PlacesUtils.bookmarks.TYPE_BOOKMARK} AND
+          r.guid = '${PlacesUtils.bookmarks.tagsGuid}'`);
+
+  // Untags a URL by removing its tag entry.
+  await db.execute(`
+    CREATE TEMP TRIGGER untagLocalPlace
+    INSTEAD OF DELETE ON localTags
+    BEGIN
+      DELETE FROM moz_bookmarks WHERE id = OLD.tagEntryId;
+
+      /* Fix the positions of the sibling tag entries. */
+      UPDATE moz_bookmarks SET
+        position = position - 1
+      WHERE parent = OLD.tagFolderId AND
+            position > OLD.tagEntryPosition;
+    END`);
+
+  // Tags a URL by creating a tag folder if it doesn't exist, then inserting a
+  // tag entry for the URL into the tag folder. `NEW.placeId` can be `NULL`, in
+  // which case we'll just create the tag folder.
+  await db.execute(`
+    CREATE TEMP TRIGGER tagLocalPlace
+    INSTEAD OF INSERT ON localTags
+    BEGIN
+      /* Ensure the tag folder exists. */
+      INSERT OR IGNORE INTO moz_bookmarks(guid, parent, position, type, title,
+                                          dateAdded, lastModified)
+      VALUES(IFNULL((SELECT b.guid FROM moz_bookmarks b
+                     JOIN moz_bookmarks p ON p.id = b.parent
+                     WHERE b.title = NEW.tag AND
+                           p.guid = '${PlacesUtils.bookmarks.tagsGuid}'),
+                    GENERATE_GUID()),
+             (SELECT id FROM moz_bookmarks
+              WHERE guid = '${PlacesUtils.bookmarks.tagsGuid}'),
+             (SELECT COUNT(*) FROM moz_bookmarks b
+              JOIN moz_bookmarks p ON p.id = b.parent
+              WHERE p.guid = '${PlacesUtils.bookmarks.tagsGuid}'),
+             ${PlacesUtils.bookmarks.TYPE_FOLDER}, NEW.tag,
+             STRFTIME('%s', 'now', 'localtime', 'utc') * 1000000,
+             STRFTIME('%s', 'now', 'localtime', 'utc') * 1000000);
+
+      /* Add a tag entry for the URL under the tag folder. Omitting the place
+         ID creates a tag folder without tagging the URL. */
+      INSERT OR IGNORE INTO moz_bookmarks(guid, parent, position, type, fk,
+                                          dateAdded, lastModified)
+      SELECT IFNULL((SELECT b.guid FROM moz_bookmarks b
+                     JOIN moz_bookmarks p ON p.id = b.parent
+                     JOIN moz_bookmarks r ON r.id = p.parent
+                     WHERE b.fk = NEW.placeId AND
+                           p.title = NEW.tag AND
+                           r.guid = '${PlacesUtils.bookmarks.tagsGuid}'),
+                    GENERATE_GUID()),
+             (SELECT b.id FROM moz_bookmarks b
+              JOIN moz_bookmarks p ON p.id = b.parent
+              WHERE p.guid = '${PlacesUtils.bookmarks.tagsGuid}' AND
+                    b.title = NEW.tag),
+             (SELECT COUNT(*) FROM moz_bookmarks b
+              JOIN moz_bookmarks p ON p.id = b.parent
+              JOIN moz_bookmarks r ON r.id = p.parent
+              WHERE p.title = NEW.tag AND
+                    r.guid = '${PlacesUtils.bookmarks.tagsGuid}'),
+             ${PlacesUtils.bookmarks.TYPE_BOOKMARK}, NEW.placeId,
+             STRFTIME('%s', 'now', 'localtime', 'utc') * 1000000,
+             STRFTIME('%s', 'now', 'localtime', 'utc') * 1000000
+        WHERE NEW.placeId NOT NULL;
+    END`);
+
+  // Locally changed items staged for upload. See `stageItemsToUpload` for
+  // a more thorough explanation of why this table exists.
+  await db.execute(`CREATE TEMP TABLE itemsToUpload(
+    guid TEXT PRIMARY KEY,
+    syncChangeCounter INTEGER NOT NULL,
+    isDeleted BOOLEAN NOT NULL DEFAULT 0,
+    parentGuid TEXT,
+    parentTitle TEXT,
+    dateAdded INTEGER,
+    type INTEGER,
+    title TEXT,
+    isQuery BOOLEAN NOT NULL DEFAULT 0,
+    url TEXT,
+    tags TEXT,
+    description TEXT,
+    loadInSidebar BOOLEAN,
+    smartBookmarkName TEXT,
+    tagFolderName TEXT,
+    keyword TEXT,
+    feedURL TEXT,
+    siteURL TEXT,
+    position INTEGER
+  ) WITHOUT ROWID`);
+
+  await db.execute(`CREATE TEMP TABLE structureToUpload(
+    guid TEXT PRIMARY KEY,
+    parentGuid TEXT NOT NULL REFERENCES itemsToUpload(guid)
+                             ON DELETE CASCADE,
+    position INTEGER NOT NULL
+  ) WITHOUT ROWID`);
+}
+
+// Converts a Sync record ID to a Places GUID. Returns `null` if the ID is
+// invalid.
+function validateGuid(recordId) {
+  let guid = PlacesSyncUtils.bookmarks.recordIdToGuid(recordId);
+  return PlacesUtils.isValidGuid(guid) ? guid : null;
+}
+
+// Converts a Sync record's last modified time to milliseconds.
+function determineServerModified(record) {
+  return Math.max(record.modified * 1000, 0) || 0;
+}
+
+// Determines a Sync record's creation date.
+function determineDateAdded(record) {
+  let serverModified = determineServerModified(record);
+  let dateAdded = PlacesSyncUtils.bookmarks.ratchetTimestampBackwards(
+    record.dateAdded, serverModified);
+  return dateAdded ? PlacesUtils.toPRTime(new Date(dateAdded)) : 0;
+}
+
+function validateTitle(rawTitle) {
+  if (typeof rawTitle != "string" || !rawTitle) {
+    return null;
+  }
+  return rawTitle.slice(0, DB_TITLE_LENGTH_MAX);
+}
+
+function validateURL(rawURL) {
+  if (typeof rawURL != "string" || rawURL.length > DB_URL_LENGTH_MAX) {
+    return null;
+  }
+  let url = null;
+  try {
+    url = new URL(rawURL);
+  } catch (ex) {}
+  return url;
+}
+
+function validateDescription(rawDescription) {
+  if (typeof rawDescription != "string" || !rawDescription) {
+    return null;
+  }
+  return rawDescription.slice(0, DB_DESCRIPTION_LENGTH_MAX);
+}
+
+function validateKeyword(rawKeyword) {
+  if (typeof rawKeyword != "string") {
+    return null;
+  }
+  let keyword = rawKeyword.trim();
+  return keyword ? keyword.toLowerCase() : null;
+}
+
+// Remove leading and trailing whitespace; ignore empty tags.
+function validateTag(rawTag) {
+  if (typeof rawTag != "string") {
+    return null;
+  }
+  let tag = rawTag.trim();
+  if (!tag || tag.length > Ci.nsITaggingService.MAX_TAG_LENGTH) {
+    return null;
+  }
+  return tag;
+}
+
+// Recursively inflates a bookmark tree from a pseudo-tree that maps
+// parents to children.
+function inflateTree(tree, pseudoTree, parentGuid) {
+  let nodes = pseudoTree.get(parentGuid);
+  if (nodes) {
+    for (let node of nodes) {
+      tree.insert(parentGuid, node);
+      inflateTree(tree, pseudoTree, node.guid);
+    }
+  }
+}
+
+class BookmarkContent {
+  constructor(title, urlHref, smartBookmarkName, position) {
+    this.title = title;
+    this.url = urlHref ? new URL(urlHref) : null;
+    this.smartBookmarkName = smartBookmarkName;
+    this.position = position;
+  }
+
+  static fromRow(row) {
+    let title = row.getResultByName("title");
+    let urlHref = row.getResultByName("url");
+    let smartBookmarkName = row.getResultByName("smartBookmarkName");
+    let position = row.getResultByName("position");
+    return new BookmarkContent(title, urlHref, smartBookmarkName, position);
+  }
+
+  hasSameURL(otherContent) {
+    return !!this.url == !!otherContent.url &&
+           this.url.href == otherContent.url.href;
+  }
+}
+
+/**
+ * The merge state indicates which node we should prefer when reconciling
+ * with Places. Recall that a merged node may point to a local node, remote
+ * node, or both.
+ */
+class BookmarkMergeState {
+  constructor(type, newStructureNode = null) {
+    this.type = type;
+    this.newStructureNode = newStructureNode;
+  }
+
+  /**
+   * Takes an existing value state, and a new node for the structure state. We
+   * use the new merge state to resolve conflicts caused by moving local items
+   * out of a remotely deleted folder, or remote items out of a locally deleted
+   * folder.
+   *
+   * Applying a new merged node bumps its local change counter, so that the
+   * merged structure is reuploaded to the server.
+   *
+   * @param  {BookmarkMergeState} oldState
+   *         The existing value state.
+   * @param  {BookmarkNode} newStructureNode
+   *         A node to use for the new structure state.
+   * @return {BookmarkMergeState}
+   *         The new merge state.
+   */
+  static new(oldState, newStructureNode) {
+    return new BookmarkMergeState(oldState.type, newStructureNode);
+  }
+
+  // Returns the structure state type: `LOCAL`, `REMOTE`, or `NEW`.
+  structure() {
+    return this.newStructureNode ? BookmarkMergeState.TYPE.NEW : this.type;
+  }
+
+  // Returns the value state type: `LOCAL` or `REMOTE`.
+  value() {
+    return this.type;
+  }
+}
+
+BookmarkMergeState.TYPE = {
+  LOCAL: 1,
+  REMOTE: 2,
+  NEW: 3,
+};
+
+/**
+ * A local merge state means no changes: we keep the local value and structure
+ * state. This could mean that the item doesn't exist on the server yet, or that
+ * it has newer local changes that we should upload.
+ *
+ * It's an error for a merged node to have a local merge state without a local
+ * node. Deciding the value state for the merged node asserts this.
+ */
+BookmarkMergeState.local = new BookmarkMergeState(
+  BookmarkMergeState.TYPE.LOCAL);
+
+/**
+ * A remote merge state means we should update Places with new value and
+ * structure state from the mirror. The item might not exist locally yet, or
+ * might have newer remote changes that we should apply.
+ *
+ * As with local, a merged node can't have a remote merge state without a
+ * remote node.
+ */
+BookmarkMergeState.remote = new BookmarkMergeState(
+  BookmarkMergeState.TYPE.REMOTE);
+
+/**
+ * A node in a local or remote bookmark tree. Nodes are lightweight: they carry
+ * enough information for the merger to resolve trivial conflicts without
+ * querying the mirror or Places for the complete value state.
+ *
+ * There are 5 kinds of nodes, one for each Sync record kind.
+ */
+class BookmarkNode {
+  constructor(guid, age, kind, needsMerge = false) {
+    this.guid = guid;
+    this.kind = kind;
+    this.age = age;
+    this.needsMerge = needsMerge;
+    this.children = [];
+  }
+
+  // Creates a virtual folder node for the Places root.
+  static root() {
+    let guid = PlacesUtils.bookmarks.rootGuid;
+    return new BookmarkNode(guid, 0, SyncedBookmarksMirror.KIND.FOLDER);
+  }
+
+  /**
+   * Creates a bookmark node from a Places row.
+   *
+   * @param  {mozIStorageRow} row
+   *         The Places row containing the node info.
+   * @param  {Number} localTimeSeconds
+   *         The current local time, in seconds, used to calculate the
+   *         item's age.
+   * @return {BookmarkNode}
+   *         A bookmark node for the local item.
+   */
+  static fromLocalRow(row, localTimeSeconds) {
+    let guid = row.getResultByName("guid");
+
+    // Note that this doesn't account for local clock skew. `localModified`
+    // is in *microseconds*.
+    let localModified = row.getResultByName("lastModified");
+    let age = Math.max(localTimeSeconds - localModified / 1000000, 0) || 0;
+
+    let kind = row.getResultByName("kind");
+
+    let syncChangeCounter = row.getResultByName("syncChangeCounter");
+    let needsMerge = syncChangeCounter > 0;
+
+    return new BookmarkNode(guid, age, kind, needsMerge);
+  }
+
+  /**
+   * Creates a bookmark node from a mirror row.
+   *
+   * @param  {mozIStorageRow} row
+   *         The mirror row containing the node info.
+   * @param  {Number} remoteTimeSeconds
+   *         The current server time, in seconds, used to calculate the
+   *         item's age.
+   * @return {BookmarkNode}
+   *         A bookmark node for the remote item.
+   */
+  static fromRemoteRow(row, remoteTimeSeconds) {
+    let guid = row.getResultByName("guid");
+
+    // `serverModified` is in *milliseconds*.
+    let serverModified = row.getResultByName("serverModified");
+    let age = Math.max(remoteTimeSeconds - serverModified / 1000, 0) || 0;
+
+    let kind = row.getResultByName("kind");
+    let needsMerge = !!row.getResultByName("needsMerge");
+
+    return new BookmarkNode(guid, age, kind, needsMerge);
+  }
+
+  isRoot() {
+    return this.guid == PlacesUtils.bookmarks.rootGuid ||
+           PlacesUtils.bookmarks.userContentRoots.includes(this.guid);
+  }
+
+  isFolder() {
+    return this.kind == SyncedBookmarksMirror.KIND.FOLDER;
+  }
+
+  newerThan(otherNode) {
+    return this.age < otherNode.age;
+  }
+
+  * descendants() {
+    for (let node of this.children) {
+      yield node;
+      if (node.isFolder()) {
+        yield* node.descendants();
+      }
+    }
+  }
+}
+
+/**
+ * A complete, rooted tree with tombstones.
+ */
+class BookmarkTree {
+  constructor(root) {
+    this.byGuid = new Map();
+    this.infosByNode = new WeakMap();
+    this.deletedGuids = new Set();
+
+    this.root = root;
+    this.byGuid.set(this.root.guid, this.root);
+  }
+
+  isDeleted(guid) {
+    return this.deletedGuids.has(guid);
+  }
+
+  nodeForGuid(guid) {
+    return this.byGuid.get(guid);
+  }
+
+  parentNodeFor(childNode) {
+    let info = this.infosByNode.get(childNode);
+    return info ? info.parentNode : null;
+  }
+
+  levelForGuid(guid) {
+    let node = this.byGuid.get(guid);
+    if (!node) {
+      return -1;
+    }
+    let info = this.infosByNode.get(node);
+    return info ? info.level : -1;
+  }
+
+  /**
+   * Inserts a node into the tree. The node must not already exist in the tree,
+   * and the node's parent must be a folder.
+   */
+  insert(parentGuid, node) {
+    if (this.byGuid.has(node.guid)) {
+      let existingNode = this.byGuid.get(node.guid);
+      MirrorLog.error("Can't replace existing node ${existingNode} with node " +
+                      "${node}", { existingNode, node });
+      throw new TypeError("Node already exists in tree");
+    }
+    let parentNode = this.byGuid.get(parentGuid);
+    if (!parentNode) {
+      MirrorLog.error("Missing parent ${parentGuid} for node ${node}",
+                      { parentGuid, node });
+      throw new TypeError("Can't insert node into nonexistent parent");
+    }
+    if (!parentNode.isFolder()) {
+      MirrorLog.error("Non-folder parent ${parentNode} for node ${node}",
+                      { parentNode, node });
+      throw new TypeError("Can't insert node into non-folder");
+    }
+
+    parentNode.children.push(node);
+    this.byGuid.set(node.guid, node);
+
+    let parentInfo = this.infosByNode.get(parentNode);
+    let level = parentInfo ? parentInfo.level + 1 : 0;
+    this.infosByNode.set(node, { parentNode, level });
+  }
+
+  noteDeleted(guid) {
+    this.deletedGuids.add(guid);
+  }
+
+  * guids() {
+    for (let [guid, node] of this.byGuid) {
+      if (node == this.root) {
+        continue;
+      }
+      yield guid;
+    }
+    for (let guid of this.deletedGuids) {
+      yield guid;
+    }
+  }
+
+  toJSON() {
+    let deleted = Array.from(this.deletedGuids);
+    return { root: this.root, deleted };
+  }
+}
+
+/**
+ * A node in a merged bookmark tree. Holds the local node, remote node,
+ * merged children, and a merge state indicating which side to prefer.
+ */
+class MergedBookmarkNode {
+  constructor(guid, localNode, remoteNode, mergeState) {
+    this.guid = guid;
+    this.localNode = localNode;
+    this.remoteNode = remoteNode;
+    this.mergeState = mergeState;
+    this.mergedChildren = [];
+  }
+
+  /**
+   * Yields the decided value and structure states of the merged node's
+   * descendants. We use these as binding parameters to populate the temporary
+   * `mergeStates` table when applying the merged tree to Places.
+   */
+  * mergeStatesParams() {
+    for (let position = 0; position < this.mergedChildren.length; ++position) {
+      let mergedChild = this.mergedChildren[position];
+      let mergeStateParam = {
+        localGuid: mergedChild.localNode ? mergedChild.localNode.guid : null,
+        // The merged GUID is different than the local GUID if we deduped a
+        // NEW local item to a remote item.
+        mergedGuid: mergedChild.guid,
+        parentGuid: this.guid,
+        position,
+        valueState: mergedChild.mergeState.value(),
+        structureState: mergedChild.mergeState.structure(),
+      };
+      yield mergeStateParam;
+      yield* mergedChild.mergeStatesParams();
+    }
+  }
+
+  /**
+   * Creates a bookmark node from this merged node.
+   *
+   * @return {BookmarkNode}
+   *         A node containing the decided value and structure state.
+   */
+  toBookmarkNode() {
+    if (MergedBookmarkNode.cachedBookmarkNodes.has(this)) {
+      return MergedBookmarkNode.cachedBookmarkNodes.get(this);
+    }
+
+    let decidedValueNode = this.decidedValue();
+    let decidedStructureState = this.mergeState.structure();
+    let needsMerge = decidedStructureState == BookmarkMergeState.TYPE.NEW ||
+                     (decidedStructureState == BookmarkMergeState.TYPE.LOCAL &&
+                      decidedValueNode.needsMerge);
+
+    let newNode = new BookmarkNode(this.guid, decidedValueNode.age,
+                                   decidedValueNode.kind, needsMerge);
+    MergedBookmarkNode.cachedBookmarkNodes.set(this, newNode);
+
+    if (newNode.isFolder()) {
+      for (let mergedChildNode of this.mergedChildren) {
+        newNode.children.push(mergedChildNode.toBookmarkNode());
+      }
+    }
+
+    return newNode;
+  }
+
+  /**
+   * Decides the value state for the merged node. Note that you can't walk the
+   * decided node's children: since the value node doesn't include structure
+   * changes from the other side, you'll depart from the merged tree. You'll
+   * want to use `toBookmarkNode` instead, which returns a node with the
+   * decided value *and* structure.
+   *
+   * @return {BookmarkNode}
+   *         The local or remote node containing the decided value state.
+   */
+  decidedValue() {
+    let valueState = this.mergeState.value();
+    switch (valueState) {
+      case BookmarkMergeState.TYPE.LOCAL:
+        if (!this.localNode) {
+          MirrorLog.error("Merged node ${guid} has local value state, but " +
+                          "no local node", this);
+          throw new TypeError(
+            "Can't take local value state without local node");
+        }
+        return this.localNode;
+
+      case BookmarkMergeState.TYPE.REMOTE:
+        if (!this.remoteNode) {
+          MirrorLog.error("Merged node ${guid} has remote value state, but " +
+                          "no remote node", this);
+          throw new TypeError(
+            "Can't take remote value state without remote node");
+        }
+        return this.remoteNode;
+    }
+    MirrorLog.error("Merged node ${guid} has unknown value state ${valueState}",
+                    { guid: this.guid, valueState });
+    throw new TypeError("Can't take unknown value state");
+  }
+}
+
+// Caches bookmark nodes containing the decided value and structure.
+MergedBookmarkNode.cachedBookmarkNodes = new WeakMap();
+
+/**
+ * A two-way merger that produces a complete merged tree from a complete local
+ * tree and a complete remote tree with changes since the last sync.
+ *
+ * This is ported almost directly from iOS. On iOS, the `ThreeWayMerger` takes a
+ * complete "mirror" tree with the server state after the last sync, and two
+ * incomplete trees with local and remote changes to the mirror: "local" and
+ * "mirror", respectively. Overlaying buffer onto mirror yields the current
+ * server tree; overlaying local onto mirror yields the complete local tree.
+ *
+ * On Desktop, our `localTree` is the union of iOS's mirror and local, and our
+ * `remoteTree` is the union of iOS's mirror and buffer. Mapping the iOS
+ * concepts to Desktop:
+ *
+ * - "Mirror" is approximately all `moz_bookmarks` where `syncChangeCounter = 0`
+ *   and `items` where `needsMerge = 0`. This is approximate because Desktop
+ *   doesn't store the shared parent for changed items.
+ * - "Local" is all `moz_bookmarks` where `syncChangeCounter > 0`.
+ * - "Buffer" is all `items` where `needsMerge = 1`.
+ *
+ * Since we don't store the shared parent, we can only do two-way merges. Also,
+ * our merger doesn't distinguish between structure and value changes, since we
+ * don't record that state in Places. The change counter notes *that* a bookmark
+ * changed, but not *how*. This means we might choose the wrong side when
+ * resolving merge conflicts, while iOS will do the right thing.
+ *
+ * Fortunately, most of our users don't organize their bookmarks into deeply
+ * nested hierarchies, or make conflicting changes on multiple devices
+ * simultaneously. Changing Places to record structure and value changes would
+ * require significant changes to the storage schema. A simpler two-way tree
+ * merge strikes a good balance between correctness and complexity.
+ */
+class BookmarkMerger {
+  constructor(localTree, newLocalContents, remoteTree, newRemoteContents) {
+    this.localTree = localTree;
+    this.newLocalContents = newLocalContents;
+    this.remoteTree = remoteTree;
+    this.newRemoteContents = newRemoteContents;
+    this.mergedGuids = new Set();
+    this.deleteLocally = new Set();
+    this.deleteRemotely = new Set();
+    this.telemetryEvents = [];
+  }
+
+  merge() {
+    let localRoot = this.localTree.nodeForGuid(PlacesUtils.bookmarks.rootGuid);
+    let remoteRoot = this.remoteTree.nodeForGuid(PlacesUtils.bookmarks.rootGuid);
+    let mergedRoot = this.mergeNode(PlacesUtils.bookmarks.rootGuid, localRoot,
+                                    remoteRoot);
+    return mergedRoot;
+  }
+
+  subsumes(tree) {
+    for (let guid of tree.guids()) {
+      if (!this.mergedGuids.has(guid) && !this.deleteLocally.has(guid) &&
+          !this.deleteRemotely.has(guid)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Merges two nodes, recursively walking folders.
+   *
+   * @param  {String} guid
+   *         The GUID to use for the merged node.
+   * @param  {BookmarkNode?} localNode
+   *         The local node. May be `null` if the node only exists remotely.
+   * @param  {BookmarkNode?} remoteNode
+   *         The remote node. May be `null` if the node only exists locally.
+   * @return {MergedBookmarkNode}
+   *         The merged node, with merged folder children.
+   */
+  mergeNode(mergedGuid, localNode, remoteNode) {
+    this.mergedGuids.add(mergedGuid);
+
+    if (localNode) {
+      if (localNode.guid != mergedGuid) {
+        // We deduped a NEW local item to a remote item.
+        this.mergedGuids.add(localNode.guid);
+      }
+
+      if (remoteNode) {
+        MirrorLog.trace("Item ${mergedGuid} exists locally as ${localNode} " +
+                        "and remotely as ${remoteNode}; merging",
+                        { mergedGuid, localNode, remoteNode });
+        let mergedNode = this.twoWayMerge(mergedGuid, localNode, remoteNode);
+        return mergedNode;
+      }
+
+      MirrorLog.trace("Item ${mergedGuid} only exists locally as " +
+                      "${localNode}; taking local state", { mergedGuid,
+                                                            localNode });
+      let mergedNode = new MergedBookmarkNode(mergedGuid, localNode, null,
+                                              BookmarkMergeState.local);
+      if (localNode.isFolder()) {
+        // The local folder doesn't exist remotely, but its children might, so
+        // we still need to recursively walk and merge them. This method will
+        // change the merge state from local to new if any children were moved
+        // or deleted.
+        this.mergeChildListsIntoMergedNode(mergedNode, localNode,
+                                           /* remoteNode */ null);
+      }
+      return mergedNode;
+    }
+
+    if (remoteNode) {
+      MirrorLog.trace("Item ${mergedGuid} only exists remotely as " +
+                      "${remoteNode}; taking remote state", { mergedGuid,
+                                                              remoteNode });
+      let mergedNode = new MergedBookmarkNode(mergedGuid, null, remoteNode,
+                                              BookmarkMergeState.remote);
+      if (remoteNode.isFolder()) {
+        // As above, a remote folder's children might still exist locally, so we
+        // need to merge them and update the merge state from remote to new if
+        // any children were moved or deleted.
+        this.mergeChildListsIntoMergedNode(mergedNode, /* localNode */ null,
+                                           remoteNode);
+      }
+      return mergedNode;
+    }
+
+    // Should never happen. We need to have at least one node for a two-way
+    // merge.
+    throw new TypeError("Can't merge two nonexistent nodes");
+  }
+
+  /**
+   * Merges two nodes that exist locally and remotely.
+   *
+   * @param  {String} mergedGuid
+   *         The GUID to use for the merged node.
+   * @param  {BookmarkNode} localNode
+   *         The existing local node.
+   * @param  {BookmarkNode} remoteNode
+   *         The existing remote node.
+   * @return {MergedBookmarkNode}
+   *         The merged node, with merged folder children.
+   */
+  twoWayMerge(mergedGuid, localNode, remoteNode) {
+    let mergeState = this.resolveTwoWayValueConflict(mergedGuid, localNode,
+                                                     remoteNode);
+    MirrorLog.trace("Merge state for ${mergedGuid} is ${mergeState}",
+                    { mergedGuid, mergeState });
+
+    let mergedNode = new MergedBookmarkNode(mergedGuid, localNode, remoteNode,
+                                            mergeState);
+
+    if (localNode.isFolder()) {
+      if (remoteNode.isFolder()) {
+        // Merging two folders, so we need to walk their children to handle
+        // structure changes.
+        MirrorLog.trace("Merging folders ${localNode} and ${remoteNode}",
+                        { localNode, remoteNode });
+        this.mergeChildListsIntoMergedNode(mergedNode, localNode, remoteNode);
+        return mergedNode;
+      }
+
+      if (remoteNode.kind == SyncedBookmarksMirror.KIND.LIVEMARK) {
+        // We allow merging local folders and remote livemarks because Places
+        // stores livemarks as empty folders with feed and site URL annotations.
+        // The livemarks service first inserts the folder, and *then* sets
+        // annotations. Since this isn't wrapped in a transaction, we might sync
+        // before the annotations are set, and upload a folder record instead
+        // of a livemark record (bug 632287), then replace the folder with a
+        // livemark on the next sync.
+        MirrorLog.trace("Merging local folder ${localNode} and remote " +
+                        "livemark ${remoteNode}", { localNode, remoteNode });
+        this.telemetryEvents.push({
+          value: "kind",
+          extra: { local: "folder", remote: "folder" },
+        });
+        return mergedNode;
+      }
+
+      MirrorLog.error("Merging local folder ${localNode} and remote " +
+                      "non-folder ${remoteNode}", { localNode, remoteNode });
+      throw new SyncedBookmarksMirror.ConsistencyError(
+        "Can't merge folder and non-folder");
+    }
+
+    if (localNode.kind == remoteNode.kind) {
+      // Merging two non-folders, so no need to walk children.
+      MirrorLog.trace("Merging non-folders ${localNode} and ${remoteNode}",
+                      { localNode, remoteNode });
+      return mergedNode;
+    }
+
+    MirrorLog.error("Merging local ${localNode} and remote ${remoteNode} " +
+                    "with different kinds", { localNode, remoteNode });
+    throw new SyncedBookmarksMirror.ConsistencyError(
+      "Can't merge different item kinds");
+  }
+
+  /**
+   * Determines the merge state for a node that exists locally and remotely.
+   *
+   * @param  {String} mergedGuid
+   *         The GUID of the merged node. This is the same as the remote GUID,
+   *         and usually the same as the local GUID. The local GUID may be
+   *         different if we're deduping a local item to a remote item.
+   * @param  {String} localNode
+   *         The local bookmark node.
+   * @param  {BookmarkNode} remoteNode
+   *         The remote bookmark node.
+   * @return {BookmarkMergeState}
+   *         The two-way merge state.
+   */
+  resolveTwoWayValueConflict(mergedGuid, localNode, remoteNode) {
+    if (!remoteNode.needsMerge) {
+      // The node wasn't changed remotely since the last sync. Keep the local
+      // state.
+      return BookmarkMergeState.local;
+    }
+    if (!localNode.needsMerge) {
+      // The node was changed remotely, but not locally. Take the remote state.
+      return BookmarkMergeState.remote;
+    }
+    // At this point, we know the item changed locally and remotely. We could
+    // query storage to determine if the value state is the same, as iOS does.
+    // However, that's an expensive check that requires joining `moz_bookmarks`,
+    // `moz_items_annos`, and `moz_places` to the mirror. It's unlikely that
+    // the value state is identical, so we skip the value check and use the
+    // timestamp to decide which node is newer.
+    let valueState = localNode.newerThan(remoteNode) ?
+                     BookmarkMergeState.local :
+                     BookmarkMergeState.remote;
+    return valueState;
+  }
+
+  /**
+   * Merges a remote child node into a merged folder node.
+   *
+   * @param  {MergedBookmarkNode} mergedNode
+   *         The merged folder node.
+   * @param  {BookmarkNode} remoteParentNode
+   *         The remote folder node.
+   * @param  {BookmarkNode} remoteChildNode
+   *         The remote child node.
+   * @return {Boolean}
+   *         `true` if the merged structure state changed because the remote
+   *         child was locally moved or deleted; `false` otherwise.
+   */
+  mergeRemoteChildIntoMergedNode(mergedNode, remoteParentNode,
+                                 remoteChildNode) {
+    if (this.mergedGuids.has(remoteChildNode.guid)) {
+      MirrorLog.trace("Remote child ${remoteChildNode} already seen in " +
+                      "another folder and merged", { remoteChildNode });
+      return false;
+    }
+
+    MirrorLog.trace("Merging remote child ${remoteChildNode} of " +
+                    "${remoteParentNode} into ${mergedNode}",
+                    { remoteChildNode, remoteParentNode, mergedNode });
+
+    // Make sure the remote child isn't locally deleted. If it is, we need
+    // to move all descendants that aren't also remotely deleted to the
+    // merged node. This handles the case where a user deletes a folder
+    // on this device, and adds a bookmark to the same folder on another
+    // device. We want to keep the folder deleted, but we also don't want
+    // to lose the new bookmark, so we move the bookmark to the deleted
+    // folder's parent.
+    let locallyDeleted = this.checkForLocalDeletionOfRemoteNode(mergedNode,
+      remoteChildNode);
+    if (locallyDeleted) {
+      return true;
+    }
+
+    // The remote child isn't locally deleted. Does it exist in the local tree?
+    let localChildNode = this.localTree.nodeForGuid(remoteChildNode.guid);
+    if (!localChildNode) {
+      // Remote child doesn't exist locally, either. Try to find a content
+      // match in the containing folder, and dedupe the local item if we can.
+      MirrorLog.trace("Remote child ${remoteChildNode} doesn't exist " +
+                      "locally; looking for content match",
+                      { remoteChildNode });
+
+      let localChildNodeByContent = this.findLocalNodeMatchingRemoteNode(
+        mergedNode, remoteChildNode);
+
+      let mergedChildNode = this.mergeNode(remoteChildNode.guid,
+                                           localChildNodeByContent,
+                                           remoteChildNode);
+      mergedNode.mergedChildren.push(mergedChildNode);
+      return false;
+    }
+
+    // Otherwise, the remote child exists in the local tree. Did it move?
+    let localParentNode = this.localTree.parentNodeFor(localChildNode);
+    if (!localParentNode) {
+      // Should never happen. The local tree must be complete.
+      MirrorLog.error("Remote child ${remoteChildNode} exists locally as " +
+                      "${localChildNode} without local parent",
+                      { remoteChildNode, localChildNode });
+      throw new SyncedBookmarksMirror.ConsistencyError(
+        "Local child node is orphan");
+    }
+
+    MirrorLog.trace("Remote child ${remoteChildNode} exists locally in " +
+                    "${localParentNode} and remotely in ${remoteParentNode}",
+                    { remoteChildNode, localParentNode, remoteParentNode });
+
+    if (localParentNode.needsMerge) {
+      if (remoteParentNode.needsMerge) {
+        MirrorLog.trace("Local ${localParentNode} and remote " +
+                        "${remoteParentNode} parents changed; comparing " +
+                        "modified times to decide parent for remote child " +
+                        "${remoteChildNode}",
+                        { localParentNode, remoteParentNode, remoteChildNode });
+
+        let latestLocalAge = Math.min(localChildNode.age,
+                                      localParentNode.age);
+        let latestRemoteAge = Math.min(remoteChildNode.age,
+                                       remoteParentNode.age);
+
+        if (latestLocalAge < latestRemoteAge) {
+          // Local move is younger, so we ignore the remote move. We'll
+          // merge the child later, when we walk its new local parent.
+          MirrorLog.trace("Ignoring older remote move for ${remoteChildNode} " +
+                          "to ${remoteParentNode} at ${latestRemoteAge}; " +
+                          "local move to ${localParentNode} at " +
+                          "${latestLocalAge} is newer",
+                          { remoteChildNode, remoteParentNode, latestRemoteAge,
+                            localParentNode, latestLocalAge });
+          return true;
+        }
+
+        // Otherwise, the remote move is younger, so we ignore the local move
+        // and merge the child now.
+        MirrorLog.trace("Taking newer remote move for ${remoteChildNode} to " +
+                        "${remoteParentNode} at ${latestRemoteAge}; local " +
+                        "move to ${localParentNode} at ${latestLocalAge} is " +
+                        "older", { remoteChildNode, remoteParentNode,
+                                   latestRemoteAge, localParentNode,
+                                   latestLocalAge });
+
+        let mergedChildNode = this.mergeNode(remoteChildNode.guid,
+                                             localChildNode, remoteChildNode);
+        mergedNode.mergedChildren.push(mergedChildNode);
+        return false;
+      }
+
+      MirrorLog.trace("Remote parent unchanged; keeping remote child " +
+                      "${remoteChildNode} in ${localParentNode}",
+                      { remoteChildNode, localParentNode });
+      return true;
+    }
+
+    MirrorLog.trace("Local parent unchanged; keeping remote child " +
+                    "${remoteChildNode} in ${remoteParentNode}",
+                    { remoteChildNode, remoteParentNode });
+
+    let mergedChildNode = this.mergeNode(remoteChildNode.guid, localChildNode,
+                                         remoteChildNode);
+    mergedNode.mergedChildren.push(mergedChildNode);
+    return false;
+  }
+
+  /**
+   * Merges a local child node into a merged folder node.
+   *
+   * @param  {MergedBookmarkNode} mergedNode
+   *         The merged folder node.
+   * @param  {BookmarkNode} localParentNode
+   *         The local folder node.
+   * @param  {BookmarkNode} localChildNode
+   *         The local child node.
+   * @return {Boolean}
+   *         `true` if the merged structure state changed because the local
+   *         child doesn't exist remotely or was locally moved; `false`
+   *         otherwise.
+   */
+  mergeLocalChildIntoMergedNode(mergedNode, localParentNode, localChildNode) {
+    if (this.mergedGuids.has(localChildNode.guid)) {
+      // We already merged the child when we walked another folder.
+      MirrorLog.trace("Local child ${localChildNode} already seen in " +
+                      "another folder and merged", { localChildNode });
+      return false;
+    }
+
+    MirrorLog.trace("Merging local child ${localChildNode} of " +
+                    "${localParentNode} into ${mergedNode}",
+                    { localChildNode, localParentNode, mergedNode });
+
+    // Now, we know we haven't seen the local child before, and it's not in
+    // this folder on the server. Check if the child is remotely deleted.
+    // If so, we need to move any new local descendants to the merged node,
+    // just as we did for new remote descendants of locally deleted parents.
+    let remotelyDeleted = this.checkForRemoteDeletionOfLocalNode(mergedNode,
+      localChildNode);
+    if (remotelyDeleted) {
+      return true;
+    }
+
+    // At this point, we know the local child isn't deleted. See if it
+    // exists in the remote tree.
+    let remoteChildNode = this.remoteTree.nodeForGuid(localChildNode.guid);
+    if (!remoteChildNode) {
+      // The local child doesn't exist remotely, but we still need to walk
+      // its children.
+      let mergedChildNode = this.mergeNode(localChildNode.guid, localChildNode,
+                                           /* remoteChildNode */ null);
+      mergedNode.mergedChildren.push(mergedChildNode);
+      return true;
+    }
+
+    // The local child exists remotely. It must have moved; otherwise, we
+    // would have seen it when we walked the remote children.
+    let remoteParentNode = this.remoteTree.parentNodeFor(remoteChildNode);
+    if (!remoteParentNode) {
+      // Should never happen. The remote tree must be complete.
+      MirrorLog.error("Local child ${localChildNode} exists remotely as " +
+                      "${remoteChildNode} without remote parent",
+                      { localChildNode, remoteChildNode });
+      throw new SyncedBookmarksMirror.ConsistencyError(
+        "Remote child node is orphan");
+    }
+
+    MirrorLog.trace("Local child ${localChildNode} exists locally in " +
+                    "${localParentNode} and remotely in ${remoteParentNode}",
+                    { localChildNode, localParentNode, remoteParentNode });
+
+    if (localParentNode.needsMerge) {
+      if (remoteParentNode.needsMerge) {
+        MirrorLog.trace("Local ${localParentNode} and remote " +
+                        "${remoteParentNode} parents changed; comparing " +
+                        "modified times to decide parent for local child " +
+                        "${localChildNode}", { localParentNode,
+                                               remoteParentNode,
+                                               localChildNode });
+
+        let latestLocalAge = Math.min(localChildNode.age,
+                                      localParentNode.age);
+        let latestRemoteAge = Math.min(remoteChildNode.age,
+                                       remoteParentNode.age);
+
+        if (latestRemoteAge <= latestLocalAge) {
+          MirrorLog.trace("Ignoring older local move for ${localChildNode} " +
+                          "to ${localParentNode} at ${latestLocalAge}; " +
+                          "remote move to ${remoteParentNode} at " +
+                          "${latestRemoteAge} is newer",
+                          { localChildNode, localParentNode, latestLocalAge,
+                            remoteParentNode, latestRemoteAge });
+          return false;
+        }
+
+        MirrorLog.trace("Taking newer local move for ${localChildNode} to " +
+                        "${localParentNode} at ${latestLocalAge}; remote " +
+                        "move to ${remoteParentNode} at ${latestRemoteAge} " +
+                        "is older", { localChildNode, localParentNode,
+                                      latestLocalAge, remoteParentNode,
+                                      latestRemoteAge });
+
+        let mergedChildNode = this.mergeNode(localChildNode.guid,
+                                             localChildNode, remoteChildNode);
+        mergedNode.mergedChildren.push(mergedChildNode);
+        return true;
+      }
+
+      MirrorLog.trace("Remote parent unchanged; keeping local child " +
+                      "${localChildNode} in local parent ${localParentNode}",
+                      { localChildNode, localParentNode });
+
+      let mergedChildNode = this.mergeNode(localChildNode.guid, localChildNode,
+                                           remoteChildNode);
+      mergedNode.mergedChildren.push(mergedChildNode);
+      return true;
+    }
+
+    MirrorLog.trace("Local parent unchanged; keeping local child " +
+                    "${localChildNode} in remote parent ${remoteParentNode}",
+                    { localChildNode, remoteParentNode });
+    return false;
+  }
+
+  /**
+   * Recursively merges the children of a local folder node and a matching
+   * remote folder node.
+   *
+   * @param {MergedBookmarkNode} mergedNode
+   *        The merged folder state. This method mutates the merged node to
+   *        append merged children, and change the node's merge state to new
+   *        if needed.
+   * @param {BookmarkNode?} localNode
+   *        The local folder node. May be `null` if the folder only exists
+   *        remotely.
+   * @param {BookmarkNode?} remoteNode
+   *        The remote folder node. May be `null` if the folder only exists
+   *        locally.
+   */
+  mergeChildListsIntoMergedNode(mergedNode, localNode, remoteNode) {
+    let mergeStateChanged = false;
+
+    // Walk and merge remote children first.
+    MirrorLog.trace("Merging remote children of ${remoteNode} into " +
+                    "${mergedNode}", { remoteNode, mergedNode });
+    if (remoteNode) {
+      for (let remoteChildNode of remoteNode.children) {
+        let remoteChildrenChanged = this.mergeRemoteChildIntoMergedNode(
+          mergedNode, remoteNode, remoteChildNode);
+        if (remoteChildrenChanged) {
+          mergeStateChanged = true;
+        }
+      }
+    }
+
+    // Now walk and merge any local children that we haven't already merged.
+    MirrorLog.trace("Merging local children of ${localNode} into " +
+                    "${mergedNode}", { localNode, mergedNode });
+    if (localNode) {
+      for (let localChildNode of localNode.children) {
+        let remoteChildrenChanged = this.mergeLocalChildIntoMergedNode(
+          mergedNode, localNode, localChildNode);
+        if (remoteChildrenChanged) {
+          mergeStateChanged = true;
+        }
+      }
+    }
+
+    // Update the merge state if we moved children orphaned on one side by a
+    // deletion on the other side, if we kept newer locally moved children,
+    // or if the child order changed. We already updated the merge state of the
+    // orphans, but we also need to flag the containing folder so that it's
+    // reuploaded to the server along with the new children.
+    if (mergeStateChanged) {
+      let newStructureNode = mergedNode.toBookmarkNode();
+      let newMergeState = BookmarkMergeState.new(mergedNode.mergeState,
+                                                 newStructureNode);
+      MirrorLog.trace("Merge state for ${mergedNode} has new structure " +
+                      "${newMergeState}", { mergedNode, newMergeState });
+      this.telemetryEvents.push({
+        value: "structure",
+        extra: { type: "new" },
+      });
+      mergedNode.mergeState = newMergeState;
+    }
+  }
+
+  /**
+   * Walks a locally deleted remote node's children, reparenting any children
+   * that aren't also deleted remotely to the merged node. Returns `true` if
+   * `remoteNode` is deleted locally; `false` if `remoteNode` is not deleted or
+   * doesn't exist locally.
+   *
+   * This is the inverse of `checkForRemoteDeletionOfLocalNode`.
+   */
+  checkForLocalDeletionOfRemoteNode(mergedNode, remoteNode) {
+    if (!this.localTree.isDeleted(remoteNode.guid)) {
+      return false;
+    }
+
+    if (remoteNode.needsMerge) {
+      if (!remoteNode.isFolder()) {
+        // If a non-folder child is deleted locally and changed remotely, we
+        // ignore the local deletion and take the remote child.
+        MirrorLog.trace("Remote non-folder ${remoteNode} deleted locally " +
+                        "and changed remotely; taking remote change",
+                        { remoteNode });
+        this.telemetryEvents.push({
+          value: "structure",
+          extra: { type: "delete", kind: "item", prefer: "remote" },
+        });
+        return false;
+      }
+      // For folders, we always take the local deletion and relocate remotely
+      // changed grandchildren to the merged node. We could use the mirror to
+      // revive the child folder, but it's easier to relocate orphaned
+      // grandchildren than to partially revive the child folder.
+      MirrorLog.trace("Remote folder ${remoteNode} deleted locally " +
+                      "and changed remotely; taking local deletion",
+                      { remoteNode });
+      this.telemetryEvents.push({
+        value: "structure",
+        extra: { type: "delete", kind: "folder", prefer: "local" },
+      });
+    } else {
+      MirrorLog.trace("Remote node ${remoteNode} deleted locally and not " +
+                       "changed remotely; taking local deletion",
+                       { remoteNode });
+    }
+
+    this.deleteRemotely.add(remoteNode.guid);
+
+    let mergedOrphanNodes = this.processRemoteOrphansForNode(mergedNode,
+                                                             remoteNode);
+    this.relocateOrphansTo(mergedNode, mergedOrphanNodes);
+    MirrorLog.trace("Relocating remote orphans ${mergedOrphanNodes} to " +
+                    "${mergedNode}", { mergedOrphanNodes, mergedNode });
+
+    return true;
+  }
+
+  /**
+   * Walks a remotely deleted local node's children, reparenting any children
+   * that aren't also deleted locally to the merged node. Returns `true` if
+   * `localNode` is deleted remotely; `false` if `localNode` is not deleted or
+   * doesn't exist locally.
+   *
+   * This is the inverse of `checkForLocalDeletionOfRemoteNode`.
+   */
+  checkForRemoteDeletionOfLocalNode(mergedNode, localNode) {
+    if (!this.remoteTree.isDeleted(localNode.guid)) {
+      return false;
+    }
+
+    if (localNode.needsMerge) {
+      if (!localNode.isFolder()) {
+        MirrorLog.trace("Local non-folder ${localNode} deleted remotely and " +
+                        "changed locally; taking local change", { localNode });
+        this.telemetryEvents.push({
+          value: "structure",
+          extra: { type: "delete", kind: "item", prefer: "local" },
+        });
+        return false;
+      }
+      MirrorLog.trace("Local folder ${localNode} deleted remotely and " +
+                      "changed locally; taking remote deletion", { localNode });
+      this.telemetryEvents.push({
+        value: "structure",
+        extra: { type: "delete", kind: "folder", prefer: "remote" },
+      });
+    } else {
+      MirrorLog.trace("Local node ${localNode} deleted remotely and not " +
+                      "changed locally; taking remote deletion", { localNode });
+    }
+
+    MirrorLog.trace("Local node ${localNode} deleted remotely; taking remote " +
+                    "deletion", { localNode });
+
+    this.deleteLocally.add(localNode.guid);
+
+    let mergedOrphanNodes = this.processLocalOrphansForNode(mergedNode,
+                                                            localNode);
+    this.relocateOrphansTo(mergedNode, mergedOrphanNodes);
+    MirrorLog.trace("Relocating local orphans ${mergedOrphanNodes} to " +
+                    "${mergedNode}", { mergedOrphanNodes, mergedNode });
+
+    return true;
+  }
+
+  /**
+   * Recursively merges all remote children of a locally deleted folder that
+   * haven't also been deleted remotely. This can happen if the user adds a
+   * bookmark to a folder on another device, and deletes that folder locally.
+   * This is the inverse of `processLocalOrphansForNode`.
+   */
+  processRemoteOrphansForNode(mergedNode, remoteNode) {
+    let remoteOrphanNodes = [];
+
+    for (let remoteChildNode of remoteNode.children) {
+      let locallyDeleted = this.checkForLocalDeletionOfRemoteNode(mergedNode,
+        remoteChildNode);
+      if (locallyDeleted) {
+        // The remote child doesn't exist locally, or is also deleted locally,
+        // so we can safely delete its parent.
+        continue;
+      }
+      remoteOrphanNodes.push(remoteChildNode);
+    }
+
+    let mergedOrphanNodes = [];
+    for (let remoteOrphanNode of remoteOrphanNodes) {
+      let localOrphanNode = this.localTree.nodeForGuid(remoteOrphanNode.guid);
+      let mergedOrphanNode = this.mergeNode(remoteOrphanNode.guid,
+                                            localOrphanNode, remoteOrphanNode);
+      mergedOrphanNodes.push(mergedOrphanNode);
+    }
+
+    return mergedOrphanNodes;
+  }
+
+  /**
+   * Recursively merges all local children of a remotely deleted folder that
+   * haven't also been deleted locally. This is the inverse of
+   * `processRemoteOrphansForNode`.
+   */
+  processLocalOrphansForNode(mergedNode, localNode) {
+    if (!localNode.isFolder()) {
+      // The local node isn't a folder, so it won't have orphans.
+      return [];
+    }
+
+    let localOrphanNodes = [];
+    for (let localChildNode of localNode.children) {
+      let remotelyDeleted = this.checkForRemoteDeletionOfLocalNode(mergedNode,
+        localChildNode);
+      if (remotelyDeleted) {
+        // The local child doesn't exist or is also deleted on the server, so we
+        // can safely delete its parent without orphaning any local children.
+        continue;
+      }
+      localOrphanNodes.push(localChildNode);
+    }
+
+    let mergedOrphanNodes = [];
+    for (let localOrphanNode of localOrphanNodes) {
+      let remoteOrphanNode = this.remoteTree.nodeForGuid(localOrphanNode.guid);
+      let mergedNode = this.mergeNode(localOrphanNode.guid,
+                                      localOrphanNode, remoteOrphanNode);
+      mergedOrphanNodes.push(mergedNode);
+    }
+
+    return mergedOrphanNodes;
+  }
+
+  /**
+   * Moves a list of merged orphan nodes to the closest surviving ancestor.
+   * Changes the merge state of the moved orphans to new, so that we reupload
+   * them along with their new parent on the next sync.
+   *
+   * @param {MergedBookmarkNode} mergedNode
+   * @param {MergedBookmarkNode[]} mergedOrphanNodes
+   */
+  relocateOrphansTo(mergedNode, mergedOrphanNodes) {
+    for (let mergedOrphanNode of mergedOrphanNodes) {
+      let newStructureNode = mergedOrphanNode.toBookmarkNode();
+      let newMergeState = BookmarkMergeState.new(mergedOrphanNode.mergeState,
+                                                 newStructureNode);
+      mergedOrphanNode.mergeState = newMergeState;
+      mergedNode.mergedChildren.push(mergedOrphanNode);
+    }
+  }
+
+  /**
+   * Finds a local node with a different GUID that matches the content of a
+   * remote node. This is used to dedupe local items that haven't been uploaded
+   * to remote items that don't exist locally.
+   *
+   * @param  {MergedBookmarkNode} mergedNode
+   *         The merged folder node.
+   * @param  {BookmarkNode} remoteChildNode
+   *         The remote child node.
+   * @return {BookmarkNode?}
+   *         A matching local child node, or `null` if there are no matching
+   *         local items.
+   */
+  findLocalNodeMatchingRemoteNode(mergedNode, remoteChildNode) {
+    let localParentNode = mergedNode.localNode;
+    if (!localParentNode) {
+      MirrorLog.trace("Merged node ${mergedNode} doesn't exist locally; no " +
+                      "potential dupes for ${remoteChildNode}",
+                      { mergedNode, remoteChildNode });
+      return null;
+    }
+    let remoteChildContent = this.newRemoteContents.get(remoteChildNode.guid);
+    if (!remoteChildContent) {
+      // The node doesn't exist locally, but it's also flagged as unchanged
+      // in the mirror.
+      return null;
+    }
+    let newLocalNode = null;
+    for (let localChildNode of localParentNode.children) {
+      if (this.mergedGuids.has(localChildNode.guid)) {
+        MirrorLog.trace("Not deduping ${localChildNode}; already seen in " +
+                        "another folder", { localChildNode });
+        continue;
+      }
+      if (!this.newLocalContents.has(localChildNode.guid)) {
+        MirrorLog.trace("Not deduping ${localChildNode}; already uploaded",
+                        { localChildNode });
+        continue;
+      }
+      let remoteCandidate = this.remoteTree.nodeForGuid(localChildNode.guid);
+      if (remoteCandidate) {
+        MirrorLog.trace("Not deduping ${localChildNode}; already exists " +
+                        "remotely", { localChildNode });
+        continue;
+      }
+      if (this.remoteTree.isDeleted(localChildNode.guid)) {
+        MirrorLog.trace("Not deduping ${localChildNode}; deleted on server",
+                        { localChildNode });
+        continue;
+      }
+      let localChildContent = this.newLocalContents.get(localChildNode.guid);
+      if (!contentsMatch(localChildNode, localChildContent, remoteChildNode,
+                         remoteChildContent)) {
+        MirrorLog.trace("${localChildNode} is not a dupe of ${remoteChildNode}",
+                        { localChildNode, remoteChildNode });
+        continue;
+      }
+      this.telemetryEvents.push({ value: "dupe" });
+      newLocalNode = localChildNode;
+      break;
+    }
+    return newLocalNode;
+  }
+}
+
+/**
+ * Determines if two new local and remote nodes are of the same kind, and have
+ * similar contents.
+ *
+ * - Bookmarks must have the same title and URL.
+ * - Smart bookmarks must have the same smart bookmark name. Other queries
+ *   must have the same title and query URL.
+ * - Folders and livemarks must have the same title.
+ * - Separators must have the same position within their parents.
+ *
+ * @param  {BookmarkNode} localNode
+ * @param  {BookmarkContent} localContent
+ * @param  {BookmarkNode} remoteNode
+ * @param  {BookmarkContent} remoteContent
+ * @return {Boolean}
+ */
+function contentsMatch(localNode, localContent, remoteNode, remoteContent) {
+  if (localNode.kind != remoteNode.kind) {
+    return false;
+  }
+  switch (localNode.kind) {
+    case SyncedBookmarksMirror.KIND.BOOKMARK:
+      return localContent.title == remoteContent.title &&
+             localContent.hasSameURL(remoteContent);
+
+    case SyncedBookmarksMirror.KIND.QUERY:
+      if (localContent.smartBookmarkName || remoteContent.smartBookmarkName) {
+        return localContent.smartBookmarkName ==
+               remoteContent.smartBookmarkName;
+      }
+      return localContent.title == remoteContent.title &&
+             localContent.hasSameURL(remoteContent);
+
+    case SyncedBookmarksMirror.KIND.FOLDER:
+    case SyncedBookmarksMirror.KIND.LIVEMARK:
+      return localContent.title == remoteContent.title;
+
+    case SyncedBookmarksMirror.KIND.SEPARATOR:
+      return localContent.position == remoteContent.position;
+  }
+  return false;
+}
+
+/**
+ * Records bookmark, annotation, and keyword observer notifications for all
+ * changes made during the merge, then fires the notifications after the merge
+ * is done.
+ *
+ * Recording bookmark changes and deletions is somewhat expensive, because we
+ * need to fetch all observer infos before writing. Making this more efficient
+ * is tracked in bug 1340498.
+ *
+ * Annotation observers don't require the extra context, so they're cheap to
+ * record and fire.
+ */
+class BookmarkObserverRecorder {
+  constructor(db, localTree, remoteTree) {
+    this.db = db;
+    this.localTree = localTree;
+    this.remoteTree = remoteTree;
+    this.addedGuids = [];
+    this.changedItemInfos = new Map();
+    this.removedItemInfos = [];
+    this.changedKeywordInfos = [];
+    this.changedAnnoInfos = [];
+    this.shouldInvalidateLivemarks = false;
+  }
+
+  /**
+   * Fires all recorded observer notifications.
+   */
+  async notifyAll() {
+    await this.notifyBookmarkObservers();
+    this.notifyAnnoObservers();
+    if (this.shouldInvalidateLivemarks) {
+      await PlacesUtils.livemarks.invalidateCachedLivemarks();
+    }
+    await this.updateFrecencies();
+  }
+
+  async updateFrecencies() {
+    await this.db.execute(`
+      UPDATE moz_places SET
+        frecency = CALCULATE_FRECENCY(id)
+      WHERE frecency = -1`);
+  }
+
+  noteItemAdded(guid) {
+    this.addedGuids.push(guid);
+  }
+
+  noteItemChanged(info) {
+    this.changedItemInfos.set(info.id, info);
+  }
+
+  noteItemsRemoved(infos) {
+    this.removedItemInfos.push(...infos);
+  }
+
+  noteKeywordChanged(info) {
+    this.changedKeywordInfos.push(info);
+  }
+
+  noteAnnoSet(id, name) {
+    if (isLivemarkAnno(name)) {
+      this.shouldInvalidateLivemarks = true;
+    }
+    this.changedAnnoInfos.push({ type: "set", id, name });
+
+  }
+
+  noteAnnoRemoved(id, name) {
+    if (isLivemarkAnno(name)) {
+      this.shouldInvalidateLivemarks = true;
+    }
+    this.changedAnnoInfos.push({ type: "removed", id, name });
+  }
+
+  async notifyBookmarkObservers() {
+    MirrorLog.debug("Fetching infos for new bookmarks");
+    let addedItemInfos = [];
+    for (let chunk of PlacesSyncUtils.chunkArray(this.addedGuids,
+      SQLITE_MAX_VARIABLE_NUMBER)) {
+
+      let rows = await this.db.execute(`
+        SELECT b.id, p.id AS parentId, b.position, b.type, h.url,
+               IFNULL(b.title, "") AS title, b.dateAdded, b.guid,
+               p.guid AS parentGuid
+        FROM moz_bookmarks b
+        JOIN moz_bookmarks p ON p.id = b.parent
+        LEFT JOIN moz_places h ON h.id = b.fk
+        WHERE b.guid IN (${new Array(chunk.length).fill("?").join(",")})`,
+        chunk);
+      for (let row of rows) {
+        let urlHref = row.getResultByName("url");
+        addedItemInfos.push({
+          id: row.getResultByName("id"),
+          parentId: row.getResultByName("parentId"),
+          position: row.getResultByName("position"),
+          type: row.getResultByName("type"),
+          uri: urlHref ? Services.io.newURI(urlHref) : null,
+          title: row.getResultByName("title"),
+          dateAdded: row.getResultByName("dateAdded"),
+          guid: row.getResultByName("guid"),
+          parentGuid: row.getResultByName("parentGuid"),
+        });
+      }
+    }
+
+    MirrorLog.debug("Fetching info for updated bookmarks");
+    let changeNotifications = [];
+    let changedIds = Array.from(this.changedItemInfos.keys());
+    for (let chunk of PlacesSyncUtils.chunkArray(changedIds,
+      SQLITE_MAX_VARIABLE_NUMBER)) {
+
+      let rows = await this.db.execute(`
+        SELECT b.id, b.lastModified, b.type, b.guid, b.position,
+               p.id AS parentId, p.guid AS parentGuid, b.title, h.url
+        FROM moz_bookmarks b
+        JOIN moz_bookmarks p ON p.id = b.parent
+        LEFT JOIN moz_places h ON h.id = b.fk
+        WHERE b.id IN (${new Array(chunk.length).fill("?").join(",")})`,
+        chunk);
+      for (let row of rows) {
+        let id = row.getResultByName("id");
+        let info = this.changedItemInfos.get(id);
+        if (!info) {
+          // The query should never return info for items we didn't request.
+          throw new TypeError(
+            "Can't note bookmark change for unexpected item ID");
+        }
+        let lastModified = row.getResultByName("lastModified");
+        let type = row.getResultByName("type");
+        let newGuid = row.getResultByName("guid");
+        if (info.oldGuid != newGuid) {
+          changeNotifications.push({
+            notification: "onItemChanged",
+            args: [id, "guid", /* isAnnotationProperty */ false,
+                   newGuid, lastModified, type, info.oldParentId,
+                   newGuid, info.oldParentGuid, info.oldGuid,
+                   PlacesUtils.bookmarks.SOURCES.SYNC],
+          });
+          PlacesUtils.invalidateCachedGuidFor(id);
+        }
+        let newPosition = row.getResultByName("position");
+        let newParentId = row.getResultByName("parentId");
+        let newParentGuid = row.getResultByName("parentGuid");
+        if (info.oldPosition != newPosition ||
+            info.oldParentId != newParentId ||
+            info.oldParentGuid != newParentGuid) {
+          changeNotifications.push({
+            notification: "onItemMoved",
+            args: [id, info.oldParentId, info.oldPosition, newParentId,
+                   newPosition, type, newGuid, info.oldParentGuid,
+                   newParentGuid, PlacesUtils.bookmarks.SOURCES.SYNC],
+          });
+        }
+        let newTitle = row.getResultByName("title");
+        if (info.oldTitle != newTitle) {
+          changeNotifications.push({
+            notification: "onItemChanged",
+            args: [id, "title", /* isAnnotationProperty */ false,
+                   newTitle, lastModified, type, newParentId,
+                   newGuid, newParentGuid, info.oldTitle,
+                   PlacesUtils.bookmarks.SOURCES.SYNC],
+          });
+        }
+        let newURLHref = row.getResultByName("url");
+        if (info.oldURLHref != newURLHref) {
+          changeNotifications.push({
+            notification: "onItemChanged",
+            args: [id, "uri", /* isAnnotationProperty */ false,
+                   newURLHref, lastModified, type, newParentId,
+                   newGuid, newParentGuid, info.oldURLHref,
+                   PlacesUtils.bookmarks.SOURCES.SYNC],
+          });
+        }
+      }
+    }
+
+    // Sort added items in level order (parents before children), and removed
+    // items in reverse level order (children before parents). This is important
+    // for cache coherence (bug 1297941).
+    addedItemInfos.sort((a, b) => {
+      let aLevel = this.remoteTree.levelForGuid(a.guid);
+      let bLevel = this.remoteTree.levelForGuid(b.guid);
+      return aLevel - bLevel;
+    });
+    this.removedItemInfos.sort((a, b) => {
+      let aLevel = this.localTree.levelForGuid(a.guid);
+      let bLevel = this.localTree.levelForGuid(b.guid);
+      return bLevel - aLevel;
+    });
+
+    MirrorLog.debug("Notifying all observers");
+    let observers = PlacesUtils.bookmarks.getObservers();
+    for (let observer of observers) {
+      this.notifyObserver(observer, "onBeginUpdateBatch");
+      for (let info of addedItemInfos) {
+        this.notifyObserver(observer, "onItemAdded", [info.id, info.parentId,
+          info.position, info.type, info.uri, info.title, info.dateAdded,
+          info.guid, info.parentGuid, PlacesUtils.bookmarks.SOURCES.SYNC]);
+      }
+      for (let { notification, args } of changeNotifications) {
+        this.notifyObserver(observer, notification, args);
+      }
+      for (let info of this.changedKeywordInfos) {
+        this.notifyObserver(observer, "onItemChanged", [info.id, "keyword",
+          /* isAnnotationProperty */ false, info.keyword, info.lastModified,
+          info.type, info.parentId, info.guid, info.parentGuid,
+          /* oldValue */ info.urlHref, PlacesUtils.bookmarks.SOURCES.SYNC]);
+      }
+      for (let info of this.removedItemInfos) {
+        this.notifyObserver(observer, "onItemRemoved", [info.id, info.parentId,
+          info.position, info.type, info.uri, info.guid, info.parentGuid,
+          PlacesUtils.bookmarks.SOURCES.SYNC]);
+      }
+      this.notifyObserver(observer, "onEndUpdateBatch");
+    }
+  }
+
+  notifyAnnoObservers() {
+    let observers = PlacesUtils.annotations.getObservers();
+    for (let observer of observers) {
+      for (let info of this.changedAnnoInfos) {
+        MirrorLog.trace("Notifying ${type} annotation observer for ${name} " +
+                        "on ${id}", info);
+        switch (info.type) {
+          case "set":
+            this.notifyObserver(observer, "onItemAnnotationSet", [
+              info.id, info.name, PlacesUtils.bookmarks.SOURCES.SYNC]);
+            break;
+
+          case "removed":
+            this.notifyObserver(observer, "onItemAnnotationRemoved", [
+              info.id, info.name, PlacesUtils.bookmarks.SOURCES.SYNC]);
+            break;
+
+          default:
+            throw new TypeError(
+              "Can't notify annotation observers with unknown change type");
+        }
+      }
+    }
+  }
+
+  notifyObserver(observer, notification, args = []) {
+    try {
+      observer[notification](...args);
+    } catch (ex) {
+      MirrorLog.warn("Error notifying bookmark observer", ex);
+    }
+  }
+}
+
+function isLivemarkAnno(name) {
+  return name == PlacesUtils.LMANNO_FEEDURI ||
+         name == PlacesUtils.LMANNO_SITEURI;
+}
+
+class BookmarkChangeRecord {
+  constructor(syncChangeCounter, cleartext) {
+    this.tombstone = cleartext.deleted === true;
+    this.counter = syncChangeCounter;
+    this.cleartext = cleartext;
+    this.synced = false;
+  }
+}
+
+// In conclusion, this is why bookmark syncing is hard.
--- a/toolkit/components/places/moz.build
+++ b/toolkit/components/places/moz.build
@@ -69,16 +69,17 @@ if CONFIG['MOZ_PLACES']:
         'History.jsm',
         'PlacesBackups.jsm',
         'PlacesDBUtils.jsm',
         'PlacesRemoteTabsAutocompleteProvider.jsm',
         'PlacesSearchAutocompleteProvider.jsm',
         'PlacesSyncUtils.jsm',
         'PlacesTransactions.jsm',
         'PlacesUtils.jsm',
+        'SyncedBookmarksMirror.jsm',
     ]
 
     EXTRA_COMPONENTS += [
         'ColorAnalyzer.js',
         'nsLivemarkService.js',
         'nsPlacesExpiration.js',
         'nsTaggingService.js',
         'PageIconProtocolHandler.js',
--- a/toolkit/components/places/tests/moz.build
+++ b/toolkit/components/places/tests/moz.build
@@ -13,16 +13,17 @@ TESTING_JS_MODULES += [
 XPCSHELL_TESTS_MANIFESTS += [
     'bookmarks/xpcshell.ini',
     'expiration/xpcshell.ini',
     'favicons/xpcshell.ini',
     'history/xpcshell.ini',
     'legacy/xpcshell.ini',
     'migration/xpcshell.ini',
     'queries/xpcshell.ini',
+    'sync/xpcshell.ini',
     'unifiedcomplete/xpcshell.ini',
     'unit/xpcshell.ini',
 ]
 
 BROWSER_CHROME_MANIFESTS += ['browser/browser.ini']
 MOCHITEST_CHROME_MANIFESTS += [
     'chrome/chrome.ini',
 ]
new file mode 100644
--- /dev/null
+++ b/toolkit/components/places/tests/sync/head_sync.js
@@ -0,0 +1,154 @@
+const { utils: Cu, interfaces: Ci, classes: Cc, results: Cr } = Components;
+
+Cu.import("resource://gre/modules/Log.jsm");
+Cu.import("resource://gre/modules/ObjectUtils.jsm");
+Cu.import("resource://gre/modules/osfile.jsm");
+Cu.import("resource://gre/modules/PlacesUtils.jsm");
+Cu.import("resource://gre/modules/PlacesSyncUtils.jsm");
+Cu.import("resource://gre/modules/Services.jsm");
+Cu.import("resource://gre/modules/SyncedBookmarksMirror.jsm");
+
+Cu.import("resource://testing-common/PlacesTestUtils.jsm");
+Cu.import("resource://testing-common/httpd.js");
+
+function run_test() {
+  let bufLog = Log.repository.getLogger("Sync.Engine.Bookmarks.Mirror");
+  bufLog.level = Log.Level.Error;
+
+  let sqliteLog = Log.repository.getLogger("Sqlite");
+  sqliteLog.level = Log.Level.Error;
+
+  let formatter = new Log.BasicFormatter();
+  let appender = new Log.DumpAppender(formatter);
+  appender.level = Log.Level.All;
+
+  for (let log of [bufLog, sqliteLog]) {
+    log.addAppender(appender);
+  }
+
+  do_get_profile();
+  run_next_test();
+}
+
+function inspectChangeRecords(changeRecords) {
+  let results = { updated: [], deleted: [] };
+  for (let [id, record] of Object.entries(changeRecords)) {
+    (record.tombstone ? results.deleted : results.updated).push(id);
+  }
+  results.updated.sort();
+  results.deleted.sort();
+  return results;
+}
+
+async function assertLocalTree(rootGuid, expected, message) {
+  function bookmarkNodeToInfo(node) {
+    let { guid, index, title, typeCode: type } = node;
+    let info = { guid, index, title, type };
+    if (node.annos) {
+      let syncableAnnos = node.annos.filter(anno => [
+        PlacesSyncUtils.bookmarks.DESCRIPTION_ANNO,
+        PlacesSyncUtils.bookmarks.SIDEBAR_ANNO,
+        PlacesSyncUtils.bookmarks.SMART_BOOKMARKS_ANNO,
+        PlacesUtils.LMANNO_FEEDURI,
+        PlacesUtils.LMANNO_SITEURI,
+      ].includes(anno.name));
+      if (syncableAnnos.length) {
+        info.annos = syncableAnnos;
+      }
+    }
+    if (node.uri) {
+      info.url = node.uri;
+    }
+    if (node.keyword) {
+      info.keyword = node.keyword;
+    }
+    if (node.children) {
+      info.children = node.children.map(bookmarkNodeToInfo);
+    }
+    return info;
+  }
+  let root = await PlacesUtils.promiseBookmarksTree(rootGuid);
+  let actual = bookmarkNodeToInfo(root);
+  if (!ObjectUtils.deepEqual(actual, expected)) {
+    do_print(`Expected structure for ${rootGuid}`, JSON.stringify(expected));
+    do_print(`Actual structure for ${rootGuid}`, JSON.stringify(actual));
+    throw new Assert.constructor.AssertionError({ actual, expected, message });
+  }
+}
+
+function makeLivemarkServer() {
+  let server = new HttpServer();
+  server.registerPrefixHandler("/feed/", do_get_file("./livemark.xml"));
+  server.start(-1);
+  return {
+    server,
+    get site() {
+      let { identity } = server;
+      let host = identity.primaryHost.includes(":") ?
+        `[${identity.primaryHost}]` : identity.primaryHost;
+      return `${identity.primaryScheme}://${host}:${identity.primaryPort}`;
+    },
+    stopServer() {
+      return new Promise(resolve => server.stop(resolve));
+    },
+  };
+}
+
+function shuffle(array) {
+  let results = [];
+  for (let i = 0; i < array.length; ++i) {
+    let randomIndex = Math.floor(Math.random() * (i + 1));
+    results[i] = results[randomIndex];
+    results[randomIndex] = array[i];
+  }
+  return results;
+}
+
+async function fetchAllKeywords(info) {
+  let entries = [];
+  await PlacesUtils.keywords.fetch(info, entry => entries.push(entry));
+  return entries;
+}
+
+async function openMirror(name) {
+  let path = OS.Path.join(OS.Constants.Path.profileDir, `${name}_buf.sqlite`);
+  let buf = await SyncedBookmarksMirror.open({
+    path,
+    recordTelemetryEvent() {},
+  });
+  return buf;
+}
+
+// A debugging helper that dumps the contents of an array of rows.
+function dumpRows(rows) {
+  let results = [];
+  for (let row of rows) {
+    let numColumns = row.numEntries;
+    let rowValues = [];
+    for (let i = 0; i < numColumns; ++i) {
+      switch (row.getTypeOfIndex(i)) {
+        case Ci.mozIStorageValueArray.VALUE_TYPE_NULL:
+          rowValues.push("NULL");
+          break;
+        case Ci.mozIStorageValueArray.VALUE_TYPE_INTEGER:
+          rowValues.push(row.getInt64(i));
+          break;
+        case Ci.mozIStorageValueArray.VALUE_TYPE_FLOAT:
+          rowValues.push(row.getDouble(i));
+          break;
+        case Ci.mozIStorageValueArray.VALUE_TYPE_TEXT:
+          rowValues.push(JSON.stringify(row.getString(i)));
+          break;
+      }
+    }
+    results.push(rowValues.join("\t"));
+  }
+  results.push("\n");
+  dump(results.join("\n"));
+}
+
+async function dumpMirrorTable(buf, table) {
+  let rows = await buf.db.execute(`SELECT * FROM ${table}`);
+  do_print(`${table} contains ${rows.length} rows`);
+  dumpRows(rows);
+}
new file mode 100644
--- /dev/null
+++ b/toolkit/components/places/tests/sync/livemark.xml
@@ -0,0 +1,17 @@
+<?xml version="1.0" encoding="utf-8"?>
+<feed xmlns="http://www.w3.org/2005/Atom">
+  <title>Livemark Feed</title>
+  <link href="https://example.com/"/>
+  <updated>2016-08-09T19:51:45.147Z</updated>
+  <author>
+    <name>John Doe</name>
+  </author>
+  <id>urn:uuid:e7947414-6ee0-4009-ae75-8b0ad3c6894b</id>
+  <entry>
+    <title>Some awesome article</title>
+    <link href="https://example.com/some-article"/>
+    <id>urn:uuid:d72ce019-0a56-4a0b-ac03-f66117d78141</id>
+    <updated>2016-08-09T19:57:22.178Z</updated>
+    <summary>My great article summary.</summary>
+  </entry>
+</feed>
new file mode 100644
--- /dev/null
+++ b/toolkit/components/places/tests/sync/test_bookmark_corruption.js
@@ -0,0 +1,982 @@
+add_task(async function test_missing_children() {
+  let buf = await openMirror("missing_childen");
+
+  do_print("Set up empty mirror");
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  do_print("Set up mirror: A > ([B] C [D E])");
+  {
+    await buf.store(shuffle([{
+      id: "menu",
+      type: "folder",
+      title: "Bookmarks Menu",
+      children: ["bookmarkBBBB", "bookmarkCCCC", "bookmarkDDDD",
+                 "bookmarkEEEE"],
+    }, {
+      id: "bookmarkCCCC",
+      type: "bookmark",
+      bmkUri: "http://example.com/c",
+      title: "C",
+    }]));
+    let changesToUpload = await buf.apply();
+    let idsToUpload = inspectChangeRecords(changesToUpload);
+    deepEqual(idsToUpload, {
+      updated: [],
+      deleted: [],
+    }, "Should not reupload menu with missing children (B D E)");
+    await assertLocalTree(PlacesUtils.bookmarks.menuGuid, {
+      guid: PlacesUtils.bookmarks.menuGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 0,
+      title: "Bookmarks Menu",
+      children: [{
+        guid: "bookmarkCCCC",
+        type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+        index: 0,
+        title: "C",
+        url: "http://example.com/c",
+      }],
+    }, "Menu children should be (C)");
+    let { missingChildren } = await buf.fetchRemoteOrphans();
+    deepEqual(missingChildren.sort(), ["bookmarkBBBB", "bookmarkDDDD",
+      "bookmarkEEEE"], "Should report (B D E) as missing");
+  }
+
+  do_print("Add (B E) to mirror");
+  {
+    await buf.store(shuffle([{
+      id: "bookmarkBBBB",
+      type: "bookmark",
+      title: "B",
+      bmkUri: "http://example.com/b",
+    }, {
+      id: "bookmarkEEEE",
+      type: "bookmark",
+      title: "E",
+      bmkUri: "http://example.com/e",
+    }]));
+    let changesToUpload = await buf.apply();
+    let idsToUpload = inspectChangeRecords(changesToUpload);
+    deepEqual(idsToUpload, {
+      updated: [],
+      deleted: [],
+    }, "Should not reupload menu with missing child D");
+    await assertLocalTree(PlacesUtils.bookmarks.menuGuid, {
+      guid: PlacesUtils.bookmarks.menuGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 0,
+      title: "Bookmarks Menu",
+      children: [{
+        guid: "bookmarkBBBB",
+        type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+        index: 0,
+        title: "B",
+        url: "http://example.com/b",
+      }, {
+        guid: "bookmarkCCCC",
+        type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+        index: 1,
+        title: "C",
+        url: "http://example.com/c",
+      }, {
+        guid: "bookmarkEEEE",
+        type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+        index: 2,
+        title: "E",
+        url: "http://example.com/e",
+      }],
+    }, "Menu children should be (B C E)");
+    let { missingChildren } = await buf.fetchRemoteOrphans();
+    deepEqual(missingChildren, ["bookmarkDDDD"],
+      "Should report (D) as missing");
+  }
+
+  do_print("Add D to mirror");
+  {
+    await buf.store([{
+      id: "bookmarkDDDD",
+      type: "bookmark",
+      title: "D",
+      bmkUri: "http://example.com/d",
+    }]);
+    let changesToUpload = await buf.apply();
+    let idsToUpload = inspectChangeRecords(changesToUpload);
+    deepEqual(idsToUpload, {
+      updated: [],
+      deleted: [],
+    }, "Should not reupload complete menu");
+    await assertLocalTree(PlacesUtils.bookmarks.menuGuid, {
+      guid: PlacesUtils.bookmarks.menuGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 0,
+      title: "Bookmarks Menu",
+      children: [{
+        guid: "bookmarkBBBB",
+        type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+        index: 0,
+        title: "B",
+        url: "http://example.com/b",
+      }, {
+        guid: "bookmarkCCCC",
+        type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+        index: 1,
+        title: "C",
+        url: "http://example.com/c",
+      }, {
+        guid: "bookmarkDDDD",
+        type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+        index: 2,
+        title: "D",
+        url: "http://example.com/d",
+      }, {
+        guid: "bookmarkEEEE",
+        type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+        index: 3,
+        title: "E",
+        url: "http://example.com/e",
+      }],
+    }, "Menu children should be (B C D E)");
+    let { missingChildren } = await buf.fetchRemoteOrphans();
+    deepEqual(missingChildren, [], "Should not report any missing children");
+  }
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
+
+add_task(async function test_new_orphan_without_local_parent() {
+  let buf = await openMirror("new_orphan_without_local_parent");
+
+  do_print("Set up empty mirror");
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  // A doesn't exist locally, so we move the bookmarks into "unfiled" without
+  // reuploading. When the partial uploader returns and uploads A, we'll
+  // move the bookmarks to the correct folder.
+  do_print("Set up mirror with [A] > (B C D)");
+  await buf.store(shuffle([{
+    id: "bookmarkBBBB",
+    type: "bookmark",
+    title: "B (remote)",
+    bmkUri: "http://example.com/b-remote",
+  }, {
+    id: "bookmarkCCCC",
+    type: "bookmark",
+    title: "C (remote)",
+    bmkUri: "http://example.com/c-remote",
+  }, {
+    id: "bookmarkDDDD",
+    type: "bookmark",
+    title: "D (remote)",
+    bmkUri: "http://example.com/d-remote",
+  }]));
+
+  do_print("Apply mirror with (B C D)");
+  {
+    let changesToUpload = await buf.apply();
+    let idsToUpload = inspectChangeRecords(changesToUpload);
+    deepEqual(idsToUpload, {
+      updated: [],
+      deleted: [],
+    }, "Should not reupload orphans (B C D)");
+  }
+
+  await assertLocalTree(PlacesUtils.bookmarks.unfiledGuid, {
+    guid: PlacesUtils.bookmarks.unfiledGuid,
+    type: PlacesUtils.bookmarks.TYPE_FOLDER,
+    index: 3,
+    title: "Other Bookmarks",
+    children: [{
+      guid: "bookmarkBBBB",
+      type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+      index: 0,
+      title: "B (remote)",
+      url: "http://example.com/b-remote",
+    }, {
+      guid: "bookmarkCCCC",
+      type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+      index: 1,
+      title: "C (remote)",
+      url: "http://example.com/c-remote",
+    }, {
+      guid: "bookmarkDDDD",
+      type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+      index: 2,
+      title: "D (remote)",
+      url: "http://example.com/d-remote",
+    }],
+  }, "Should move (B C D) to unfiled");
+
+  // A is an orphan because we don't have E locally, but we should move
+  // (B C D) into A.
+  do_print("Add [E] > A to mirror");
+  await buf.store([{
+    id: "folderAAAAAA",
+    type: "folder",
+    title: "A",
+    children: ["bookmarkDDDD", "bookmarkCCCC", "bookmarkBBBB"],
+  }]);
+
+  do_print("Apply mirror with A");
+  {
+    let changesToUpload = await buf.apply();
+    let idsToUpload = inspectChangeRecords(changesToUpload);
+    deepEqual(idsToUpload, {
+      updated: [],
+      deleted: [],
+    }, "Should not reupload orphan A");
+  }
+
+  await assertLocalTree(PlacesUtils.bookmarks.unfiledGuid, {
+    guid: PlacesUtils.bookmarks.unfiledGuid,
+    type: PlacesUtils.bookmarks.TYPE_FOLDER,
+    index: 3,
+    title: "Other Bookmarks",
+    children: [{
+      guid: "folderAAAAAA",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 0,
+      title: "A",
+      children: [{
+        guid: "bookmarkDDDD",
+        type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+        index: 0,
+        title: "D (remote)",
+        url: "http://example.com/d-remote",
+      }, {
+        guid: "bookmarkCCCC",
+        type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+        index: 1,
+        title: "C (remote)",
+        url: "http://example.com/c-remote",
+      }, {
+        guid: "bookmarkBBBB",
+        type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+        index: 2,
+        title: "B (remote)",
+        url: "http://example.com/b-remote",
+      }],
+    }],
+  }, "Should move (D C B) into A");
+
+  do_print("Add E to mirror");
+  await buf.store([{
+    id: "folderEEEEEE",
+    type: "folder",
+    title: "E",
+    children: ["folderAAAAAA"],
+  }]);
+
+  do_print("Apply mirror with E");
+  {
+    let changesToUpload = await buf.apply();
+    let idsToUpload = inspectChangeRecords(changesToUpload);
+    deepEqual(idsToUpload, {
+      updated: [],
+      deleted: [],
+    }, "Should not reupload orphan E");
+  }
+
+  // E is still in unfiled because we don't have a record for the menu.
+  await assertLocalTree(PlacesUtils.bookmarks.unfiledGuid, {
+    guid: PlacesUtils.bookmarks.unfiledGuid,
+    type: PlacesUtils.bookmarks.TYPE_FOLDER,
+    index: 3,
+    title: "Other Bookmarks",
+    children: [{
+      guid: "folderEEEEEE",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 0,
+      title: "E",
+      children: [{
+        guid: "folderAAAAAA",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        index: 0,
+        title: "A",
+        children: [{
+          guid: "bookmarkDDDD",
+          type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+          index: 0,
+          title: "D (remote)",
+          url: "http://example.com/d-remote",
+        }, {
+          guid: "bookmarkCCCC",
+          type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+          index: 1,
+          title: "C (remote)",
+          url: "http://example.com/c-remote",
+        }, {
+          guid: "bookmarkBBBB",
+          type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+          index: 2,
+          title: "B (remote)",
+          url: "http://example.com/b-remote",
+        }],
+      }],
+    }],
+  }, "Should move A into E");
+
+  do_print("Add Menu > E to mirror");
+  await buf.store([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: ["folderEEEEEE"],
+  }]);
+
+  do_print("Apply mirror with menu");
+  {
+    let changesToUpload = await buf.apply();
+    let idsToUpload = inspectChangeRecords(changesToUpload);
+    deepEqual(idsToUpload, {
+      updated: [],
+      deleted: [],
+    }, "Should not reupload after forming complete tree");
+  }
+
+  await assertLocalTree(PlacesUtils.bookmarks.rootGuid, {
+    guid: PlacesUtils.bookmarks.rootGuid,
+    type: PlacesUtils.bookmarks.TYPE_FOLDER,
+    index: 0,
+    title: "",
+    children: [{
+      guid: PlacesUtils.bookmarks.menuGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 0,
+      title: "Bookmarks Menu",
+      children: [{
+        guid: "folderEEEEEE",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        index: 0,
+        title: "E",
+        children: [{
+          guid: "folderAAAAAA",
+          type: PlacesUtils.bookmarks.TYPE_FOLDER,
+          index: 0,
+          title: "A",
+          children: [{
+            guid: "bookmarkDDDD",
+            type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+            index: 0,
+            title: "D (remote)",
+            url: "http://example.com/d-remote",
+          }, {
+            guid: "bookmarkCCCC",
+            type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+            index: 1,
+            title: "C (remote)",
+            url: "http://example.com/c-remote",
+          }, {
+            guid: "bookmarkBBBB",
+            type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+            index: 2,
+            title: "B (remote)",
+            url: "http://example.com/b-remote",
+          }],
+        }],
+      }],
+    }, {
+      guid: PlacesUtils.bookmarks.toolbarGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 1,
+      title: "Bookmarks Toolbar",
+    }, {
+      guid: PlacesUtils.bookmarks.unfiledGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 3,
+      title: "Other Bookmarks",
+    }, {
+      guid: PlacesUtils.bookmarks.mobileGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 4,
+      title: "mobile",
+    }],
+  }, "Should form complete tree after applying E");
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
+
+add_task(async function test_move_into_orphaned() {
+  let buf = await openMirror("move_into_orphaned");
+
+  do_print("Set up mirror: Menu > (A B (C > (D (E > F))))");
+  await PlacesUtils.bookmarks.insertTree({
+    guid: PlacesUtils.bookmarks.menuGuid,
+    children: [{
+      guid: "bookmarkAAAA",
+      url: "http://example.com/a",
+      title: "A",
+    }, {
+      guid: "bookmarkBBBB",
+      url: "http://example.com/b",
+      title: "B",
+    }, {
+      guid: "folderCCCCCC",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      title: "C",
+      children: [{
+        guid: "bookmarkDDDD",
+        title: "D",
+        url: "http://example.com/d",
+      }, {
+        guid: "folderEEEEEE",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        title: "E",
+        children: [{
+          guid: "bookmarkFFFF",
+          title: "F",
+          url: "http://example.com/f",
+        }],
+      }],
+    }],
+  });
+  await buf.store([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: ["bookmarkAAAA", "bookmarkBBBB", "folderCCCCCC"],
+  }, {
+    id: "bookmarkAAAA",
+    type: "bookmark",
+    title: "A",
+    bmkUri: "http://example.com/a",
+  }, {
+    id: "bookmarkBBBB",
+    type: "bookmark",
+    title: "B",
+    bmkUri: "http://example.com/b",
+  }, {
+    id: "folderCCCCCC",
+    type: "folder",
+    title: "C",
+    children: ["bookmarkDDDD", "folderEEEEEE"],
+  }, {
+    id: "bookmarkDDDD",
+    type: "bookmark",
+    title: "D",
+    bmkUri: "http://example.com/d",
+  }, {
+    id: "folderEEEEEE",
+    type: "folder",
+    title: "E",
+    children: ["bookmarkFFFF"],
+  }, {
+    id: "bookmarkFFFF",
+    type: "bookmark",
+    title: "F",
+    bmkUri: "http://example.com/f",
+  }], { needsMerge: false });
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  do_print("Make local changes: delete D, add E > I");
+  await PlacesUtils.bookmarks.remove("bookmarkDDDD");
+  await PlacesUtils.bookmarks.insert({
+    guid: "bookmarkIIII",
+    parentGuid: "folderEEEEEE",
+    title: "I (local)",
+    url: "http://example.com/i",
+  });
+
+  // G doesn't exist on the server.
+  do_print("Set up mirror: ([G] > A (C > (D H E))), (C > H)");
+  await buf.store(shuffle([{
+    id: "bookmarkAAAA",
+    type: "bookmark",
+    title: "A",
+    bmkUri: "http://example.com/a",
+  }, {
+    id: "folderCCCCCC",
+    type: "folder",
+    title: "C",
+    children: ["bookmarkDDDD", "bookmarkHHHH", "folderEEEEEE"],
+  }, {
+    id: "bookmarkHHHH",
+    type: "bookmark",
+    title: "H (remote)",
+    bmkUri: "http://example.com/h-remote",
+  }]));
+
+  do_print("Apply mirror");
+  let changesToUpload = await buf.apply();
+  let idsToUpload = inspectChangeRecords(changesToUpload);
+  deepEqual(idsToUpload, {
+    updated: ["bookmarkIIII", "folderCCCCCC", "folderEEEEEE"],
+    deleted: ["bookmarkDDDD"],
+  }, "Should upload records for (I C E); tombstone for D");
+
+  await assertLocalTree(PlacesUtils.bookmarks.rootGuid, {
+    guid: PlacesUtils.bookmarks.rootGuid,
+    type: PlacesUtils.bookmarks.TYPE_FOLDER,
+    index: 0,
+    title: "",
+    children: [{
+      guid: PlacesUtils.bookmarks.menuGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 0,
+      title: "Bookmarks Menu",
+      children: [{
+        // A remains in its original place, since we don't use the `parentid`,
+        // and we don't have a record for G.
+        guid: "bookmarkAAAA",
+        type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+        index: 0,
+        title: "A",
+        url: "http://example.com/a",
+      }, {
+        guid: "bookmarkBBBB",
+        type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+        index: 1,
+        title: "B",
+        url: "http://example.com/b",
+      }, {
+        // C exists on the server, so we take its children and order. D was
+        // deleted locally, and doesn't exist remotely. C is also a child of
+        // G, but we don't have a record for it on the server.
+        guid: "folderCCCCCC",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        index: 2,
+        title: "C",
+        children: [{
+          guid: "bookmarkHHHH",
+          type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+          index: 0,
+          title: "H (remote)",
+          url: "http://example.com/h-remote",
+        }, {
+          guid: "folderEEEEEE",
+          type: PlacesUtils.bookmarks.TYPE_FOLDER,
+          index: 1,
+          title: "E",
+          children: [{
+            guid: "bookmarkFFFF",
+            type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+            index: 0,
+            title: "F",
+            url: "http://example.com/f",
+          }, {
+            guid: "bookmarkIIII",
+            type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+            index: 1,
+            title: "I (local)",
+            url: "http://example.com/i",
+          }],
+        }],
+      }],
+    }, {
+      guid: PlacesUtils.bookmarks.toolbarGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 1,
+      title: "Bookmarks Toolbar",
+    }, {
+      guid: PlacesUtils.bookmarks.unfiledGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 3,
+      title: "Other Bookmarks",
+    }, {
+      guid: PlacesUtils.bookmarks.mobileGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 4,
+      title: "mobile",
+    }],
+  }, "Should treat local tree as canonical if server is missing new parent");
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
+
+add_task(async function test_new_orphan_with_local_parent() {
+  let buf = await openMirror("new_orphan_with_local_parent");
+
+  do_print("Set up mirror: A > (B D)");
+  await PlacesUtils.bookmarks.insertTree({
+    guid: PlacesUtils.bookmarks.menuGuid,
+    children: [{
+      guid: "folderAAAAAA",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      title: "A",
+      children: [{
+        guid: "bookmarkBBBB",
+        url: "http://example.com/b",
+        title: "B",
+      }, {
+        guid: "bookmarkEEEE",
+        url: "http://example.com/e",
+        title: "E",
+      }],
+    }],
+  });
+  await buf.store(shuffle([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: ["folderAAAAAA"],
+  }, {
+    id: "folderAAAAAA",
+    type: "folder",
+    title: "A",
+    children: ["bookmarkBBBB", "bookmarkEEEE"],
+  }, {
+    id: "bookmarkBBBB",
+    type: "bookmark",
+    title: "B",
+    bmkUri: "http://example.com/b",
+  }, {
+    id: "bookmarkEEEE",
+    type: "bookmark",
+    title: "E",
+    bmkUri: "http://example.com/e",
+  }]), { needsMerge: false });
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  // Simulate a partial write by another device that uploaded only B and C. A
+  // exists locally, so we can move B and C into the correct folder, but not
+  // the correct positions.
+  do_print("Set up mirror with orphans: [A] > (C D)");
+  await buf.store([{
+    id: "bookmarkDDDD",
+    type: "bookmark",
+    title: "D (remote)",
+    bmkUri: "http://example.com/d-remote",
+  }, {
+    id: "bookmarkCCCC",
+    type: "bookmark",
+    title: "C (remote)",
+    bmkUri: "http://example.com/c-remote",
+  }]);
+
+  do_print("Apply mirror with (C D)");
+  {
+    let changesToUpload = await buf.apply();
+    let idsToUpload = inspectChangeRecords(changesToUpload);
+    deepEqual(idsToUpload, {
+      updated: [],
+      deleted: [],
+    }, "Should not reupload orphans (C D)");
+  }
+
+  await assertLocalTree(PlacesUtils.bookmarks.rootGuid, {
+    guid: PlacesUtils.bookmarks.rootGuid,
+    type: PlacesUtils.bookmarks.TYPE_FOLDER,
+    index: 0,
+    title: "",
+    children: [{
+      guid: PlacesUtils.bookmarks.menuGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 0,
+      title: "Bookmarks Menu",
+      children: [{
+        guid: "folderAAAAAA",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        index: 0,
+        title: "A",
+        children: [{
+          guid: "bookmarkBBBB",
+          type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+          index: 0,
+          title: "B",
+          url: "http://example.com/b",
+        }, {
+          guid: "bookmarkEEEE",
+          type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+          index: 1,
+          title: "E",
+          url: "http://example.com/e",
+        }],
+      }],
+    }, {
+      guid: PlacesUtils.bookmarks.toolbarGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 1,
+      title: "Bookmarks Toolbar",
+    }, {
+      guid: PlacesUtils.bookmarks.unfiledGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 3,
+      title: "Other Bookmarks",
+      children: [{
+        guid: "bookmarkCCCC",
+        type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+        index: 0,
+        title: "C (remote)",
+        url: "http://example.com/c-remote",
+      }, {
+        guid: "bookmarkDDDD",
+        type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+        index: 1,
+        title: "D (remote)",
+        url: "http://example.com/d-remote",
+      }],
+    }, {
+      guid: PlacesUtils.bookmarks.mobileGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 4,
+      title: "mobile",
+    }],
+  }, "Should move (C D) to unfiled");
+
+  // The partial uploader returns and uploads A.
+  do_print("Add A to mirror");
+  await buf.store([{
+    id: "folderAAAAAA",
+    type: "folder",
+    title: "A",
+    children: ["bookmarkCCCC", "bookmarkDDDD", "bookmarkEEEE", "bookmarkBBBB"],
+  }]);
+
+  do_print("Apply mirror with A");
+  {
+    let changesToUpload = await buf.apply();
+    let idsToUpload = inspectChangeRecords(changesToUpload);
+    deepEqual(idsToUpload, {
+      updated: [],
+      deleted: [],
+    }, "Should not reupload orphan A");
+  }
+
+  await assertLocalTree("folderAAAAAA", {
+    guid: "folderAAAAAA",
+    type: PlacesUtils.bookmarks.TYPE_FOLDER,
+    index: 0,
+    title: "A",
+    children: [{
+      guid: "bookmarkCCCC",
+      type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+      index: 0,
+      title: "C (remote)",
+      url: "http://example.com/c-remote",
+    }, {
+      guid: "bookmarkDDDD",
+      type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+      index: 1,
+      title: "D (remote)",
+      url: "http://example.com/d-remote",
+    }, {
+      guid: "bookmarkEEEE",
+      type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+      index: 2,
+      title: "E",
+      url: "http://example.com/e",
+    }, {
+      guid: "bookmarkBBBB",
+      type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+      index: 3,
+      title: "B",
+      url: "http://example.com/b",
+    }],
+  }, "Should update child positions once A exists in mirror");
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
+
+add_task(async function test_tombstone_as_child() {
+  // TODO(kitcambridge): Add a folder that mentions a tombstone in its
+  // `children`.
+});
+
+add_task(async function test_left_pane_root() {
+  // TODO(kitcambridge): Add a left pane root to the server. We ignore and
+  // remove the Places root, so the left pane queries will be orphaned and
+  // moved to unfiled. Consider adding heuristics to remove them later.
+});
+
+add_task(async function test_partial_cycle() {
+  let buf = await openMirror("partial_cycle");
+
+  do_print("Set up mirror: Menu > A > B > C");
+  await PlacesUtils.bookmarks.insertTree({
+    guid: PlacesUtils.bookmarks.menuGuid,
+    children: [{
+      guid: "folderAAAAAA",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      title: "A",
+      children: [{
+        guid: "folderBBBBBB",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        title: "B",
+        children: [{
+          guid: "bookmarkCCCC",
+          url: "http://example.com/c",
+          title: "C",
+        }],
+      }],
+    }],
+  });
+  await buf.store(shuffle([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: ["folderAAAAAA"],
+  }, {
+    id: "folderAAAAAA",
+    type: "folder",
+    title: "A",
+    children: ["folderBBBBBB"],
+  }, {
+    id: "folderBBBBBB",
+    type: "folder",
+    title: "B",
+    children: ["bookmarkCCCC"],
+  }, {
+    id: "bookmarkCCCC",
+    type: "bookmark",
+    title: "C",
+    bmkUri: "http://example.com/c",
+  }]), { needsMerge: false });
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  // Try to create a cycle: move A into B, and B into the menu, but don't upload
+  // a record for the menu. B is still a child of A locally. Since we ignore the
+  // `parentid`, we'll move (B A) into unfiled.
+  do_print("Set up mirror: A > C");
+  await buf.store([{
+    id: "folderAAAAAA",
+    type: "folder",
+    title: "A (remote)",
+    children: ["bookmarkCCCC"],
+  }, {
+    id: "folderBBBBBB",
+    type: "folder",
+    title: "B (remote)",
+    children: ["folderAAAAAA"],
+  }]);
+
+  do_print("Apply mirror");
+  let changesToUpload = await buf.apply();
+  let idsToUpload = inspectChangeRecords(changesToUpload);
+  deepEqual(idsToUpload, { updated: [], deleted: [] },
+    "Should not mark any local items for upload");
+
+  await assertLocalTree(PlacesUtils.bookmarks.rootGuid, {
+    guid: PlacesUtils.bookmarks.rootGuid,
+    type: PlacesUtils.bookmarks.TYPE_FOLDER,
+    index: 0,
+    title: "",
+    children: [{
+      guid: PlacesUtils.bookmarks.menuGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 0,
+      title: "Bookmarks Menu",
+    }, {
+      guid: PlacesUtils.bookmarks.toolbarGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 1,
+      title: "Bookmarks Toolbar",
+    }, {
+      guid: PlacesUtils.bookmarks.unfiledGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 3,
+      title: "Other Bookmarks",
+      children: [{
+        guid: "folderBBBBBB",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        index: 0,
+        title: "B (remote)",
+        children: [{
+          guid: "folderAAAAAA",
+          type: PlacesUtils.bookmarks.TYPE_FOLDER,
+          index: 0,
+          title: "A (remote)",
+          children: [{
+            guid: "bookmarkCCCC",
+            type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+            index: 0,
+            title: "C",
+            url: "http://example.com/c",
+          }],
+        }],
+      }],
+    }, {
+      guid: PlacesUtils.bookmarks.mobileGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 4,
+      title: "mobile",
+    }],
+  }, "Should move A and B to unfiled");
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
+
+add_task(async function test_complete_cycle() {
+  let buf = await openMirror("complete_cycle");
+
+  do_print("Set up empty mirror");
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  // This test is order-dependent. We shouldn't recurse infinitely, but,
+  // depending on the order of the records, we might ignore the circular
+  // subtree because there's nothing linking it back to the rest of the
+  // tree.
+  do_print("Set up mirror: Menu > A > B > C > A");
+  await buf.store([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: ["folderAAAAAA"],
+  }, {
+    id: "folderAAAAAA",
+    type: "folder",
+    title: "A",
+    children: ["folderBBBBBB"],
+  }, {
+    id: "folderBBBBBB",
+    type: "folder",
+    title: "B",
+    children: ["folderCCCCCC"],
+  }, {
+    id: "folderCCCCCC",
+    type: "folder",
+    title: "C",
+    children: ["folderDDDDDD"],
+  }, {
+    id: "folderDDDDDD",
+    type: "folder",
+    title: "D",
+    children: ["folderAAAAAA"],
+  }]);
+
+  do_print("Apply mirror");
+  let changesToUpload = await buf.apply();
+  let idsToUpload = inspectChangeRecords(changesToUpload);
+  deepEqual(idsToUpload, { updated: [], deleted: [] },
+    "Should not mark any local items for upload");
+
+  await assertLocalTree(PlacesUtils.bookmarks.rootGuid, {
+    guid: PlacesUtils.bookmarks.rootGuid,
+    type: PlacesUtils.bookmarks.TYPE_FOLDER,
+    index: 0,
+    title: "",
+    children: [{
+      guid: PlacesUtils.bookmarks.menuGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 0,
+      title: "Bookmarks Menu",
+    }, {
+      guid: PlacesUtils.bookmarks.toolbarGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 1,
+      title: "Bookmarks Toolbar",
+    }, {
+      guid: PlacesUtils.bookmarks.unfiledGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 3,
+      title: "Other Bookmarks",
+    }, {
+      guid: PlacesUtils.bookmarks.mobileGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 4,
+      title: "mobile",
+    }],
+  }, "Should not be confused into creating a cycle");
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
new file mode 100644
--- /dev/null
+++ b/toolkit/components/places/tests/sync/test_bookmark_deduping.js
@@ -0,0 +1,436 @@
+add_task(async function test_duping() {
+  let buf = await openMirror("duping");
+
+  do_print("Set up mirror");
+  await PlacesUtils.bookmarks.insertTree({
+    guid: PlacesUtils.bookmarks.menuGuid,
+    children: [{
+      // Shouldn't dupe to `folderA11111` because its sync status is "NORMAL".
+      guid: "folderAAAAAA",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      title: "A",
+      children: [{
+        // Shouldn't dupe to `bookmarkG111`.
+        guid: "bookmarkGGGG",
+        url: "http://example.com/g",
+        title: "G",
+      }],
+    }],
+  });
+  await buf.store(shuffle([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: ["folderAAAAAA"],
+  }, {
+    id: "folderAAAAAA",
+    type: "folder",
+    title: "A",
+    children: ["bookmarkGGGG"],
+  }, {
+    id: "bookmarkGGGG",
+    type: "bookmark",
+    title: "G",
+    url: "http://example.com/g",
+  }]), { needsMerge: false });
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  do_print("Insert local dupes");
+  await PlacesUtils.bookmarks.insertTree({
+    guid: PlacesUtils.bookmarks.menuGuid,
+    children: [{
+      // Should dupe to `folderB11111`.
+      guid: "folderBBBBBB",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      title: "B",
+      children: [{
+        // Should dupe to `bookmarkC222`.
+        guid: "bookmarkC111",
+        url: "http://example.com/c",
+        title: "C",
+      }, {
+        // Should dupe to `separatorF11` because the positions are the same.
+        guid: "separatorFFF",
+        type: PlacesUtils.bookmarks.TYPE_SEPARATOR,
+      }],
+    }, {
+      // Shouldn't dupe to `separatorE11`, because the positions are different.
+      guid: "separatorEEE",
+      type: PlacesUtils.bookmarks.TYPE_SEPARATOR,
+    }, {
+      // Shouldn't dupe to `bookmarkC222` because the parents are different.
+      guid: "bookmarkCCCC",
+      url: "http://example.com/c",
+      title: "C",
+    }, {
+      // Should dupe to `queryD111111`.
+      guid: "queryDDDDDDD",
+      url: "place:sort=8&maxResults=10",
+      title: "Most Visited",
+      annos: [{
+        name: PlacesSyncUtils.bookmarks.SMART_BOOKMARKS_ANNO,
+        value: "MostVisited",
+      }],
+    }],
+  });
+  // Not a candidate for `bookmarkH111` because we didn't dupe `folderAAAAAA`.
+  await PlacesUtils.bookmarks.insert({
+    parentGuid: "folderAAAAAA",
+    guid: "bookmarkHHHH",
+    url: "http://example.com/h",
+    title: "H",
+  });
+
+  do_print("Set up mirror");
+  await buf.store(shuffle([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: ["folderAAAAAA", "folderB11111", "folderA11111",
+               "separatorE11", "queryD111111"],
+  }, {
+    id: "folderB11111",
+    type: "folder",
+    title: "B",
+    children: ["bookmarkC222", "separatorF11"],
+  }, {
+    id: "bookmarkC222",
+    type: "bookmark",
+    bmkUri: "http://example.com/c",
+    title: "C",
+  }, {
+    id: "separatorF11",
+    type: "separator",
+  }, {
+    id: "folderA11111",
+    type: "folder",
+    title: "A",
+    children: ["bookmarkG111"],
+  }, {
+    id: "bookmarkG111",
+    type: "bookmark",
+    bmkUri: "http://example.com/g",
+    title: "G",
+  }, {
+    id: "separatorE11",
+    type: "separator",
+  }, {
+    id: "queryD111111",
+    type: "query",
+    bmkUri: "place:maxResults=10&sort=8",
+    title: "Most Visited",
+    queryId: "MostVisited",
+  }]));
+
+  do_print("Apply mirror");
+  let changesToUpload = await buf.apply();
+  let idsToUpload = inspectChangeRecords(changesToUpload);
+  deepEqual(idsToUpload, {
+    updated: ["bookmarkCCCC", "bookmarkHHHH", "folderAAAAAA", "menu",
+              "separatorEEE"],
+    deleted: [],
+  }, "Should not upload deduped local records");
+
+  await assertLocalTree(PlacesUtils.bookmarks.rootGuid, {
+    guid: PlacesUtils.bookmarks.rootGuid,
+    type: PlacesUtils.bookmarks.TYPE_FOLDER,
+    index: 0,
+    title: "",
+    children: [{
+      guid: PlacesUtils.bookmarks.menuGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 0,
+      title: "Bookmarks Menu",
+      children: [{
+        guid: "folderAAAAAA",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        index: 0,
+        title: "A",
+        children: [{
+          guid: "bookmarkGGGG",
+          type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+          index: 0,
+          title: "G",
+          url: "http://example.com/g",
+        }, {
+          guid: "bookmarkHHHH",
+          type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+          index: 1,
+          title: "H",
+          url: "http://example.com/h",
+        }],
+      }, {
+        guid: "folderB11111",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        index: 1,
+        title: "B",
+        children: [{
+          guid: "bookmarkC222",
+          type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+          index: 0,
+          title: "C",
+          url: "http://example.com/c",
+        }, {
+          guid: "separatorF11",
+          type: PlacesUtils.bookmarks.TYPE_SEPARATOR,
+          index: 1,
+          title: "",
+        }],
+      }, {
+        guid: "folderA11111",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        index: 2,
+        title: "A",
+        children: [{
+          guid: "bookmarkG111",
+          type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+          index: 0,
+          title: "G",
+          url: "http://example.com/g",
+        }],
+      }, {
+        guid: "separatorE11",
+        type: PlacesUtils.bookmarks.TYPE_SEPARATOR,
+        index: 3,
+        title: "",
+      }, {
+        guid: "queryD111111",
+        type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+        index: 4,
+        title: "Most Visited",
+        url: "place:maxResults=10&sort=8",
+        annos: [{
+          name: PlacesSyncUtils.bookmarks.SMART_BOOKMARKS_ANNO,
+          flags: 0,
+          expires: PlacesUtils.annotations.EXPIRE_NEVER,
+          value: "MostVisited",
+        }],
+      }, {
+        guid: "separatorEEE",
+        type: PlacesUtils.bookmarks.TYPE_SEPARATOR,
+        index: 5,
+        title: "",
+      }, {
+        guid: "bookmarkCCCC",
+        type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+        index: 6,
+        title: "C",
+        url: "http://example.com/c",
+      }],
+    }, {
+      guid: PlacesUtils.bookmarks.toolbarGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 1,
+      title: "Bookmarks Toolbar",
+    }, {
+      guid: PlacesUtils.bookmarks.unfiledGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 3,
+      title: "Other Bookmarks",
+    }, {
+      guid: PlacesUtils.bookmarks.mobileGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 4,
+      title: "mobile",
+    }],
+  }, "Should dedupe matching NEW bookmarks");
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
+
+add_task(async function test_applying_two_empty_folders_doesnt_smush() {
+  let buf = await openMirror("applying_two_empty_folders_doesnt_smush");
+
+  do_print("Set up empty mirror");
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  do_print("Set up mirror");
+  await buf.store(shuffle([{
+    id: "mobile",
+    type: "folder",
+    title: "mobile",
+    children: ["emptyempty01", "emptyempty02"],
+  }, {
+    id: "emptyempty01",
+    type: "folder",
+    title: "Empty",
+  }, {
+    id: "emptyempty02",
+    type: "folder",
+    title: "Empty",
+  }]));
+
+  do_print("Apply mirror");
+  let changesToUpload = await buf.apply();
+  let idsToUpload = inspectChangeRecords(changesToUpload);
+  deepEqual(idsToUpload, {
+    updated: [],
+    deleted: [],
+  }, "Should not upload records for remote-only value changes");
+
+  await assertLocalTree(PlacesUtils.bookmarks.mobileGuid, {
+    guid: PlacesUtils.bookmarks.mobileGuid,
+    type: PlacesUtils.bookmarks.TYPE_FOLDER,
+    index: 4,
+    title: "mobile",
+    children: [{
+      guid: "emptyempty01",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 0,
+      title: "Empty",
+    }, {
+      guid: "emptyempty02",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 1,
+      title: "Empty",
+    }],
+  }, "Should not smush 1 and 2");
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
+
+add_task(async function test_applying_two_empty_folders_matches_only_one() {
+  let buf = await openMirror("applying_two_empty_folders_doesnt_smush");
+
+  do_print("Set up empty mirror");
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  do_print("Set up local");
+  await PlacesUtils.bookmarks.insertTree({
+    guid: PlacesUtils.bookmarks.mobileGuid,
+    children: [{
+      guid: "emptyempty02",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      title: "Empty",
+    }, {
+      guid: "emptyemptyL0",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      title: "Empty",
+    }],
+  });
+
+  do_print("Set up mirror");
+  await buf.store(shuffle([{
+    id: "mobile",
+    type: "folder",
+    title: "mobile",
+    children: ["emptyempty01", "emptyempty02", "emptyempty03"],
+  }, {
+    id: "emptyempty01",
+    type: "folder",
+    title: "Empty",
+  }, {
+    id: "emptyempty02",
+    type: "folder",
+    title: "Empty",
+  }, {
+    id: "emptyempty03",
+    type: "folder",
+    title: "Empty",
+  }]));
+
+  do_print("Apply mirror");
+  let changesToUpload = await buf.apply();
+  let idsToUpload = inspectChangeRecords(changesToUpload);
+  deepEqual(idsToUpload, {
+    updated: [],
+    deleted: [],
+  }, "Should not upload records after applying empty folders");
+
+  await assertLocalTree(PlacesUtils.bookmarks.mobileGuid, {
+    guid: PlacesUtils.bookmarks.mobileGuid,
+    type: PlacesUtils.bookmarks.TYPE_FOLDER,
+    index: 4,
+    title: "mobile",
+    children: [{
+      guid: "emptyempty01",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 0,
+      title: "Empty",
+    }, {
+      guid: "emptyempty02",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 1,
+      title: "Empty",
+    }, {
+      guid: "emptyempty03",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 2,
+      title: "Empty",
+    }],
+  }, "Should apply 1 and dedupe L0 to 3");
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
+
+// Bug 747699.
+add_task(async function test_duping_mobile_bookmarks() {
+  let buf = await openMirror("duping_mobile_bookmarks");
+
+  do_print("Set up empty mirror with localized mobile root title");
+  let mobileInfo = await PlacesUtils.bookmarks.fetch(
+    PlacesUtils.bookmarks.mobileGuid);
+  await PlacesUtils.bookmarks.update({
+    guid: PlacesUtils.bookmarks.mobileGuid,
+    title: "Favoritos do celular",
+  });
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  do_print("Set up local");
+  await PlacesUtils.bookmarks.insert({
+    guid: "bookmarkAAA1",
+    parentGuid: PlacesUtils.bookmarks.mobileGuid,
+    title: "A",
+    url: "http://example.com/a",
+  });
+
+  do_print("Set up mirror");
+  await buf.store(shuffle([{
+    id: "mobile",
+    type: "folder",
+    title: "Mobile Bookmarks",
+    children: ["bookmarkAAAA"],
+  }, {
+    id: "bookmarkAAAA",
+    type: "bookmark",
+    title: "A",
+    bmkUri: "http://example.com/a",
+  }]));
+
+  do_print("Apply mirror");
+  let changesToUpload = await buf.apply();
+  let idsToUpload = inspectChangeRecords(changesToUpload);
+  deepEqual(idsToUpload, {
+    updated: [],
+    deleted: [],
+  }, "Should not upload records after applying deduped mobile bookmark");
+
+  await assertLocalTree(PlacesUtils.bookmarks.mobileGuid, {
+    guid: PlacesUtils.bookmarks.mobileGuid,
+    type: PlacesUtils.bookmarks.TYPE_FOLDER,
+    index: 4,
+    title: "Mobile Bookmarks",
+    children: [{
+      guid: "bookmarkAAAA",
+      type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+      index: 0,
+      title: "A",
+      url: "http://example.com/a",
+    }],
+  }, "Should dedupe A1 to A with different parent title");
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  // Restore the original mobile root title.
+  await PlacesUtils.bookmarks.update({
+    guid: PlacesUtils.bookmarks.mobileGuid,
+    title: mobileInfo.title,
+  });
+  await PlacesSyncUtils.bookmarks.reset();
+});
new file mode 100644
--- /dev/null
+++ b/toolkit/components/places/tests/sync/test_bookmark_deletion.js
@@ -0,0 +1,579 @@
+add_task(async function test_complex_orphaning() {
+  let buf = await openMirror("complex_orphaning");
+
+  // On iOS, the mirror exists as a separate table. On Desktop, we have a
+  // shadow mirror of synced local bookmarks without new changes.
+  do_print("Set up mirror: ((Toolbar > A > B) (Menu > G > C > D))");
+  await PlacesUtils.bookmarks.insertTree({
+    guid: PlacesUtils.bookmarks.toolbarGuid,
+    children: [{
+      guid: "folderAAAAAA",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      title: "A",
+      children: [{
+        guid: "folderBBBBBB",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        title: "B",
+      }],
+    }],
+  });
+  await buf.store(shuffle([{
+    id: "toolbar",
+    type: "folder",
+    title: "Bookmarks Toolbar",
+    children: ["folderAAAAAA"],
+  }, {
+    id: "folderAAAAAA",
+    type: "folder",
+    title: "A",
+    children: ["folderBBBBBB"],
+  }, {
+    id: "folderBBBBBB",
+    type: "folder",
+    title: "B",
+  }]), { needsMerge: false });
+  await PlacesUtils.bookmarks.insertTree({
+    guid: PlacesUtils.bookmarks.menuGuid,
+    children: [{
+      guid: "folderGGGGGG",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      title: "G",
+      children: [{
+        guid: "folderCCCCCC",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        title: "C",
+        children: [{
+          guid: "folderDDDDDD",
+          type: PlacesUtils.bookmarks.TYPE_FOLDER,
+          title: "D",
+        }],
+      }],
+    }],
+  });
+  await buf.store(shuffle([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: ["folderGGGGGG"],
+  }, {
+    id: "folderGGGGGG",
+    type: "folder",
+    title: "G",
+    children: ["folderCCCCCC"],
+  }, {
+    id: "folderCCCCCC",
+    type: "folder",
+    title: "C",
+    children: ["folderDDDDDD"],
+  }, {
+    id: "folderDDDDDD",
+    type: "folder",
+    title: "D",
+  }]), { needsMerge: false });
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  do_print("Make local changes: delete D, add B > E");
+  await PlacesUtils.bookmarks.remove("folderDDDDDD");
+  await PlacesUtils.bookmarks.insert({
+    guid: "bookmarkEEEE",
+    parentGuid: "folderBBBBBB",
+    title: "E",
+    url: "http://example.com/e",
+  });
+
+  do_print("Set up mirror: delete B, add D > F");
+  await buf.store(shuffle([{
+    id: "folderBBBBBB",
+    deleted: true,
+  }, {
+    id: "folderAAAAAA",
+    type: "folder",
+    title: "A",
+  }, {
+    id: "folderDDDDDD",
+    type: "folder",
+    children: ["bookmarkFFFF"],
+  }, {
+    id: "bookmarkFFFF",
+    type: "bookmark",
+    title: "F",
+    bmkUri: "http://example.com/f",
+  }]));
+
+  do_print("Apply mirror");
+  let changesToUpload = await buf.apply();
+  let idsToUpload = inspectChangeRecords(changesToUpload);
+  deepEqual(idsToUpload, {
+    updated: ["bookmarkEEEE", "bookmarkFFFF", "folderAAAAAA", "folderCCCCCC"],
+    deleted: ["folderDDDDDD"],
+  }, "Should upload new records for (A > E), (C > F); tombstone for D");
+
+  await assertLocalTree(PlacesUtils.bookmarks.rootGuid, {
+    guid: PlacesUtils.bookmarks.rootGuid,
+    type: PlacesUtils.bookmarks.TYPE_FOLDER,
+    index: 0,
+    title: "",
+    children: [{
+      guid: PlacesUtils.bookmarks.menuGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 0,
+      title: "Bookmarks Menu",
+      children: [{
+        guid: "folderGGGGGG",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        index: 0,
+        title: "G",
+        children: [{
+          guid: "folderCCCCCC",
+          type: PlacesUtils.bookmarks.TYPE_FOLDER,
+          index: 0,
+          title: "C",
+          children: [{
+            // D was deleted, so F moved to C, the closest surviving parent.
+            guid: "bookmarkFFFF",
+            type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+            index: 0,
+            title: "F",
+            url: "http://example.com/f",
+          }],
+        }],
+      }],
+    }, {
+      guid: PlacesUtils.bookmarks.toolbarGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 1,
+      title: "Bookmarks Toolbar",
+      children: [{
+        guid: "folderAAAAAA",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        index: 0,
+        title: "A",
+        children: [{
+          // B was deleted, so E moved to A.
+          guid: "bookmarkEEEE",
+          type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+          index: 0,
+          title: "E",
+          url: "http://example.com/e",
+        }],
+      }],
+    }, {
+      guid: PlacesUtils.bookmarks.unfiledGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 3,
+      title: "Other Bookmarks",
+    }, {
+      guid: PlacesUtils.bookmarks.mobileGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 4,
+      title: "mobile",
+    }],
+  }, "Should move orphans to closest surviving parent");
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
+
+add_task(async function test_locally_modified_remotely_deleted() {
+  let buf = await openMirror("locally_modified_remotely_deleted");
+
+  do_print("Set up mirror");
+  await PlacesUtils.bookmarks.insertTree({
+    guid: PlacesUtils.bookmarks.menuGuid,
+    children: [{
+      guid: "bookmarkAAAA",
+      title: "A",
+      url: "http://example.com/a",
+    }, {
+      guid: "folderBBBBBB",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      title: "B",
+      children: [{
+        guid: "bookmarkCCCC",
+        title: "C",
+        url: "http://example.com/c",
+      }, {
+        guid: "folderDDDDDD",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        title: "D",
+        children: [{
+          guid: "bookmarkEEEE",
+          title: "E",
+          url: "http://example.com/e",
+        }],
+      }],
+    }],
+  });
+  await buf.store([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: ["bookmarkAAAA", "folderBBBBBB"],
+  }, {
+    id: "bookmarkAAAA",
+    type: "bookmark",
+    title: "A",
+    bmkUri: "http://example.com/a",
+  }, {
+    id: "folderBBBBBB",
+    type: "folder",
+    title: "B",
+    children: ["bookmarkCCCC", "folderDDDDDD"],
+  }, {
+    id: "bookmarkCCCC",
+    type: "bookmark",
+    title: "C",
+    bmkUri: "http://example.com/c",
+  }, {
+    id: "folderDDDDDD",
+    type: "folder",
+    title: "D",
+    children: ["bookmarkEEEE"],
+  }, {
+    id: "bookmarkEEEE",
+    type: "bookmark",
+    title: "E",
+    bmkUri: "http://example.com/e",
+  }], { needsMerge: false });
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  do_print("Set up local: change A; B > ((D > F) G)");
+  await PlacesUtils.bookmarks.update({
+    guid: "bookmarkAAAA",
+    title: "A (local)",
+    url: "http://example.com/a-local",
+  });
+  await PlacesUtils.bookmarks.insert({
+    guid: "bookmarkFFFF",
+    parentGuid: "folderDDDDDD",
+    title: "F (local)",
+    url: "http://example.com/f-local",
+  });
+  await PlacesUtils.bookmarks.insert({
+    guid: "bookmarkGGGG",
+    parentGuid: "folderBBBBBB",
+    title: "G (local)",
+    url: "http://example.com/g-local",
+  });
+
+  do_print("Set up mirror: delete A, B");
+  await buf.store([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: [],
+  }, {
+    id: "bookmarkAAAA",
+    deleted: true,
+  }, {
+    id: "folderBBBBBB",
+    deleted: true,
+  }, {
+    id: "bookmarkCCCC",
+    deleted: true,
+  }, {
+    id: "folderDDDDDD",
+    deleted: true,
+  }, {
+    id: "bookmarkEEEE",
+    deleted: true,
+  }]);
+
+  do_print("Apply mirror");
+  let changesToUpload = await buf.apply();
+  let idsToUpload = inspectChangeRecords(changesToUpload);
+  deepEqual(idsToUpload, {
+    updated: ["bookmarkAAAA", "bookmarkFFFF", "bookmarkGGGG", "menu"],
+    deleted: [],
+  }, "Should upload A, relocated local orphans, and menu");
+
+  await assertLocalTree(PlacesUtils.bookmarks.menuGuid, {
+    guid: PlacesUtils.bookmarks.menuGuid,
+    type: PlacesUtils.bookmarks.TYPE_FOLDER,
+    index: 0,
+    title: "Bookmarks Menu",
+    children: [{
+      guid: "bookmarkAAAA",
+      type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+      index: 0,
+      title: "A (local)",
+      url: "http://example.com/a-local",
+    }, {
+      guid: "bookmarkFFFF",
+      type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+      index: 1,
+      title: "F (local)",
+      url: "http://example.com/f-local",
+    }, {
+      guid: "bookmarkGGGG",
+      type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+      index: 2,
+      title: "G (local)",
+      url: "http://example.com/g-local",
+    }],
+  }, "Should restore A and relocate (F G) to menu");
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
+
+add_task(async function test_locally_deleted_remotely_modified() {
+  let buf = await openMirror("locally_deleted_remotely_modified");
+
+  do_print("Set up mirror");
+  await PlacesUtils.bookmarks.insertTree({
+    guid: PlacesUtils.bookmarks.menuGuid,
+    children: [{
+      guid: "bookmarkAAAA",
+      title: "A",
+      url: "http://example.com/a",
+    }, {
+      guid: "folderBBBBBB",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      title: "B",
+      children: [{
+        guid: "bookmarkCCCC",
+        title: "C",
+        url: "http://example.com/c",
+      }, {
+        guid: "folderDDDDDD",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        title: "D",
+        children: [{
+          guid: "bookmarkEEEE",
+          title: "E",
+          url: "http://example.com/e",
+        }],
+      }],
+    }],
+  });
+  await buf.store([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: ["bookmarkAAAA", "folderBBBBBB"],
+  }, {
+    id: "bookmarkAAAA",
+    type: "bookmark",
+    title: "A",
+    bmkUri: "http://example.com/a",
+  }, {
+    id: "folderBBBBBB",
+    type: "folder",
+    title: "B",
+    children: ["bookmarkCCCC", "folderDDDDDD"],
+  }, {
+    id: "bookmarkCCCC",
+    type: "bookmark",
+    title: "C",
+    bmkUri: "http://example.com/c",
+  }, {
+    id: "folderDDDDDD",
+    type: "folder",
+    title: "D",
+    children: ["bookmarkEEEE"],
+  }, {
+    id: "bookmarkEEEE",
+    type: "bookmark",
+    title: "E",
+    bmkUri: "http://example.com/e",
+  }], { needsMerge: false });
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  do_print("Set up local: delete A, B");
+  await PlacesUtils.bookmarks.remove("bookmarkAAAA");
+  await PlacesUtils.bookmarks.remove("folderBBBBBB");
+
+  do_print("Set up mirror: change A; B > ((D > F) G)");
+  await buf.store([{
+    id: "bookmarkAAAA",
+    type: "bookmark",
+    title: "A (remote)",
+    bmkUri: "http://example.com/a-remote",
+  }, {
+    id: "folderBBBBBB",
+    type: "folder",
+    title: "B (remote)",
+    children: ["bookmarkCCCC", "folderDDDDDD", "bookmarkGGGG"],
+  }, {
+    id: "folderDDDDDD",
+    type: "folder",
+    title: "D",
+    children: ["bookmarkEEEE", "bookmarkFFFF"],
+  }, {
+    id: "bookmarkFFFF",
+    type: "bookmark",
+    title: "F (remote)",
+    bmkUri: "http://example.com/f-remote",
+  }, {
+    id: "bookmarkGGGG",
+    type: "bookmark",
+    title: "G (remote)",
+    bmkUri: "http://example.com/g-remote",
+  }]);
+
+  do_print("Apply mirror");
+  let changesToUpload = await buf.apply();
+  let idsToUpload = inspectChangeRecords(changesToUpload);
+  deepEqual(idsToUpload, {
+    updated: ["bookmarkFFFF", "bookmarkGGGG", "menu"],
+    deleted: ["bookmarkCCCC", "bookmarkEEEE", "folderBBBBBB", "folderDDDDDD"],
+  }, "Should upload relocated remote orphans and menu");
+
+  await assertLocalTree(PlacesUtils.bookmarks.menuGuid, {
+    guid: PlacesUtils.bookmarks.menuGuid,
+    type: PlacesUtils.bookmarks.TYPE_FOLDER,
+    index: 0,
+    title: "Bookmarks Menu",
+    children: [{
+      guid: "bookmarkAAAA",
+      type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+      index: 0,
+      title: "A (remote)",
+      url: "http://example.com/a-remote",
+    }, {
+      guid: "bookmarkFFFF",
+      type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+      index: 1,
+      title: "F (remote)",
+      url: "http://example.com/f-remote",
+    }, {
+      guid: "bookmarkGGGG",
+      type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+      index: 2,
+      title: "G (remote)",
+      url: "http://example.com/g-remote",
+    }],
+  }, "Should restore A and relocate (F G) to menu");
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
+
+add_task(async function test_move_to_new_then_delete() {
+  let buf = await openMirror("move_to_new_then_delete");
+
+  do_print("Set up mirror: A > B > (C D)");
+  await PlacesUtils.bookmarks.insertTree({
+    guid: PlacesUtils.bookmarks.menuGuid,
+    children: [{
+      guid: "folderAAAAAA",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      title: "A",
+      children: [{
+        guid: "folderBBBBBB",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        title: "B",
+        children: [{
+          guid: "bookmarkCCCC",
+          url: "http://example.com/c",
+          title: "C",
+        }, {
+          guid: "bookmarkDDDD",
+          url: "http://example.com/d",
+          title: "D",
+        }],
+      }],
+    }],
+  });
+  await buf.store(shuffle([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: ["folderAAAAAA"],
+  }, {
+    id: "folderAAAAAA",
+    type: "folder",
+    title: "A",
+    children: ["folderBBBBBB"],
+  }, {
+    id: "folderBBBBBB",
+    type: "folder",
+    title: "B",
+    children: ["bookmarkCCCC", "bookmarkDDDD"],
+  }, {
+    id: "bookmarkCCCC",
+    type: "bookmark",
+    title: "C",
+    bmkUri: "http://example.com/c",
+  }, {
+    id: "bookmarkDDDD",
+    type: "bookmark",
+    title: "D",
+    bmkUri: "http://example.com/d",
+  }]), { needsMerge: false });
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  do_print("Make local changes: E > A, delete E");
+  await PlacesUtils.bookmarks.insert({
+    parentGuid: PlacesUtils.bookmarks.toolbarGuid,
+    guid: "folderEEEEEE",
+    type: PlacesUtils.bookmarks.TYPE_FOLDER,
+    title: "E",
+  });
+  await PlacesUtils.bookmarks.update({
+    guid: "folderAAAAAA",
+    parentGuid: "folderEEEEEE",
+    index: PlacesUtils.bookmarks.DEFAULT_INDEX,
+  });
+  // E isn't synced, so we shouldn't upload a tombstone.
+  await PlacesUtils.bookmarks.remove("folderEEEEEE");
+
+  do_print("Change B in mirror");
+  await buf.store([{
+    id: "bookmarkCCCC",
+    type: "bookmark",
+    title: "C (remote)",
+    bmkUri: "http://example.com/c-remote",
+  }]);
+
+  do_print("Apply mirror");
+  let changesToUpload = await buf.apply();
+  let idsToUpload = inspectChangeRecords(changesToUpload);
+  deepEqual(idsToUpload, {
+    updated: ["bookmarkCCCC", "menu", "toolbar"],
+    deleted: ["bookmarkDDDD", "folderAAAAAA", "folderBBBBBB"],
+  }, "Should upload records for Menu > C, Toolbar");
+
+  await assertLocalTree(PlacesUtils.bookmarks.rootGuid, {
+    guid: PlacesUtils.bookmarks.rootGuid,
+    type: PlacesUtils.bookmarks.TYPE_FOLDER,
+    index: 0,
+    title: "",
+    children: [{
+      guid: PlacesUtils.bookmarks.menuGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 0,
+      title: "Bookmarks Menu",
+      children: [{
+        guid: "bookmarkCCCC",
+        type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+        index: 0,
+        title: "C (remote)",
+        url: "http://example.com/c-remote",
+      }],
+    }, {
+      guid: PlacesUtils.bookmarks.toolbarGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 1,
+      title: "Bookmarks Toolbar",
+    }, {
+      guid: PlacesUtils.bookmarks.unfiledGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 3,
+      title: "Other Bookmarks",
+    }, {
+      guid: PlacesUtils.bookmarks.mobileGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 4,
+      title: "mobile",
+    }],
+  }, "Should move C to closest surviving parent");
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
new file mode 100644
--- /dev/null
+++ b/toolkit/components/places/tests/sync/test_bookmark_kinds.js
@@ -0,0 +1,202 @@
+add_task(async function test_livemarks() {
+  let { site, stopServer } = makeLivemarkServer();
+
+  try {
+    let buf = await openMirror("livemarks");
+
+    do_print("Set up mirror");
+    await PlacesUtils.bookmarks.insertTree({
+      guid: PlacesUtils.bookmarks.menuGuid,
+      children: [{
+        guid: "livemarkAAAA",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        title: "A",
+        annos: [{
+          name: PlacesUtils.LMANNO_FEEDURI,
+          value: site + "/feed/a",
+        }],
+      }],
+    });
+    await buf.store(shuffle([{
+      id: "menu",
+      type: "folder",
+      title: "Bookmarks Menu",
+      children: ["livemarkAAAA"],
+    }, {
+      id: "livemarkAAAA",
+      type: "livemark",
+      title: "A",
+      feedUri: site + "/feed/a",
+    }]), { needsMerge: false });
+    await PlacesTestUtils.markBookmarksAsSynced();
+
+    do_print("Make local changes");
+    await PlacesUtils.livemarks.addLivemark({
+      parentGuid: PlacesUtils.bookmarks.toolbarGuid,
+      guid: "livemarkBBBB",
+      title: "B",
+      feedURI: Services.io.newURI(site + "/feed/b-local"),
+      siteURI: Services.io.newURI(site + "/site/b-local"),
+    });
+
+    do_print("Set up mirror");
+    await buf.store(shuffle([{
+      id: "livemarkAAAA",
+      type: "livemark",
+      title: "A (remote)",
+      feedUri: site + "/feed/a-remote",
+    }, {
+      id: "toolbar",
+      type: "folder",
+      title: "Bookmarks Toolbar",
+      children: ["livemarkCCCC", "livemarkB111"],
+    }, {
+      id: "livemarkCCCC",
+      type: "livemark",
+      title: "C (remote)",
+      feedUri: site + "/feed/c-remote",
+    }, {
+      id: "livemarkB111",
+      type: "livemark",
+      title: "B",
+      feedUri: site + "/feed/b-remote",
+    }]));
+
+    do_print("Apply mirror");
+    let changesToUpload = await buf.apply();
+    let idsToUpload = inspectChangeRecords(changesToUpload);
+    deepEqual(idsToUpload, {
+      updated: [],
+      deleted: [],
+    }, "Should not upload any local records");
+
+    await assertLocalTree(PlacesUtils.bookmarks.rootGuid, {
+      guid: PlacesUtils.bookmarks.rootGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 0,
+      title: "",
+      children: [{
+        guid: PlacesUtils.bookmarks.menuGuid,
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        index: 0,
+        title: "Bookmarks Menu",
+        children: [{
+          guid: "livemarkAAAA",
+          type: PlacesUtils.bookmarks.TYPE_FOLDER,
+          index: 0,
+          title: "A (remote)",
+          annos: [{
+            name: PlacesUtils.LMANNO_FEEDURI,
+            flags: 0,
+            expires: PlacesUtils.annotations.EXPIRE_NEVER,
+            value: site + "/feed/a-remote",
+          }],
+        }],
+      }, {
+        guid: PlacesUtils.bookmarks.toolbarGuid,
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        index: 1,
+        title: "Bookmarks Toolbar",
+        children: [{
+          guid: "livemarkCCCC",
+          type: PlacesUtils.bookmarks.TYPE_FOLDER,
+          index: 0,
+          title: "C (remote)",
+          annos: [{
+            name: PlacesUtils.LMANNO_FEEDURI,
+            flags: 0,
+            expires: PlacesUtils.annotations.EXPIRE_NEVER,
+            value: site + "/feed/c-remote",
+          }],
+        }, {
+          guid: "livemarkB111",
+          type: PlacesUtils.bookmarks.TYPE_FOLDER,
+          index: 1,
+          title: "B",
+          annos: [{
+            name: PlacesUtils.LMANNO_FEEDURI,
+            flags: 0,
+            expires: PlacesUtils.annotations.EXPIRE_NEVER,
+            value: site + "/feed/b-remote",
+          }],
+        }],
+      }, {
+        guid: PlacesUtils.bookmarks.unfiledGuid,
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        index: 3,
+        title: "Other Bookmarks",
+      }, {
+        guid: PlacesUtils.bookmarks.mobileGuid,
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        index: 4,
+        title: "mobile",
+      }],
+    }, "Should apply and dedupe livemarks");
+
+    let cLivemark = await PlacesUtils.livemarks.getLivemark({
+      guid: "livemarkCCCC",
+    });
+    equal(cLivemark.title, "C (remote)", "Should set livemark C title");
+    ok(cLivemark.feedURI.equals(Services.io.newURI(site + "/feed/c-remote")),
+      "Should set livemark C feed URL");
+
+    let bLivemark = await PlacesUtils.livemarks.getLivemark({
+      guid: "livemarkB111",
+    });
+    ok(bLivemark.feedURI.equals(Services.io.newURI(site + "/feed/b-remote")),
+      "Should set deduped livemark B feed URL");
+    strictEqual(bLivemark.siteURI, null,
+      "Should remove deduped livemark B site URL");
+
+    await buf.finalize();
+  } finally {
+    await stopServer();
+  }
+
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
+
+// Bug 632287.
+add_task(async function test_mismatched_types() {
+  let buf = await openMirror("mismatched_types");
+
+  do_print("Set up mirror");
+  await PlacesUtils.bookmarks.insertTree({
+    guid: PlacesUtils.bookmarks.menuGuid,
+    children: [{
+      guid: "l1nZZXfB8nC7",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      title: "Innerst i Sneglehode",
+    }],
+  });
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  do_print("Set up mirror");
+  await buf.store([{
+    "id": "l1nZZXfB8nC7",
+    "type": "livemark",
+    "siteUri": "http://sneglehode.wordpress.com/",
+    "feedUri": "http://sneglehode.wordpress.com/feed/",
+    "parentName": "Bookmarks Toolbar",
+    "title": "Innerst i Sneglehode",
+    "description": null,
+    "children":
+      ["HCRq40Rnxhrd", "YeyWCV1RVsYw", "GCceVZMhvMbP", "sYi2hevdArlF",
+       "vjbZlPlSyGY8", "UtjUhVyrpeG6", "rVq8WMG2wfZI", "Lx0tcy43ZKhZ",
+       "oT74WwV8_j4P", "IztsItWVSo3-"],
+    "parentid": "toolbar"
+  }]);
+
+  do_print("Apply mirror");
+  let changesToUpload = await buf.apply();
+  let idsToUpload = inspectChangeRecords(changesToUpload);
+  deepEqual(idsToUpload, {
+    updated: [],
+    deleted: [],
+  }, "Should not reupload merged livemark");
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
new file mode 100644
--- /dev/null
+++ b/toolkit/components/places/tests/sync/test_bookmark_structure_changes.js
@@ -0,0 +1,567 @@
+add_task(async function test_value_structure_conflict() {
+  let buf = await openMirror("value_structure_conflict");
+
+  do_print("Set up mirror");
+  await PlacesUtils.bookmarks.insertTree({
+    guid: PlacesUtils.bookmarks.menuGuid,
+    children: [{
+      guid: "folderAAAAAA",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      title: "A",
+      children: [{
+        guid: "bookmarkBBBB",
+        url: "http://example.com/b",
+        title: "B",
+      }, {
+        guid: "bookmarkCCCC",
+        url: "http://example.com/c",
+        title: "C",
+      }],
+    }, {
+      guid: "folderDDDDDD",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      title: "D",
+      children: [{
+        guid: "bookmarkEEEE",
+        url: "http://example.com/e",
+        title: "E",
+      }],
+    }],
+  });
+  await buf.store(shuffle([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: ["folderAAAAAA", "folderDDDDDD"],
+    modified: Date.now() / 1000 - 60,
+  }, {
+    id: "folderAAAAAA",
+    type: "folder",
+    title: "A",
+    children: ["bookmarkBBBB", "bookmarkCCCC"],
+    modified: Date.now() / 1000 - 60,
+  }, {
+    id: "bookmarkBBBB",
+    type: "bookmark",
+    title: "B",
+    bmkUri: "http://example.com/b",
+    modified: Date.now() / 1000 - 60,
+  }, {
+    id: "bookmarkCCCC",
+    type: "bookmark",
+    title: "C",
+    bmkUri: "http://example.com/c",
+    modified: Date.now() / 1000 - 60,
+  }, {
+    id: "folderDDDDDD",
+    type: "folder",
+    title: "D",
+    children: ["bookmarkEEEE"],
+    modified: Date.now() / 1000 - 60,
+  }, {
+    id: "bookmarkEEEE",
+    type: "bookmark",
+    title: "E",
+    bmkUri: "http://example.com/e",
+    modified: Date.now() / 1000 - 60,
+  }]), { needsMerge: false });
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  do_print("Make local value change");
+  await PlacesUtils.bookmarks.update({
+    guid: "folderAAAAAA",
+    title: "A (local)",
+  });
+
+  do_print("Make local structure change");
+  await PlacesUtils.bookmarks.update({
+    guid: "bookmarkBBBB",
+    parentGuid: "folderDDDDDD",
+    index: 0,
+  });
+
+  do_print("Make remote value change");
+  await buf.store([{
+    id: "folderDDDDDD",
+    type: "folder",
+    title: "D (remote)",
+    children: ["bookmarkEEEE"],
+    modified: Date.now() / 1000 + 60,
+  }]);
+
+  do_print("Apply mirror");
+  let changesToUpload = await buf.apply({
+    remoteTimeSeconds: Date.now() / 1000,
+  });
+  let idsToUpload = inspectChangeRecords(changesToUpload);
+  deepEqual(idsToUpload, {
+    updated: ["bookmarkBBBB", "folderAAAAAA", "folderDDDDDD"],
+    deleted: [],
+  }, "Should upload records for merged and new local items");
+
+  await assertLocalTree(PlacesUtils.bookmarks.menuGuid, {
+    guid: PlacesUtils.bookmarks.menuGuid,
+    type: PlacesUtils.bookmarks.TYPE_FOLDER,
+    index: 0,
+    title: "Bookmarks Menu",
+    children: [{
+      guid: "folderAAAAAA",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 0,
+      title: "A (local)",
+      children: [{
+        guid: "bookmarkCCCC",
+        type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+        index: 0,
+        title: "C",
+        url: "http://example.com/c",
+      }],
+    }, {
+      guid: "folderDDDDDD",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 1,
+      title: "D (remote)",
+      children: [{
+        guid: "bookmarkEEEE",
+        type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+        index: 0,
+        title: "E",
+        url: "http://example.com/e",
+      }, {
+        guid: "bookmarkBBBB",
+        type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+        index: 1,
+        title: "B",
+        url: "http://example.com/b",
+      }],
+    }],
+  }, "Should reconcile structure and value changes");
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
+
+add_task(async function test_move() {
+  let buf = await openMirror("move");
+
+  await PlacesUtils.bookmarks.insertTree({
+    guid: PlacesUtils.bookmarks.menuGuid,
+    children: [{
+      guid: "devFolder___",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      title: "Dev",
+      children: [{
+        guid: "mdnBmk______",
+        title: "MDN",
+        url: "https://developer.mozilla.org",
+      }, {
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        guid: "mozFolder___",
+        title: "Mozilla",
+        children: [{
+          guid: "fxBmk_______",
+          title: "Get Firefox!",
+          url: "http://getfirefox.com/",
+        }, {
+          guid: "nightlyBmk__",
+          title: "Nightly",
+          url: "https://nightly.mozilla.org",
+        }],
+      }, {
+        guid: "wmBmk_______",
+        title: "Webmaker",
+        url: "https://webmaker.org",
+      }],
+    }, {
+      guid: "bzBmk_______",
+      title: "Bugzilla",
+      url: "https://bugzilla.mozilla.org",
+    }]
+  });
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  await buf.store(shuffle([{
+    id: "unfiled",
+    type: "folder",
+    title: "Other Bookmarks",
+    children: ["mozFolder___"],
+  }, {
+    id: "toolbar",
+    type: "folder",
+    title: "Bookmarks Toolbar",
+    children: ["devFolder___"],
+  }, {
+    id: "devFolder___",
+    // Moving to toolbar.
+    type: "folder",
+    title: "Dev",
+    children: ["bzBmk_______", "wmBmk_______"],
+  }, {
+    // Moving to "Mozilla".
+    id: "mdnBmk______",
+    type: "bookmark",
+    title: "MDN",
+    bmkUri: "https://developer.mozilla.org",
+  }, {
+    // Rearranging children and moving to unfiled.
+    id: "mozFolder___",
+    type: "folder",
+    title: "Mozilla",
+    children: ["nightlyBmk__", "mdnBmk______", "fxBmk_______"],
+  }, {
+    id: "fxBmk_______",
+    type: "bookmark",
+    title: "Get Firefox!",
+    bmkUri: "http://getfirefox.com/",
+  }, {
+    id: "nightlyBmk__",
+    type: "bookmark",
+    title: "Nightly",
+    bmkUri: "https://nightly.mozilla.org",
+  }, {
+    id: "wmBmk_______",
+    type: "bookmark",
+    title: "Webmaker",
+    bmkUri: "https://webmaker.org",
+  }, {
+    id: "bzBmk_______",
+    type: "bookmark",
+    title: "Bugzilla",
+    bmkUri: "https://bugzilla.mozilla.org",
+  }]));
+
+  do_print("Apply mirror");
+  let changesToUpload = await buf.apply();
+  let idsToUpload = inspectChangeRecords(changesToUpload);
+  deepEqual(idsToUpload, {
+    updated: [],
+    deleted: [],
+  }, "Should not upload records for remotely moved items");
+
+  await assertLocalTree(PlacesUtils.bookmarks.rootGuid, {
+    guid: PlacesUtils.bookmarks.rootGuid,
+    type: PlacesUtils.bookmarks.TYPE_FOLDER,
+    index: 0,
+    title: "",
+    children: [{
+      guid: PlacesUtils.bookmarks.menuGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 0,
+      title: "Bookmarks Menu",
+    }, {
+      guid: PlacesUtils.bookmarks.toolbarGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 1,
+      title: "Bookmarks Toolbar",
+      children: [{
+        guid: "devFolder___",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        index: 0,
+        title: "Dev",
+        children: [{
+          guid: "bzBmk_______",
+          type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+          index: 0,
+          title: "Bugzilla",
+          url: "https://bugzilla.mozilla.org/",
+        }, {
+          guid: "wmBmk_______",
+          type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+          index: 1,
+          title: "Webmaker",
+          url: "https://webmaker.org/",
+        }],
+      }],
+    }, {
+      guid: PlacesUtils.bookmarks.unfiledGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 3,
+      title: "Other Bookmarks",
+      children: [{
+        guid: "mozFolder___",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        index: 0,
+        title: "Mozilla",
+        children: [{
+          guid: "nightlyBmk__",
+          type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+          index: 0,
+          title: "Nightly",
+          url: "https://nightly.mozilla.org/",
+        }, {
+          guid: "mdnBmk______",
+          type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+          index: 1,
+          title: "MDN",
+          url: "https://developer.mozilla.org/",
+        }, {
+          guid: "fxBmk_______",
+          type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+          index: 2,
+          title: "Get Firefox!",
+          url: "http://getfirefox.com/",
+        }],
+      }],
+    }, {
+      guid: PlacesUtils.bookmarks.mobileGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 4,
+      title: "mobile",
+    }],
+  }, "Should move and reorder bookmarks to match remote");
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
+
+add_task(async function test_move_into_parent_sibling() {
+  // This test moves a bookmark that exists locally into a new folder that only
+  // exists remotely, and is a later sibling of the local parent. This ensures
+  // we set up the local structure before applying structure changes.
+  let buf = await openMirror("move_into_parent_sibling");
+
+  do_print("Set up mirror: Menu > A > B");
+  await PlacesUtils.bookmarks.insertTree({
+    guid: PlacesUtils.bookmarks.menuGuid,
+    children: [{
+      guid: "folderAAAAAA",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      title: "A",
+      children: [{
+        guid: "bookmarkBBBB",
+        url: "http://example.com/b",
+        title: "B",
+      }],
+    }],
+  });
+  await buf.store(shuffle([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: ["folderAAAAAA"],
+  }, {
+    id: "folderAAAAAA",
+    type: "folder",
+    title: "A",
+    children: ["bookmarkBBBB"],
+  }, {
+    id: "bookmarkBBBB",
+    type: "bookmark",
+    title: "B",
+    bmkUri: "http://example.com/b",
+  }]), { needsMerge: false });
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  do_print("Set up mirror: Menu > (A (B > C))");
+  await buf.store([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: ["folderAAAAAA", "folderCCCCCC"],
+  }, {
+    id: "folderAAAAAA",
+    type: "folder",
+    title: "A",
+  }, {
+    id: "folderCCCCCC",
+    type: "folder",
+    title: "C",
+    children: ["bookmarkBBBB"],
+  }]);
+
+  do_print("Apply mirror");
+  let changesToUpload = await buf.apply();
+  let idsToUpload = inspectChangeRecords(changesToUpload);
+  deepEqual(idsToUpload, {
+    updated: [],
+    deleted: [],
+  }, "Should not upload records for remote-only structure changes");
+
+  await assertLocalTree(PlacesUtils.bookmarks.menuGuid, {
+    guid: PlacesUtils.bookmarks.menuGuid,
+    type: PlacesUtils.bookmarks.TYPE_FOLDER,
+    index: 0,
+    title: "Bookmarks Menu",
+    children: [{
+      guid: "folderAAAAAA",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 0,
+      title: "A",
+    }, {
+      guid: "folderCCCCCC",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 1,
+      title: "C",
+      children: [{
+        guid: "bookmarkBBBB",
+        type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+        index: 0,
+        title: "B",
+        url: "http://example.com/b",
+      }],
+    }],
+  }, "Should set up local structure correctly");
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
+
+add_task(async function test_complex_move_with_additions() {
+  let buf = await openMirror("complex_move_with_additions");
+
+  do_print("Set up local and remote mirrors: Menu > A > (B C)");
+  await PlacesUtils.bookmarks.insertTree({
+    guid: PlacesUtils.bookmarks.menuGuid,
+    children: [{
+      guid: "folderAAAAAA",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      title: "A",
+      children: [{
+        guid: "bookmarkBBBB",
+        url: "http://example.com/b",
+        title: "B",
+      }, {
+        guid: "bookmarkCCCC",
+        url: "http://example.com/c",
+        title: "C",
+      }],
+    }],
+  });
+  await buf.store(shuffle([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: ["folderAAAAAA"],
+  }, {
+    id: "folderAAAAAA",
+    type: "folder",
+    title: "A",
+    children: ["bookmarkBBBB", "bookmarkCCCC"],
+  }, {
+    id: "bookmarkBBBB",
+    type: "bookmark",
+    title: "B",
+    bmkUri: "http://example.com/b",
+  }, {
+    id: "bookmarkCCCC",
+    type: "bookmark",
+    title: "C",
+    bmkUri: "http://example.com/c",
+  }]), { needsMerge: false });
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  do_print("Make local change: Menu > A > (B C D)");
+  await PlacesUtils.bookmarks.insert({
+    guid: "bookmarkDDDD",
+    parentGuid: "folderAAAAAA",
+    title: "D (local)",
+    url: "http://example.com/d-local",
+  });
+
+  do_print("Set up mirror: ((Menu > C) (Toolbar > A > (B E)))");
+  await buf.store(shuffle([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: ["bookmarkCCCC"],
+  }, {
+    id: "toolbar",
+    type: "folder",
+    title: "Bookmarks Toolbar",
+    children: ["folderAAAAAA"],
+  }, {
+    id: "folderAAAAAA",
+    type: "folder",
+    title: "A",
+    children: ["bookmarkBBBB", "bookmarkEEEE"],
+  }, {
+    id: "bookmarkCCCC",
+    type: "bookmark",
+    title: "C",
+    bmkUri: "http://example.com/c",
+  }, {
+    id: "bookmarkEEEE",
+    type: "bookmark",
+    title: "E",
+    bmkUri: "http://example.com/e",
+  }]));
+
+  do_print("Apply mirror");
+  let changesToUpload = await buf.apply();
+  let idsToUpload = inspectChangeRecords(changesToUpload);
+  deepEqual(idsToUpload, {
+    updated: ["bookmarkDDDD", "folderAAAAAA"],
+    deleted: [],
+  }, "Should upload new records for (A D)");
+
+  await assertLocalTree(PlacesUtils.bookmarks.rootGuid, {
+    guid: PlacesUtils.bookmarks.rootGuid,
+    type: PlacesUtils.bookmarks.TYPE_FOLDER,
+    index: 0,
+    title: "",
+    children: [{
+      guid: PlacesUtils.bookmarks.menuGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 0,
+      title: "Bookmarks Menu",
+      children: [{
+        guid: "bookmarkCCCC",
+        type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+        index: 0,
+        title: "C",
+        url: "http://example.com/c",
+      }],
+    }, {
+      guid: PlacesUtils.bookmarks.toolbarGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 1,
+      title: "Bookmarks Toolbar",
+      children: [{
+        // We can guarantee child order (B E D), since we always walk remote
+        // children first, and the remote folder A record is newer than the
+        // local folder. If the local folder were newer, the order would be
+        // (D B E).
+        guid: "folderAAAAAA",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        index: 0,
+        title: "A",
+        children: [{
+          guid: "bookmarkBBBB",
+          type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+          index: 0,
+          title: "B",
+          url: "http://example.com/b",
+        }, {
+          guid: "bookmarkEEEE",
+          type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+          index: 1,
+          title: "E",
+          url: "http://example.com/e",
+        }, {
+          guid: "bookmarkDDDD",
+          type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+          index: 2,
+          title: "D (local)",
+          url: "http://example.com/d-local",
+        }],
+      }],
+    }, {
+      guid: PlacesUtils.bookmarks.unfiledGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 3,
+      title: "Other Bookmarks",
+    }, {
+      guid: PlacesUtils.bookmarks.mobileGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 4,
+      title: "mobile",
+    }],
+  }, "Should take remote order and preserve local children");
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
new file mode 100644
--- /dev/null
+++ b/toolkit/components/places/tests/sync/test_bookmark_value_changes.js
@@ -0,0 +1,894 @@
+add_task(async function test_value_combo() {
+  let buf = await openMirror("value_combo");
+
+  do_print("Set up mirror with existing bookmark to update");
+  await PlacesUtils.bookmarks.insertTree({
+    guid: PlacesUtils.bookmarks.menuGuid,
+    children: [{
+      guid: "mozBmk______",
+      url: "https://mozilla.org",
+      title: "Mozilla",
+      tags: ["moz", "dot", "org"],
+    }],
+  });
+  await buf.store(shuffle([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: ["mozBmk______"],
+  }, {
+    id: "mozBmk______",
+    type: "bookmark",
+    title: "Mozilla",
+    bmkUri: "https://mozilla.org",
+    tags: ["moz", "dot", "org"],
+  }]), { needsMerge: false });
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  do_print("Insert new bookmark to upload");
+  await PlacesUtils.bookmarks.insertTree({
+    guid: PlacesUtils.bookmarks.toolbarGuid,
+    children: [{
+      guid: "bzBmk_______",
+      url: "https://bugzilla.mozilla.org",
+      title: "Bugzilla",
+      tags: ["new", "tag"],
+    }],
+  });
+
+  do_print("Insert bookmarks and folder into mirror");
+  await buf.store(shuffle([{
+    id: "mozBmk______",
+    type: "bookmark",
+    title: "Mozilla home page",
+    bmkUri: "https://mozilla.org",
+    tags: ["browsers"],
+  }, {
+    id: "toolbar",
+    type: "folder",
+    title: "Bookmarks Toolbar",
+    children: ["fxBmk_______", "tFolder_____"],
+  }, {
+    id: "fxBmk_______",
+    type: "bookmark",
+    title: "Get Firefox",
+    bmkUri: "http://getfirefox.com",
+    tags: ["taggy", "browsers"],
+  }, {
+    id: "tFolder_____",
+    type: "folder",
+    title: "Mail",
+    children: ["tbBmk_______"],
+  }, {
+    id: "tbBmk_______",
+    type: "bookmark",
+    title: "Get Thunderbird",
+    bmkUri: "http://getthunderbird.com",
+    keyword: "tb",
+  }]));
+
+  do_print("Apply mirror");
+  let changesToUpload = await buf.apply();
+  let idsToUpload = inspectChangeRecords(changesToUpload);
+  deepEqual(idsToUpload, {
+    updated: ["bzBmk_______", "toolbar"],
+    deleted: [],
+  }, "Should upload new local bookmarks and parents");
+
+  let fxBmk = await PlacesUtils.bookmarks.fetch("fxBmk_______");
+  ok(fxBmk, "New Firefox bookmark should exist");
+  equal(fxBmk.parentGuid, PlacesUtils.bookmarks.toolbarGuid,
+    "Should add Firefox bookmark to toolbar");
+  let fxTags = PlacesUtils.tagging.getTagsForURI(
+    Services.io.newURI("http://getfirefox.com"));
+  deepEqual(fxTags.sort(), ["browsers", "taggy"],
+    "Should tag new Firefox bookmark");
+
+  let folder = await PlacesUtils.bookmarks.fetch("tFolder_____");
+  ok(folder, "New folder should exist");
+  equal(folder.parentGuid, PlacesUtils.bookmarks.toolbarGuid,
+    "Should add new folder to toolbar");
+
+  let tbBmk = await PlacesUtils.bookmarks.fetch("tbBmk_______");
+  ok(tbBmk, "Should insert Thunderbird child bookmark");
+  equal(tbBmk.parentGuid, folder.guid,
+    "Should add Thunderbird bookmark to new folder");
+  let keywordInfo = await PlacesUtils.keywords.fetch("tb");
+  equal(keywordInfo.url.href, "http://getthunderbird.com/",
+    "Should set keyword for Thunderbird bookmark");
+
+  let updatedBmk = await PlacesUtils.bookmarks.fetch("mozBmk______");
+  equal(updatedBmk.title, "Mozilla home page",
+    "Should rename Mozilla bookmark");
+  equal(updatedBmk.parentGuid, PlacesUtils.bookmarks.menuGuid,
+    "Should not move Mozilla bookmark");
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
+
+add_task(async function test_value_only_changes() {
+  let buf = await openMirror("value_only_changes");
+
+  do_print("Set up mirror");
+  await PlacesUtils.bookmarks.insertTree({
+    guid: PlacesUtils.bookmarks.menuGuid,
+    children: [{
+      guid: "folderAAAAAA",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      title: "A",
+      children: [{
+        guid: "bookmarkBBBB",
+        url: "http://example.com/b",
+        title: "B",
+      }, {
+        guid: "bookmarkCCCC",
+        url: "http://example.com/c",
+        title: "C",
+      }, {
+        guid: "folderJJJJJJ",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        title: "J",
+        children: [{
+          guid: "bookmarkKKKK",
+          url: "http://example.com/k",
+          title: "K",
+        }],
+      }, {
+        guid: "bookmarkDDDD",
+        url: "http://example.com/d",
+        title: "D",
+      }, {
+        guid: "bookmarkEEEE",
+        url: "http://example.com/e",
+        title: "E",
+      }],
+    }, {
+      guid: "folderFFFFFF",
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      title: "F",
+      children: [{
+        guid: "bookmarkGGGG",
+        url: "http://example.com/g",
+        title: "G",
+      }, {
+        guid: "folderHHHHHH",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        title: "H",
+        children: [{
+          guid: "bookmarkIIII",
+          url: "http://example.com/i",
+          title: "I",
+        }],
+      }],
+    }],
+  });
+  await buf.store(shuffle([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: ["folderAAAAAA", "folderFFFFFF"],
+  }, {
+    id: "folderAAAAAA",
+    type: "folder",
+    title: "A",
+    children: ["bookmarkBBBB", "bookmarkCCCC", "folderJJJJJJ", "bookmarkDDDD",
+               "bookmarkEEEE"],
+  }, {
+    id: "bookmarkBBBB",
+    type: "bookmark",
+    title: "B",
+    bmkUri: "http://example.com/b",
+  }, {
+    id: "bookmarkCCCC",
+    type: "bookmark",
+    title: "C",
+    bmkUri: "http://example.com/c",
+  }, {
+    id: "folderJJJJJJ",
+    type: "folder",
+    title: "J",
+    children: ["bookmarkKKKK"],
+  }, {
+    id: "bookmarkKKKK",
+    type: "bookmark",
+    title: "K",
+    bmkUri: "http://example.com/k",
+  }, {
+    id: "bookmarkDDDD",
+    type: "bookmark",
+    title: "D",
+    bmkUri: "http://example.com/d",
+  }, {
+    id: "bookmarkEEEE",
+    type: "bookmark",
+    title: "E",
+    bmkUri: "http://example.com/e",
+  }, {
+    id: "folderFFFFFF",
+    type: "folder",
+    title: "F",
+    children: ["bookmarkGGGG", "folderHHHHHH"],
+  }, {
+    id: "bookmarkGGGG",
+    type: "bookmark",
+    title: "G",
+    bmkUri: "http://example.com/g",
+  }, {
+    id: "folderHHHHHH",
+    type: "folder",
+    title: "H",
+    children: ["bookmarkIIII"],
+  }, {
+    id: "bookmarkIIII",
+    type: "bookmark",
+    title: "I",
+    bmkUri: "http://example.com/i",
+  }]), { needsMerge: false });
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  do_print("Set up mirror");
+  await buf.store(shuffle([{
+    id: "bookmarkCCCC",
+    type: "bookmark",
+    title: "C (remote)",
+    bmkUri: "http://example.com/c-remote",
+  }, {
+    id: "bookmarkEEEE",
+    type: "bookmark",
+    title: "E (remote)",
+    bmkUri: "http://example.com/e-remote",
+  }, {
+    id: "bookmarkIIII",
+    type: "bookmark",
+    title: "I (remote)",
+    bmkUri: "http://example.com/i-remote",
+  }, {
+    id: "folderFFFFFF",
+    type: "folder",
+    title: "F (remote)",
+    children: ["bookmarkGGGG", "folderHHHHHH"],
+  }]));
+
+  do_print("Apply mirror");
+  let changesToUpload = await buf.apply();
+  let idsToUpload = inspectChangeRecords(changesToUpload);
+  deepEqual(idsToUpload, {
+    updated: [],
+    deleted: [],
+  }, "Should not upload records for remote-only value changes");
+
+  await assertLocalTree(PlacesUtils.bookmarks.rootGuid, {
+    guid: PlacesUtils.bookmarks.rootGuid,
+    type: PlacesUtils.bookmarks.TYPE_FOLDER,
+    index: 0,
+    title: "",
+    children: [{
+      guid: PlacesUtils.bookmarks.menuGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 0,
+      title: "Bookmarks Menu",
+      children: [{
+        guid: "folderAAAAAA",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        index: 0,
+        title: "A",
+        children: [{
+          guid: "bookmarkBBBB",
+          type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+          index: 0,
+          title: "B",
+          url: "http://example.com/b",
+        }, {
+          guid: "bookmarkCCCC",
+          type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+          index: 1,
+          title: "C (remote)",
+          url: "http://example.com/c-remote",
+        }, {
+          guid: "folderJJJJJJ",
+          type: PlacesUtils.bookmarks.TYPE_FOLDER,
+          index: 2,
+          title: "J",
+          children: [{
+            guid: "bookmarkKKKK",
+            type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+            index: 0,
+            title: "K",
+            url: "http://example.com/k",
+          }],
+        }, {
+          guid: "bookmarkDDDD",
+          type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+          index: 3,
+          title: "D",
+          url: "http://example.com/d",
+        }, {
+          guid: "bookmarkEEEE",
+          type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+          index: 4,
+          title: "E (remote)",
+          url: "http://example.com/e-remote",
+        }],
+      }, {
+        guid: "folderFFFFFF",
+        type: PlacesUtils.bookmarks.TYPE_FOLDER,
+        index: 1,
+        title: "F (remote)",
+        children: [{
+          guid: "bookmarkGGGG",
+          type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+          index: 0,
+          title: "G",
+          url: "http://example.com/g",
+        }, {
+          guid: "folderHHHHHH",
+          type: PlacesUtils.bookmarks.TYPE_FOLDER,
+          index: 1,
+          title: "H",
+          children: [{
+            guid: "bookmarkIIII",
+            type: PlacesUtils.bookmarks.TYPE_BOOKMARK,
+            index: 0,
+            title: "I (remote)",
+            url: "http://example.com/i-remote",
+          }],
+        }],
+      }],
+    }, {
+      guid: PlacesUtils.bookmarks.toolbarGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 1,
+      title: "Bookmarks Toolbar",
+    }, {
+      guid: PlacesUtils.bookmarks.unfiledGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 3,
+      title: "Other Bookmarks",
+    }, {
+      guid: PlacesUtils.bookmarks.mobileGuid,
+      type: PlacesUtils.bookmarks.TYPE_FOLDER,
+      index: 4,
+      title: "mobile",
+    }],
+  }, "Should not change structure for value-only changes");
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
+
+add_task(async function test_keywords() {
+  let buf = await openMirror("keywords");
+
+  do_print("Set up mirror");
+  await PlacesUtils.bookmarks.insertTree({
+    guid: PlacesUtils.bookmarks.menuGuid,
+    children: [{
+      guid: "bookmarkAAAA",
+      title: "A",
+      url: "http://example.com/a",
+      keyword: "one",
+    }, {
+      guid: "bookmarkBBBB",
+      title: "B",
+      url: "http://example.com/b",
+      keyword: "two",
+    }, {
+      guid: "bookmarkCCCC",
+      title: "C",
+      url: "http://example.com/c",
+    }, {
+      guid: "bookmarkDDDD",
+      title: "D",
+      url: "http://example.com/d",
+      keyword: "three",
+    }],
+  });
+  await buf.store(shuffle([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: ["bookmarkAAAA", "bookmarkBBBB", "bookmarkCCCC", "bookmarkDDDD"],
+  }, {
+    id: "bookmarkAAAA",
+    type: "bookmark",
+    title: "A",
+    bmkUri: "http://example.com/a",
+    keyword: "one",
+  }, {
+    id: "bookmarkBBBB",
+    type: "bookmark",
+    title: "B",
+    bmkUri: "http://example.com/b",
+    keyword: "two",
+  }, {
+    id: "bookmarkCCCC",
+    type: "bookmark",
+    title: "C",
+    bmkUri: "http://example.com/c",
+  }, {
+    id: "bookmarkDDDD",
+    type: "bookmark",
+    title: "D",
+    bmkUri: "http://example.com/d",
+    keyword: "three",
+  }]), { needsMerge: false });
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  do_print("Change keywords remotely");
+  await buf.store(shuffle([{
+    id: "bookmarkAAAA",
+    type: "bookmark",
+    title: "A",
+    bmkUri: "http://example.com/a",
+    keyword: "two",
+  }, {
+    id: "bookmarkBBBB",
+    type: "bookmark",
+    title: "B",
+    bmkUri: "http://example.com/b",
+  }]));
+
+  do_print("Change keywords locally");
+  await PlacesUtils.keywords.insert({
+    keyword: "four",
+    url: "http://example.com/c",
+  });
+  await PlacesUtils.keywords.remove("three");
+
+  do_print("Apply mirror");
+  let changesToUpload = await buf.apply();
+  let idsToUpload = inspectChangeRecords(changesToUpload);
+  deepEqual(idsToUpload, {
+    updated: ["bookmarkAAAA", "bookmarkCCCC", "bookmarkDDDD"],
+    deleted: [],
+  }, "Should reupload all local records with changed keywords");
+
+  let entryForOne = await PlacesUtils.keywords.fetch("one");
+  ok(!entryForOne, "Should remove existing keyword from A");
+
+  let entriesForTwo = await fetchAllKeywords("two");
+  deepEqual(entriesForTwo.map(entry => ({
+    keyword: entry.keyword,
+    url: entry.url.href,
+  })), [{
+    keyword: "two",
+    url: "http://example.com/a",
+  }], "Should move keyword for B to A");
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
+
+add_task(async function test_keywords_complex() {
+  let buf = await openMirror("keywords_complex");
+
+  do_print("Set up mirror");
+  await PlacesUtils.bookmarks.insertTree({
+    guid: PlacesUtils.bookmarks.menuGuid,
+    children: [{
+      guid: "bookmarkBBBB",
+      title: "B",
+      url: "http://example.com/b",
+      keyword: "four",
+    }, {
+      guid: "bookmarkCCCC",
+      title: "C",
+      url: "http://example.com/c",
+      keyword: "five",
+    }, {
+      guid: "bookmarkDDDD",
+      title: "D",
+      url: "http://example.com/d",
+    }, {
+      guid: "bookmarkEEEE",
+      title: "E",
+      url: "http://example.com/e",
+      keyword: "three",
+    }],
+  });
+  await buf.store(shuffle([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: ["bookmarkBBBB", "bookmarkCCCC", "bookmarkDDDD", "bookmarkEEEE"],
+  }, {
+    id: "bookmarkBBBB",
+    type: "bookmark",
+    title: "B",
+    bmkUri: "http://example.com/b",
+    keyword: "four",
+  }, {
+    id: "bookmarkCCCC",
+    type: "bookmark",
+    title: "C",
+    bmkUri: "http://example.com/c",
+    keyword: "five",
+  }, {
+    id: "bookmarkDDDD",
+    type: "bookmark",
+    title: "D",
+    bmkUri: "http://example.com/d",
+  }, {
+    id: "bookmarkEEEE",
+    type: "bookmark",
+    title: "E",
+    bmkUri: "http://example.com/e",
+    keyword: "three",
+  }]), { needsMerge: false });
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  await buf.store(shuffle([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: ["bookmarkAAAA", "bookmarkAAA1", "bookmarkBBBB", "bookmarkCCCC",
+               "bookmarkDDDD", "bookmarkEEEE"],
+  }, {
+    id: "bookmarkAAAA",
+    type: "bookmark",
+    title: "A",
+    bmkUri: "http://example.com/a",
+    keyword: "one",
+  }, {
+    id: "bookmarkAAA1",
+    type: "bookmark",
+    title: "A (copy)",
+    bmkUri: "http://example.com/a",
+    keyword: "two",
+  }, {
+    id: "bookmarkBBB1",
+    type: "bookmark",
+    title: "B",
+    bmkUri: "http://example.com/b",
+  }, {
+    id: "bookmarkCCCC",
+    type: "bookmark",
+    title: "C (remote)",
+    bmkUri: "http://example.com/c-remote",
+    keyword: "six",
+  }]));
+
+  do_print("Apply mirror");
+  let changesToUpload = await buf.apply();
+  let idsToUpload = inspectChangeRecords(changesToUpload);
+  let expectedIdsToUpload = {
+    updated: ["bookmarkBBBB", "bookmarkCCCC"],
+    deleted: [],
+  };
+
+  // We'll take the keyword of either "bookmarkAAAA" or "bookmarkAAA1",
+  // depending on which we see first, and reupload the other.
+  let entriesForOne = await fetchAllKeywords("one");
+  let entriesForTwo = await fetchAllKeywords("two");
+  if (entriesForOne.length) {
+    expectedIdsToUpload.updated.push("bookmarkAAA1");
+    ok(!entriesForTwo.length, "Should drop conflicting keyword from A1");
+    deepEqual(entriesForOne.map(keyword => keyword.url.href),
+      ["http://example.com/a"], "Should use A keyword for A and A1");
+  } else {
+    expectedIdsToUpload.updated.push("bookmarkAAAA");
+    ok(!entriesForOne.length, "Should drop conflicting keyword from A");
+    deepEqual(entriesForTwo.map(keyword => keyword.url.href),
+      ["http://example.com/a"], "Should use A1 keyword for A and A1");
+  }
+  expectedIdsToUpload.updated.sort();
+  deepEqual(idsToUpload, expectedIdsToUpload,
+    "Should reupload all local records with corrected keywords");
+
+  let entriesForFour = await fetchAllKeywords("four");
+  ok(!entriesForFour.length, "Should remove all keywords for B");
+
+  let entriesForOldC = await fetchAllKeywords({
+    url: "http://example.com/c",
+  });
+  ok(!entriesForOldC.length, "Should remove all keywords from old C URL");
+  let entriesForNewC = await fetchAllKeywords({
+    url: "http://example.com/c-remote",
+  });
+  deepEqual(entriesForNewC.map(entry => entry.keyword), ["six"],
+    "Should add new keyword to new C URL");
+
+  let entriesForD = await fetchAllKeywords("http://example.com/d");
+  ok(!entriesForD.length, "Should not add keywords to D");
+
+  let entriesForThree = await fetchAllKeywords("three");
+  deepEqual(entriesForThree.map(keyword => keyword.url.href),
+    ["http://example.com/e"], "Should not change keywords for E");
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
+
+add_task(async function test_tags() {
+  let buf = await openMirror("tags");
+
+  do_print("Set up mirror");
+  await PlacesUtils.bookmarks.insertTree({
+    guid: PlacesUtils.bookmarks.menuGuid,
+    children: [{
+      guid: "bookmarkAAAA",
+      title: "A",
+      url: "http://example.com/a",
+      tags: ["one", "two", "three", "four"],
+    }, {
+      guid: "bookmarkBBBB",
+      title: "B",
+      url: "http://example.com/b",
+      tags: ["five", "six"],
+    }, {
+      guid: "bookmarkCCCC",
+      title: "C",
+      url: "http://example.com/c",
+    }, {
+      guid: "bookmarkDDDD",
+      title: "D",
+      url: "http://example.com/d",
+      tags: ["seven", "eight", "nine"],
+    }],
+  });
+  await buf.store(shuffle([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: ["bookmarkAAAA", "bookmarkBBBB", "bookmarkCCCC", "bookmarkDDDD"],
+  }, {
+    id: "bookmarkAAAA",
+    type: "bookmark",
+    title: "A",
+    bmkUri: "http://example.com/a",
+    tags: ["one", "two", "three", "four"],
+  }, {
+    id: "bookmarkBBBB",
+    type: "bookmark",
+    title: "B",
+    bmkUri: "http://example.com/b",
+    tags: ["five", "six"],
+  }, {
+    id: "bookmarkCCCC",
+    type: "bookmark",
+    title: "C",
+    bmkUri: "http://example.com/c",
+  }, {
+    id: "bookmarkDDDD",
+    type: "bookmark",
+    title: "D",
+    bmkUri: "http://example.com/d",
+    tags: ["seven", "eight", "nine"],
+  }]), { needsMerge: false });
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  do_print("Change tags remotely");
+  await buf.store(shuffle([{
+    id: "bookmarkAAAA",
+    type: "bookmark",
+    title: "A",
+    bmkUri: "http://example.com/a",
+    tags: ["one", "two", "ten"],
+  }, {
+    id: "bookmarkBBBB",
+    type: "bookmark",
+    title: "B",
+    bmkUri: "http://example.com/b",
+    tags: [],
+  }]));
+
+  do_print("Change tags locally");
+  PlacesUtils.tagging.tagURI(Services.io.newURI(
+    "http://example.com/c"), ["eleven", "twelve"]);
+  PlacesUtils.tagging.untagURI(Services.io.newURI(
+    "http://example.com/d"), null);
+
+  do_print("Apply mirror");
+  let changesToUpload = await buf.apply();
+  let idsToUpload = inspectChangeRecords(changesToUpload);
+  deepEqual(idsToUpload, {
+    updated: ["bookmarkCCCC", "bookmarkDDDD"],
+    deleted: [],
+  }, "Should upload local records with new tags");
+
+  deepEqual(changesToUpload.bookmarkCCCC.cleartext.tags.sort(),
+    ["eleven", "twelve"], "Should upload record with new tags for C");
+  ok(!changesToUpload.bookmarkDDDD.cleartext.tags,
+    "Should upload record for D with tags removed");
+
+  let tagsForA = PlacesUtils.tagging.getTagsForURI(
+    Services.io.newURI("http://example.com/a"));
+  deepEqual(tagsForA.sort(), ["one", "ten", "two"], "Should change tags for A");
+
+  let tagsForB = PlacesUtils.tagging.getTagsForURI(
+    Services.io.newURI("http://example.com/b"));
+  deepEqual(tagsForB, [], "Should remove all tags from B");
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
+
+add_task(async function test_rewrite_tag_queries() {
+  let buf = await openMirror("rewrite_tag_queries");
+
+  do_print("Set up mirror");
+  await PlacesUtils.bookmarks.insertTree({
+    guid: PlacesUtils.bookmarks.menuGuid,
+    children: [{
+      guid: "bookmarkAAAA",
+      title: "A",
+      url: "http://example.com/a",
+    }, {
+      guid: "bookmarkDDDD",
+      title: "D",
+      url: "http://example.com/d",
+      tags: ["kitty"],
+    }],
+  });
+  await buf.store([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: ["bookmarkAAAA", "bookmarkDDDD"],
+  }, {
+    id: "bookmarkAAAA",
+    type: "bookmark",
+    title: "A",
+    bmkUri: "http://example.com/a",
+  }, {
+    id: "bookmarkDDDD",
+    type: "bookmark",
+    title: "D",
+    bmkUri: "http://example.com/d",
+    tags: ["kitty"],
+  }], { needsMerge: false });
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  do_print("Add tag queries for new and existing tags");
+  await buf.store([{
+    id: "toolbar",
+    type: "folder",
+    title: "Bookmarks Toolbar",
+    children: ["queryBBBBBBB", "queryCCCCCCC", "bookmarkEEEE"],
+  }, {
+    id: "queryBBBBBBB",
+    type: "query",
+    title: "Tagged stuff",
+    bmkUri: "place:type=7&folder=999",
+    folderName: "taggy",
+  }, {
+    id: "queryCCCCCCC",
+    type: "query",
+    title: "Cats",
+    bmkUri: "place:type=7&folder=888",
+    folderName: "kitty",
+  }, {
+    id: "bookmarkEEEE",
+    type: "bookmark",
+    title: "E",
+    bmkUri: "http://example.com/e",
+    tags: ["taggy"],
+  }]);
+
+  do_print("Apply mirror");
+  let changesToUpload = await buf.apply();
+  deepEqual(changesToUpload, {}, "Should not reupload any local records");
+
+  let urisWithTaggy = PlacesUtils.tagging.getURIsForTag("taggy");
+  deepEqual(urisWithTaggy.map(uri => uri.spec).sort(), ["http://example.com/e"],
+    "Should insert bookmark with new tag");
+
+  let urisWithKitty = PlacesUtils.tagging.getURIsForTag("kitty");
+  deepEqual(urisWithKitty.map(uri => uri.spec).sort(), ["http://example.com/d"],
+    "Should retain existing tag");
+
+  let { root: toolbarContainer } = PlacesUtils.getFolderContents(
+    PlacesUtils.toolbarFolderId, false, true);
+  equal(toolbarContainer.childCount, 3,
+    "Should add queries and bookmark to toolbar");
+
+  let containerForB = PlacesUtils.asContainer(toolbarContainer.getChild(0));
+  containerForB.containerOpen = true;
+  for (let i = 0; i < containerForB.childCount; ++i) {
+    let child = containerForB.getChild(i);
+    equal(child.uri, "http://example.com/e",
+      `Rewritten tag query B should have tagged child node at ${i}`);
+  }
+  containerForB.containerOpen = false;
+
+  let containerForC = PlacesUtils.asContainer(toolbarContainer.getChild(1));
+  containerForC.containerOpen = true;
+  for (let i = 0; i < containerForC.childCount; ++i) {
+    let child = containerForC.getChild(i);
+    equal(child.uri, "http://example.com/d",
+      `Rewritten tag query C should have tagged child node at ${i}`);
+  }
+  containerForC.containerOpen = false;
+
+  toolbarContainer.containerOpen = false;
+
+  await buf.finalize();
+  await PlacesUtils.bookmarks.eraseEverything();
+  await PlacesSyncUtils.bookmarks.reset();
+});
+
+add_task(async function test_date_added() {
+  let buf = await openMirror("date_added");
+
+  let aDateAdded = new Date(Date.now() - 1 * 24 * 60 * 60 * 1000);
+  let bDateAdded = new Date();
+
+  do_print("Set up mirror");
+  await PlacesUtils.bookmarks.insertTree({
+    guid: PlacesUtils.bookmarks.menuGuid,
+    children: [{
+      guid: "bookmarkAAAA",
+      dateAdded: aDateAdded,
+      title: "A",
+      url: "http://example.com/a",
+    }, {
+      guid: "bookmarkBBBB",
+      dateAdded: bDateAdded,
+      title: "B",
+      url: "http://example.com/b",
+    }],
+  });
+  await buf.store([{
+    id: "menu",
+    type: "folder",
+    title: "Bookmarks Menu",
+    children: ["bookmarkAAAA", "bookmarkBBBB"],
+  }, {
+    id: "bookmarkAAAA",
+    type: "bookmark",
+    title: "A",
+    dateAdded: aDateAdded.getTime(),
+    bmkUri: "http://example.com/a",
+  }, {
+    id: "bookmarkBBBB",
+    type: "bookmark",
+    title: "A",
+    dateAdded: bDateAdded.getTime(),
+    bmkUri: "http://example.com/a",
+  }], { needsMerge: false });
+  await PlacesTestUtils.markBookmarksAsSynced();
+
+  do_print("Set up mirror");
+  let bNewDateAdded = new Date(bDateAdded.getTime() - 1 * 60 * 60 * 1000);
+  await buf.store([{
+    id: "bookmarkAAAA",
+    type: "bookmark",
+    title: "A (remote)",
+    dateAdded: Date.now(),
+    bmkUri: "http://example.com/a",
+  }, {
+    id: "bookmarkBBBB",
+    type: "bookmark",
+    title: "B (remote)",
+    dateAdded: bNewDateAdded.getTime(),
+    bmkUri: "http://example.com/b",
+  }]);
+
+  do_print("Apply mirror");
+  let changesToUpload = await buf.apply();
+  let idsToUpload = inspectChangeRecords(changesToUpload);
+  deepEqual(idsToUpload, {
+    updated: ["bookmarkAAAA"],
+    deleted: []
+  }, "Should flag A for weak reupload");
+
+  let changeCounter = changesToUpload.bookmarkAAAA.counter;
+  strictEqual(changeCounter, 0, "Should not bump change counter for A");
+
+  let aInfo = await PlacesUtils.bookmarks.fetch("bookmarkAAAA");
+  equal(aInfo.title, "A (remote)", "Should change local title for A");
+  deepEqual(aInfo.dateAdded, aDateAdded,
+    "Should not change date added for A to newer remote date");
+
+  let bInfo = await PlacesUtils.bookmarks.fetch("bookmarkBBBB");
+  equal(bInfo.title, "B (remote)", "Should change local title for B");
+  deepEqual(bInfo.dateAdded, bNewDateAdded,
+    "Should take older date added for B");
+});
+
new file mode 100644
--- /dev/null
+++ b/toolkit/components/places/tests/sync/xpcshell.ini
@@ -0,0 +1,11 @@
+[DEFAULT]
+head = head_sync.js
+support-files =
+  livemark.xml
+
+[test_bookmark_corruption.js]
+[test_bookmark_deduping.js]
+[test_bookmark_deletion.js]
+[test_bookmark_kinds.js]
+[test_bookmark_structure_changes.js]
+[test_bookmark_value_changes.js]
--- a/tools/lint/eslint/modules.json
+++ b/tools/lint/eslint/modules.json
@@ -187,16 +187,17 @@
   "status.js": ["Status"],
   "storageserver.js": ["ServerBSO", "StorageServerCallback", "StorageServerCollection", "StorageServer", "storageServerForUsers"],
   "StructuredLog.jsm": ["StructuredLogger", "StructuredFormatter"],
   "StyleEditorUtil.jsm": ["getString", "assert", "log", "text", "wire", "showFilePicker"],
   "subprocess_common.jsm": ["BaseProcess", "PromiseWorker", "SubprocessConstants"],
   "subprocess_unix.jsm": ["SubprocessImpl"],
   "subprocess_win.jsm": ["SubprocessImpl"],
   "sync.jsm": ["Authentication"],
+  "SyncedBookmarksMirror.jsm": ["SyncedBookmarksMirror"],
   "tabs.js": ["TabEngine", "TabSetRecord"],
   "tabs.jsm": ["BrowserTabs"],
   "tcpsocket_test.jsm": ["createSocket", "createServer", "enablePrefsAndPermissions", "socketCompartmentInstanceOfArrayBuffer"],
   "telemetry.js": ["SyncTelemetry"],
   "test.jsm": ["Foo"],
   "test2.jsm": ["Bar"],
   "test_bug883784.jsm": ["Test"],
   "Timer.jsm": ["setTimeout", "setTimeoutWithTarget", "clearTimeout", "setInterval", "setIntervalWithTarget", "clearInterval"],